From 033d2bc52469ff658757f0eb762ed0e52fcc6d28 Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Wed, 16 Dec 2020 09:54:09 -0600 Subject: [PATCH 01/32] Added initial static_reduction_map files. Copied existing static_map files and just renamed all references to static_map to static_reduction_map. --- include/cuco/detail/static_reduction_map.inl | 396 ++++++++ .../detail/static_reduction_map_kernels.cuh | 386 ++++++++ include/cuco/static_reduction_map.cuh | 929 ++++++++++++++++++ tests/CMakeLists.txt | 12 +- .../static_reduction_map_test.cu | 355 +++++++ 5 files changed, 2070 insertions(+), 8 deletions(-) create mode 100644 include/cuco/detail/static_reduction_map.inl create mode 100644 include/cuco/detail/static_reduction_map_kernels.cuh create mode 100644 include/cuco/static_reduction_map.cuh create mode 100644 tests/static_reduction_map/static_reduction_map_test.cu diff --git a/include/cuco/detail/static_reduction_map.inl b/include/cuco/detail/static_reduction_map.inl new file mode 100644 index 000000000..243032f6b --- /dev/null +++ b/include/cuco/detail/static_reduction_map.inl @@ -0,0 +1,396 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed 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. + */ + +namespace cuco { + +/**---------------------------------------------------------------------------* + * @brief Enumeration of the possible results of attempting to insert into + *a hash bucket + *---------------------------------------------------------------------------**/ +enum class insert_result { + CONTINUE, ///< Insert did not succeed, continue trying to insert + SUCCESS, ///< New pair inserted successfully + DUPLICATE ///< Insert did not succeed, key is already present +}; + +template +static_reduction_map::static_reduction_map(std::size_t capacity, + Key empty_key_sentinel, + Value empty_value_sentinel, + Allocator const& alloc) + : capacity_{capacity}, + empty_key_sentinel_{empty_key_sentinel}, + empty_value_sentinel_{empty_value_sentinel}, + slot_allocator_{alloc} +{ + slots_ = std::allocator_traits::allocate(slot_allocator_, capacity); + + auto constexpr block_size = 256; + auto constexpr stride = 4; + auto const grid_size = (capacity + stride * block_size - 1) / (stride * block_size); + detail::initialize + <<>>(slots_, empty_key_sentinel, empty_value_sentinel, capacity); + + CUCO_CUDA_TRY(cudaMallocManaged(&num_successes_, sizeof(atomic_ctr_type))); +} + +template +static_reduction_map::~static_reduction_map() +{ + std::allocator_traits::deallocate(slot_allocator_, slots_, capacity_); + CUCO_CUDA_TRY(cudaFree(num_successes_)); +} + +template +template +void static_reduction_map::insert(InputIt first, + InputIt last, + Hash hash, + KeyEqual key_equal) +{ + auto num_keys = std::distance(first, last); + auto const block_size = 128; + auto const stride = 1; + auto const tile_size = 4; + auto const grid_size = (tile_size * num_keys + stride * block_size - 1) / (stride * block_size); + auto view = get_device_mutable_view(); + + *num_successes_ = 0; + int device_id; + CUCO_CUDA_TRY(cudaGetDevice(&device_id)); + CUCO_CUDA_TRY(cudaMemPrefetchAsync(num_successes_, sizeof(atomic_ctr_type), device_id)); + + detail::insert + <<>>(first, first + num_keys, num_successes_, view, hash, key_equal); + CUCO_CUDA_TRY(cudaDeviceSynchronize()); + + size_ += num_successes_->load(cuda::std::memory_order_relaxed); +} + +template +template +void static_reduction_map::find( + InputIt first, InputIt last, OutputIt output_begin, Hash hash, KeyEqual key_equal) noexcept +{ + auto num_keys = std::distance(first, last); + auto const block_size = 128; + auto const stride = 1; + auto const tile_size = 4; + auto const grid_size = (tile_size * num_keys + stride * block_size - 1) / (stride * block_size); + auto view = get_device_view(); + + detail::find + <<>>(first, last, output_begin, view, hash, key_equal); + CUCO_CUDA_TRY(cudaDeviceSynchronize()); +} + +template +template +void static_reduction_map::contains( + InputIt first, InputIt last, OutputIt output_begin, Hash hash, KeyEqual key_equal) noexcept +{ + auto num_keys = std::distance(first, last); + auto const block_size = 128; + auto const stride = 1; + auto const tile_size = 4; + auto const grid_size = (tile_size * num_keys + stride * block_size - 1) / (stride * block_size); + auto view = get_device_view(); + + detail::contains + <<>>(first, last, output_begin, view, hash, key_equal); + CUCO_CUDA_TRY(cudaDeviceSynchronize()); +} + +template +template +__device__ bool static_reduction_map::device_mutable_view::insert( + value_type const& insert_pair, Hash hash, KeyEqual key_equal) noexcept +{ + auto current_slot{initial_slot(insert_pair.first, hash)}; + + while (true) { + using cuda::std::memory_order_relaxed; + auto expected_key = this->get_empty_key_sentinel(); + auto expected_value = this->get_empty_value_sentinel(); + auto& slot_key = current_slot->first; + auto& slot_value = current_slot->second; + + bool key_success = + slot_key.compare_exchange_strong(expected_key, insert_pair.first, memory_order_relaxed); + bool value_success = + slot_value.compare_exchange_strong(expected_value, insert_pair.second, memory_order_relaxed); + + if (key_success) { + while (not value_success) { + value_success = + slot_value.compare_exchange_strong(expected_value = this->get_empty_value_sentinel(), + insert_pair.second, + memory_order_relaxed); + } + return true; + } else if (value_success) { + slot_value.store(this->get_empty_value_sentinel(), memory_order_relaxed); + } + + // if the key was already inserted by another thread, than this instance is a + // duplicate, so the insert fails + if (key_equal(insert_pair.first, expected_key)) { return false; } + + // if we couldn't insert the key, but it wasn't a duplicate, then there must + // have been some other key there, so we keep looking for a slot + current_slot = next_slot(current_slot); + } +} + +template +template +__device__ bool static_reduction_map::device_mutable_view::insert( + CG g, value_type const& insert_pair, Hash hash, KeyEqual key_equal) noexcept +{ + auto current_slot = initial_slot(g, insert_pair.first, hash); + + while (true) { + key_type const existing_key = current_slot->first; + + // The user provide `key_equal` can never be used to compare against `empty_key_sentinel` as the + // sentinel is not a valid key value. Therefore, first check for the sentinel + auto const slot_is_empty = (existing_key == this->get_empty_key_sentinel()); + + // the key we are trying to insert is already in the map, so we return with failure to insert + if (g.ballot(not slot_is_empty and key_equal(existing_key, insert_pair.first))) { + return false; + } + + auto const window_contains_empty = g.ballot(slot_is_empty); + + // we found an empty slot, but not the key we are inserting, so this must + // be an empty slot into which we can insert the key + if (window_contains_empty) { + // the first lane in the group with an empty slot will attempt the insert + insert_result status{insert_result::CONTINUE}; + uint32_t src_lane = __ffs(window_contains_empty) - 1; + + if (g.thread_rank() == src_lane) { + using cuda::std::memory_order_relaxed; + auto expected_key = this->get_empty_key_sentinel(); + auto expected_value = this->get_empty_value_sentinel(); + auto& slot_key = current_slot->first; + auto& slot_value = current_slot->second; + + bool key_success = + slot_key.compare_exchange_strong(expected_key, insert_pair.first, memory_order_relaxed); + bool value_success = slot_value.compare_exchange_strong( + expected_value, insert_pair.second, memory_order_relaxed); + + if (key_success) { + while (not value_success) { + value_success = + slot_value.compare_exchange_strong(expected_value = this->get_empty_value_sentinel(), + insert_pair.second, + memory_order_relaxed); + } + status = insert_result::SUCCESS; + } else if (value_success) { + slot_value.store(this->get_empty_value_sentinel(), memory_order_relaxed); + } + + // our key was already present in the slot, so our key is a duplicate + if (key_equal(insert_pair.first, expected_key)) { status = insert_result::DUPLICATE; } + // another key was inserted in the slot we wanted to try + // so we need to try the next empty slot in the window + } + + uint32_t res_status = g.shfl(static_cast(status), src_lane); + status = static_cast(res_status); + + // successful insert + if (status == insert_result::SUCCESS) { return true; } + // duplicate present during insert + if (status == insert_result::DUPLICATE) { return false; } + // if we've gotten this far, a different key took our spot + // before we could insert. We need to retry the insert on the + // same window + } + // if there are no empty slots in the current window, + // we move onto the next window + else { + current_slot = next_slot(g, current_slot); + } + } +} + +template +template +__device__ typename static_reduction_map::device_view::iterator +static_reduction_map::device_view::find(Key const& k, + Hash hash, + KeyEqual key_equal) noexcept +{ + auto current_slot = initial_slot(k, hash); + + while (true) { + auto const existing_key = current_slot->first.load(cuda::std::memory_order_relaxed); + // Key doesn't exist, return end() + if (existing_key == this->get_empty_key_sentinel()) { return this->end(); } + + // Key exists, return iterator to location + if (key_equal(existing_key, k)) { return current_slot; } + + current_slot = next_slot(current_slot); + } +} + +template +template +__device__ typename static_reduction_map::device_view::const_iterator +static_reduction_map::device_view::find(Key const& k, + Hash hash, + KeyEqual key_equal) const + noexcept +{ + auto current_slot = initial_slot(k, hash); + + while (true) { + auto const existing_key = current_slot->first.load(cuda::std::memory_order_relaxed); + // Key doesn't exist, return end() + if (existing_key == this->get_empty_key_sentinel()) { return this->end(); } + + // Key exists, return iterator to location + if (key_equal(existing_key, k)) { return current_slot; } + + current_slot = next_slot(current_slot); + } +} + +template +template +__device__ typename static_reduction_map::device_view::iterator +static_reduction_map::device_view::find(CG g, + Key const& k, + Hash hash, + KeyEqual key_equal) noexcept +{ + auto current_slot = initial_slot(g, k, hash); + + while (true) { + auto const existing_key = current_slot->first.load(cuda::std::memory_order_relaxed); + + // The user provide `key_equal` can never be used to compare against `empty_key_sentinel` as the + // sentinel is not a valid key value. Therefore, first check for the sentinel + auto const slot_is_empty = (existing_key == this->get_empty_key_sentinel()); + + // the key we were searching for was found by one of the threads, + // so we return an iterator to the entry + auto const exists = g.ballot(not slot_is_empty and key_equal(existing_key, k)); + if (exists) { + uint32_t src_lane = __ffs(exists) - 1; + // TODO: This shouldn't cast an iterator to an int to shuffle. Instead, get the index of the + // current_slot and shuffle that instead. + intptr_t res_slot = g.shfl(reinterpret_cast(current_slot), src_lane); + return reinterpret_cast(res_slot); + } + + // we found an empty slot, meaning that the key we're searching for isn't present + if (g.ballot(slot_is_empty)) { return this->end(); } + + // otherwise, all slots in the current window are full with other keys, so we move onto the + // next window + current_slot = next_slot(g, current_slot); + } +} + +template +template +__device__ typename static_reduction_map::device_view::const_iterator +static_reduction_map::device_view::find( + CG g, Key const& k, Hash hash, KeyEqual key_equal) const noexcept +{ + auto current_slot = initial_slot(g, k, hash); + + while (true) { + auto const existing_key = current_slot->first.load(cuda::std::memory_order_relaxed); + + // The user provide `key_equal` can never be used to compare against `empty_key_sentinel` as the + // sentinel is not a valid key value. Therefore, first check for the sentinel + auto const slot_is_empty = (existing_key == this->get_empty_key_sentinel()); + + // the key we were searching for was found by one of the threads, so we return an iterator to + // the entry + auto const exists = g.ballot(not slot_is_empty and key_equal(existing_key, k)); + if (exists) { + uint32_t src_lane = __ffs(exists) - 1; + // TODO: This shouldn't cast an iterator to an int to shuffle. Instead, get the index of the + // current_slot and shuffle that instead. + intptr_t res_slot = g.shfl(reinterpret_cast(current_slot), src_lane); + return reinterpret_cast(res_slot); + } + + // we found an empty slot, meaning that the key we're searching + // for isn't in this submap, so we should move onto the next one + if (g.ballot(slot_is_empty)) { return this->end(); } + + // otherwise, all slots in the current window are full with other keys, + // so we move onto the next window in the current submap + + current_slot = next_slot(g, current_slot); + } +} + +template +template +__device__ bool static_reduction_map::device_view::contains( + Key const& k, Hash hash, KeyEqual key_equal) noexcept +{ + auto current_slot = initial_slot(k, hash); + + while (true) { + auto const existing_key = current_slot->first.load(cuda::std::memory_order_relaxed); + + if (existing_key == empty_key_sentinel_) { return false; } + + if (key_equal(existing_key, k)) { return true; } + + current_slot = next_slot(current_slot); + } +} + +template +template +__device__ bool static_reduction_map::device_view::contains( + CG g, Key const& k, Hash hash, KeyEqual key_equal) noexcept +{ + auto current_slot = initial_slot(g, k, hash); + + while (true) { + key_type const existing_key = current_slot->first.load(cuda::std::memory_order_relaxed); + + // The user provide `key_equal` can never be used to compare against `empty_key_sentinel` as the + // sentinel is not a valid key value. Therefore, first check for the sentinel + auto const slot_is_empty = (existing_key == this->get_empty_key_sentinel()); + + // the key we were searching for was found by one of the threads, so we return an iterator to + // the entry + if (g.ballot(not slot_is_empty and key_equal(existing_key, k))) { return true; } + + // we found an empty slot, meaning that the key we're searching for isn't present + if (g.ballot(slot_is_empty)) { return false; } + + // otherwise, all slots in the current window are full with other keys, so we move onto the next + // window + current_slot = next_slot(g, current_slot); + } +} +} // namespace cuco diff --git a/include/cuco/detail/static_reduction_map_kernels.cuh b/include/cuco/detail/static_reduction_map_kernels.cuh new file mode 100644 index 000000000..6ded5e99d --- /dev/null +++ b/include/cuco/detail/static_reduction_map_kernels.cuh @@ -0,0 +1,386 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed 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. + */ + +namespace cuco { +namespace detail { +namespace cg = cooperative_groups; + +/** + * @brief Initializes each slot in the flat `slots` storage to contain `k` and `v`. + * + * Each space in `slots` that can hold a key value pair is initialized to a + * `pair_atomic_type` containing the key `k` and the value `v`. + * + * @tparam atomic_key_type Type of the `Key` atomic container + * @tparam atomic_mapped_type Type of the `Value` atomic container + * @tparam Key key type + * @tparam Value value type + * @tparam pair_atomic_type key/value pair type + * @param slots Pointer to flat storage for the map's key/value pairs + * @param k Key to which all keys in `slots` are initialized + * @param v Value to which all values in `slots` are initialized + * @param size Size of the storage pointed to by `slots` + */ +template +__global__ void initialize(pair_atomic_type* const slots, Key k, Value v, std::size_t size) +{ + auto tid = threadIdx.x + blockIdx.x * blockDim.x; + while (tid < size) { + new (&slots[tid].first) atomic_key_type{k}; + new (&slots[tid].second) atomic_mapped_type{v}; + tid += gridDim.x * blockDim.x; + } +} + +/** + * @brief Inserts all key/value pairs in the range `[first, last)`. + * + * If multiple keys in `[first, last)` compare equal, it is unspecified which + * element is inserted. + * + * @tparam block_size + * @tparam InputIt Device accessible input iterator whose `value_type` is + * convertible to the map's `value_type` + * @tparam atomicT Type of atomic storage + * @tparam viewT Type of device view allowing access of hash map storage + * @tparam Hash Unary callable type + * @tparam KeyEqual Binary callable type + * @param first Beginning of the sequence of key/value pairs + * @param last End of the sequence of key/value pairs + * @param num_successes The number of successfully inserted key/value pairs + * @param view Mutable device view used to access the hash map's slot storage + * @param hash The unary function to apply to hash each key + * @param key_equal The binary function used to compare two keys for equality + */ +template +__global__ void insert( + InputIt first, InputIt last, atomicT* num_successes, viewT view, Hash hash, KeyEqual key_equal) +{ + typedef cub::BlockReduce BlockReduce; + __shared__ typename BlockReduce::TempStorage temp_storage; + std::size_t thread_num_successes = 0; + + auto tid = blockDim.x * blockIdx.x + threadIdx.x; + auto it = first + tid; + + while (it < last) { + typename viewT::value_type const insert_pair{*it}; + if (view.insert(insert_pair, hash, key_equal)) { thread_num_successes++; } + it += gridDim.x * blockDim.x; + } + + // compute number of successfully inserted elements for each block + // and atomically add to the grand total + std::size_t block_num_successes = BlockReduce(temp_storage).Sum(thread_num_successes); + if (threadIdx.x == 0) { *num_successes += block_num_successes; } +} + +/** + * @brief Inserts all key/value pairs in the range `[first, last)`. + * + * If multiple keys in `[first, last)` compare equal, it is unspecified which + * element is inserted. Uses the CUDA Cooperative Groups API to leverage groups + * of multiple threads to perform each key/value insertion. This provides a + * significant boost in throughput compared to the non Cooperative Group + * `insert` at moderate to high load factors. + * + * @tparam block_size + * @tparam tile_size The number of threads in the Cooperative Groups used to perform + * inserts + * @tparam InputIt Device accessible input iterator whose `value_type` is + * convertible to the map's `value_type` + * @tparam atomicT Type of atomic storage + * @tparam viewT Type of device view allowing access of hash map storage + * @tparam Hash Unary callable type + * @tparam KeyEqual Binary callable type + * @param first Beginning of the sequence of key/value pairs + * @param last End of the sequence of key/value pairs + * @param num_successes The number of successfully inserted key/value pairs + * @param view Mutable device view used to access the hash map's slot storage + * @param hash The unary function to apply to hash each key + * @param key_equal The binary function used to compare two keys for equality + */ +template +__global__ void insert( + InputIt first, InputIt last, atomicT* num_successes, viewT view, Hash hash, KeyEqual key_equal) +{ + typedef cub::BlockReduce BlockReduce; + __shared__ typename BlockReduce::TempStorage temp_storage; + std::size_t thread_num_successes = 0; + + auto tile = cg::tiled_partition(cg::this_thread_block()); + auto tid = blockDim.x * blockIdx.x + threadIdx.x; + auto it = first + tid / tile_size; + + while (it < last) { + // force conversion to value_type + typename viewT::value_type const insert_pair{*it}; + if (view.insert(tile, insert_pair, hash, key_equal) && tile.thread_rank() == 0) { + thread_num_successes++; + } + it += (gridDim.x * blockDim.x) / tile_size; + } + + // compute number of successfully inserted elements for each block + // and atomically add to the grand total + std::size_t block_num_successes = BlockReduce(temp_storage).Sum(thread_num_successes); + if (threadIdx.x == 0) { *num_successes += block_num_successes; } +} + +/** + * @brief Finds the values corresponding to all keys in the range `[first, last)`. + * + * If the key `*(first + i)` exists in the map, copies its associated value to `(output_begin + i)`. + * Else, copies the empty value sentinel. + * @tparam block_size The size of the thread block + * @tparam Value The type of the mapped value for the map + * @tparam InputIt Device accessible input iterator whose `value_type` is + * convertible to the map's `key_type` + * @tparam OutputIt Device accessible output iterator whose `value_type` is + * convertible to the map's `mapped_type` + * @tparam viewT Type of device view allowing access of hash map storage + * @tparam Hash Unary callable type + * @tparam KeyEqual Binary callable type + * @param first Beginning of the sequence of keys + * @param last End of the sequence of keys + * @param output_begin Beginning of the sequence of values retrieved for each key + * @param view Device view used to access the hash map's slot storage + * @param hash The unary function to apply to hash each key + * @param key_equal The binary function to compare two keys for equality + */ +template +__global__ void find( + InputIt first, InputIt last, OutputIt output_begin, viewT view, Hash hash, KeyEqual key_equal) +{ + auto tid = blockDim.x * blockIdx.x + threadIdx.x; + auto key_idx = tid; + __shared__ Value writeBuffer[block_size]; + + while (first + key_idx < last) { + auto key = *(first + key_idx); + auto found = view.find(key, hash, key_equal); + + /* + * The ld.relaxed.gpu instruction used in view.find causes L1 to + * flush more frequently, causing increased sector stores from L2 to global memory. + * By writing results to shared memory and then synchronizing before writing back + * to global, we no longer rely on L1, preventing the increase in sector stores from + * L2 to global and improving performance. + */ + writeBuffer[threadIdx.x] = found->second.load(cuda::std::memory_order_relaxed); + __syncthreads(); + *(output_begin + key_idx) = writeBuffer[threadIdx.x]; + key_idx += gridDim.x * blockDim.x; + } +} + +/** + * @brief Finds the values corresponding to all keys in the range `[first, last)`. + * + * If the key `*(first + i)` exists in the map, copies its associated value to `(output_begin + i)`. + * Else, copies the empty value sentinel. Uses the CUDA Cooperative Groups API to leverage groups + * of multiple threads to find each key. This provides a significant boost in throughput compared + * to the non Cooperative Group `find` at moderate to high load factors. + * + * @tparam block_size The size of the thread block + * @tparam tile_size The number of threads in the Cooperative Groups used to perform + * inserts + * @tparam Value The type of the mapped value for the map + * @tparam InputIt Device accessible input iterator whose `value_type` is + * convertible to the map's `key_type` + * @tparam OutputIt Device accessible output iterator whose `value_type` is + * convertible to the map's `mapped_type` + * @tparam viewT Type of device view allowing access of hash map storage + * @tparam Hash Unary callable type + * @tparam KeyEqual Binary callable type + * @param first Beginning of the sequence of keys + * @param last End of the sequence of keys + * @param output_begin Beginning of the sequence of values retrieved for each key + * @param view Device view used to access the hash map's slot storage + * @param hash The unary function to apply to hash each key + * @param key_equal The binary function to compare two keys for equality + */ +template +__global__ void find( + InputIt first, InputIt last, OutputIt output_begin, viewT view, Hash hash, KeyEqual key_equal) +{ + auto tile = cg::tiled_partition(cg::this_thread_block()); + auto tid = blockDim.x * blockIdx.x + threadIdx.x; + auto key_idx = tid / tile_size; + __shared__ Value writeBuffer[block_size]; + + while (first + key_idx < last) { + auto key = *(first + key_idx); + auto found = view.find(tile, key, hash, key_equal); + + /* + * The ld.relaxed.gpu instruction used in view.find causes L1 to + * flush more frequently, causing increased sector stores from L2 to global memory. + * By writing results to shared memory and then synchronizing before writing back + * to global, we no longer rely on L1, preventing the increase in sector stores from + * L2 to global and improving performance. + */ + if (tile.thread_rank() == 0) { + writeBuffer[threadIdx.x / tile_size] = found->second.load(cuda::std::memory_order_relaxed); + } + __syncthreads(); + if (tile.thread_rank() == 0) { + *(output_begin + key_idx) = writeBuffer[threadIdx.x / tile_size]; + } + key_idx += (gridDim.x * blockDim.x) / tile_size; + } +} + +/** + * @brief Indicates whether the keys in the range `[first, last)` are contained in the map. + * + * Writes a `bool` to `(output + i)` indicating if the key `*(first + i)` exists in the map. + * + * @tparam block_size The size of the thread block + * @tparam InputIt Device accessible input iterator whose `value_type` is + * convertible to the map's `key_type` + * @tparam OutputIt Device accessible output iterator whose `value_type` is + * convertible to the map's `mapped_type` + * @tparam viewT Type of device view allowing access of hash map storage + * @tparam Hash Unary callable type + * @tparam KeyEqual Binary callable type + * @param first Beginning of the sequence of keys + * @param last End of the sequence of keys + * @param output_begin Beginning of the sequence of booleans for the presence of each key + * @param view Device view used to access the hash map's slot storage + * @param hash The unary function to apply to hash each key + * @param key_equal The binary function to compare two keys for equality + */ +template +__global__ void contains( + InputIt first, InputIt last, OutputIt output_begin, viewT view, Hash hash, KeyEqual key_equal) +{ + auto tid = blockDim.x * blockIdx.x + threadIdx.x; + auto key_idx = tid; + __shared__ bool writeBuffer[block_size]; + + while (first + key_idx < last) { + auto key = *(first + key_idx); + + /* + * The ld.relaxed.gpu instruction used in view.find causes L1 to + * flush more frequently, causing increased sector stores from L2 to global memory. + * By writing results to shared memory and then synchronizing before writing back + * to global, we no longer rely on L1, preventing the increase in sector stores from + * L2 to global and improving performance. + */ + writeBuffer[threadIdx.x] = view.contains(key, hash, key_equal); + __syncthreads(); + *(output_begin + key_idx) = writeBuffer[threadIdx.x]; + key_idx += gridDim.x * blockDim.x; + } +} + +/** + * @brief Indicates whether the keys in the range `[first, last)` are contained in the map. + * + * Writes a `bool` to `(output + i)` indicating if the key `*(first + i)` exists in the map. + * Uses the CUDA Cooperative Groups API to leverage groups of multiple threads to perform the + * contains operation for each key. This provides a significant boost in throughput compared + * to the non Cooperative Group `contains` at moderate to high load factors. + * + * @tparam block_size The size of the thread block + * @tparam tile_size The number of threads in the Cooperative Groups used to perform + * inserts + * @tparam InputIt Device accessible input iterator whose `value_type` is + * convertible to the map's `key_type` + * @tparam OutputIt Device accessible output iterator whose `value_type` is + * convertible to the map's `mapped_type` + * @tparam viewT Type of device view allowing access of hash map storage + * @tparam Hash Unary callable type + * @tparam KeyEqual Binary callable type + * @param first Beginning of the sequence of keys + * @param last End of the sequence of keys + * @param output_begin Beginning of the sequence of booleans for the presence of each key + * @param view Device view used to access the hash map's slot storage + * @param hash The unary function to apply to hash each key + * @param key_equal The binary function to compare two keys for equality + */ +template +__global__ void contains( + InputIt first, InputIt last, OutputIt output_begin, viewT view, Hash hash, KeyEqual key_equal) +{ + auto tile = cg::tiled_partition(cg::this_thread_block()); + auto tid = blockDim.x * blockIdx.x + threadIdx.x; + auto key_idx = tid / tile_size; + __shared__ bool writeBuffer[block_size]; + + while (first + key_idx < last) { + auto key = *(first + key_idx); + auto found = view.contains(tile, key, hash, key_equal); + + /* + * The ld.relaxed.gpu instruction used in view.find causes L1 to + * flush more frequently, causing increased sector stores from L2 to global memory. + * By writing results to shared memory and then synchronizing before writing back + * to global, we no longer rely on L1, preventing the increase in sector stores from + * L2 to global and improving performance. + */ + if (tile.thread_rank() == 0) { writeBuffer[threadIdx.x / tile_size] = found; } + __syncthreads(); + if (tile.thread_rank() == 0) { + *(output_begin + key_idx) = writeBuffer[threadIdx.x / tile_size]; + } + key_idx += (gridDim.x * blockDim.x) / tile_size; + } +} + +} // namespace detail +} // namespace cuco \ No newline at end of file diff --git a/include/cuco/static_reduction_map.cuh b/include/cuco/static_reduction_map.cuh new file mode 100644 index 000000000..241ef480d --- /dev/null +++ b/include/cuco/static_reduction_map.cuh @@ -0,0 +1,929 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed 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 + +#include +#include +#ifndef CUDART_VERSION +#error CUDART_VERSION Undefined! +#elif (CUDART_VERSION >= 11000) // including with CUDA 10.2 leads to compilation errors +#include +#endif + +#include +#include +#include +#include + +namespace cuco { + +/** + * @brief Possible reduction operations that can be performed by a `static_reduction_map`. + * + * `GENERIC` allows for any associative binary reduction operation, but may have worse performance + * compared to one of the native operations. + * + */ +enum class reduction_op { + SUM, ///< Addition + SUB, ///< Subtraction + MIN, ///< Minimum value + MAX, ///< Maximum value + AND, ///< Bitwise AND + OR, ///< Bitwise OR + XOR, ///< Bitwise XOR + GENERIC ///< User-defined, associative binary operation +}; + +/** + * @brief A GPU-accelerated, unordered, associative container of key-value + * pairs with unique keys. + * + * Allows constant time concurrent inserts or concurrent find operations (not + * concurrent insert and find) from threads in device code. + * + * Current limitations: + * - Requires keys that are Arithmetic + * - Does not support erasing keys + * - Capacity is fixed and will not grow automatically + * - Requires the user to specify sentinel values for both key and mapped value + * to indicate empty slots + * - Does not support concurrent insert and find operations + * + * The `static_reduction_map` supports two types of operations: + * - Host-side "bulk" operations + * - Device-side "singular" operations + * + * The host-side bulk operations include `insert`, `find`, and `contains`. These + * APIs should be used when there are a large number of keys to insert or lookup + * in the map. For example, given a range of keys specified by device-accessible + * iterators, the bulk `insert` function will insert all keys into the map. + * + * The singular device-side operations allow individual threads to to perform + * independent insert or find/contains operations from device code. These + * operations are accessed through non-owning, trivially copyable "view" types: + * `device_view` and `mutable_device_view`. The `device_view` class is an + * immutable view that allows only non-modifying operations such as `find` or + * `contains`. The `mutable_device_view` class only allows `insert` operations. + * The two types are separate to prevent erroneous concurrent insert/find + * operations. + * + * Example: + * \code{.cpp} + * int empty_key_sentinel = -1; + * int empty_value_sentine = -1; + * + * // Constructs a map with 100,000 slots using -1 and -1 as the empty key/value + * // sentinels. Note the capacity is chosen knowing we will insert 50,000 keys, + * // for an load factor of 50%. + * static_reduction_map m{100'000, empty_key_sentinel, empty_value_sentinel}; + * + * // Create a sequence of pairs {{0,0}, {1,1}, ... {i,i}} + * thrust::device_vector> pairs(50,000); + * thrust::transform(thrust::make_counting_iterator(0), + * thrust::make_counting_iterator(pairs.size()), + * pairs.begin(), + * []__device__(auto i){ return thrust::make_pair(i,i); }; + * + * + * // Inserts all pairs into the map + * m.insert(pairs.begin(), pairs.end()); + * + * // Get a `device_view` and passes it to a kernel where threads may perform + * // `find/contains` lookups + * kernel<<<...>>>(m.get_device_view()); + * \endcode + * + * + * @tparam Key Arithmetic type used for key + * @tparam Value Type of the mapped values + * @tparam Scope The scope in which insert/find operations will be performed by + * individual threads. + * @tparam Allocator Type of allocator used for device storage + */ +template > +class static_reduction_map { + static_assert(std::is_arithmetic::value, "Unsupported, non-arithmetic key type."); + + public: + using value_type = cuco::pair_type; + using key_type = Key; + using mapped_type = Value; + using atomic_key_type = cuda::atomic; + using atomic_mapped_type = cuda::atomic; + using pair_atomic_type = cuco::pair_type; + using atomic_ctr_type = cuda::atomic; + using allocator_type = Allocator; + using slot_allocator_type = + typename std::allocator_traits::rebind_alloc; + + static_reduction_map(static_reduction_map const&) = delete; + static_reduction_map(static_reduction_map&&) = delete; + static_reduction_map& operator=(static_reduction_map const&) = delete; + static_reduction_map& operator=(static_reduction_map&&) = delete; + + /** + * @brief Construct a fixed-size map with the specified capacity and sentinel values. + * @brief Construct a statically sized map with the specified number of slots + * and sentinel values. + * + * The capacity of the map is fixed. Insert operations will not automatically + * grow the map. Attempting to insert more unique keys than the capacity of + * the map results in undefined behavior. + * + * Performance begins to degrade significantly beyond a load factor of ~70%. + * For best performance, choose a capacity that will keep the load factor + * below 70%. E.g., if inserting `N` unique keys, choose a capacity of + * `N * (1/0.7)`. + * + * The `empty_key_sentinel` and `empty_value_sentinel` values are reserved and + * undefined behavior results from attempting to insert any key/value pair + * that contains either. + * + * @param capacity The total number of slots in the map + * @param empty_key_sentinel The reserved key value for empty slots + * @param empty_value_sentinel The reserved mapped value for empty slots + * @param alloc Allocator used for allocating device storage + */ + static_reduction_map(std::size_t capacity, + Key empty_key_sentinel, + Value empty_value_sentinel, + Allocator const& alloc = Allocator{}); + + /** + * @brief Destroys the map and frees its contents. + * + */ + ~static_reduction_map(); + + /** + * @brief Inserts all key/value pairs in the range `[first, last)`. + * + * If multiple keys in `[first, last)` compare equal, it is unspecified which + * element is inserted. + * + * @tparam InputIt Device accessible input iterator whose `value_type` is + * convertible to the map's `value_type` + * @tparam Hash Unary callable type + * @tparam KeyEqual Binary callable type + * @param first Beginning of the sequence of key/value pairs + * @param last End of the sequence of key/value pairs + * @param hash The unary function to apply to hash each key + * @param key_equal The binary function to compare two keys for equality + */ + template , + typename KeyEqual = thrust::equal_to> + void insert(InputIt first, InputIt last, Hash hash = Hash{}, KeyEqual key_equal = KeyEqual{}); + + /** + * @brief Finds the values corresponding to all keys in the range `[first, last)`. + * + * If the key `*(first + i)` exists in the map, copies its associated value to `(output_begin + + * i)`. Else, copies the empty value sentinel. + * + * @tparam InputIt Device accessible input iterator whose `value_type` is + * convertible to the map's `key_type` + * @tparam OutputIt Device accessible output iterator whose `value_type` is + * convertible to the map's `mapped_type` + * @tparam Hash Unary callable type + * @tparam KeyEqual Binary callable type + * @param first Beginning of the sequence of keys + * @param last End of the sequence of keys + * @param output_begin Beginning of the sequence of values retrieved for each key + * @param hash The unary function to apply to hash each key + * @param key_equal The binary function to compare two keys for equality + */ + template , + typename KeyEqual = thrust::equal_to> + void find(InputIt first, + InputIt last, + OutputIt output_begin, + Hash hash = Hash{}, + KeyEqual key_equal = KeyEqual{}) noexcept; + + /** + * @brief Indicates whether the keys in the range `[first, last)` are contained in the map. + * + * Writes a `bool` to `(output + i)` indicating if the key `*(first + i)` exists in the map. + * + * @tparam InputIt Device accessible input iterator whose `value_type` is + * convertible to the map's `key_type` + * @tparam OutputIt Device accessible output iterator whose `value_type` is + * convertible to the map's `mapped_type` + * @tparam Hash Unary callable type + * @tparam KeyEqual Binary callable type + * @param first Beginning of the sequence of keys + * @param last End of the sequence of keys + * @param output_begin Beginning of the sequence of booleans for the presence of each key + * @param hash The unary function to apply to hash each key + * @param key_equal The binary function to compare two keys for equality + */ + template , + typename KeyEqual = thrust::equal_to> + void contains(InputIt first, + InputIt last, + OutputIt output_begin, + Hash hash = Hash{}, + KeyEqual key_equal = KeyEqual{}) noexcept; + + private: + class device_view_base { + protected: + // Import member type definitions from `static_reduction_map` + using value_type = value_type; + using key_type = Key; + using mapped_type = Value; + using iterator = pair_atomic_type*; + using const_iterator = pair_atomic_type const*; + + private: + pair_atomic_type* slots_{}; ///< Pointer to flat slots storage + std::size_t capacity_{}; ///< Total number of slots + Key empty_key_sentinel_{}; ///< Key value that represents an empty slot + Value empty_value_sentinel_{}; ///< Initial Value of empty slot + + protected: + __host__ __device__ device_view_base(pair_atomic_type* slots, + std::size_t capacity, + Key empty_key_sentinel, + Value empty_value_sentinel) noexcept + : slots_{slots}, + capacity_{capacity}, + empty_key_sentinel_{empty_key_sentinel}, + empty_value_sentinel_{empty_value_sentinel} + { + } + + /** + * @brief Gets slots array. + * + * @return Slots array + */ + __device__ pair_atomic_type* get_slots() noexcept { return slots_; } + + /** + * @brief Gets slots array. + * + * @return Slots array + */ + __device__ pair_atomic_type const* get_slots() const noexcept { return slots_; } + + /** + * @brief Returns the initial slot for a given key `k` + * + * @tparam Hash Unary callable type + * @param k The key to get the slot for + * @param hash The unary callable used to hash the key + * @return Pointer to the initial slot for `k` + */ + template + __device__ iterator initial_slot(Key const& k, Hash hash) noexcept + { + return &slots_[hash(k) % capacity_]; + } + + /** + * @brief Returns the initial slot for a given key `k` + * + * @tparam Hash Unary callable type + * @param k The key to get the slot for + * @param hash The unary callable used to hash the key + * @return Pointer to the initial slot for `k` + */ + template + __device__ const_iterator initial_slot(Key const& k, Hash hash) const noexcept + { + return &slots_[hash(k) % capacity_]; + } + + /** + * @brief Returns the initial slot for a given key `k` + * + * To be used for Cooperative Group based probing. + * + * @tparam CG Cooperative Group type + * @tparam Hash Unary callable type + * @param g the Cooperative Group for which the initial slot is needed + * @param k The key to get the slot for + * @param hash The unary callable used to hash the key + * @return Pointer to the initial slot for `k` + */ + template + __device__ iterator initial_slot(CG g, Key const& k, Hash hash) noexcept + { + return &slots_[(hash(k) + g.thread_rank()) % capacity_]; + } + + /** + * @brief Returns the initial slot for a given key `k` + * + * To be used for Cooperative Group based probing. + * + * @tparam CG Cooperative Group type + * @tparam Hash Unary callable type + * @param g the Cooperative Group for which the initial slot is needed + * @param k The key to get the slot for + * @param hash The unary callable used to hash the key + * @return Pointer to the initial slot for `k` + */ + template + __device__ const_iterator initial_slot(CG g, Key const& k, Hash hash) const noexcept + { + return &slots_[(hash(k) + g.thread_rank()) % capacity_]; + } + + /** + * @brief Given a slot `s`, returns the next slot. + * + * If `s` is the last slot, wraps back around to the first slot. + * + * @param s The slot to advance + * @return The next slot after `s` + */ + __device__ iterator next_slot(iterator s) noexcept { return (++s < end()) ? s : begin_slot(); } + + /** + * @brief Given a slot `s`, returns the next slot. + * + * If `s` is the last slot, wraps back around to the first slot. + * + * @param s The slot to advance + * @return The next slot after `s` + */ + __device__ const_iterator next_slot(const_iterator s) const noexcept + { + return (++s < end()) ? s : begin_slot(); + } + + /** + * @brief Given a slot `s`, returns the next slot. + * + * If `s` is the last slot, wraps back around to the first slot. To + * be used for Cooperative Group based probing. + * + * @tparam CG The Cooperative Group type + * @param g The Cooperative Group for which the next slot is needed + * @param s The slot to advance + * @return The next slot after `s` + */ + template + __device__ iterator next_slot(CG g, iterator s) noexcept + { + uint32_t index = s - slots_; + return &slots_[(index + g.size()) % capacity_]; + } + + /** + * @brief Given a slot `s`, returns the next slot. + * + * If `s` is the last slot, wraps back around to the first slot. To + * be used for Cooperative Group based probing. + * + * @tparam CG The Cooperative Group type + * @param g The Cooperative Group for which the next slot is needed + * @param s The slot to advance + * @return The next slot after `s` + */ + template + __device__ const_iterator next_slot(CG g, const_iterator s) const noexcept + { + uint32_t index = s - slots_; + return &slots_[(index + g.size()) % capacity_]; + } + + public: + /** + * @brief Gets the maximum number of elements the hash map can hold. + * + * @return The maximum number of elements the hash map can hold + */ + __host__ __device__ std::size_t get_capacity() const noexcept { return capacity_; } + + /** + * @brief Gets the sentinel value used to represent an empty key slot. + * + * @return The sentinel value used to represent an empty key slot + */ + __host__ __device__ Key get_empty_key_sentinel() const noexcept { return empty_key_sentinel_; } + + /** + * @brief Gets the sentinel value used to represent an empty value slot. + * + * @return The sentinel value used to represent an empty value slot + */ + __host__ __device__ Value get_empty_value_sentinel() const noexcept + { + return empty_value_sentinel_; + } + + /** + * @brief Returns iterator to the first slot. + * + * @note Unlike `std::map::begin()`, the `begin_slot()` iterator does _not_ point to the first + * occupied slot. Instead, it refers to the first slot in the array of contiguous slot storage. + * Iterating from `begin_slot()` to `end_slot()` will iterate over all slots, including those + * both empty and filled. + * + * There is no `begin()` iterator to avoid confusion as it is not possible to provide an + * iterator over only the filled slots. + * + * @return Iterator to the first slot + */ + __device__ iterator begin_slot() noexcept { return slots_; } + + /** + * @brief Returns iterator to the first slot. + * + * @note Unlike `std::map::begin()`, the `begin_slot()` iterator does _not_ point to the first + * occupied slot. Instead, it refers to the first slot in the array of contiguous slot storage. + * Iterating from `begin_slot()` to `end_slot()` will iterate over all slots, including those + * both empty and filled. + * + * There is no `begin()` iterator to avoid confusion as it is not possible to provide an + * iterator over only the filled slots. + * + * @return Iterator to the first slot + */ + __device__ const_iterator begin_slot() const noexcept { return slots_; } + + /** + * @brief Returns a const_iterator to one past the last slot. + * + * @return A const_iterator to one past the last slot + */ + __host__ __device__ const_iterator end_slot() const noexcept { return slots_ + capacity_; } + + /** + * @brief Returns an iterator to one past the last slot. + * + * @return An iterator to one past the last slot + */ + __host__ __device__ iterator end_slot() noexcept { return slots_ + capacity_; } + + /** + * @brief Returns a const_iterator to one past the last slot. + * + * `end()` calls `end_slot()` and is provided for convenience for those familiar with checking + * an iterator returned from `find()` against the `end()` iterator. + * + * @return A const_iterator to one past the last slot + */ + __host__ __device__ const_iterator end() const noexcept { return end_slot(); } + + /** + * @brief Returns an iterator to one past the last slot. + * + * `end()` calls `end_slot()` and is provided for convenience for those familiar with checking + * an iterator returned from `find()` against the `end()` iterator. + * + * @return An iterator to one past the last slot + */ + __host__ __device__ iterator end() noexcept { return end_slot(); } + }; + + public: + /** + * @brief Mutable, non-owning view-type that may be used in device code to + * perform singular inserts into the map. + * + * `device_mutable_view` is trivially-copyable and is intended to be passed by + * value. + * + * Example: + * \code{.cpp} + * cuco::static_reduction_map m{100'000, -1, -1}; + * + * // Inserts a sequence of pairs {{0,0}, {1,1}, ... {i,i}} + * thrust::for_each(thrust::make_counting_iterator(0), + * thrust::make_counting_iterator(50'000), + * [map = m.get_mutable_device_view()] + * __device__ (auto i) mutable { + * map.insert(thrust::make_pair(i,i)); + * }); + * \endcode + */ + class device_mutable_view : public device_view_base { + public: + using value_type = typename device_view_base::value_type; + using key_type = typename device_view_base::key_type; + using mapped_type = typename device_view_base::mapped_type; + using iterator = typename device_view_base::iterator; + using const_iterator = typename device_view_base::const_iterator; + /** + * @brief Construct a mutable view of the first `capacity` slots of the + * slots array pointed to by `slots`. + * + * @param slots Pointer to beginning of initialized slots array + * @param capacity The number of slots viewed by this object + * @param empty_key_sentinel The reserved value for keys to represent empty + * slots + * @param empty_value_sentinel The reserved value for mapped values to + * represent empty slots + */ + __host__ __device__ device_mutable_view(pair_atomic_type* slots, + std::size_t capacity, + Key empty_key_sentinel, + Value empty_value_sentinel) noexcept + : device_view_base{slots, capacity, empty_key_sentinel, empty_value_sentinel} + { + } + + /** + * @brief Inserts the specified key/value pair into the map. + * + * Returns a pair consisting of an iterator to the inserted element (or to + * the element that prevented the insertion) and a `bool` denoting whether + * the insertion took place. + * + * @tparam Hash Unary callable type + * @tparam KeyEqual Binary callable type + * @param insert_pair The pair to insert + * @param hash The unary callable used to hash the key + * @param key_equal The binary callable used to compare two keys for + * equality + * @return `true` if the insert was successful, `false` otherwise. + */ + template , + typename KeyEqual = thrust::equal_to> + __device__ bool insert(value_type const& insert_pair, + Hash hash = Hash{}, + KeyEqual key_equal = KeyEqual{}) noexcept; + /** + * @brief Inserts the specified key/value pair into the map. + * + * Returns a pair consisting of an iterator to the inserted element (or to + * the element that prevented the insertion) and a `bool` denoting whether + * the insertion took place. Uses the CUDA Cooperative Groups API to + * to leverage multiple threads to perform a single insert. This provides a + * significant boost in throughput compared to the non Cooperative Group + * `insert` at moderate to high load factors. + * + * @tparam Cooperative Group type + * @tparam Hash Unary callable type + * @tparam KeyEqual Binary callable type + * + * @param g The Cooperative Group that performs the insert + * @param insert_pair The pair to insert + * @param hash The unary callable used to hash the key + * @param key_equal The binary callable used to compare two keys for + * equality + * @return `true` if the insert was successful, `false` otherwise. + */ + template , + typename KeyEqual = thrust::equal_to> + __device__ bool insert(CG g, + value_type const& insert_pair, + Hash hash = Hash{}, + KeyEqual key_equal = KeyEqual{}) noexcept; + + }; // class device mutable view + + /** + * @brief Non-owning view-type that may be used in device code to + * perform singular find and contains operations for the map. + * + * `device_view` is trivially-copyable and is intended to be passed by + * value. + * + */ + class device_view : public device_view_base { + public: + using value_type = typename device_view_base::value_type; + using key_type = typename device_view_base::key_type; + using mapped_type = typename device_view_base::mapped_type; + using iterator = typename device_view_base::iterator; + using const_iterator = typename device_view_base::const_iterator; + /** + * @brief Construct a view of the first `capacity` slots of the + * slots array pointed to by `slots`. + * + * @param slots Pointer to beginning of initialized slots array + * @param capacity The number of slots viewed by this object + * @param empty_key_sentinel The reserved value for keys to represent empty + * slots + * @param empty_value_sentinel The reserved value for mapped values to + * represent empty slots + */ + __host__ __device__ device_view(pair_atomic_type* slots, + std::size_t capacity, + Key empty_key_sentinel, + Value empty_value_sentinel) noexcept + : device_view_base{slots, capacity, empty_key_sentinel, empty_value_sentinel} + { + } + + /** + * @brief Makes a copy of given `device_view` using non-owned memory. + * + * This function is intended to be used to create shared memory copies of small static maps, + * although global memory can be used as well. + * + * Example: + * @code{.cpp} + * template + * __global__ void use_device_view(const typename MapType::device_view device_view, + * map_key_t const* const keys_to_search, + * map_value_t* const values_found, + * const size_t number_of_elements) + * { + * const size_t index = blockIdx.x * blockDim.x + threadIdx.x; + * + * __shared__ typename MapType::pair_atomic_type sm_buffer[CAPACITY]; + * + * auto g = cg::this_thread_block(); + * + * const map_t::device_view sm_static_reduction_map = device_view.make_copy(g, + * sm_buffer); + * + * for (size_t i = g.thread_rank(); i < number_of_elements; i += g.size()) + * { + * values_found[i] = sm_static_reduction_map.find(keys_to_search[i])->second; + * } + * } + * @endcode + * + * @tparam CG The type of the cooperative thread group + * @param g The ooperative thread group used to copy the slots + * @param source_device_view `device_view` to copy from + * @param memory_to_use Array large enough to support `capacity` elements. Object does not take + * the ownership of the memory + * @return Copy of passed `device_view` + */ + template + __device__ static device_view make_copy(CG g, + pair_atomic_type* const memory_to_use, + device_view source_device_view) noexcept + { +#ifndef CUDART_VERSION +#error CUDART_VERSION Undefined! +#elif (CUDART_VERSION >= 11000) + __shared__ cuda::barrier barrier; + if (g.thread_rank() == 0) { init(&barrier, g.size()); } + g.sync(); + + cuda::memcpy_async(g, + memory_to_use, + source_device_view.get_slots(), + sizeof(pair_atomic_type) * source_device_view.get_capacity(), + barrier); + + barrier.arrive_and_wait(); +#else + pair_atomic_type const* const slots_ptr = source_device_view.get_slots(); + for (std::size_t i = g.thread_rank(); i < source_device_view.get_capacity(); i += g.size()) { + new (&memory_to_use[i].first) + atomic_key_type{slots_ptr[i].first.load(cuda::memory_order_relaxed)}; + new (&memory_to_use[i].second) + atomic_mapped_type{slots_ptr[i].second.load(cuda::memory_order_relaxed)}; + } + g.sync(); +#endif + + return device_view(memory_to_use, + source_device_view.get_capacity(), + source_device_view.get_empty_key_sentinel(), + source_device_view.get_empty_value_sentinel()); + } + + /** + * @brief Finds the value corresponding to the key `k`. + * + * Returns an iterator to the pair whose key is equivalent to `k`. + * If no such pair exists, returns `end()`. + * + * @tparam Hash Unary callable type + * @tparam KeyEqual Binary callable type + * @param k The key to search for + * @param hash The unary callable used to hash the key + * @param key_equal The binary callable used to compare two keys + * for equality + * @return An iterator to the position at which the key/value pair + * containing `k` was inserted + */ + template , + typename KeyEqual = thrust::equal_to> + __device__ iterator find(Key const& k, + Hash hash = Hash{}, + KeyEqual key_equal = KeyEqual{}) noexcept; + + /** @brief Finds the value corresponding to the key `k`. + * + * Returns a const_iterator to the pair whose key is equivalent to `k`. + * If no such pair exists, returns `end()`. + * + * @tparam Hash Unary callable type + * @tparam KeyEqual Binary callable type + * @param k The key to search for + * @param hash The unary callable used to hash the key + * @param key_equal The binary callable used to compare two keys + * for equality + * @return An iterator to the position at which the key/value pair + * containing `k` was inserted + */ + template , + typename KeyEqual = thrust::equal_to> + __device__ const_iterator find(Key const& k, + Hash hash = Hash{}, + KeyEqual key_equal = KeyEqual{}) const noexcept; + + /** + * @brief Finds the value corresponding to the key `k`. + * + * Returns an iterator to the pair whose key is equivalent to `k`. + * If no such pair exists, returns `end()`. Uses the CUDA Cooperative Groups API to + * to leverage multiple threads to perform a single find. This provides a + * significant boost in throughput compared to the non Cooperative Group + * `find` at moderate to high load factors. + * + * @tparam CG Cooperative Group type + * @tparam Hash Unary callable type + * @tparam KeyEqual Binary callable type + * @param g The Cooperative Group used to perform the find + * @param k The key to search for + * @param hash The unary callable used to hash the key + * @param key_equal The binary callable used to compare two keys + * for equality + * @return An iterator to the position at which the key/value pair + * containing `k` was inserted + */ + template , + typename KeyEqual = thrust::equal_to> + __device__ iterator + find(CG g, Key const& k, Hash hash = Hash{}, KeyEqual key_equal = KeyEqual{}) noexcept; + + /** + * @brief Finds the value corresponding to the key `k`. + * + * Returns a const_iterator to the pair whose key is equivalent to `k`. + * If no such pair exists, returns `end()`. Uses the CUDA Cooperative Groups API to + * to leverage multiple threads to perform a single find. This provides a + * significant boost in throughput compared to the non Cooperative Group + * `find` at moderate to high load factors. + * + * @tparam CG Cooperative Group type + * @tparam Hash Unary callable type + * @tparam KeyEqual Binary callable type + * @param g The Cooperative Group used to perform the find + * @param k The key to search for + * @param hash The unary callable used to hash the key + * @param key_equal The binary callable used to compare two keys + * for equality + * @return An iterator to the position at which the key/value pair + * containing `k` was inserted + */ + template , + typename KeyEqual = thrust::equal_to> + __device__ const_iterator + find(CG g, Key const& k, Hash hash = Hash{}, KeyEqual key_equal = KeyEqual{}) const noexcept; + + /** + * @brief Indicates whether the key `k` was inserted into the map. + * + * If the key `k` was inserted into the map, find returns + * true. Otherwise, it returns false. + * + * @tparam Hash Unary callable type + * @tparam KeyEqual Binary callable type + * @param k The key to search for + * @param hash The unary callable used to hash the key + * @param key_equal The binary callable used to compare two keys + * for equality + * @return A boolean indicating whether the key/value pair + * containing `k` was inserted + */ + template , + typename KeyEqual = thrust::equal_to> + __device__ bool contains(Key const& k, + Hash hash = Hash{}, + KeyEqual key_equal = KeyEqual{}) noexcept; + + /** + * @brief Indicates whether the key `k` was inserted into the map. + * + * If the key `k` was inserted into the map, find returns + * true. Otherwise, it returns false. Uses the CUDA Cooperative Groups API to + * to leverage multiple threads to perform a single contains operation. This provides a + * significant boost in throughput compared to the non Cooperative Group + * `contains` at moderate to high load factors. + * + * @tparam CG Cooperative Group type + * @tparam Hash Unary callable type + * @tparam KeyEqual Binary callable type + * @param g The Cooperative Group used to perform the contains operation + * @param k The key to search for + * @param hash The unary callable used to hash the key + * @param key_equal The binary callable used to compare two keys + * for equality + * @return A boolean indicating whether the key/value pair + * containing `k` was inserted + */ + template , + typename KeyEqual = thrust::equal_to> + __device__ bool contains(CG g, + Key const& k, + Hash hash = Hash{}, + KeyEqual key_equal = KeyEqual{}) noexcept; + }; // class device_view + + /** + * @brief Gets the maximum number of elements the hash map can hold. + * + * @return The maximum number of elements the hash map can hold + */ + std::size_t get_capacity() const noexcept { return capacity_; } + + /** + * @brief Gets the number of elements in the hash map. + * + * @return The number of elements in the map + */ + std::size_t get_size() const noexcept { return size_; } + + /** + * @brief Gets the load factor of the hash map. + * + * @return The load factor of the hash map + */ + float get_load_factor() const noexcept { return static_cast(size_) / capacity_; } + + /** + * @brief Gets the sentinel value used to represent an empty key slot. + * + * @return The sentinel value used to represent an empty key slot + */ + Key get_empty_key_sentinel() const noexcept { return empty_key_sentinel_; } + + /** + * @brief Gets the sentinel value used to represent an empty value slot. + * + * @return The sentinel value used to represent an empty value slot + */ + Value get_empty_value_sentinel() const noexcept { return empty_value_sentinel_; } + + /** + * @brief Constructs a device_view object based on the members of the `static_reduction_map` + * object. + * + * @return A device_view object based on the members of the `static_reduction_map` object + */ + device_view get_device_view() const noexcept + { + return device_view(slots_, capacity_, empty_key_sentinel_, empty_value_sentinel_); + } + + /** + * @brief Constructs a device_mutable_view object based on the members of the + * `static_reduction_map` object + * + * @return A device_mutable_view object based on the members of the `static_reduction_map` object + */ + device_mutable_view get_device_mutable_view() const noexcept + { + return device_mutable_view(slots_, capacity_, empty_key_sentinel_, empty_value_sentinel_); + } + + private: + pair_atomic_type* slots_{nullptr}; ///< Pointer to flat slots storage + std::size_t capacity_{}; ///< Total number of slots + std::size_t size_{}; ///< Number of keys in map + Key empty_key_sentinel_{}; ///< Key value that represents an empty slot + Value empty_value_sentinel_{}; ///< Initial value of empty slot + atomic_ctr_type* num_successes_{}; ///< Number of successfully inserted keys on insert + slot_allocator_type slot_allocator_{}; ///< Allocator used to allocate slots +}; +} // namespace cuco + +#include \ No newline at end of file diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index 66c1682ed..32d77b2a8 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -48,13 +48,9 @@ endfunction(ConfigureTest) ################################################################################################### ### test sources ################################################################################## ################################################################################################### -set(STATIC_MAP_TEST_SRC - "${CMAKE_CURRENT_SOURCE_DIR}/static_map/static_map_test.cu") -ConfigureTest(STATIC_MAP_TEST "${STATIC_MAP_TEST_SRC}") -#################################################################################################### -set(DYNAMIC_MAP_TEST_SRC - "${CMAKE_CURRENT_SOURCE_DIR}/dynamic_map/dynamic_map_test.cu") +ConfigureTest(STATIC_MAP_TEST "${CMAKE_CURRENT_SOURCE_DIR}/static_map/static_map_test.cu") -ConfigureTest(DYNAMIC_MAP_TEST "${DYNAMIC_MAP_TEST_SRC}") -#################################################################################################### \ No newline at end of file +ConfigureTest(STATIC_REDUCTION_MAP_TEST "${CMAKE_CURRENT_SOURCE_DIR}/static_reduction_map/static_reduction_map_test.cu") + +ConfigureTest(DYNAMIC_MAP_TEST "${CMAKE_CURRENT_SOURCE_DIR}/dynamic_map/dynamic_map_test.cu") \ No newline at end of file diff --git a/tests/static_reduction_map/static_reduction_map_test.cu b/tests/static_reduction_map/static_reduction_map_test.cu new file mode 100644 index 000000000..d69d581fc --- /dev/null +++ b/tests/static_reduction_map/static_reduction_map_test.cu @@ -0,0 +1,355 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed 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 + +namespace { +namespace cg = cooperative_groups; + +// Thrust logical algorithms (any_of/all_of/none_of) don't work with device +// lambdas: See https://github.com/thrust/thrust/issues/1062 +template +bool all_of(Iterator begin, Iterator end, Predicate p) +{ + auto size = thrust::distance(begin, end); + return size == thrust::count_if(begin, end, p); +} + +template +bool any_of(Iterator begin, Iterator end, Predicate p) +{ + return thrust::count_if(begin, end, p) > 0; +} + +template +bool none_of(Iterator begin, Iterator end, Predicate p) +{ + return not all_of(begin, end, p); +} +} // namespace + +enum class dist_type { UNIQUE, UNIFORM, GAUSSIAN }; + +template +static void generate_keys(OutputIt output_begin, OutputIt output_end) +{ + auto num_keys = std::distance(output_begin, output_end); + + std::random_device rd; + std::mt19937 gen{rd()}; + + switch (Dist) { + case dist_type::UNIQUE: + for (auto i = 0; i < num_keys; ++i) { + output_begin[i] = i; + } + break; + case dist_type::UNIFORM: + for (auto i = 0; i < num_keys; ++i) { + output_begin[i] = std::abs(static_cast(gen())); + } + break; + case dist_type::GAUSSIAN: + std::normal_distribution<> dg{1e9, 1e7}; + for (auto i = 0; i < num_keys; ++i) { + output_begin[i] = std::abs(static_cast(dg(gen))); + } + break; + } +} + +TEMPLATE_TEST_CASE_SIG("Unique sequence of keys", + "", + ((typename T, dist_type Dist), T, Dist), + (int32_t, dist_type::UNIQUE), + (int64_t, dist_type::UNIQUE), + (int32_t, dist_type::UNIFORM), + (int64_t, dist_type::UNIFORM), + (int32_t, dist_type::GAUSSIAN), + (int64_t, dist_type::GAUSSIAN)) +{ + using Key = T; + using Value = T; + + constexpr std::size_t num_keys{50'000'000}; + cuco::static_reduction_map map{100'000'000, -1, -1}; + + auto m_view = map.get_device_mutable_view(); + auto view = map.get_device_view(); + + std::vector h_keys(num_keys); + std::vector h_values(num_keys); + std::vector> h_pairs(num_keys); + + generate_keys(h_keys.begin(), h_keys.end()); + + for (auto i = 0; i < num_keys; ++i) { + Key key = h_keys[i]; + Value val = h_keys[i]; + h_pairs[i].first = key; + h_pairs[i].second = val; + h_values[i] = val; + } + + thrust::device_vector d_keys(h_keys); + thrust::device_vector d_values(h_values); + thrust::device_vector> d_pairs(h_pairs); + thrust::device_vector d_results(num_keys); + thrust::device_vector d_contained(num_keys); + + // bulk function test cases + SECTION("All inserted keys-value pairs should be correctly recovered during find") + { + map.insert(d_pairs.begin(), d_pairs.end()); + map.find(d_keys.begin(), d_keys.end(), d_results.begin()); + auto zip = thrust::make_zip_iterator(thrust::make_tuple(d_results.begin(), d_values.begin())); + + REQUIRE(all_of(zip, zip + num_keys, [] __device__(auto const& p) { + return thrust::get<0>(p) == thrust::get<1>(p); + })); + } + + SECTION("All inserted keys-value pairs should be contained") + { + map.insert(d_pairs.begin(), d_pairs.end()); + map.contains(d_keys.begin(), d_keys.end(), d_contained.begin()); + + REQUIRE( + all_of(d_contained.begin(), d_contained.end(), [] __device__(bool const& b) { return b; })); + } + + SECTION("Non-inserted keys-value pairs should not be contained") + { + map.contains(d_keys.begin(), d_keys.end(), d_contained.begin()); + + REQUIRE( + none_of(d_contained.begin(), d_contained.end(), [] __device__(bool const& b) { return b; })); + } + + SECTION("Inserting unique keys should return insert success.") + { + if (Dist == dist_type::UNIQUE) { + REQUIRE(all_of(d_pairs.begin(), + d_pairs.end(), + [m_view] __device__(cuco::pair_type const& pair) mutable { + return m_view.insert(pair); + })); + } + } + + SECTION("Cannot find any key in an empty hash map with non-const view") + { + SECTION("non-const view") + { + REQUIRE(all_of(d_pairs.begin(), + d_pairs.end(), + [view] __device__(cuco::pair_type const& pair) mutable { + return view.find(pair.first) == view.end(); + })); + } + SECTION("const view") + { + REQUIRE(all_of( + d_pairs.begin(), d_pairs.end(), [view] __device__(cuco::pair_type const& pair) { + return view.find(pair.first) == view.end(); + })); + } + } + + SECTION("Keys are all found after inserting many keys.") + { + // Bulk insert keys + thrust::for_each(thrust::device, + d_pairs.begin(), + d_pairs.end(), + [m_view] __device__(cuco::pair_type const& pair) mutable { + m_view.insert(pair); + }); + + SECTION("non-const view") + { + // All keys should be found + REQUIRE(all_of(d_pairs.begin(), + d_pairs.end(), + [view] __device__(cuco::pair_type const& pair) mutable { + auto const found = view.find(pair.first); + return (found != view.end()) and (found->first.load() == pair.first and + found->second.load() == pair.second); + })); + } + SECTION("const view") + { + // All keys should be found + REQUIRE(all_of( + d_pairs.begin(), d_pairs.end(), [view] __device__(cuco::pair_type const& pair) { + auto const found = view.find(pair.first); + return (found != view.end()) and + (found->first.load() == pair.first and found->second.load() == pair.second); + })); + } + } +} + +template +__global__ void shared_memory_test_kernel( + typename MapType::device_view const* const device_views, + typename MapType::device_view::key_type const* const insterted_keys, + typename MapType::device_view::mapped_type const* const inserted_values, + const size_t number_of_elements, + bool* const keys_exist, + bool* const keys_and_values_correct) +{ + // Each block processes one map + const size_t map_id = blockIdx.x; + const size_t offset = map_id * number_of_elements; + + __shared__ typename MapType::pair_atomic_type sm_buffer[CAPACITY]; + + auto g = cg::this_thread_block(); + typename MapType::device_view sm_device_view = + MapType::device_view::make_copy(g, sm_buffer, device_views[map_id]); + + for (int i = g.thread_rank(); i < number_of_elements; i += g.size()) { + auto found_pair_it = sm_device_view.find(insterted_keys[offset + i]); + + if (found_pair_it != sm_device_view.end()) { + keys_exist[offset + i] = true; + if (found_pair_it->first == insterted_keys[offset + i] and + found_pair_it->second == inserted_values[offset + i]) { + keys_and_values_correct[offset + i] = true; + } else { + keys_and_values_correct[offset + i] = false; + } + } else { + keys_exist[offset + i] = false; + keys_and_values_correct[offset + i] = true; + } + } +} + +TEMPLATE_TEST_CASE_SIG("Shared memory static map", + "", + ((typename T, dist_type Dist), T, Dist), + (int32_t, dist_type::UNIQUE), + (int64_t, dist_type::UNIQUE), + (int32_t, dist_type::UNIFORM), + (int64_t, dist_type::UNIFORM), + (int32_t, dist_type::GAUSSIAN), + (int64_t, dist_type::GAUSSIAN)) +{ + using KeyType = T; + using ValueType = T; + using MapType = cuco::static_reduction_map; + using DeviceViewType = typename MapType::device_view; + using DeviceViewIteratorType = typename DeviceViewType::iterator; + + constexpr std::size_t number_of_maps = 1000; + constexpr std::size_t elements_in_map = 500; + constexpr std::size_t map_capacity = 2 * elements_in_map; + + // one array for all maps, first elements_in_map element belong to map 0, second to map 1 and so + // on + std::vector h_keys(number_of_maps * elements_in_map); + std::vector h_values(number_of_maps * elements_in_map); + std::vector> h_pairs(number_of_maps * elements_in_map); + + // using std::unique_ptr because static_reduction_map does not have copy/move + // constructor/assignment operator yet + std::vector> maps; + + for (std::size_t map_id = 0; map_id < number_of_maps; ++map_id) { + const std::size_t offset = map_id * elements_in_map; + + generate_keys(h_keys.begin() + offset, + h_keys.begin() + offset + elements_in_map); + + for (std::size_t i = 0; i < elements_in_map; ++i) { + KeyType key = h_keys[offset + i]; + ValueType val = key < std::numeric_limits::max() ? key + 1 : 0; + h_values[offset + i] = val; + h_pairs[offset + i].first = key; + h_pairs[offset + i].second = val; + } + + maps.push_back(std::make_unique(map_capacity, -1, -1)); + } + + thrust::device_vector d_keys(h_keys); + thrust::device_vector d_values(h_values); + thrust::device_vector> d_pairs(h_pairs); + + SECTION("Keys are all found after insertion.") + { + std::vector h_device_views; + for (std::size_t map_id = 0; map_id < number_of_maps; ++map_id) { + const std::size_t offset = map_id * elements_in_map; + + MapType* map = maps[map_id].get(); + map->insert(d_pairs.begin() + offset, d_pairs.begin() + offset + elements_in_map); + h_device_views.push_back(map->get_device_view()); + } + thrust::device_vector d_device_views(h_device_views); + + thrust::device_vector d_keys_exist(number_of_maps * elements_in_map); + thrust::device_vector d_keys_and_values_correct(number_of_maps * elements_in_map); + + shared_memory_test_kernel + <<>>(d_device_views.data().get(), + d_keys.data().get(), + d_values.data().get(), + elements_in_map, + d_keys_exist.data().get(), + d_keys_and_values_correct.data().get()); + + REQUIRE(d_keys_exist.size() == d_keys_and_values_correct.size()); + auto zip = thrust::make_zip_iterator( + thrust::make_tuple(d_keys_exist.begin(), d_keys_and_values_correct.begin())); + + REQUIRE(all_of(zip, zip + d_keys_exist.size(), [] __device__(auto const& z) { + return thrust::get<0>(z) and thrust::get<1>(z); + })); + } + + SECTION("No key is found before insertion.") + { + std::vector h_device_views; + for (std::size_t map_id = 0; map_id < number_of_maps; ++map_id) { + h_device_views.push_back(maps[map_id].get()->get_device_view()); + } + thrust::device_vector d_device_views(h_device_views); + + thrust::device_vector d_keys_exist(number_of_maps * elements_in_map); + thrust::device_vector d_keys_and_values_correct(number_of_maps * elements_in_map); + + shared_memory_test_kernel + <<>>(d_device_views.data().get(), + d_keys.data().get(), + d_values.data().get(), + elements_in_map, + d_keys_exist.data().get(), + d_keys_and_values_correct.data().get()); + + REQUIRE(none_of(d_keys_exist.begin(), d_keys_exist.end(), [] __device__(const bool key_found) { + return key_found; + })); + } +} \ No newline at end of file From fe606cd60d27b645d2c551fb607652658c204c41 Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Mon, 4 Jan 2021 14:57:20 -0600 Subject: [PATCH 02/32] Add template parameter for reduction binary op. --- include/cuco/detail/static_reduction_map.inl | 172 +++++++----- include/cuco/static_reduction_map.cuh | 54 ++-- .../static_reduction_map_test.cu | 263 +----------------- 3 files changed, 134 insertions(+), 355 deletions(-) diff --git a/include/cuco/detail/static_reduction_map.inl b/include/cuco/detail/static_reduction_map.inl index 243032f6b..be28e0f28 100644 --- a/include/cuco/detail/static_reduction_map.inl +++ b/include/cuco/detail/static_reduction_map.inl @@ -26,14 +26,17 @@ enum class insert_result { DUPLICATE ///< Insert did not succeed, key is already present }; -template -static_reduction_map::static_reduction_map(std::size_t capacity, - Key empty_key_sentinel, - Value empty_value_sentinel, - Allocator const& alloc) +template +static_reduction_map::static_reduction_map( + std::size_t capacity, Key empty_key_sentinel, ReductionOp reduction_op, Allocator const& alloc) : capacity_{capacity}, empty_key_sentinel_{empty_key_sentinel}, - empty_value_sentinel_{empty_value_sentinel}, + empty_value_sentinel_{ReductionOp::identity}, + op_{reduction_op}, slot_allocator_{alloc} { slots_ = std::allocator_traits::allocate(slot_allocator_, capacity); @@ -41,25 +44,33 @@ static_reduction_map::static_reduction_map(std::si auto constexpr block_size = 256; auto constexpr stride = 4; auto const grid_size = (capacity + stride * block_size - 1) / (stride * block_size); - detail::initialize - <<>>(slots_, empty_key_sentinel, empty_value_sentinel, capacity); + detail::initialize<<>>( + slots_, get_empty_key_sentinel(), get_empty_value_sentinel(), get_capacity()); CUCO_CUDA_TRY(cudaMallocManaged(&num_successes_, sizeof(atomic_ctr_type))); } -template -static_reduction_map::~static_reduction_map() +template +static_reduction_map::~static_reduction_map() { std::allocator_traits::deallocate(slot_allocator_, slots_, capacity_); CUCO_CUDA_TRY(cudaFree(num_successes_)); } -template +template template -void static_reduction_map::insert(InputIt first, - InputIt last, - Hash hash, - KeyEqual key_equal) +void static_reduction_map::insert(InputIt first, + InputIt last, + Hash hash, + KeyEqual key_equal) { auto num_keys = std::distance(first, last); auto const block_size = 128; @@ -80,9 +91,13 @@ void static_reduction_map::insert(InputIt first, size_ += num_successes_->load(cuda::std::memory_order_relaxed); } -template +template template -void static_reduction_map::find( +void static_reduction_map::find( InputIt first, InputIt last, OutputIt output_begin, Hash hash, KeyEqual key_equal) noexcept { auto num_keys = std::distance(first, last); @@ -97,9 +112,13 @@ void static_reduction_map::find( CUCO_CUDA_TRY(cudaDeviceSynchronize()); } -template +template template -void static_reduction_map::contains( +void static_reduction_map::contains( InputIt first, InputIt last, OutputIt output_begin, Hash hash, KeyEqual key_equal) noexcept { auto num_keys = std::distance(first, last); @@ -114,9 +133,14 @@ void static_reduction_map::contains( CUCO_CUDA_TRY(cudaDeviceSynchronize()); } -template +template template -__device__ bool static_reduction_map::device_mutable_view::insert( +__device__ Value +static_reduction_map::device_mutable_view::insert( value_type const& insert_pair, Hash hash, KeyEqual key_equal) noexcept { auto current_slot{initial_slot(insert_pair.first, hash)}; @@ -128,26 +152,12 @@ __device__ bool static_reduction_map::device_mutab auto& slot_key = current_slot->first; auto& slot_value = current_slot->second; - bool key_success = + auto const key_success = slot_key.compare_exchange_strong(expected_key, insert_pair.first, memory_order_relaxed); - bool value_success = - slot_value.compare_exchange_strong(expected_value, insert_pair.second, memory_order_relaxed); - - if (key_success) { - while (not value_success) { - value_success = - slot_value.compare_exchange_strong(expected_value = this->get_empty_value_sentinel(), - insert_pair.second, - memory_order_relaxed); - } - return true; - } else if (value_success) { - slot_value.store(this->get_empty_value_sentinel(), memory_order_relaxed); - } - // if the key was already inserted by another thread, than this instance is a - // duplicate, so the insert fails - if (key_equal(insert_pair.first, expected_key)) { return false; } + if (key_success or key_equal(insert_pair.first, expected_key)) { + // return do_op{}(slot_value, insert_pair.second); + } // if we couldn't insert the key, but it wasn't a duplicate, then there must // have been some other key there, so we keep looking for a slot @@ -155,9 +165,14 @@ __device__ bool static_reduction_map::device_mutab } } -template +template template -__device__ bool static_reduction_map::device_mutable_view::insert( +__device__ bool +static_reduction_map::device_mutable_view::insert( CG g, value_type const& insert_pair, Hash hash, KeyEqual key_equal) noexcept { auto current_slot = initial_slot(g, insert_pair.first, hash); @@ -232,12 +247,16 @@ __device__ bool static_reduction_map::device_mutab } } -template +template template -__device__ typename static_reduction_map::device_view::iterator -static_reduction_map::device_view::find(Key const& k, - Hash hash, - KeyEqual key_equal) noexcept +__device__ + typename static_reduction_map::device_view::iterator + static_reduction_map::device_view::find( + Key const& k, Hash hash, KeyEqual key_equal) noexcept { auto current_slot = initial_slot(k, hash); @@ -253,13 +272,16 @@ static_reduction_map::device_view::find(Key const& } } -template +template template -__device__ typename static_reduction_map::device_view::const_iterator -static_reduction_map::device_view::find(Key const& k, - Hash hash, - KeyEqual key_equal) const - noexcept +__device__ typename static_reduction_map::device_view:: + const_iterator + static_reduction_map::device_view::find( + Key const& k, Hash hash, KeyEqual key_equal) const noexcept { auto current_slot = initial_slot(k, hash); @@ -275,13 +297,16 @@ static_reduction_map::device_view::find(Key const& } } -template +template template -__device__ typename static_reduction_map::device_view::iterator -static_reduction_map::device_view::find(CG g, - Key const& k, - Hash hash, - KeyEqual key_equal) noexcept +__device__ + typename static_reduction_map::device_view::iterator + static_reduction_map::device_view::find( + CG g, Key const& k, Hash hash, KeyEqual key_equal) noexcept { auto current_slot = initial_slot(g, k, hash); @@ -312,11 +337,16 @@ static_reduction_map::device_view::find(CG g, } } -template +template template -__device__ typename static_reduction_map::device_view::const_iterator -static_reduction_map::device_view::find( - CG g, Key const& k, Hash hash, KeyEqual key_equal) const noexcept +__device__ typename static_reduction_map::device_view:: + const_iterator + static_reduction_map::device_view::find( + CG g, Key const& k, Hash hash, KeyEqual key_equal) const noexcept { auto current_slot = initial_slot(g, k, hash); @@ -349,9 +379,14 @@ static_reduction_map::device_view::find( } } -template +template template -__device__ bool static_reduction_map::device_view::contains( +__device__ bool +static_reduction_map::device_view::contains( Key const& k, Hash hash, KeyEqual key_equal) noexcept { auto current_slot = initial_slot(k, hash); @@ -367,9 +402,14 @@ __device__ bool static_reduction_map::device_view: } } -template +template template -__device__ bool static_reduction_map::device_view::contains( +__device__ bool +static_reduction_map::device_view::contains( CG g, Key const& k, Hash hash, KeyEqual key_equal) noexcept { auto current_slot = initial_slot(g, k, hash); diff --git a/include/cuco/static_reduction_map.cuh b/include/cuco/static_reduction_map.cuh index 241ef480d..d66c6cf4a 100644 --- a/include/cuco/static_reduction_map.cuh +++ b/include/cuco/static_reduction_map.cuh @@ -38,22 +38,16 @@ namespace cuco { -/** - * @brief Possible reduction operations that can be performed by a `static_reduction_map`. - * - * `GENERIC` allows for any associative binary reduction operation, but may have worse performance - * compared to one of the native operations. - * - */ -enum class reduction_op { - SUM, ///< Addition - SUB, ///< Subtraction - MIN, ///< Minimum value - MAX, ///< Maximum value - AND, ///< Bitwise AND - OR, ///< Bitwise OR - XOR, ///< Bitwise XOR - GENERIC ///< User-defined, associative binary operation +template +struct reduce_add { + using value_type = T; + static constexpr T identity = 0; + + template + T apply(cuda::atomic& slot, T2 const& value) + { + return slot.fetch_add(value); + } }; /** @@ -122,7 +116,8 @@ enum class reduction_op { * individual threads. * @tparam Allocator Type of allocator used for device storage */ -template > @@ -171,8 +166,8 @@ class static_reduction_map { */ static_reduction_map(std::size_t capacity, Key empty_key_sentinel, - Value empty_value_sentinel, - Allocator const& alloc = Allocator{}); + ReductionOp reduction_op = {}, + Allocator const& alloc = Allocator{}); /** * @brief Destroys the map and frees its contents. @@ -270,16 +265,18 @@ class static_reduction_map { std::size_t capacity_{}; ///< Total number of slots Key empty_key_sentinel_{}; ///< Key value that represents an empty slot Value empty_value_sentinel_{}; ///< Initial Value of empty slot + ReductionOp op_{}; ///< Binary operation reduction function object protected: __host__ __device__ device_view_base(pair_atomic_type* slots, std::size_t capacity, Key empty_key_sentinel, - Value empty_value_sentinel) noexcept + ReductionOp reduction_op) noexcept : slots_{slots}, capacity_{capacity}, empty_key_sentinel_{empty_key_sentinel}, - empty_value_sentinel_{empty_value_sentinel} + empty_value_sentinel_{ReductionOp::identity}, + op_{reduction_op} { } @@ -552,8 +549,8 @@ class static_reduction_map { __host__ __device__ device_mutable_view(pair_atomic_type* slots, std::size_t capacity, Key empty_key_sentinel, - Value empty_value_sentinel) noexcept - : device_view_base{slots, capacity, empty_key_sentinel, empty_value_sentinel} + ReductionOp reduction_op = {}) noexcept + : device_view_base{slots, capacity, empty_key_sentinel, reduction_op} { } @@ -574,9 +571,9 @@ class static_reduction_map { */ template , typename KeyEqual = thrust::equal_to> - __device__ bool insert(value_type const& insert_pair, - Hash hash = Hash{}, - KeyEqual key_equal = KeyEqual{}) noexcept; + __device__ Value insert(value_type const& insert_pair, + Hash hash = Hash{}, + KeyEqual key_equal = KeyEqual{}) noexcept; /** * @brief Inserts the specified key/value pair into the map. * @@ -637,8 +634,8 @@ class static_reduction_map { __host__ __device__ device_view(pair_atomic_type* slots, std::size_t capacity, Key empty_key_sentinel, - Value empty_value_sentinel) noexcept - : device_view_base{slots, capacity, empty_key_sentinel, empty_value_sentinel} + ReductionOp reduction_op = {}) noexcept + : device_view_base{slots, capacity, empty_key_sentinel, reduction_op} { } @@ -922,6 +919,7 @@ class static_reduction_map { Key empty_key_sentinel_{}; ///< Key value that represents an empty slot Value empty_value_sentinel_{}; ///< Initial value of empty slot atomic_ctr_type* num_successes_{}; ///< Number of successfully inserted keys on insert + ReductionOp op_{}; ///< Binary operation reduction function object slot_allocator_type slot_allocator_{}; ///< Allocator used to allocate slots }; } // namespace cuco diff --git a/tests/static_reduction_map/static_reduction_map_test.cu b/tests/static_reduction_map/static_reduction_map_test.cu index d69d581fc..9d709a6c6 100644 --- a/tests/static_reduction_map/static_reduction_map_test.cu +++ b/tests/static_reduction_map/static_reduction_map_test.cu @@ -90,266 +90,7 @@ TEMPLATE_TEST_CASE_SIG("Unique sequence of keys", using Key = T; using Value = T; - constexpr std::size_t num_keys{50'000'000}; - cuco::static_reduction_map map{100'000'000, -1, -1}; + constexpr std::size_t num_slots{50'000'000}; + cuco::static_reduction_map, Key, Value> map{num_slots, -1}; - auto m_view = map.get_device_mutable_view(); - auto view = map.get_device_view(); - - std::vector h_keys(num_keys); - std::vector h_values(num_keys); - std::vector> h_pairs(num_keys); - - generate_keys(h_keys.begin(), h_keys.end()); - - for (auto i = 0; i < num_keys; ++i) { - Key key = h_keys[i]; - Value val = h_keys[i]; - h_pairs[i].first = key; - h_pairs[i].second = val; - h_values[i] = val; - } - - thrust::device_vector d_keys(h_keys); - thrust::device_vector d_values(h_values); - thrust::device_vector> d_pairs(h_pairs); - thrust::device_vector d_results(num_keys); - thrust::device_vector d_contained(num_keys); - - // bulk function test cases - SECTION("All inserted keys-value pairs should be correctly recovered during find") - { - map.insert(d_pairs.begin(), d_pairs.end()); - map.find(d_keys.begin(), d_keys.end(), d_results.begin()); - auto zip = thrust::make_zip_iterator(thrust::make_tuple(d_results.begin(), d_values.begin())); - - REQUIRE(all_of(zip, zip + num_keys, [] __device__(auto const& p) { - return thrust::get<0>(p) == thrust::get<1>(p); - })); - } - - SECTION("All inserted keys-value pairs should be contained") - { - map.insert(d_pairs.begin(), d_pairs.end()); - map.contains(d_keys.begin(), d_keys.end(), d_contained.begin()); - - REQUIRE( - all_of(d_contained.begin(), d_contained.end(), [] __device__(bool const& b) { return b; })); - } - - SECTION("Non-inserted keys-value pairs should not be contained") - { - map.contains(d_keys.begin(), d_keys.end(), d_contained.begin()); - - REQUIRE( - none_of(d_contained.begin(), d_contained.end(), [] __device__(bool const& b) { return b; })); - } - - SECTION("Inserting unique keys should return insert success.") - { - if (Dist == dist_type::UNIQUE) { - REQUIRE(all_of(d_pairs.begin(), - d_pairs.end(), - [m_view] __device__(cuco::pair_type const& pair) mutable { - return m_view.insert(pair); - })); - } - } - - SECTION("Cannot find any key in an empty hash map with non-const view") - { - SECTION("non-const view") - { - REQUIRE(all_of(d_pairs.begin(), - d_pairs.end(), - [view] __device__(cuco::pair_type const& pair) mutable { - return view.find(pair.first) == view.end(); - })); - } - SECTION("const view") - { - REQUIRE(all_of( - d_pairs.begin(), d_pairs.end(), [view] __device__(cuco::pair_type const& pair) { - return view.find(pair.first) == view.end(); - })); - } - } - - SECTION("Keys are all found after inserting many keys.") - { - // Bulk insert keys - thrust::for_each(thrust::device, - d_pairs.begin(), - d_pairs.end(), - [m_view] __device__(cuco::pair_type const& pair) mutable { - m_view.insert(pair); - }); - - SECTION("non-const view") - { - // All keys should be found - REQUIRE(all_of(d_pairs.begin(), - d_pairs.end(), - [view] __device__(cuco::pair_type const& pair) mutable { - auto const found = view.find(pair.first); - return (found != view.end()) and (found->first.load() == pair.first and - found->second.load() == pair.second); - })); - } - SECTION("const view") - { - // All keys should be found - REQUIRE(all_of( - d_pairs.begin(), d_pairs.end(), [view] __device__(cuco::pair_type const& pair) { - auto const found = view.find(pair.first); - return (found != view.end()) and - (found->first.load() == pair.first and found->second.load() == pair.second); - })); - } - } } - -template -__global__ void shared_memory_test_kernel( - typename MapType::device_view const* const device_views, - typename MapType::device_view::key_type const* const insterted_keys, - typename MapType::device_view::mapped_type const* const inserted_values, - const size_t number_of_elements, - bool* const keys_exist, - bool* const keys_and_values_correct) -{ - // Each block processes one map - const size_t map_id = blockIdx.x; - const size_t offset = map_id * number_of_elements; - - __shared__ typename MapType::pair_atomic_type sm_buffer[CAPACITY]; - - auto g = cg::this_thread_block(); - typename MapType::device_view sm_device_view = - MapType::device_view::make_copy(g, sm_buffer, device_views[map_id]); - - for (int i = g.thread_rank(); i < number_of_elements; i += g.size()) { - auto found_pair_it = sm_device_view.find(insterted_keys[offset + i]); - - if (found_pair_it != sm_device_view.end()) { - keys_exist[offset + i] = true; - if (found_pair_it->first == insterted_keys[offset + i] and - found_pair_it->second == inserted_values[offset + i]) { - keys_and_values_correct[offset + i] = true; - } else { - keys_and_values_correct[offset + i] = false; - } - } else { - keys_exist[offset + i] = false; - keys_and_values_correct[offset + i] = true; - } - } -} - -TEMPLATE_TEST_CASE_SIG("Shared memory static map", - "", - ((typename T, dist_type Dist), T, Dist), - (int32_t, dist_type::UNIQUE), - (int64_t, dist_type::UNIQUE), - (int32_t, dist_type::UNIFORM), - (int64_t, dist_type::UNIFORM), - (int32_t, dist_type::GAUSSIAN), - (int64_t, dist_type::GAUSSIAN)) -{ - using KeyType = T; - using ValueType = T; - using MapType = cuco::static_reduction_map; - using DeviceViewType = typename MapType::device_view; - using DeviceViewIteratorType = typename DeviceViewType::iterator; - - constexpr std::size_t number_of_maps = 1000; - constexpr std::size_t elements_in_map = 500; - constexpr std::size_t map_capacity = 2 * elements_in_map; - - // one array for all maps, first elements_in_map element belong to map 0, second to map 1 and so - // on - std::vector h_keys(number_of_maps * elements_in_map); - std::vector h_values(number_of_maps * elements_in_map); - std::vector> h_pairs(number_of_maps * elements_in_map); - - // using std::unique_ptr because static_reduction_map does not have copy/move - // constructor/assignment operator yet - std::vector> maps; - - for (std::size_t map_id = 0; map_id < number_of_maps; ++map_id) { - const std::size_t offset = map_id * elements_in_map; - - generate_keys(h_keys.begin() + offset, - h_keys.begin() + offset + elements_in_map); - - for (std::size_t i = 0; i < elements_in_map; ++i) { - KeyType key = h_keys[offset + i]; - ValueType val = key < std::numeric_limits::max() ? key + 1 : 0; - h_values[offset + i] = val; - h_pairs[offset + i].first = key; - h_pairs[offset + i].second = val; - } - - maps.push_back(std::make_unique(map_capacity, -1, -1)); - } - - thrust::device_vector d_keys(h_keys); - thrust::device_vector d_values(h_values); - thrust::device_vector> d_pairs(h_pairs); - - SECTION("Keys are all found after insertion.") - { - std::vector h_device_views; - for (std::size_t map_id = 0; map_id < number_of_maps; ++map_id) { - const std::size_t offset = map_id * elements_in_map; - - MapType* map = maps[map_id].get(); - map->insert(d_pairs.begin() + offset, d_pairs.begin() + offset + elements_in_map); - h_device_views.push_back(map->get_device_view()); - } - thrust::device_vector d_device_views(h_device_views); - - thrust::device_vector d_keys_exist(number_of_maps * elements_in_map); - thrust::device_vector d_keys_and_values_correct(number_of_maps * elements_in_map); - - shared_memory_test_kernel - <<>>(d_device_views.data().get(), - d_keys.data().get(), - d_values.data().get(), - elements_in_map, - d_keys_exist.data().get(), - d_keys_and_values_correct.data().get()); - - REQUIRE(d_keys_exist.size() == d_keys_and_values_correct.size()); - auto zip = thrust::make_zip_iterator( - thrust::make_tuple(d_keys_exist.begin(), d_keys_and_values_correct.begin())); - - REQUIRE(all_of(zip, zip + d_keys_exist.size(), [] __device__(auto const& z) { - return thrust::get<0>(z) and thrust::get<1>(z); - })); - } - - SECTION("No key is found before insertion.") - { - std::vector h_device_views; - for (std::size_t map_id = 0; map_id < number_of_maps; ++map_id) { - h_device_views.push_back(maps[map_id].get()->get_device_view()); - } - thrust::device_vector d_device_views(h_device_views); - - thrust::device_vector d_keys_exist(number_of_maps * elements_in_map); - thrust::device_vector d_keys_and_values_correct(number_of_maps * elements_in_map); - - shared_memory_test_kernel - <<>>(d_device_views.data().get(), - d_keys.data().get(), - d_values.data().get(), - elements_in_map, - d_keys_exist.data().get(), - d_keys_and_values_correct.data().get()); - - REQUIRE(none_of(d_keys_exist.begin(), d_keys_exist.end(), [] __device__(const bool key_found) { - return key_found; - })); - } -} \ No newline at end of file From fd3b98f981d5742ea0dd98c3faa10e7eb7d6bb15 Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Mon, 4 Jan 2021 15:19:33 -0600 Subject: [PATCH 03/32] Fix static_assert for ReductionOp::value_type. --- include/cuco/static_reduction_map.cuh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/include/cuco/static_reduction_map.cuh b/include/cuco/static_reduction_map.cuh index d66c6cf4a..a33de7026 100644 --- a/include/cuco/static_reduction_map.cuh +++ b/include/cuco/static_reduction_map.cuh @@ -123,6 +123,8 @@ template > class static_reduction_map { static_assert(std::is_arithmetic::value, "Unsupported, non-arithmetic key type."); + static_assert(std::is_same::value, + "Type mismatch between ReductionOp::value_type and Value"); public: using value_type = cuco::pair_type; From a3678fbd9417787f0c7818a992d4d1e6284eace6 Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Mon, 4 Jan 2021 21:49:02 -0600 Subject: [PATCH 04/32] CG reduction insert implementation. --- include/cuco/detail/static_reduction_map.inl | 106 ++++++++----------- include/cuco/static_reduction_map.cuh | 2 +- 2 files changed, 46 insertions(+), 62 deletions(-) diff --git a/include/cuco/detail/static_reduction_map.inl b/include/cuco/detail/static_reduction_map.inl index be28e0f28..140c728a4 100644 --- a/include/cuco/detail/static_reduction_map.inl +++ b/include/cuco/detail/static_reduction_map.inl @@ -156,7 +156,7 @@ static_reduction_map::device_mutable_ slot_key.compare_exchange_strong(expected_key, insert_pair.first, memory_order_relaxed); if (key_success or key_equal(insert_pair.first, expected_key)) { - // return do_op{}(slot_value, insert_pair.second); + return op_.apply(slot_value, insert_pair.second); } // if we couldn't insert the key, but it wasn't a duplicate, then there must @@ -171,77 +171,61 @@ template template -__device__ bool +__device__ void static_reduction_map::device_mutable_view::insert( CG g, value_type const& insert_pair, Hash hash, KeyEqual key_equal) noexcept { auto current_slot = initial_slot(g, insert_pair.first, hash); + auto& slot_key = current_slot->first; + auto& slot_value = current_slot->second; while (true) { - key_type const existing_key = current_slot->first; + auto const current_key = slot_key.load(cuda::std::memory_order_relaxed); - // The user provide `key_equal` can never be used to compare against `empty_key_sentinel` as the - // sentinel is not a valid key value. Therefore, first check for the sentinel - auto const slot_is_empty = (existing_key == this->get_empty_key_sentinel()); + // The user provided `key_equal` should never be used to compare against `empty_key_sentinel` as + // the sentinel is not a valid key value. Therefore, first check for the sentinel + // TODO: Use memcmp + auto const slot_is_empty = (current_key == this->get_empty_key_sentinel()); - // the key we are trying to insert is already in the map, so we return with failure to insert - if (g.ballot(not slot_is_empty and key_equal(existing_key, insert_pair.first))) { - return false; - } + auto const key_exists = not slot_is_empty and key_equal(current_key, insert_pair.first); - auto const window_contains_empty = g.ballot(slot_is_empty); + // Key already exists, aggregate with it's value + if (key_exists) { op_.apply(slot_value, insert_pair.second); } - // we found an empty slot, but not the key we are inserting, so this must - // be an empty slot into which we can insert the key - if (window_contains_empty) { + // If key already exists in the CG window, all threads exit + if (g.ballot(key_exists)) { return; } + + auto const window_empty_mask = g.ballot(slot_is_empty); + + if (window_empty_mask) { // the first lane in the group with an empty slot will attempt the insert - insert_result status{insert_result::CONTINUE}; - uint32_t src_lane = __ffs(window_contains_empty) - 1; - - if (g.thread_rank() == src_lane) { - using cuda::std::memory_order_relaxed; - auto expected_key = this->get_empty_key_sentinel(); - auto expected_value = this->get_empty_value_sentinel(); - auto& slot_key = current_slot->first; - auto& slot_value = current_slot->second; - - bool key_success = - slot_key.compare_exchange_strong(expected_key, insert_pair.first, memory_order_relaxed); - bool value_success = slot_value.compare_exchange_strong( - expected_value, insert_pair.second, memory_order_relaxed); - - if (key_success) { - while (not value_success) { - value_success = - slot_value.compare_exchange_strong(expected_value = this->get_empty_value_sentinel(), - insert_pair.second, - memory_order_relaxed); + auto const src_lane = __ffs(window_empty_mask) - 1; + + auto const thread_success = [&]() { + if (g.thread_rank() == src_lane) { + auto expected_key = this->get_empty_key_sentinel(); + + auto const key_success = slot_key.compare_exchange_strong( + expected_key, insert_pair.first, cuda::memory_order_relaxed); + + if (key_success or key_equal(insert_pair.first, expected_key)) { + op_.apply(slot_value, insert_pair.second); + return true; } - status = insert_result::SUCCESS; - } else if (value_success) { - slot_value.store(this->get_empty_value_sentinel(), memory_order_relaxed); } + return false; + }(); - // our key was already present in the slot, so our key is a duplicate - if (key_equal(insert_pair.first, expected_key)) { status = insert_result::DUPLICATE; } - // another key was inserted in the slot we wanted to try - // so we need to try the next empty slot in the window - } + auto const src_success = g.shfl(thread_success, src_lane); - uint32_t res_status = g.shfl(static_cast(status), src_lane); - status = static_cast(res_status); + if (src_success) { return; } - // successful insert - if (status == insert_result::SUCCESS) { return true; } - // duplicate present during insert - if (status == insert_result::DUPLICATE) { return false; } // if we've gotten this far, a different key took our spot // before we could insert. We need to retry the insert on the // same window - } - // if there are no empty slots in the current window, - // we move onto the next window - else { + } else { + // if there are no empty slots in the current window, + // we move onto the next window current_slot = next_slot(g, current_slot); } } @@ -313,8 +297,8 @@ __device__ while (true) { auto const existing_key = current_slot->first.load(cuda::std::memory_order_relaxed); - // The user provide `key_equal` can never be used to compare against `empty_key_sentinel` as the - // sentinel is not a valid key value. Therefore, first check for the sentinel + // The user provide `key_equal` can never be used to compare against `empty_key_sentinel` as + // the sentinel is not a valid key value. Therefore, first check for the sentinel auto const slot_is_empty = (existing_key == this->get_empty_key_sentinel()); // the key we were searching for was found by one of the threads, @@ -353,8 +337,8 @@ __device__ typename static_reduction_mapfirst.load(cuda::std::memory_order_relaxed); - // The user provide `key_equal` can never be used to compare against `empty_key_sentinel` as the - // sentinel is not a valid key value. Therefore, first check for the sentinel + // The user provide `key_equal` can never be used to compare against `empty_key_sentinel` as + // the sentinel is not a valid key value. Therefore, first check for the sentinel auto const slot_is_empty = (existing_key == this->get_empty_key_sentinel()); // the key we were searching for was found by one of the threads, so we return an iterator to @@ -417,8 +401,8 @@ static_reduction_map::device_view::co while (true) { key_type const existing_key = current_slot->first.load(cuda::std::memory_order_relaxed); - // The user provide `key_equal` can never be used to compare against `empty_key_sentinel` as the - // sentinel is not a valid key value. Therefore, first check for the sentinel + // The user provide `key_equal` can never be used to compare against `empty_key_sentinel` as + // the sentinel is not a valid key value. Therefore, first check for the sentinel auto const slot_is_empty = (existing_key == this->get_empty_key_sentinel()); // the key we were searching for was found by one of the threads, so we return an iterator to @@ -428,8 +412,8 @@ static_reduction_map::device_view::co // we found an empty slot, meaning that the key we're searching for isn't present if (g.ballot(slot_is_empty)) { return false; } - // otherwise, all slots in the current window are full with other keys, so we move onto the next - // window + // otherwise, all slots in the current window are full with other keys, so we move onto the + // next window current_slot = next_slot(g, current_slot); } } diff --git a/include/cuco/static_reduction_map.cuh b/include/cuco/static_reduction_map.cuh index a33de7026..47b21d6f2 100644 --- a/include/cuco/static_reduction_map.cuh +++ b/include/cuco/static_reduction_map.cuh @@ -600,7 +600,7 @@ class static_reduction_map { template , typename KeyEqual = thrust::equal_to> - __device__ bool insert(CG g, + __device__ void insert(CG g, value_type const& insert_pair, Hash hash = Hash{}, KeyEqual key_equal = KeyEqual{}) noexcept; From 5a65bf61674077971dd0a65bfe870f75bcfeb1da Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Tue, 5 Jan 2021 09:02:51 -0600 Subject: [PATCH 05/32] Cleanup of CG insert. --- include/cuco/detail/static_reduction_map.inl | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/include/cuco/detail/static_reduction_map.inl b/include/cuco/detail/static_reduction_map.inl index 140c728a4..d833d2650 100644 --- a/include/cuco/detail/static_reduction_map.inl +++ b/include/cuco/detail/static_reduction_map.inl @@ -201,7 +201,7 @@ static_reduction_map::device_mutable_ // the first lane in the group with an empty slot will attempt the insert auto const src_lane = __ffs(window_empty_mask) - 1; - auto const thread_success = [&]() { + auto const update_success = [&]() { if (g.thread_rank() == src_lane) { auto expected_key = this->get_empty_key_sentinel(); @@ -216,16 +216,12 @@ static_reduction_map::device_mutable_ return false; }(); - auto const src_success = g.shfl(thread_success, src_lane); + // If the update succeeded, the thread group exits + if (g.shfl(update_success, src_lane)) { return; } - if (src_success) { return; } - - // if we've gotten this far, a different key took our spot - // before we could insert. We need to retry the insert on the - // same window + // A different key took the current slot. Look for an empty slot in the current window } else { - // if there are no empty slots in the current window, - // we move onto the next window + // No empty slots in the current window, move onto the next window current_slot = next_slot(g, current_slot); } } From 28e09953cd2ceb1fadf6b8ca93aa379624887b9d Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Tue, 5 Jan 2021 14:49:16 -0600 Subject: [PATCH 06/32] Pass reduction op to device view ctors. --- include/cuco/static_reduction_map.cuh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/include/cuco/static_reduction_map.cuh b/include/cuco/static_reduction_map.cuh index 47b21d6f2..1a94b8270 100644 --- a/include/cuco/static_reduction_map.cuh +++ b/include/cuco/static_reduction_map.cuh @@ -900,7 +900,7 @@ class static_reduction_map { */ device_view get_device_view() const noexcept { - return device_view(slots_, capacity_, empty_key_sentinel_, empty_value_sentinel_); + return device_view(slots_, capacity_, empty_key_sentinel_, op_); } /** @@ -911,7 +911,7 @@ class static_reduction_map { */ device_mutable_view get_device_mutable_view() const noexcept { - return device_mutable_view(slots_, capacity_, empty_key_sentinel_, empty_value_sentinel_); + return device_mutable_view(slots_, capacity_, empty_key_sentinel_, op_); } private: From 8dc64ee9f4f06392e3c526a2507c2b8d7f1dd8dd Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Tue, 5 Jan 2021 14:50:19 -0600 Subject: [PATCH 07/32] Add pair ctor for constructing from two elements. --- include/cuco/detail/pair.cuh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/include/cuco/detail/pair.cuh b/include/cuco/detail/pair.cuh index 8bc6ec6b4..8ed10b32b 100644 --- a/include/cuco/detail/pair.cuh +++ b/include/cuco/detail/pair.cuh @@ -65,6 +65,10 @@ struct alignas(detail::pair_alignment()) pair { : first{p.first}, second{p.second} { } + __host__ __device__ constexpr pair(First const& f, Second const& s) noexcept + : first{f}, second{s} + { + } }; template From 573bce28f00b2fd57749537d684f39ad09d08148 Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Tue, 5 Jan 2021 14:50:38 -0600 Subject: [PATCH 08/32] Allow bulk insert kernel to work on iterators over tuples. --- include/cuco/detail/static_reduction_map_kernels.cuh | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/include/cuco/detail/static_reduction_map_kernels.cuh b/include/cuco/detail/static_reduction_map_kernels.cuh index 6ded5e99d..9849efb44 100644 --- a/include/cuco/detail/static_reduction_map_kernels.cuh +++ b/include/cuco/detail/static_reduction_map_kernels.cuh @@ -142,7 +142,10 @@ __global__ void insert( while (it < last) { // force conversion to value_type - typename viewT::value_type const insert_pair{*it}; + typename viewT::value_type const insert_pair{ + static_cast(thrust::get<0>(*it)), + static_cast(thrust::get<1>(*it))}; + if (view.insert(tile, insert_pair, hash, key_equal) && tile.thread_rank() == 0) { thread_num_successes++; } From d9236e588e1eb87051b891de707b1326624c8795 Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Tue, 5 Jan 2021 15:22:16 -0600 Subject: [PATCH 09/32] Add device decorator to reduction op definition. --- include/cuco/static_reduction_map.cuh | 1 + 1 file changed, 1 insertion(+) diff --git a/include/cuco/static_reduction_map.cuh b/include/cuco/static_reduction_map.cuh index 1a94b8270..bac8132ae 100644 --- a/include/cuco/static_reduction_map.cuh +++ b/include/cuco/static_reduction_map.cuh @@ -44,6 +44,7 @@ struct reduce_add { static constexpr T identity = 0; template + __device__ T apply(cuda::atomic& slot, T2 const& value) { return slot.fetch_add(value); From 89ed44e656e4ab16981127d4d7449788b1c390e1 Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Tue, 5 Jan 2021 15:22:48 -0600 Subject: [PATCH 10/32] Add get_op function to allow accessing the op from the derived types. --- include/cuco/static_reduction_map.cuh | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/include/cuco/static_reduction_map.cuh b/include/cuco/static_reduction_map.cuh index bac8132ae..5f2f0341d 100644 --- a/include/cuco/static_reduction_map.cuh +++ b/include/cuco/static_reduction_map.cuh @@ -283,6 +283,12 @@ class static_reduction_map { { } + /** + * @brief Gets the binary op + * + */ + __device__ ReductionOp get_op() const { return op_; } + /** * @brief Gets slots array. * From e28db800db2afca8df262592f68e1a698cfb12e3 Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Tue, 5 Jan 2021 15:23:18 -0600 Subject: [PATCH 11/32] Make insert return a bool after all. We need to return a bool so we can keep track of how many unique keys were inserted in a bulk insert. --- include/cuco/static_reduction_map.cuh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/include/cuco/static_reduction_map.cuh b/include/cuco/static_reduction_map.cuh index 5f2f0341d..c8e2ecc13 100644 --- a/include/cuco/static_reduction_map.cuh +++ b/include/cuco/static_reduction_map.cuh @@ -607,7 +607,7 @@ class static_reduction_map { template , typename KeyEqual = thrust::equal_to> - __device__ void insert(CG g, + __device__ bool insert(CG g, value_type const& insert_pair, Hash hash = Hash{}, KeyEqual key_equal = KeyEqual{}) noexcept; From 0eeac206df5acfaf869bd611c66216230dc63fb8 Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Tue, 5 Jan 2021 15:23:45 -0600 Subject: [PATCH 12/32] Use get_op in implementation. --- include/cuco/detail/static_reduction_map.inl | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/include/cuco/detail/static_reduction_map.inl b/include/cuco/detail/static_reduction_map.inl index d833d2650..cf98e46b3 100644 --- a/include/cuco/detail/static_reduction_map.inl +++ b/include/cuco/detail/static_reduction_map.inl @@ -156,7 +156,7 @@ static_reduction_map::device_mutable_ slot_key.compare_exchange_strong(expected_key, insert_pair.first, memory_order_relaxed); if (key_success or key_equal(insert_pair.first, expected_key)) { - return op_.apply(slot_value, insert_pair.second); + return this->get_op().apply(slot_value, insert_pair.second); } // if we couldn't insert the key, but it wasn't a duplicate, then there must @@ -190,7 +190,7 @@ static_reduction_map::device_mutable_ auto const key_exists = not slot_is_empty and key_equal(current_key, insert_pair.first); // Key already exists, aggregate with it's value - if (key_exists) { op_.apply(slot_value, insert_pair.second); } + if (key_exists) { this->get_op().apply(slot_value, insert_pair.second); } // If key already exists in the CG window, all threads exit if (g.ballot(key_exists)) { return; } @@ -209,7 +209,7 @@ static_reduction_map::device_mutable_ expected_key, insert_pair.first, cuda::memory_order_relaxed); if (key_success or key_equal(insert_pair.first, expected_key)) { - op_.apply(slot_value, insert_pair.second); + this->get_op().apply(slot_value, insert_pair.second); return true; } } From fa31c8117abe243ff9ac55c9edda8ac69719ceef Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Tue, 5 Jan 2021 15:24:02 -0600 Subject: [PATCH 13/32] Make insert return a bool. --- include/cuco/detail/static_reduction_map.inl | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/include/cuco/detail/static_reduction_map.inl b/include/cuco/detail/static_reduction_map.inl index cf98e46b3..2c1e434aa 100644 --- a/include/cuco/detail/static_reduction_map.inl +++ b/include/cuco/detail/static_reduction_map.inl @@ -171,7 +171,7 @@ template template -__device__ void +__device__ bool static_reduction_map::device_mutable_view::insert( CG g, value_type const& insert_pair, Hash hash, KeyEqual key_equal) noexcept { @@ -193,7 +193,7 @@ static_reduction_map::device_mutable_ if (key_exists) { this->get_op().apply(slot_value, insert_pair.second); } // If key already exists in the CG window, all threads exit - if (g.ballot(key_exists)) { return; } + if (g.ballot(key_exists)) { return false; } auto const window_empty_mask = g.ballot(slot_is_empty); @@ -217,7 +217,7 @@ static_reduction_map::device_mutable_ }(); // If the update succeeded, the thread group exits - if (g.shfl(update_success, src_lane)) { return; } + if (g.shfl(update_success, src_lane)) { return true; } // A different key took the current slot. Look for an empty slot in the current window } else { From ab81b2b7d38ab4d414681b35b9b905685738f9d9 Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Tue, 5 Jan 2021 16:07:52 -0600 Subject: [PATCH 14/32] Correct insert to return if the key was the first key inserted. --- include/cuco/detail/static_reduction_map.inl | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/include/cuco/detail/static_reduction_map.inl b/include/cuco/detail/static_reduction_map.inl index 2c1e434aa..9025ceefa 100644 --- a/include/cuco/detail/static_reduction_map.inl +++ b/include/cuco/detail/static_reduction_map.inl @@ -190,7 +190,9 @@ static_reduction_map::device_mutable_ auto const key_exists = not slot_is_empty and key_equal(current_key, insert_pair.first); // Key already exists, aggregate with it's value - if (key_exists) { this->get_op().apply(slot_value, insert_pair.second); } + if (key_exists) { + this->get_op().apply(slot_value, insert_pair.second); + } // If key already exists in the CG window, all threads exit if (g.ballot(key_exists)) { return false; } @@ -210,14 +212,16 @@ static_reduction_map::device_mutable_ if (key_success or key_equal(insert_pair.first, expected_key)) { this->get_op().apply(slot_value, insert_pair.second); - return true; + return key_success; } } return false; }(); // If the update succeeded, the thread group exits - if (g.shfl(update_success, src_lane)) { return true; } + if (g.shfl(update_success, src_lane)) { + return true; + } // A different key took the current slot. Look for an empty slot in the current window } else { From 46f9b73794a28cef4b4dfc7dd5489e0966741b7f Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Tue, 5 Jan 2021 16:08:02 -0600 Subject: [PATCH 15/32] First test verifying size passed. --- tests/static_reduction_map/static_reduction_map_test.cu | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/tests/static_reduction_map/static_reduction_map_test.cu b/tests/static_reduction_map/static_reduction_map_test.cu index 9d709a6c6..958571129 100644 --- a/tests/static_reduction_map/static_reduction_map_test.cu +++ b/tests/static_reduction_map/static_reduction_map_test.cu @@ -93,4 +93,12 @@ TEMPLATE_TEST_CASE_SIG("Unique sequence of keys", constexpr std::size_t num_slots{50'000'000}; cuco::static_reduction_map, Key, Value> map{num_slots, -1}; + SECTION("Inserting all the same key should sum all of their corresponding values") { + thrust::device_vector keys(100, 42); + thrust::device_vector values(keys.size(), 1); + auto zip = thrust::make_zip_iterator(thrust::make_tuple(keys.begin(), values.begin())); + auto zip_end = zip + keys.size(); + map.insert(zip, zip_end); + REQUIRE(map.get_size() == 1); + } } From 8aebabbbdf79a307d61f9e3a085fbd060362e5e2 Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Wed, 6 Jan 2021 13:01:49 -0600 Subject: [PATCH 16/32] Update CG insert logic. The mapped value is updated in the case of a new insert or updating an existing key, but we need to track if the insert was the first time that key was inserted. --- include/cuco/detail/static_reduction_map.inl | 33 ++++++++++---------- 1 file changed, 17 insertions(+), 16 deletions(-) diff --git a/include/cuco/detail/static_reduction_map.inl b/include/cuco/detail/static_reduction_map.inl index 9025ceefa..f5e26ac58 100644 --- a/include/cuco/detail/static_reduction_map.inl +++ b/include/cuco/detail/static_reduction_map.inl @@ -190,9 +190,7 @@ static_reduction_map::device_mutable_ auto const key_exists = not slot_is_empty and key_equal(current_key, insert_pair.first); // Key already exists, aggregate with it's value - if (key_exists) { - this->get_op().apply(slot_value, insert_pair.second); - } + if (key_exists) { this->get_op().apply(slot_value, insert_pair.second); } // If key already exists in the CG window, all threads exit if (g.ballot(key_exists)) { return false; } @@ -203,24 +201,27 @@ static_reduction_map::device_mutable_ // the first lane in the group with an empty slot will attempt the insert auto const src_lane = __ffs(window_empty_mask) - 1; - auto const update_success = [&]() { - if (g.thread_rank() == src_lane) { - auto expected_key = this->get_empty_key_sentinel(); + auto const attempt_update = [&]() { + auto expected_key = this->get_empty_key_sentinel(); - auto const key_success = slot_key.compare_exchange_strong( - expected_key, insert_pair.first, cuda::memory_order_relaxed); + auto const key_success = slot_key.compare_exchange_strong( + expected_key, insert_pair.first, cuda::memory_order_relaxed); - if (key_success or key_equal(insert_pair.first, expected_key)) { - this->get_op().apply(slot_value, insert_pair.second); - return key_success; - } + if (key_success or key_equal(insert_pair.first, expected_key)) { + this->get_op().apply(slot_value, insert_pair.second); + return key_success ? insert_result::SUCCESS : insert_result::DUPLICATE; } - return false; - }(); + return insert_result::CONTINUE; + }; + + auto const update_result = + (g.thread_rank() == src_lane) ? attempt_update() : insert_result::CONTINUE; + + auto const window_result = g.shfl(update_result, src_lane); // If the update succeeded, the thread group exits - if (g.shfl(update_success, src_lane)) { - return true; + if (window_result != insert_result::CONTINUE) { + return (window_result == insert_result::SUCCESS); } // A different key took the current slot. Look for an empty slot in the current window From 9fb930ec216d6c2a074eb7d537a2572c5686a585 Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Wed, 6 Jan 2021 13:01:57 -0600 Subject: [PATCH 17/32] Add more tests. --- .../static_reduction_map_test.cu | 44 +++++++++++++------ 1 file changed, 30 insertions(+), 14 deletions(-) diff --git a/tests/static_reduction_map/static_reduction_map_test.cu b/tests/static_reduction_map/static_reduction_map_test.cu index 958571129..144c02ec8 100644 --- a/tests/static_reduction_map/static_reduction_map_test.cu +++ b/tests/static_reduction_map/static_reduction_map_test.cu @@ -80,25 +80,41 @@ static void generate_keys(OutputIt output_begin, OutputIt output_end) TEMPLATE_TEST_CASE_SIG("Unique sequence of keys", "", ((typename T, dist_type Dist), T, Dist), - (int32_t, dist_type::UNIQUE), - (int64_t, dist_type::UNIQUE), - (int32_t, dist_type::UNIFORM), - (int64_t, dist_type::UNIFORM), - (int32_t, dist_type::GAUSSIAN), - (int64_t, dist_type::GAUSSIAN)) + (int32_t, dist_type::UNIQUE)) { using Key = T; using Value = T; - constexpr std::size_t num_slots{50'000'000}; + constexpr std::size_t num_slots{200}; cuco::static_reduction_map, Key, Value> map{num_slots, -1}; - SECTION("Inserting all the same key should sum all of their corresponding values") { - thrust::device_vector keys(100, 42); - thrust::device_vector values(keys.size(), 1); - auto zip = thrust::make_zip_iterator(thrust::make_tuple(keys.begin(), values.begin())); - auto zip_end = zip + keys.size(); - map.insert(zip, zip_end); - REQUIRE(map.get_size() == 1); + SECTION("Inserting identical keys") + { + thrust::device_vector keys(100, 42); + thrust::device_vector values(keys.size(), 1); + auto zip = thrust::make_zip_iterator(thrust::make_tuple(keys.begin(), values.begin())); + auto zip_end = zip + keys.size(); + map.insert(zip, zip_end); + + SECTION("There should only be one key in the map") { REQUIRE(map.get_size() == 1); } + + SECTION("Map should contain the inserted key") + { + thrust::device_vector contained(keys.size()); + map.contains(keys.begin(), keys.end(), contained.begin()); + REQUIRE(all_of(contained.begin(), contained.end(), [] __device__(bool c) { return c; })); + } + + SECTION("Found value should equal aggregate of inserted values") + { + thrust::device_vector found(keys.size()); + map.find(keys.begin(), keys.end(), found.begin()); + auto const expected_aggregate = keys.size(); // All keys inserted "1", so the + // sum aggregate should be + // equal to the number of keys inserted + REQUIRE(all_of(found.begin(), found.end(), [expected_aggregate] __device__(Value v) { + return v == expected_aggregate; + })); + } } } From 24261b2a9b4e327409a26869804dea8abc78a993 Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Thu, 7 Jan 2021 11:55:12 -0600 Subject: [PATCH 18/32] Add test for inserting all unique keys. --- .../static_reduction_map_test.cu | 113 +++++++++--------- 1 file changed, 55 insertions(+), 58 deletions(-) diff --git a/tests/static_reduction_map/static_reduction_map_test.cu b/tests/static_reduction_map/static_reduction_map_test.cu index 144c02ec8..084b24563 100644 --- a/tests/static_reduction_map/static_reduction_map_test.cu +++ b/tests/static_reduction_map/static_reduction_map_test.cu @@ -23,8 +23,6 @@ #include namespace { -namespace cg = cooperative_groups; - // Thrust logical algorithms (any_of/all_of/none_of) don't work with device // lambdas: See https://github.com/thrust/thrust/issues/1062 template @@ -47,74 +45,73 @@ bool none_of(Iterator begin, Iterator end, Predicate p) } } // namespace -enum class dist_type { UNIQUE, UNIFORM, GAUSSIAN }; - -template -static void generate_keys(OutputIt output_begin, OutputIt output_end) +TEMPLATE_TEST_CASE_SIG("Insert all identical keys", + "", + ((typename Key, typename Value), Key, Value), + (int32_t, int32_t)) { - auto num_keys = std::distance(output_begin, output_end); + constexpr std::size_t num_slots{200}; + cuco::static_reduction_map, Key, Value> map{num_slots, -1}; - std::random_device rd; - std::mt19937 gen{rd()}; + thrust::device_vector keys(100, 42); + thrust::device_vector values(keys.size(), 1); + auto zip = thrust::make_zip_iterator(thrust::make_tuple(keys.begin(), values.begin())); + auto zip_end = zip + keys.size(); + map.insert(zip, zip_end); + + SECTION("There should only be one key in the map") { REQUIRE(map.get_size() == 1); } + + SECTION("Map should contain the inserted key") + { + thrust::device_vector contained(keys.size()); + map.contains(keys.begin(), keys.end(), contained.begin()); + REQUIRE(all_of(contained.begin(), contained.end(), [] __device__(bool c) { return c; })); + } - switch (Dist) { - case dist_type::UNIQUE: - for (auto i = 0; i < num_keys; ++i) { - output_begin[i] = i; - } - break; - case dist_type::UNIFORM: - for (auto i = 0; i < num_keys; ++i) { - output_begin[i] = std::abs(static_cast(gen())); - } - break; - case dist_type::GAUSSIAN: - std::normal_distribution<> dg{1e9, 1e7}; - for (auto i = 0; i < num_keys; ++i) { - output_begin[i] = std::abs(static_cast(dg(gen))); - } - break; + SECTION("Found value should equal aggregate of inserted values") + { + thrust::device_vector found(keys.size()); + map.find(keys.begin(), keys.end(), found.begin()); + auto const expected_aggregate = keys.size(); // All keys inserted "1", so the + // sum aggregate should be + // equal to the number of keys inserted + REQUIRE(all_of(found.begin(), found.end(), [expected_aggregate] __device__(Value v) { + return v == expected_aggregate; + })); } } -TEMPLATE_TEST_CASE_SIG("Unique sequence of keys", +TEMPLATE_TEST_CASE_SIG("Insert all unique keys", "", - ((typename T, dist_type Dist), T, Dist), - (int32_t, dist_type::UNIQUE)) + ((typename Key, typename Value), Key, Value), + (int32_t, int32_t)) { - using Key = T; - using Value = T; - - constexpr std::size_t num_slots{200}; + constexpr std::size_t num_keys = 100; + constexpr std::size_t num_slots{num_keys * 3}; cuco::static_reduction_map, Key, Value> map{num_slots, -1}; - SECTION("Inserting identical keys") - { - thrust::device_vector keys(100, 42); - thrust::device_vector values(keys.size(), 1); - auto zip = thrust::make_zip_iterator(thrust::make_tuple(keys.begin(), values.begin())); - auto zip_end = zip + keys.size(); - map.insert(zip, zip_end); + auto keys_begin = thrust::make_counting_iterator(0); + auto values_begin = thrust::make_counting_iterator(0); + auto zip = thrust::make_zip_iterator(thrust::make_tuple(keys_begin, values_begin)); + auto zip_end = zip + num_keys; + map.insert(zip, zip_end); - SECTION("There should only be one key in the map") { REQUIRE(map.get_size() == 1); } + SECTION("Size of map should equal number of inserted keys") + { + REQUIRE(map.get_size() == num_keys); + } - SECTION("Map should contain the inserted key") - { - thrust::device_vector contained(keys.size()); - map.contains(keys.begin(), keys.end(), contained.begin()); - REQUIRE(all_of(contained.begin(), contained.end(), [] __device__(bool c) { return c; })); - } + SECTION("Map should contain the inserted keys") + { + thrust::device_vector contained(num_keys); + map.contains(keys_begin, keys_begin + num_keys, contained.begin()); + REQUIRE(all_of(contained.begin(), contained.end(), [] __device__(bool c) { return c; })); + } - SECTION("Found value should equal aggregate of inserted values") - { - thrust::device_vector found(keys.size()); - map.find(keys.begin(), keys.end(), found.begin()); - auto const expected_aggregate = keys.size(); // All keys inserted "1", so the - // sum aggregate should be - // equal to the number of keys inserted - REQUIRE(all_of(found.begin(), found.end(), [expected_aggregate] __device__(Value v) { - return v == expected_aggregate; - })); - } + SECTION("Found value should equal inserted value") + { + thrust::device_vector found(num_keys); + map.find(keys_begin, keys_begin + num_keys, found.begin()); + REQUIRE(thrust::equal(thrust::device, values_begin, values_begin + num_keys, found.begin())); } } From e635e3179d4f9d8186e033acbe21d8c5275b213c Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Thu, 7 Jan 2021 11:55:19 -0600 Subject: [PATCH 19/32] Use relaxed fetch_add. --- include/cuco/static_reduction_map.cuh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/include/cuco/static_reduction_map.cuh b/include/cuco/static_reduction_map.cuh index c8e2ecc13..34fa2cba0 100644 --- a/include/cuco/static_reduction_map.cuh +++ b/include/cuco/static_reduction_map.cuh @@ -47,7 +47,7 @@ struct reduce_add { __device__ T apply(cuda::atomic& slot, T2 const& value) { - return slot.fetch_add(value); + return slot.fetch_add(value, cuda::memory_order_relaxed); } }; From d749445c6b967384aaf23e27b419ebcab26883b1 Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Thu, 7 Jan 2021 13:07:19 -0600 Subject: [PATCH 20/32] Update the slot references each iteration. --- include/cuco/detail/static_reduction_map.inl | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/include/cuco/detail/static_reduction_map.inl b/include/cuco/detail/static_reduction_map.inl index f5e26ac58..86b500e4e 100644 --- a/include/cuco/detail/static_reduction_map.inl +++ b/include/cuco/detail/static_reduction_map.inl @@ -176,10 +176,10 @@ static_reduction_map::device_mutable_ CG g, value_type const& insert_pair, Hash hash, KeyEqual key_equal) noexcept { auto current_slot = initial_slot(g, insert_pair.first, hash); - auto& slot_key = current_slot->first; - auto& slot_value = current_slot->second; while (true) { + auto& slot_key = current_slot->first; + auto& slot_value = current_slot->second; auto const current_key = slot_key.load(cuda::std::memory_order_relaxed); // The user provided `key_equal` should never be used to compare against `empty_key_sentinel` as From ca9f7d6b4362cad46521e3f9bce1b6b9926f2345 Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Thu, 7 Jan 2021 13:07:30 -0600 Subject: [PATCH 21/32] Increase size of unique key test. --- tests/static_reduction_map/static_reduction_map_test.cu | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/static_reduction_map/static_reduction_map_test.cu b/tests/static_reduction_map/static_reduction_map_test.cu index 084b24563..9a61b8b4d 100644 --- a/tests/static_reduction_map/static_reduction_map_test.cu +++ b/tests/static_reduction_map/static_reduction_map_test.cu @@ -86,8 +86,8 @@ TEMPLATE_TEST_CASE_SIG("Insert all unique keys", ((typename Key, typename Value), Key, Value), (int32_t, int32_t)) { - constexpr std::size_t num_keys = 100; - constexpr std::size_t num_slots{num_keys * 3}; + constexpr std::size_t num_keys = 10000; + constexpr std::size_t num_slots{num_keys * 2}; cuco::static_reduction_map, Key, Value> map{num_slots, -1}; auto keys_begin = thrust::make_counting_iterator(0); From 9eebd172295416e1bb3a598b95da6418450ff19b Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Thu, 7 Jan 2021 14:33:20 -0600 Subject: [PATCH 22/32] Make map size function of number of keys. --- tests/static_reduction_map/static_reduction_map_test.cu | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/static_reduction_map/static_reduction_map_test.cu b/tests/static_reduction_map/static_reduction_map_test.cu index 9a61b8b4d..bb57f0847 100644 --- a/tests/static_reduction_map/static_reduction_map_test.cu +++ b/tests/static_reduction_map/static_reduction_map_test.cu @@ -50,11 +50,12 @@ TEMPLATE_TEST_CASE_SIG("Insert all identical keys", ((typename Key, typename Value), Key, Value), (int32_t, int32_t)) { - constexpr std::size_t num_slots{200}; - cuco::static_reduction_map, Key, Value> map{num_slots, -1}; - thrust::device_vector keys(100, 42); thrust::device_vector values(keys.size(), 1); + + auto const num_slots{keys.size() * 2}; + cuco::static_reduction_map, Key, Value> map{num_slots, -1}; + auto zip = thrust::make_zip_iterator(thrust::make_tuple(keys.begin(), values.begin())); auto zip_end = zip + keys.size(); map.insert(zip, zip_end); From 212b8f6dbc212d11d48e9aee6d14cfd24bd83927 Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Thu, 7 Jan 2021 16:51:18 -0600 Subject: [PATCH 23/32] Add other agg ops. --- include/cuco/static_reduction_map.cuh | 41 ++++++++++++++++++++++++++- 1 file changed, 40 insertions(+), 1 deletion(-) diff --git a/include/cuco/static_reduction_map.cuh b/include/cuco/static_reduction_map.cuh index 34fa2cba0..470589724 100644 --- a/include/cuco/static_reduction_map.cuh +++ b/include/cuco/static_reduction_map.cuh @@ -45,12 +45,51 @@ struct reduce_add { template __device__ - T apply(cuda::atomic& slot, T2 const& value) + T apply(cuda::atomic& slot, T2 const& value) const { return slot.fetch_add(value, cuda::memory_order_relaxed); } }; +template +struct reduce_sub { + using value_type = T; + static constexpr T identity = 0; + + template + __device__ + T apply(cuda::atomic& slot, T2 const& value) const + { + return slot.fetch_sub(value, cuda::memory_order_relaxed); + } +}; + +template +struct reduce_min { + using value_type = T; + static constexpr T identity = std::numeric_limits::max(); + + template + __device__ + T apply(cuda::atomic& slot, T2 const& value) const + { + return slot.fetch_min(value, cuda::memory_order_relaxed); + } +}; + +template +struct reduce_max { + using value_type = T; + static constexpr T identity = std::numeric_limits::lowest(); + + template + __device__ + T apply(cuda::atomic& slot, T2 const& value) const + { + return slot.fetch_max(value, cuda::memory_order_relaxed); + } +}; + /** * @brief A GPU-accelerated, unordered, associative container of key-value * pairs with unique keys. From cda527a52043c5e34115d71c854851fcbb97a542 Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Thu, 7 Jan 2021 17:01:12 -0600 Subject: [PATCH 24/32] Add custom binary op. --- include/cuco/static_reduction_map.cuh | 27 +++++++++++++++++++-------- 1 file changed, 19 insertions(+), 8 deletions(-) diff --git a/include/cuco/static_reduction_map.cuh b/include/cuco/static_reduction_map.cuh index 470589724..c8dca31b0 100644 --- a/include/cuco/static_reduction_map.cuh +++ b/include/cuco/static_reduction_map.cuh @@ -44,8 +44,7 @@ struct reduce_add { static constexpr T identity = 0; template - __device__ - T apply(cuda::atomic& slot, T2 const& value) const + __device__ T apply(cuda::atomic& slot, T2 const& value) const { return slot.fetch_add(value, cuda::memory_order_relaxed); } @@ -57,8 +56,7 @@ struct reduce_sub { static constexpr T identity = 0; template - __device__ - T apply(cuda::atomic& slot, T2 const& value) const + __device__ T apply(cuda::atomic& slot, T2 const& value) const { return slot.fetch_sub(value, cuda::memory_order_relaxed); } @@ -70,8 +68,7 @@ struct reduce_min { static constexpr T identity = std::numeric_limits::max(); template - __device__ - T apply(cuda::atomic& slot, T2 const& value) const + __device__ T apply(cuda::atomic& slot, T2 const& value) const { return slot.fetch_min(value, cuda::memory_order_relaxed); } @@ -83,13 +80,27 @@ struct reduce_max { static constexpr T identity = std::numeric_limits::lowest(); template - __device__ - T apply(cuda::atomic& slot, T2 const& value) const + __device__ T apply(cuda::atomic& slot, T2 const& value) const { return slot.fetch_max(value, cuda::memory_order_relaxed); } }; +template +struct custom_op { + using value_type = T; + static constexpr T identity = Identity; + + Op op; + + template + __device__ T apply(cuda::atomic& slot, T2 const& value) const + { + auto old = slot.load(cuda::memory_order_relaxed); + while (not slot.compare_exchange_strong(old, op(old, value), cuda::memory_order_relaxed)) {} + } +}; + /** * @brief A GPU-accelerated, unordered, associative container of key-value * pairs with unique keys. From 7c1af0f4f42f3565b16dd17aefcd194568c38624 Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Thu, 7 Jan 2021 17:17:00 -0600 Subject: [PATCH 25/32] Return old value in custom op. --- include/cuco/static_reduction_map.cuh | 1 + 1 file changed, 1 insertion(+) diff --git a/include/cuco/static_reduction_map.cuh b/include/cuco/static_reduction_map.cuh index c8dca31b0..b13c1f5af 100644 --- a/include/cuco/static_reduction_map.cuh +++ b/include/cuco/static_reduction_map.cuh @@ -98,6 +98,7 @@ struct custom_op { { auto old = slot.load(cuda::memory_order_relaxed); while (not slot.compare_exchange_strong(old, op(old, value), cuda::memory_order_relaxed)) {} + return old; } }; From 3f1b59d9362f0ad517199c539c947d5e32692f81 Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Thu, 8 Apr 2021 09:55:38 -0500 Subject: [PATCH 26/32] reduction map benchmarks. --- benchmarks/CMakeLists.txt | 7 +- benchmarks/hash_table/static_map_bench.cu | 132 +++++++++--------- .../hash_table/static_reduction_map_bench.cu | 130 +++++++++++++++++ examples/CMakeLists.txt | 2 +- tests/CMakeLists.txt | 2 +- 5 files changed, 206 insertions(+), 67 deletions(-) create mode 100644 benchmarks/hash_table/static_reduction_map_bench.cu diff --git a/benchmarks/CMakeLists.txt b/benchmarks/CMakeLists.txt index 45b02848d..f9464a6eb 100644 --- a/benchmarks/CMakeLists.txt +++ b/benchmarks/CMakeLists.txt @@ -28,6 +28,8 @@ if("${GPU_ARCHS}" STREQUAL "") evaluate_gpu_archs(GPU_ARCHS) endif() +message("GPU_ARCHS = ${GPU_ARCHS}") + ################################################################################################### # - compiler function ----------------------------------------------------------------------------- @@ -35,7 +37,7 @@ function(ConfigureBench BENCH_NAME BENCH_SRC) add_executable(${BENCH_NAME} "${BENCH_SRC}") set_target_properties(${BENCH_NAME} PROPERTIES POSITION_INDEPENDENT_CODE ON - CUDA_ARCHITECTURES ${GPU_ARCHS} + CUDA_ARCHITECTURES "${GPU_ARCHS}" RUNTIME_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/gbenchmarks") target_include_directories(${BENCH_NAME} PRIVATE "${CMAKE_CURRENT_SOURCE_DIR}") @@ -58,6 +60,9 @@ ConfigureBench(DYNAMIC_MAP_BENCH "${DYNAMIC_MAP_BENCH_SRC}") set(STATIC_MAP_BENCH_SRC "${CMAKE_CURRENT_SOURCE_DIR}/hash_table/static_map_bench.cu") ConfigureBench(STATIC_MAP_BENCH "${STATIC_MAP_BENCH_SRC}") +################################################################################################### +ConfigureBench(STATIC_REDUCTION_MAP_BENCH "${CMAKE_CURRENT_SOURCE_DIR}/hash_table/static_reduction_map_bench.cu") + ################################################################################################### set(RBK_BENCH_SRC "${CMAKE_CURRENT_SOURCE_DIR}/reduce_by_key/reduce_by_key.cu") ConfigureBench(RBK_BENCH "${RBK_BENCH_SRC}") diff --git a/benchmarks/hash_table/static_map_bench.cu b/benchmarks/hash_table/static_map_bench.cu index 165465518..563769df6 100644 --- a/benchmarks/hash_table/static_map_bench.cu +++ b/benchmarks/hash_table/static_map_bench.cu @@ -15,40 +15,38 @@ */ #include -#include "cuco/static_map.cuh" -#include #include -#include +#include +#include #include +#include #include +#include "cuco/static_map.cuh" -enum class dist_type { - UNIQUE, - UNIFORM, - GAUSSIAN -}; +enum class dist_type { UNIQUE, UNIFORM, GAUSSIAN }; -template -static void generate_keys(OutputIt output_begin, OutputIt output_end) { +template +static void generate_keys(OutputIt output_begin, OutputIt output_end) +{ auto num_keys = std::distance(output_begin, output_end); - + std::random_device rd; std::mt19937 gen{rd()}; - switch(Dist) { + switch (Dist) { case dist_type::UNIQUE: - for(auto i = 0; i < num_keys; ++i) { + for (auto i = 0; i < num_keys; ++i) { output_begin[i] = i; } break; case dist_type::UNIFORM: - for(auto i = 0; i < num_keys; ++i) { + for (auto i = 0; i < num_keys; ++i) { output_begin[i] = std::abs(static_cast(gen())); } break; case dist_type::GAUSSIAN: std::normal_distribution<> dg{1e9, 1e7}; - for(auto i = 0; i < num_keys; ++i) { + for (auto i = 0; i < num_keys; ++i) { output_begin[i] = std::abs(static_cast(dg(gen))); } break; @@ -59,88 +57,84 @@ static void generate_keys(OutputIt output_begin, OutputIt output_end) { * @brief Generates input sizes and hash table occupancies * */ -static void generate_size_and_occupancy(benchmark::internal::Benchmark* b) { - for (auto size = 100'000'000; size <= 100'000'000; size *= 10) { - for (auto occupancy = 10; occupancy <= 90; occupancy += 10) { +static void generate_size_and_occupancy(benchmark::internal::Benchmark* b) +{ + for (auto size = 4096; size <= 1 << 28; size *= 2) { + for (auto occupancy = 60; occupancy <= 60; occupancy += 10) { b->Args({size, occupancy}); } } } - - template -static void BM_static_map_insert(::benchmark::State& state) { +static void BM_static_map_insert(::benchmark::State& state) +{ using map_type = cuco::static_map; - + std::size_t num_keys = state.range(0); - float occupancy = state.range(1) / float{100}; - std::size_t size = num_keys / occupancy; + float occupancy = state.range(1) / float{100}; + std::size_t size = num_keys / occupancy; + + std::vector h_keys(num_keys); + std::vector> h_pairs(num_keys); - std::vector h_keys( num_keys ); - std::vector> h_pairs( num_keys ); - generate_keys(h_keys.begin(), h_keys.end()); - - for(auto i = 0; i < num_keys; ++i) { - Key key = h_keys[i]; - Value val = h_keys[i]; - h_pairs[i].first = key; + + for (auto i = 0; i < num_keys; ++i) { + Key key = h_keys[i]; + Value val = h_keys[i]; + h_pairs[i].first = key; h_pairs[i].second = val; } - thrust::device_vector> d_pairs( h_pairs ); + thrust::device_vector> d_pairs(h_pairs); - for(auto _ : state) { - state.ResumeTiming(); - state.PauseTiming(); + for (auto _ : state) { map_type map{size, -1, -1}; - state.ResumeTiming(); - - map.insert(d_pairs.begin(), d_pairs.end()); - state.PauseTiming(); + { + cuda_event_timer raii{state}; + map.insert(d_pairs.begin(), d_pairs.end()); + } } - state.SetBytesProcessed((sizeof(Key) + sizeof(Value)) * - int64_t(state.iterations()) * + state.SetBytesProcessed((sizeof(Key) + sizeof(Value)) * int64_t(state.iterations()) * int64_t(state.range(0))); } - - template -static void BM_static_map_search_all(::benchmark::State& state) { +static void BM_static_map_search_all(::benchmark::State& state) +{ using map_type = cuco::static_map; - + std::size_t num_keys = state.range(0); - float occupancy = state.range(1) / float{100}; - std::size_t size = num_keys / occupancy; + float occupancy = state.range(1) / float{100}; + std::size_t size = num_keys / occupancy; map_type map{size, -1, -1}; auto view = map.get_device_mutable_view(); - std::vector h_keys( num_keys ); - std::vector h_values( num_keys ); - std::vector> h_pairs ( num_keys ); - std::vector h_results (num_keys); + std::vector h_keys(num_keys); + std::vector h_values(num_keys); + std::vector> h_pairs(num_keys); + std::vector h_results(num_keys); generate_keys(h_keys.begin(), h_keys.end()); - - for(auto i = 0; i < num_keys; ++i) { - Key key = h_keys[i]; - Value val = h_keys[i]; - h_pairs[i].first = key; + + for (auto i = 0; i < num_keys; ++i) { + Key key = h_keys[i]; + Value val = h_keys[i]; + h_pairs[i].first = key; h_pairs[i].second = val; } - thrust::device_vector d_keys( h_keys ); - thrust::device_vector d_results( num_keys); - thrust::device_vector> d_pairs( h_pairs ); + thrust::device_vector d_keys(h_keys); + thrust::device_vector d_results(num_keys); + thrust::device_vector> d_pairs(h_pairs); map.insert(d_pairs.begin(), d_pairs.end()); - - for(auto _ : state) { + + for (auto _ : state) { map.find(d_keys.begin(), d_keys.end(), d_results.begin()); } @@ -148,52 +142,62 @@ static void BM_static_map_search_all(::benchmark::State& state) { int64_t(state.range(0))); } - - BENCHMARK_TEMPLATE(BM_static_map_insert, int32_t, int32_t, dist_type::UNIQUE) ->Unit(benchmark::kMillisecond) + ->UseManualTime() ->Apply(generate_size_and_occupancy); BENCHMARK_TEMPLATE(BM_static_map_search_all, int32_t, int32_t, dist_type::UNIQUE) ->Unit(benchmark::kMillisecond) + ->UseManualTime() ->Apply(generate_size_and_occupancy); BENCHMARK_TEMPLATE(BM_static_map_insert, int32_t, int32_t, dist_type::UNIFORM) ->Unit(benchmark::kMillisecond) + ->UseManualTime() ->Apply(generate_size_and_occupancy); BENCHMARK_TEMPLATE(BM_static_map_search_all, int32_t, int32_t, dist_type::UNIFORM) ->Unit(benchmark::kMillisecond) + ->UseManualTime() ->Apply(generate_size_and_occupancy); BENCHMARK_TEMPLATE(BM_static_map_insert, int32_t, int32_t, dist_type::GAUSSIAN) ->Unit(benchmark::kMillisecond) + ->UseManualTime() ->Apply(generate_size_and_occupancy); BENCHMARK_TEMPLATE(BM_static_map_search_all, int32_t, int32_t, dist_type::GAUSSIAN) ->Unit(benchmark::kMillisecond) + ->UseManualTime() ->Apply(generate_size_and_occupancy); BENCHMARK_TEMPLATE(BM_static_map_insert, int64_t, int64_t, dist_type::UNIQUE) ->Unit(benchmark::kMillisecond) + ->UseManualTime() ->Apply(generate_size_and_occupancy); BENCHMARK_TEMPLATE(BM_static_map_search_all, int64_t, int64_t, dist_type::UNIQUE) ->Unit(benchmark::kMillisecond) + ->UseManualTime() ->Apply(generate_size_and_occupancy); BENCHMARK_TEMPLATE(BM_static_map_insert, int64_t, int64_t, dist_type::UNIFORM) ->Unit(benchmark::kMillisecond) + ->UseManualTime() ->Apply(generate_size_and_occupancy); BENCHMARK_TEMPLATE(BM_static_map_search_all, int64_t, int64_t, dist_type::UNIFORM) ->Unit(benchmark::kMillisecond) + ->UseManualTime() ->Apply(generate_size_and_occupancy); BENCHMARK_TEMPLATE(BM_static_map_insert, int64_t, int64_t, dist_type::GAUSSIAN) ->Unit(benchmark::kMillisecond) + ->UseManualTime() ->Apply(generate_size_and_occupancy); BENCHMARK_TEMPLATE(BM_static_map_search_all, int64_t, int64_t, dist_type::GAUSSIAN) ->Unit(benchmark::kMillisecond) + ->UseManualTime() ->Apply(generate_size_and_occupancy); \ No newline at end of file diff --git a/benchmarks/hash_table/static_reduction_map_bench.cu b/benchmarks/hash_table/static_reduction_map_bench.cu new file mode 100644 index 000000000..92a2ab788 --- /dev/null +++ b/benchmarks/hash_table/static_reduction_map_bench.cu @@ -0,0 +1,130 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed 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 +#include "cuco/static_reduction_map.cuh" + +enum class dist_type { UNIQUE, UNIFORM, GAUSSIAN }; + +template +static void generate_keys(OutputIt output_begin, OutputIt output_end) +{ + auto num_keys = std::distance(output_begin, output_end); + + std::random_device rd; + std::mt19937 gen{rd()}; + + switch (Dist) { + case dist_type::UNIQUE: + for (auto i = 0; i < num_keys; ++i) { + output_begin[i] = i; + } + break; + case dist_type::UNIFORM: + for (auto i = 0; i < num_keys; ++i) { + output_begin[i] = std::abs(static_cast(gen())); + } + break; + case dist_type::GAUSSIAN: + std::normal_distribution<> dg{1e9, 1e7}; + for (auto i = 0; i < num_keys; ++i) { + output_begin[i] = std::abs(static_cast(dg(gen))); + } + break; + } +} + +/** + * @brief Generates input sizes and hash table occupancies + * + */ +static void generate_size_and_occupancy(benchmark::internal::Benchmark* b) +{ + for (auto size = 4096; size <= 1 << 28; size *= 2) { + for (auto occupancy = 60; occupancy <= 60; occupancy += 10) { + b->Args({size, occupancy}); + } + } +} + +template typename ReductionOp> +static void BM_static_map_insert(::benchmark::State& state) +{ + using map_type = cuco::static_reduction_map, Key, Value>; + + std::size_t num_keys = state.range(0); + float occupancy = state.range(1) / float{100}; + std::size_t size = num_keys / occupancy; + + std::vector h_keys(num_keys); + std::vector> h_pairs(num_keys); + + generate_keys(h_keys.begin(), h_keys.end()); + + thrust::device_vector d_keys(h_keys); + thrust::device_vector d_values(h_keys); + + auto pairs_begin = + thrust::make_zip_iterator(thrust::make_tuple(d_keys.begin(), d_values.begin())); + auto pairs_end = pairs_begin + num_keys; + + for (auto _ : state) { + map_type map{size, -1}; + { + cuda_event_timer raii{state}; + map.insert(pairs_begin, pairs_end); + } + } + + state.SetBytesProcessed((sizeof(Key) + sizeof(Value)) * int64_t(state.iterations()) * + int64_t(state.range(0))); +} + +BENCHMARK_TEMPLATE(BM_static_map_insert, int32_t, int32_t, dist_type::UNIQUE, cuco::reduce_add) + ->Unit(benchmark::kMillisecond) + ->UseManualTime() + ->Apply(generate_size_and_occupancy); + +BENCHMARK_TEMPLATE(BM_static_map_insert, int32_t, int32_t, dist_type::UNIFORM, cuco::reduce_add) + ->Unit(benchmark::kMillisecond) + ->UseManualTime() + ->Apply(generate_size_and_occupancy); + +BENCHMARK_TEMPLATE(BM_static_map_insert, int32_t, int32_t, dist_type::GAUSSIAN, cuco::reduce_add) + ->Unit(benchmark::kMillisecond) + ->UseManualTime() + ->Apply(generate_size_and_occupancy); + +BENCHMARK_TEMPLATE(BM_static_map_insert, int64_t, int64_t, dist_type::UNIQUE, cuco::reduce_add) + ->Unit(benchmark::kMillisecond) + ->UseManualTime() + ->Apply(generate_size_and_occupancy); + +BENCHMARK_TEMPLATE(BM_static_map_insert, int64_t, int64_t, dist_type::UNIFORM, cuco::reduce_add) + ->Unit(benchmark::kMillisecond) + ->UseManualTime() + ->Apply(generate_size_and_occupancy); + +BENCHMARK_TEMPLATE(BM_static_map_insert, int64_t, int64_t, dist_type::GAUSSIAN, cuco::reduce_add) + ->Unit(benchmark::kMillisecond) + ->UseManualTime() + ->Apply(generate_size_and_occupancy); \ No newline at end of file diff --git a/examples/CMakeLists.txt b/examples/CMakeLists.txt index a70b53da8..e840e1905 100644 --- a/examples/CMakeLists.txt +++ b/examples/CMakeLists.txt @@ -15,7 +15,7 @@ endif() function(ConfigureExample EXAMPLE_NAME EXAMPLE_SRC) add_executable(${EXAMPLE_NAME} "${EXAMPLE_SRC}") set_target_properties(${EXAMPLE_NAME} PROPERTIES - CUDA_ARCHITECTURES ${GPU_ARCHS} + CUDA_ARCHITECTURES "${GPU_ARCHS}" RUNTIME_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/examples") target_include_directories(${EXAMPLE_NAME} PRIVATE "${CMAKE_CURRENT_SOURCE_DIR}") diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index 32d77b2a8..471a15a7a 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -39,7 +39,7 @@ function(ConfigureTest TEST_NAME TEST_SRC) $) # Link in the CatchMain object file target_link_libraries(${TEST_NAME} Catch2::Catch2 cuco) set_target_properties(${TEST_NAME} PROPERTIES - CUDA_ARCHITECTURES ${GPU_ARCHS} + CUDA_ARCHITECTURES "${GPU_ARCHS}" RUNTIME_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/tests") target_compile_options(${TEST_NAME} PRIVATE --expt-extended-lambda --expt-relaxed-constexpr) catch_discover_tests(${TEST_NAME}) From 2a38d70c4fc01d98a324d627b41bff739b8f5a66 Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Thu, 13 May 2021 11:57:30 -0500 Subject: [PATCH 27/32] Remove redundant ctor. --- include/cuco/detail/pair.cuh | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/include/cuco/detail/pair.cuh b/include/cuco/detail/pair.cuh index de75ad680..dfdf7632e 100644 --- a/include/cuco/detail/pair.cuh +++ b/include/cuco/detail/pair.cuh @@ -68,8 +68,8 @@ struct is_thrust_pair_like_impl : std::false_type { template struct is_thrust_pair_like_impl(std::declval())), - decltype(thrust::get<1>(std::declval()))>> + std::void_t(std::declval())), + decltype(thrust::get<1>(std::declval()))>> : std::conditional_t::value == 2, std::true_type, std::false_type> { }; @@ -116,10 +116,6 @@ struct alignas(detail::pair_alignment()) pair { thrust::get<1>(thrust::raw_reference_cast(t))} { } - __host__ __device__ constexpr pair(First const& f, Second const& s) noexcept - : first{f}, second{s} - { - } }; template From f2d1a2607c9e36b56bbac5baea47dc5066d78327 Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Thu, 13 May 2021 11:57:59 -0500 Subject: [PATCH 28/32] Add initial static_reduction_map example. --- examples/CMakeLists.txt | 2 + examples/static_reduction_map.cu | 82 ++++++++++++++++++++++++++++++++ 2 files changed, 84 insertions(+) create mode 100644 examples/static_reduction_map.cu diff --git a/examples/CMakeLists.txt b/examples/CMakeLists.txt index e840e1905..be1a760e6 100644 --- a/examples/CMakeLists.txt +++ b/examples/CMakeLists.txt @@ -28,3 +28,5 @@ endfunction(ConfigureExample) ################################################################################################### ConfigureExample(STATIC_MAP_EXAMPLE "${CMAKE_CURRENT_SOURCE_DIR}/static_map/static_map_example.cu") + +ConfigureExample(STATIC_REDUCTION_MAP_EXAMPLE "${CMAKE_CURRENT_SOURCE_DIR}/static_reduction_map.cu") diff --git a/examples/static_reduction_map.cu b/examples/static_reduction_map.cu new file mode 100644 index 000000000..c3921ad10 --- /dev/null +++ b/examples/static_reduction_map.cu @@ -0,0 +1,82 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed 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 +#include + +#include + +/** + * @file host_bulk_example.cu + * @brief Demonstrates usage of the static_map "bulk" host APIs. + * + * The bulk APIs are only invocable from the host and are used for doing operations like insert or + * find on a set of keys. + * + */ + +int main(void) +{ + using Key = int; + using Value = int; + + // Empty slots are represented by reserved "sentinel" values. These values should be selected such + // that they never occur in your input data. + Key const empty_key_sentinel = -1; + + // Number of key/value pairs to be inserted + std::size_t num_keys = 50'000; + + // Compute capacity based on a 50% load factor + auto const load_factor = 0.5; + std::size_t const capacity = std::ceil(num_keys / load_factor); + + // Constructs a map each key with "capacity" slots using -1 as the + // empty key sentinel. The initial payload value for empty slots is determined by the identity of + // the reduction operation. By using the `reduce_add` operation, all values associated with a + // given key will be summed. + cuco::static_reduction_map, Key, Value> map{capacity, empty_key_sentinel}; + + // Create a sequence of random keys in `[0, num_keys/2]` + thrust::device_vector insert_keys(num_keys); + thrust::transform(thrust::device, + thrust::make_counting_iterator(0), + thrust::make_counting_iterator(insert_keys.size()), + insert_keys.begin(), + [=] __device__(auto i) { + thrust::default_random_engine rng(i); + thrust::uniform_int_distribution dist{std::size_t{0}, num_keys/2}; + return dist(rng); + }); + + // Insert each key with a payload of `1` to count the number of times each key was inserted by + // using the `reduce_add` op + auto zipped = thrust::make_zip_iterator( + thrust::make_tuple(insert_keys.begin(), thrust::make_constant_iterator(1))); + + // Inserts all pairs into the map, accumulating the payloads with the `reduce_add` operation + map.insert(zipped, zipped + insert_keys.size()); + + std::cout << "Num unique keys: " << map.get_size() << std::endl; + +} \ No newline at end of file From 3c797013e3504a3ce9396430f553be91d7a3bec0 Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Thu, 13 May 2021 11:58:15 -0500 Subject: [PATCH 29/32] Remove cuda_memcmp header. --- include/cuco/static_reduction_map.cuh | 1 - 1 file changed, 1 deletion(-) diff --git a/include/cuco/static_reduction_map.cuh b/include/cuco/static_reduction_map.cuh index b13c1f5af..ebddaebc7 100644 --- a/include/cuco/static_reduction_map.cuh +++ b/include/cuco/static_reduction_map.cuh @@ -24,7 +24,6 @@ #include #include -#include #ifndef CUDART_VERSION #error CUDART_VERSION Undefined! #elif (CUDART_VERSION >= 11000) // including with CUDA 10.2 leads to compilation errors From 8261d939e6f962b10093b37fb58410c8a9d7223e Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Wed, 19 May 2021 12:06:54 -0500 Subject: [PATCH 30/32] Add unsafe accessors to raw slots via reinterpret_cast. --- include/cuco/static_reduction_map.cuh | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/include/cuco/static_reduction_map.cuh b/include/cuco/static_reduction_map.cuh index ebddaebc7..3f98737c8 100644 --- a/include/cuco/static_reduction_map.cuh +++ b/include/cuco/static_reduction_map.cuh @@ -972,6 +972,19 @@ class static_reduction_map { } private: + /// Unsafe access to the slots stripping away their atomic-ness to allow non-atomic access. This + /// is a temporary solution until we have atomic_ref + value_type* raw_slots_begin() noexcept { return reinterpret_cast(slots_); } + + value_type const* raw_slots_begin() const noexcept + { + return reinterpret_cast(slots_); + } + + value_type* raw_slots_end() noexcept { return raw_slots_begin() + get_capacity(); } + + value_type const* raw_slots_end() const noexcept { return raw_slots_begin() + get_capacity(); } + pair_atomic_type* slots_{nullptr}; ///< Pointer to flat slots storage std::size_t capacity_{}; ///< Total number of slots std::size_t size_{}; ///< Number of keys in map From c6daa09029a9305203c7438376039a817185cad9 Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Wed, 19 May 2021 12:07:20 -0500 Subject: [PATCH 31/32] Add retreive_all implementation. --- include/cuco/detail/static_reduction_map.inl | 39 ++++++++++++++++++++ include/cuco/static_reduction_map.cuh | 23 ++++++++++++ 2 files changed, 62 insertions(+) diff --git a/include/cuco/detail/static_reduction_map.inl b/include/cuco/detail/static_reduction_map.inl index 86b500e4e..bd9907ebc 100644 --- a/include/cuco/detail/static_reduction_map.inl +++ b/include/cuco/detail/static_reduction_map.inl @@ -112,6 +112,45 @@ void static_reduction_map::find( CUCO_CUDA_TRY(cudaDeviceSynchronize()); } +namespace detail { +template +struct slot_to_tuple { + template + __device__ thrust::tuple operator()(S const& s) + { + return thrust::tuple(s.first, s.second); + } +}; + +template +struct slot_is_filled { + Key empty_key_sentinel; + template + __device__ bool operator()(S const& s) + { + return thrust::get<0>(s) != empty_key_sentinel; + } +}; +} // namespace detail + +template +template +void static_reduction_map::retrieve_all( + KeyOut keys_out, ValueOut values_out) +{ + // Convert pair_type to thrust::tuple to allow assigning to a zip iterator + auto begin = thrust::make_transform_iterator(raw_slots_begin(), detail::slot_to_tuple{}); + auto end = begin + get_capacity(); + auto filled = detail::slot_is_filled{get_empty_key_sentinel()}; + auto zipped_out = thrust::make_zip_iterator(thrust::make_tuple(keys_out, values_out)); + + thrust::copy_if(thrust::device, begin, end, zipped_out, filled); +} + template #include #include +#include +#include +#include +#include #include #ifndef CUDART_VERSION @@ -276,6 +280,25 @@ class static_reduction_map { Hash hash = Hash{}, KeyEqual key_equal = KeyEqual{}) noexcept; + /** + * @brief Retrieves all of the keys and their associated values. + * + * The order in which keys are returned is implementation defined and not guaranteed to be + * consistent between subsequent calls to `retrieve_all`. + * + * Behavior is undefined if the range beginning at `keys_out` or `values_out` is not large enough + * to contain the number of keys in the map. + * + * @tparam KeyOut Device accessible random access output iterator whose `value_type` is + * convertible from `key_type`. + * @tparam ValueOut Device accesible random access output iterator whose `value_type` is + * convertible from `mapped_type`. + * @param keys_out Beginning output iterator for keys + * @param values_out Beginning output iterator for values + */ + template + void retrieve_all(KeyOut keys_out, ValueOut values_out); + /** * @brief Indicates whether the keys in the range `[first, last)` are contained in the map. * From 62a99ab6fb9f28614624879168eba468dcc648de Mon Sep 17 00:00:00 2001 From: Jake Hemstad Date: Wed, 19 May 2021 12:07:33 -0500 Subject: [PATCH 32/32] Add retrieve_all to example. --- examples/static_reduction_map.cu | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/examples/static_reduction_map.cu b/examples/static_reduction_map.cu index c3921ad10..f152ceb78 100644 --- a/examples/static_reduction_map.cu +++ b/examples/static_reduction_map.cu @@ -45,7 +45,7 @@ int main(void) Key const empty_key_sentinel = -1; // Number of key/value pairs to be inserted - std::size_t num_keys = 50'000; + std::size_t num_keys = 257; // Compute capacity based on a 50% load factor auto const load_factor = 0.5; @@ -64,8 +64,9 @@ int main(void) thrust::make_counting_iterator(insert_keys.size()), insert_keys.begin(), [=] __device__(auto i) { - thrust::default_random_engine rng(i); - thrust::uniform_int_distribution dist{std::size_t{0}, num_keys/2}; + thrust::default_random_engine rng; + thrust::uniform_int_distribution dist{0, 10}; + rng.discard(i); return dist(rng); }); @@ -79,4 +80,12 @@ int main(void) std::cout << "Num unique keys: " << map.get_size() << std::endl; + thrust::device_vector unique_keys(map.get_size()); + thrust::device_vector count_per_key(map.get_size()); + + map.retrieve_all(unique_keys.begin(), count_per_key.begin()); + + for (int i = 0; i < unique_keys.size(); ++i) { + std::cout << "Key: " << unique_keys[i] << " Count: " << count_per_key[i] << std::endl; + } } \ No newline at end of file