diff --git a/cub/agent/agent_scan_by_key.cuh b/cub/agent/agent_scan_by_key.cuh new file mode 100644 index 0000000000..2cb26fd68c --- /dev/null +++ b/cub/agent/agent_scan_by_key.cuh @@ -0,0 +1,458 @@ +/****************************************************************************** + * Copyright (c) 2021, NVIDIA CORPORATION. All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * * Redistributions of source code must retain the above copyright + * notice, this list of conditions and the following disclaimer. + * * Redistributions in binary form must reproduce the above copyright + * notice, this list of conditions and the following disclaimer in the + * documentation and/or other materials provided with the distribution. + * * Neither the name of the NVIDIA CORPORATION nor the + * names of its contributors may be used to endorse or promote products + * derived from this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND + * ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED + * WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL NVIDIA CORPORATION BE LIABLE FOR ANY + * DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES + * (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; + * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND + * ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS + * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + ******************************************************************************/ + +/** + * \file + * AgentScanByKey implements a stateful abstraction of CUDA thread blocks for participating in device-wide prefix scan by key. + */ + +#pragma once + +#include + +#include "single_pass_scan_operators.cuh" +#include "../block/block_load.cuh" +#include "../block/block_store.cuh" +#include "../block/block_scan.cuh" +#include "../block/block_discontinuity.cuh" +#include "../config.cuh" +#include "../iterator/cache_modified_input_iterator.cuh" + + +CUB_NAMESPACE_BEGIN + + +/****************************************************************************** + * Tuning policy types + ******************************************************************************/ + +/** + * Parameterizable tuning policy type for AgentScanByKey + */ + +template +struct AgentScanByKeyPolicy +{ + enum + { + BLOCK_THREADS = _BLOCK_THREADS, + ITEMS_PER_THREAD = _ITEMS_PER_THREAD, + }; + + static const BlockLoadAlgorithm LOAD_ALGORITHM = _LOAD_ALGORITHM; + static const CacheLoadModifier LOAD_MODIFIER = _LOAD_MODIFIER; + static const BlockScanAlgorithm SCAN_ALGORITHM = _SCAN_ALGORITHM; + static const BlockStoreAlgorithm STORE_ALGORITHM = _STORE_ALGORITHM; +}; + + +/****************************************************************************** + * Thread block abstractions + ******************************************************************************/ + +/** + * \brief AgentScanByKey implements a stateful abstraction of CUDA thread blocks for participating in device-wide prefix scan by key. + */ +template < + typename AgentScanByKeyPolicyT, ///< Parameterized AgentScanPolicyT tuning policy type + typename KeysInputIteratorT, ///< Random-access input iterator type + typename ValuesInputIteratorT, ///< Random-access input iterator type + typename ValuesOutputIteratorT, ///< Random-access output iterator type + typename EqualityOp, ///< Equality functor type + typename ScanOpT, ///< Scan functor type + typename InitValueT, ///< The init_value element for ScanOpT type (cub::NullType for inclusive scan) + typename OffsetT> ///< Signed integer type for global offsets +struct AgentScanByKey +{ + //--------------------------------------------------------------------- + // Types and constants + //--------------------------------------------------------------------- + + using KeyT = typename std::iterator_traits::value_type; + using InputT = typename std::iterator_traits::value_type; + + // The output value type -- used as the intermediate accumulator + // Per https://wg21.link/P0571, use InitValueT if provided, otherwise the + // input iterator's value type. + using OutputT = + typename If::VALUE, InputT, InitValueT>::Type; + + using SizeValuePairT = KeyValuePair; + using KeyValuePairT = KeyValuePair; + using ReduceBySegmentOpT = ReduceBySegmentOp; + + using ScanTileStateT = ReduceByKeyScanTileState; + + // Constants + enum + { + IS_INCLUSIVE = Equals::VALUE, // Inclusive scan if no init_value type is provided + BLOCK_THREADS = AgentScanByKeyPolicyT::BLOCK_THREADS, + ITEMS_PER_THREAD = AgentScanByKeyPolicyT::ITEMS_PER_THREAD, + ITEMS_PER_TILE = BLOCK_THREADS * ITEMS_PER_THREAD, + }; + + using WrappedKeysInputIteratorT = typename If::VALUE, + CacheModifiedInputIterator, // Wrap the native input pointer with CacheModifiedInputIterator + KeysInputIteratorT>::Type; + using WrappedValuesInputIteratorT = typename If::VALUE, + CacheModifiedInputIterator, // Wrap the native input pointer with CacheModifiedInputIterator + ValuesInputIteratorT>::Type; + + using BlockLoadKeysT = BlockLoad; + using BlockLoadValuesT = BlockLoad; + using BlockStoreValuesT = BlockStore; + using BlockDiscontinuityKeysT = BlockDiscontinuity; + + using TilePrefixCallbackT = TilePrefixCallbackOp; + using BlockScanT = BlockScan; + + union TempStorage + { + struct ScanStorage + { + typename BlockScanT::TempStorage scan; + typename TilePrefixCallbackT::TempStorage prefix; + typename BlockDiscontinuityKeysT::TempStorage discontinuity; + } scan_storage; + + typename BlockLoadKeysT::TempStorage load_keys; + typename BlockLoadValuesT::TempStorage load_values; + typename BlockStoreValuesT::TempStorage store_values; + }; + + //--------------------------------------------------------------------- + // Per-thread fields + //--------------------------------------------------------------------- + + TempStorage & storage; + WrappedKeysInputIteratorT d_keys_in; + WrappedValuesInputIteratorT d_values_in; + ValuesOutputIteratorT d_values_out; + InequalityWrapper inequality_op; + ScanOpT scan_op; + ReduceBySegmentOpT pair_scan_op; + InitValueT init_value; + + //--------------------------------------------------------------------- + // Block scan utility methods (first tile) + //--------------------------------------------------------------------- + + // Exclusive scan specialization + __device__ __forceinline__ + void ScanTile( + SizeValuePairT (&scan_items)[ITEMS_PER_THREAD], + SizeValuePairT &tile_aggregate, + Int2Type /* is_inclusive */) + { + BlockScanT(storage.scan_storage.scan) + .ExclusiveScan(scan_items, scan_items, pair_scan_op, tile_aggregate); + } + + // Inclusive scan specialization + __device__ __forceinline__ + void ScanTile( + SizeValuePairT (&scan_items)[ITEMS_PER_THREAD], + SizeValuePairT &tile_aggregate, + Int2Type /* is_inclusive */) + { + BlockScanT(storage.scan_storage.scan) + .InclusiveScan(scan_items, scan_items, pair_scan_op, tile_aggregate); + } + + //--------------------------------------------------------------------- + // Block scan utility methods (subsequent tiles) + //--------------------------------------------------------------------- + + // Exclusive scan specialization (with prefix from predecessors) + __device__ __forceinline__ + void ScanTile( + SizeValuePairT (&scan_items)[ITEMS_PER_THREAD], + SizeValuePairT & tile_aggregate, + TilePrefixCallbackT &prefix_op, + Int2Type /* is_incclusive */) + { + BlockScanT(storage.scan_storage.scan) + .ExclusiveScan(scan_items, scan_items, pair_scan_op, prefix_op); + tile_aggregate = prefix_op.GetBlockAggregate(); + } + + // Inclusive scan specialization (with prefix from predecessors) + __device__ __forceinline__ + void ScanTile( + SizeValuePairT (&scan_items)[ITEMS_PER_THREAD], + SizeValuePairT & tile_aggregate, + TilePrefixCallbackT &prefix_op, + Int2Type /* is_inclusive */) + { + BlockScanT(storage.scan_storage.scan) + .InclusiveScan(scan_items, scan_items, pair_scan_op, prefix_op); + tile_aggregate = prefix_op.GetBlockAggregate(); + } + + //--------------------------------------------------------------------- + // Zip utility methods + //--------------------------------------------------------------------- + + template + __device__ __forceinline__ + void ZipValuesAndFlags( + OffsetT num_remaining, + OutputT (&values)[ITEMS_PER_THREAD], + OffsetT (&segment_flags)[ITEMS_PER_THREAD], + SizeValuePairT (&scan_items)[ITEMS_PER_THREAD]) + { + // Zip values and segment_flags + #pragma unroll + for (int ITEM = 0; ITEM < ITEMS_PER_THREAD; ++ITEM) + { + // Set segment_flags for first out-of-bounds item, zero for others + if (IS_LAST_TILE && + OffsetT(threadIdx.x * ITEMS_PER_THREAD) + ITEM == num_remaining) + segment_flags[ITEM] = 1; + + scan_items[ITEM].value = values[ITEM]; + scan_items[ITEM].key = segment_flags[ITEM]; + } + } + + __device__ __forceinline__ + void UnzipValues( + OutputT (&values)[ITEMS_PER_THREAD], + SizeValuePairT (&scan_items)[ITEMS_PER_THREAD]) + { + // Zip values and segment_flags + #pragma unroll + for (int ITEM = 0; ITEM < ITEMS_PER_THREAD; ++ITEM) + { + values[ITEM] = scan_items[ITEM].value; + } + } + + template::VALUE, typename std::enable_if::type=0> + __device__ __forceinline__ + void AddInitToScan( + OutputT (&items)[ITEMS_PER_THREAD], + OffsetT (&flags)[ITEMS_PER_THREAD]) + { + #pragma unroll + for (int ITEM = 0; ITEM < ITEMS_PER_THREAD; ++ITEM) + { + items[ITEM] = flags[ITEM] ? init_value : scan_op(init_value, items[ITEM]); + } + } + + template::VALUE, typename std::enable_if::type=0> + __device__ __forceinline__ + void AddInitToScan( + OutputT (&/*items*/)[ITEMS_PER_THREAD], + OffsetT (&/*flags*/)[ITEMS_PER_THREAD]) + {} + + //--------------------------------------------------------------------- + // Cooperatively scan a device-wide sequence of tiles with other CTAs + //--------------------------------------------------------------------- + + // Process a tile of input (dynamic chained scan) + // + template + __device__ __forceinline__ + void ConsumeTile( + OffsetT /*num_items*/, + OffsetT num_remaining, + int tile_idx, + OffsetT tile_base, + ScanTileStateT& tile_state) + { + // Load items + KeyT keys[ITEMS_PER_THREAD]; + OutputT values[ITEMS_PER_THREAD]; + OffsetT segment_flags[ITEMS_PER_THREAD]; + SizeValuePairT scan_items[ITEMS_PER_THREAD]; + + if (IS_LAST_TILE) + { + // Fill last element with the first element + // because collectives are not suffix guarded + BlockLoadKeysT(storage.load_keys) + .Load(d_keys_in + tile_base, + keys, + num_remaining, + *(d_keys_in + tile_base)); + } + else + { + BlockLoadKeysT(storage.load_keys) + .Load(d_keys_in + tile_base, keys); + } + + CTA_SYNC(); + + if (IS_LAST_TILE) + { + // Fill last element with the first element + // because collectives are not suffix guarded + BlockLoadValuesT(storage.load_values) + .Load(d_values_in + tile_base, + values, + num_remaining, + *(d_values_in + tile_base)); + } + else + { + BlockLoadValuesT(storage.load_values) + .Load(d_values_in + tile_base, values); + } + + CTA_SYNC(); + + // first tile + if (tile_idx == 0) + { + BlockDiscontinuityKeysT(storage.scan_storage.discontinuity) + .FlagHeads(segment_flags, keys, inequality_op); + + // Zip values and segment_flags + ZipValuesAndFlags(num_remaining, + values, + segment_flags, + scan_items); + + // Exclusive scan of values and segment_flags + SizeValuePairT tile_aggregate; + ScanTile(scan_items, tile_aggregate, Int2Type()); + + if (threadIdx.x == 0) + { + if (!IS_LAST_TILE) + tile_state.SetInclusive(0, tile_aggregate); + + scan_items[0].key = 0; + } + } + else + { + KeyT tile_pred_key = (threadIdx.x == 0) ? d_keys_in[tile_base - 1] : KeyT(); + BlockDiscontinuityKeysT(storage.scan_storage.discontinuity) + .FlagHeads(segment_flags, keys, inequality_op, tile_pred_key); + + // Zip values and segment_flags + ZipValuesAndFlags(num_remaining, + values, + segment_flags, + scan_items); + + SizeValuePairT tile_aggregate; + TilePrefixCallbackT prefix_op(tile_state, storage.scan_storage.prefix, pair_scan_op, tile_idx); + ScanTile(scan_items, tile_aggregate, prefix_op, Int2Type()); + } + + CTA_SYNC(); + + UnzipValues(values, scan_items); + + AddInitToScan(values, segment_flags); + + // Store items + if (IS_LAST_TILE) + { + BlockStoreValuesT(storage.store_values) + .Store(d_values_out + tile_base, values, num_remaining); + } + else + { + BlockStoreValuesT(storage.store_values) + .Store(d_values_out + tile_base, values); + } + } + + //--------------------------------------------------------------------- + // Constructor + //--------------------------------------------------------------------- + + // Dequeue and scan tiles of items as part of a dynamic chained scan + // with Init functor + __device__ __forceinline__ + AgentScanByKey( + TempStorage & storage, + KeysInputIteratorT d_keys_in, + ValuesInputIteratorT d_values_in, + ValuesOutputIteratorT d_values_out, + EqualityOp equality_op, + ScanOpT scan_op, + InitValueT init_value) + : + storage(storage), + d_keys_in(d_keys_in), + d_values_in(d_values_in), + d_values_out(d_values_out), + inequality_op(equality_op), + scan_op(scan_op), + pair_scan_op(scan_op), + init_value(init_value) + {} + + /** + * Scan tiles of items as part of a dynamic chained scan + */ + __device__ __forceinline__ void ConsumeRange( + OffsetT num_items, ///< Total number of input items + ScanTileStateT& tile_state, ///< Global tile state descriptor + int start_tile) ///< The starting tile for the current grid + { + int tile_idx = blockIdx.x; + OffsetT tile_base = OffsetT(ITEMS_PER_TILE) * tile_idx; + OffsetT num_remaining = num_items - tile_base; + + if (num_remaining > ITEMS_PER_TILE) + { + // Not the last tile (full) + ConsumeTile(num_items, + num_remaining, + tile_idx, + tile_base, + tile_state); + } + else if (num_remaining > 0) + { + // The last tile (possibly partially-full) + ConsumeTile(num_items, + num_remaining, + tile_idx, + tile_base, + tile_state); + } + } +}; + + +CUB_NAMESPACE_END diff --git a/cub/device/device_scan.cuh b/cub/device/device_scan.cuh index 53e3e0adc0..171f07dd71 100644 --- a/cub/device/device_scan.cuh +++ b/cub/device/device_scan.cuh @@ -38,7 +38,9 @@ #include #include "../config.cuh" +#include "../thread/thread_operators.cuh" #include "dispatch/dispatch_scan.cuh" +#include "dispatch/dispatch_scan_by_key.cuh" CUB_NAMESPACE_BEGIN @@ -145,8 +147,8 @@ struct DeviceScan static cudaError_t ExclusiveSum( void *d_temp_storage, ///< [in] Device-accessible allocation of temporary storage. When NULL, the required allocation size is written to \p temp_storage_bytes and no work is done. size_t &temp_storage_bytes, ///< [in,out] Reference to size in bytes of \p d_temp_storage allocation - InputIteratorT d_in, ///< [in] Pointer to the input sequence of data items - OutputIteratorT d_out, ///< [out] Pointer to the output sequence of data items + InputIteratorT d_in, ///< [in] Random-access iterator to the input sequence of data items + OutputIteratorT d_out, ///< [out] Random-access iterator to the output sequence of data items int num_items, ///< [in] Total number of input items (i.e., the length of \p d_in) cudaStream_t stream = 0, ///< [in] [optional] CUDA stream to launch kernels within. Default is stream0. bool debug_synchronous = false) ///< [in] [optional] Whether or not to synchronize the stream after every kernel launch to check for errors. May cause significant slowdown. Default is \p false. @@ -191,6 +193,7 @@ struct DeviceScan * \par * \code * #include // or equivalently + * #include // for INT_MAX * * // CustomMin functor * struct CustomMin @@ -206,19 +209,19 @@ struct DeviceScan * int num_items; // e.g., 7 * int *d_in; // e.g., [8, 6, 7, 5, 3, 0, 9] * int *d_out; // e.g., [ , , , , , , ] - * CustomMin min_op + * CustomMin min_op; * ... * * // Determine temporary device storage requirements for exclusive prefix scan * void *d_temp_storage = NULL; * size_t temp_storage_bytes = 0; - * cub::DeviceScan::ExclusiveScan(d_temp_storage, temp_storage_bytes, d_in, d_out, min_op, (int) MAX_INT, num_items); + * cub::DeviceScan::ExclusiveScan(d_temp_storage, temp_storage_bytes, d_in, d_out, min_op, (int) INT_MAX, num_items); * * // Allocate temporary storage for exclusive prefix scan * cudaMalloc(&d_temp_storage, temp_storage_bytes); * * // Run exclusive prefix min-scan - * cub::DeviceScan::ExclusiveScan(d_temp_storage, temp_storage_bytes, d_in, d_out, min_op, (int) MAX_INT, num_items); + * cub::DeviceScan::ExclusiveScan(d_temp_storage, temp_storage_bytes, d_in, d_out, min_op, (int) INT_MAX, num_items); * * // d_out <-- [2147483647, 8, 6, 6, 5, 3, 0] * @@ -227,7 +230,7 @@ struct DeviceScan * \tparam InputIteratorT [inferred] Random-access input iterator type for reading scan inputs \iterator * \tparam OutputIteratorT [inferred] Random-access output iterator type for writing scan outputs \iterator * \tparam ScanOp [inferred] Binary scan functor type having member T operator()(const T &a, const T &b) - * \tparam Identity [inferred] Type of the \p identity value used Binary scan functor type having member T operator()(const T &a, const T &b) + * \tparam InitValueT [inferred] Type of the \p init_value used Binary scan functor type having member T operator()(const T &a, const T &b) */ template < typename InputIteratorT, @@ -238,8 +241,8 @@ struct DeviceScan static cudaError_t ExclusiveScan( void *d_temp_storage, ///< [in] Device-accessible allocation of temporary storage. When NULL, the required allocation size is written to \p temp_storage_bytes and no work is done. size_t &temp_storage_bytes, ///< [in,out] Reference to size in bytes of \p d_temp_storage allocation - InputIteratorT d_in, ///< [in] Pointer to the input sequence of data items - OutputIteratorT d_out, ///< [out] Pointer to the output sequence of data items + InputIteratorT d_in, ///< [in] Random-access iterator to the input sequence of data items + OutputIteratorT d_out, ///< [out] Random-access iterator to the output sequence of data items ScanOpT scan_op, ///< [in] Binary scan functor InitValueT init_value, ///< [in] Initial value to seed the exclusive scan (and is assigned to *d_out) int num_items, ///< [in] Total number of input items (i.e., the length of \p d_in) @@ -351,8 +354,8 @@ struct DeviceScan static cudaError_t InclusiveSum( void* d_temp_storage, ///< [in] Device-accessible allocation of temporary storage. When NULL, the required allocation size is written to \p temp_storage_bytes and no work is done. size_t& temp_storage_bytes, ///< [in,out] Reference to size in bytes of \p d_temp_storage allocation - InputIteratorT d_in, ///< [in] Pointer to the input sequence of data items - OutputIteratorT d_out, ///< [out] Pointer to the output sequence of data items + InputIteratorT d_in, ///< [in] Random-access iterator to the input sequence of data items + OutputIteratorT d_out, ///< [out] Random-access iterator to the output sequence of data items int num_items, ///< [in] Total number of input items (i.e., the length of \p d_in) cudaStream_t stream = 0, ///< [in] [optional] CUDA stream to launch kernels within. Default is stream0. bool debug_synchronous = false) ///< [in] [optional] Whether or not to synchronize the stream after every kernel launch to check for errors. May cause significant slowdown. Default is \p false. @@ -390,6 +393,7 @@ struct DeviceScan * \par * \code * #include // or equivalently + * #include // for INT_MAX * * // CustomMin functor * struct CustomMin @@ -435,8 +439,8 @@ struct DeviceScan static cudaError_t InclusiveScan( void *d_temp_storage, ///< [in] Device-accessible allocation of temporary storage. When NULL, the required allocation size is written to \p temp_storage_bytes and no work is done. size_t &temp_storage_bytes, ///< [in,out] Reference to size in bytes of \p d_temp_storage allocation - InputIteratorT d_in, ///< [in] Pointer to the input sequence of data items - OutputIteratorT d_out, ///< [out] Pointer to the output sequence of data items + InputIteratorT d_in, ///< [in] Random-access iterator to the input sequence of data items + OutputIteratorT d_out, ///< [out] Random-access iterator to the output sequence of data items ScanOpT scan_op, ///< [in] Binary scan functor int num_items, ///< [in] Total number of input items (i.e., the length of \p d_in) cudaStream_t stream = 0, ///< [in] [optional] CUDA stream to launch kernels within. Default is stream0. @@ -457,6 +461,397 @@ struct DeviceScan debug_synchronous); } + /** + * \brief Computes a device-wide exclusive prefix sum-by-key with key equality + * defined by \p equality_op . The value of 0 is applied as the initial value, + * and is assigned to the beginning of each segment in \p d_values_out . + * + * \par + * - Supports non-commutative sum operators. + * - Provides "run-to-run" determinism for pseudo-associative reduction + * (e.g., addition of floating point types) on the same GPU device. + * However, results for pseudo-associative reduction may be inconsistent + * from one device to a another device of a different compute-capability + * because CUB can employ different tile-sizing for different architectures. + * - \devicestorage + * + * \par Snippet + * The code snippet below illustrates the exclusive prefix sum-by-key of an \p int device vector. + * \par + * \code + * #include // or equivalently + * + * // Declare, allocate, and initialize device-accessible pointers for input and output + * int num_items; // e.g., 7 + * int *d_keys_in; // e.g., [0, 0, 1, 1, 1, 2, 2] + * int *d_values_in; // e.g., [8, 6, 7, 5, 3, 0, 9] + * int *d_values_out; // e.g., [ , , , , , , ] + * ... + * + * // Determine temporary device storage requirements + * void *d_temp_storage = NULL; + * size_t temp_storage_bytes = 0; + * cub::DeviceScan::ExclusiveSumByKey(d_temp_storage, temp_storage_bytes, d_keys_in, d_values_in, d_values_out, num_items); + * + * // Allocate temporary storage + * cudaMalloc(&d_temp_storage, temp_storage_bytes); + * + * // Run exclusive prefix sum + * cub::DeviceScan::ExclusiveSumByKey(d_temp_storage, temp_storage_bytes, d_keys_in, d_values_in, d_values_out, num_items); + * + * // d_values_out <-- [0, 8, 0, 7, 12, 0, 0] + * + * \endcode + * + * \tparam KeysInputIteratorT [inferred] Random-access input iterator type for reading scan keys inputs \iterator + * \tparam ValuesInputIteratorT [inferred] Random-access input iterator type for reading scan values inputs \iterator + * \tparam ValuesOutputIteratorT [inferred] Random-access output iterator type for writing scan values outputs \iterator + * \tparam EqualityOpT [inferred][/b] Functor type having member T operator()(const T &a, const T &b) for binary operations that defines the equality of keys + */ + template < + typename KeysInputIteratorT, + typename ValuesInputIteratorT, + typename ValuesOutputIteratorT, + typename EqualityOpT = Equality> + CUB_RUNTIME_FUNCTION + static cudaError_t ExclusiveSumByKey( + void *d_temp_storage, ///< [in] %Device-accessible allocation of temporary storage. When NULL, the required allocation size is written to \p temp_storage_bytes and no work is done. + size_t &temp_storage_bytes, ///< [in,out] Reference to size in bytes of \p d_temp_storage allocation + KeysInputIteratorT d_keys_in, ///< [in] Random-access input iterator to the input sequence of key items + ValuesInputIteratorT d_values_in, ///< [in] Random-access input iterator to the input sequence of value items + ValuesOutputIteratorT d_values_out, ///< [out] Random-access output iterator to the output sequence of value items + int num_items, ///< [in] Total number of input items (i.e., the length of \p d_keys_in and \p d_values_in) + EqualityOpT equality_op = EqualityOpT(), ///< [in] Binary functor that defines the equality of keys. Default is cub::Equality(). + cudaStream_t stream=0, ///< [in] [optional] CUDA stream to launch kernels within. Default is stream0. + bool debug_synchronous=false) ///< [in] [optional] Whether or not to synchronize the stream after every kernel launch to check for errors. May cause significant slowdown. Default is \p false. + { + // Signed integer type for global offsets + typedef int OffsetT; + + // The output value type -- used as the intermediate accumulator + // Use the input value type per https://wg21.link/P0571 + typedef typename std::iterator_traits::value_type OutputT; + + // Initial value + OutputT init_value = 0; + + return DispatchScanByKey< + KeysInputIteratorT, ValuesInputIteratorT, ValuesOutputIteratorT, EqualityOpT, Sum, OutputT, OffsetT> + ::Dispatch( + d_temp_storage, + temp_storage_bytes, + d_keys_in, + d_values_in, + d_values_out, + equality_op, + Sum(), + init_value, + num_items, + stream, + debug_synchronous); + } + + /** + * \brief Computes a device-wide exclusive prefix scan-by-key using the specified binary \p scan_op functor. + * The key equality is defined by \p equality_op . The \p init_value value is applied as the initial value, + * and is assigned to the beginning of each segment in \p d_values_out . + * + * \par + * - Supports non-commutative scan operators. + * - Provides "run-to-run" determinism for pseudo-associative reduction + * (e.g., addition of floating point types) on the same GPU device. + * However, results for pseudo-associative reduction may be inconsistent + * from one device to a another device of a different compute-capability + * because CUB can employ different tile-sizing for different architectures. + * - \devicestorage + * + * \par Snippet + * The code snippet below illustrates the exclusive prefix min-scan-by-key of an \p int device vector + * \par + * \code + * #include // or equivalently + * #include // for INT_MAX + * + * // CustomMin functor + * struct CustomMin + * { + * template + * CUB_RUNTIME_FUNCTION __forceinline__ + * T operator()(const T &a, const T &b) const { + * return (b < a) ? b : a; + * } + * }; + * + * // CustomEqual functor + * struct CustomEqual + * { + * template + * CUB_RUNTIME_FUNCTION __forceinline__ + * T operator()(const T &a, const T &b) const { + * return a == b; + * } + * }; + * + * // Declare, allocate, and initialize device-accessible pointers for input and output + * int num_items; // e.g., 7 + * int *d_keys_in; // e.g., [0, 0, 1, 1, 1, 2, 2] + * int *d_values_in; // e.g., [8, 6, 7, 5, 3, 0, 9] + * int *d_values_out; // e.g., [ , , , , , , ] + * CustomMin min_op; + * CustomEqual equality_op; + * ... + * + * // Determine temporary device storage requirements for exclusive prefix scan + * void *d_temp_storage = NULL; + * size_t temp_storage_bytes = 0; + * cub::DeviceScan::ExclusiveScanByKey(d_temp_storage, temp_storage_bytes, d_keys_in, d_values_in, d_values_out, min_op, (int) INT_MAX, num_items, equality_op); + * + * // Allocate temporary storage for exclusive prefix scan + * cudaMalloc(&d_temp_storage, temp_storage_bytes); + * + * // Run exclusive prefix min-scan + * cub::DeviceScan::ExclusiveScanByKey(d_temp_storage, temp_storage_bytes, d_keys_in, d_values_in, d_values_out, min_op, (int) INT_MAX, num_items, equality_op); + * + * // d_values_out <-- [2147483647, 8, 2147483647, 7, 5, 2147483647, 0] + * + * \endcode + * + * \tparam KeysInputIteratorT [inferred] Random-access input iterator type for reading scan keys inputs \iterator + * \tparam ValuesInputIteratorT [inferred] Random-access input iterator type for reading scan values inputs \iterator + * \tparam ValuesOutputIteratorT [inferred] Random-access output iterator type for writing scan values outputs \iterator + * \tparam ScanOp [inferred] Binary scan functor type having member T operator()(const T &a, const T &b) + * \tparam InitValueT [inferred] Type of the \p init_value value used in Binary scan functor type having member T operator()(const T &a, const T &b) + * \tparam EqualityOpT [inferred][/b] Functor type having member T operator()(const T &a, const T &b) for binary operations that defines the equality of keys + */ + template < + typename KeysInputIteratorT, + typename ValuesInputIteratorT, + typename ValuesOutputIteratorT, + typename ScanOpT, + typename InitValueT, + typename EqualityOpT = Equality> + CUB_RUNTIME_FUNCTION + static cudaError_t ExclusiveScanByKey( + void *d_temp_storage, ///< [in] %Device-accessible allocation of temporary storage. When NULL, the required allocation size is written to \p temp_storage_bytes and no work is done. + size_t &temp_storage_bytes, ///< [in,out] Reference to size in bytes of \p d_temp_storage allocation + KeysInputIteratorT d_keys_in, ///< [in] Random-access input iterator to the input sequence of key items + ValuesInputIteratorT d_values_in, ///< [in] Random-access input iterator to the input sequence of value items + ValuesOutputIteratorT d_values_out, ///< [out] Random-access output iterator to the output sequence of value items + ScanOpT scan_op, ///< [in] Binary scan functor + InitValueT init_value, ///< [in] Initial value to seed the exclusive scan (and is assigned to the beginning of each segment in \p d_values_out) + int num_items, ///< [in] Total number of input items (i.e., the length of \p d_keys_in and \p d_values_in) + EqualityOpT equality_op = EqualityOpT(), ///< [in] Binary functor that defines the equality of keys. Default is cub::Equality(). + cudaStream_t stream=0, ///< [in] [optional] CUDA stream to launch kernels within. Default is stream0. + bool debug_synchronous=false) ///< [in] [optional] Whether or not to synchronize the stream after every kernel launch to check for errors. May cause significant slowdown. Default is \p false. + { + // Signed integer type for global offsets + typedef int OffsetT; + + return DispatchScanByKey< + KeysInputIteratorT, ValuesInputIteratorT, ValuesOutputIteratorT, EqualityOpT, ScanOpT, InitValueT, OffsetT> + ::Dispatch( + d_temp_storage, + temp_storage_bytes, + d_keys_in, + d_values_in, + d_values_out, + equality_op, + scan_op, + init_value, + num_items, + stream, + debug_synchronous); + } + + /** + * \brief Computes a device-wide inclusive prefix sum-by-key with key equality defined by \p equality_op . + * + * \par + * - Supports non-commutative sum operators. + * - Provides "run-to-run" determinism for pseudo-associative reduction + * (e.g., addition of floating point types) on the same GPU device. + * However, results for pseudo-associative reduction may be inconsistent + * from one device to a another device of a different compute-capability + * because CUB can employ different tile-sizing for different architectures. + * - \devicestorage + * + * \par Snippet + * The code snippet below illustrates the inclusive prefix sum-by-key of an \p int device vector. + * \par + * \code + * #include // or equivalently + * + * // Declare, allocate, and initialize device-accessible pointers for input and output + * int num_items; // e.g., 7 + * int *d_keys_in; // e.g., [0, 0, 1, 1, 1, 2, 2] + * int *d_values_in; // e.g., [8, 6, 7, 5, 3, 0, 9] + * int *d_values_out; // e.g., [ , , , , , , ] + * ... + * + * // Determine temporary device storage requirements for inclusive prefix sum + * void *d_temp_storage = NULL; + * size_t temp_storage_bytes = 0; + * cub::DeviceScan::InclusiveSumByKey(d_temp_storage, temp_storage_bytes, d_keys_in, d_values_in, d_values_out, num_items); + * + * // Allocate temporary storage for inclusive prefix sum + * cudaMalloc(&d_temp_storage, temp_storage_bytes); + * + * // Run inclusive prefix sum + * cub::DeviceScan::InclusiveSumByKey(d_temp_storage, temp_storage_bytes, d_keys_in, d_values_in, d_values_out, num_items); + * + * // d_out <-- [8, 14, 7, 12, 15, 0, 9] + * + * \endcode + * + * \tparam KeysInputIteratorT [inferred] Random-access input iterator type for reading scan keys inputs \iterator + * \tparam ValuesInputIteratorT [inferred] Random-access input iterator type for reading scan values inputs \iterator + * \tparam ValuesOutputIteratorT [inferred] Random-access output iterator type for writing scan values outputs \iterator + * \tparam EqualityOpT [inferred][/b] Functor type having member T operator()(const T &a, const T &b) for binary operations that defines the equality of keys + */ + template < + typename KeysInputIteratorT, + typename ValuesInputIteratorT, + typename ValuesOutputIteratorT, + typename EqualityOpT = Equality> + CUB_RUNTIME_FUNCTION + static cudaError_t InclusiveSumByKey( + void *d_temp_storage, ///< [in] %Device-accessible allocation of temporary storage. When NULL, the required allocation size is written to \p temp_storage_bytes and no work is done. + size_t &temp_storage_bytes, ///< [in,out] Reference to size in bytes of \p d_temp_storage allocation + KeysInputIteratorT d_keys_in, ///< [in] Random-access input iterator to the input sequence of key items + ValuesInputIteratorT d_values_in, ///< [in] Random-access input iterator to the input sequence of value items + ValuesOutputIteratorT d_values_out, ///< [out] Random-access output iterator to the output sequence of value items + int num_items, ///< [in] Total number of input items (i.e., the length of \p d_keys_in and \p d_values_in) + EqualityOpT equality_op = EqualityOpT(), ///< [in] Binary functor that defines the equality of keys. Default is cub::Equality(). + cudaStream_t stream=0, ///< [in] [optional] CUDA stream to launch kernels within. Default is stream0. + bool debug_synchronous=false) ///< [in] [optional] Whether or not to synchronize the stream after every kernel launch to check for errors. May cause significant slowdown. Default is \p false. + { + // Signed integer type for global offsets + typedef int OffsetT; + + return DispatchScanByKey< + KeysInputIteratorT, ValuesInputIteratorT, ValuesOutputIteratorT, EqualityOpT, Sum, NullType, OffsetT> + ::Dispatch( + d_temp_storage, + temp_storage_bytes, + d_keys_in, + d_values_in, + d_values_out, + equality_op, + Sum(), + NullType(), + num_items, + stream, + debug_synchronous); + } + + /** + * \brief Computes a device-wide inclusive prefix scan-by-key using the specified binary \p scan_op functor. + * The key equality is defined by \p equality_op . + * + * \par + * - Supports non-commutative scan operators. + * - Provides "run-to-run" determinism for pseudo-associative reduction + * (e.g., addition of floating point types) on the same GPU device. + * However, results for pseudo-associative reduction may be inconsistent + * from one device to a another device of a different compute-capability + * because CUB can employ different tile-sizing for different architectures. + * - \devicestorage + * + * \par Snippet + * The code snippet below illustrates the inclusive prefix min-scan-by-key of an \p int device vector. + * \par + * \code + * #include // or equivalently + * #include // for INT_MAX + * + * // CustomMin functor + * struct CustomMin + * { + * template + * CUB_RUNTIME_FUNCTION __forceinline__ + * T operator()(const T &a, const T &b) const { + * return (b < a) ? b : a; + * } + * }; + * + * // CustomEqual functor + * struct CustomEqual + * { + * template + * CUB_RUNTIME_FUNCTION __forceinline__ + * T operator()(const T &a, const T &b) const { + * return a == b; + * } + * }; + * + * // Declare, allocate, and initialize device-accessible pointers for input and output + * int num_items; // e.g., 7 + * int *d_keys_in; // e.g., [0, 0, 1, 1, 1, 2, 2] + * int *d_values_in; // e.g., [8, 6, 7, 5, 3, 0, 9] + * int *d_values_out; // e.g., [ , , , , , , ] + * CustomMin min_op; + * CustomEqual equality_op; + * ... + * + * // Determine temporary device storage requirements for inclusive prefix scan + * void *d_temp_storage = NULL; + * size_t temp_storage_bytes = 0; + * cub::DeviceScan::InclusiveScanByKey(d_temp_storage, temp_storage_bytes, d_keys_in, d_values_in, d_values_out, min_op, num_items, equality_op); + * + * // Allocate temporary storage for inclusive prefix scan + * cudaMalloc(&d_temp_storage, temp_storage_bytes); + * + * // Run inclusive prefix min-scan + * cub::DeviceScan::InclusiveScanByKey(d_temp_storage, temp_storage_bytes, d_keys_in, d_values_in, d_values_out, min_op, num_items, equality_op); + * + * // d_out <-- [8, 6, 7, 5, 3, 0, 0] + * + * \endcode + * + * \tparam KeysInputIteratorT [inferred] Random-access input iterator type for reading scan keys inputs \iterator + * \tparam ValuesInputIteratorT [inferred] Random-access input iterator type for reading scan values inputs \iterator + * \tparam ValuesOutputIteratorT [inferred] Random-access output iterator type for writing scan values outputs \iterator + * \tparam ScanOp [inferred] Binary scan functor type having member T operator()(const T &a, const T &b) + * \tparam EqualityOpT [inferred][/b] Functor type having member T operator()(const T &a, const T &b) for binary operations that defines the equality of keys + */ + template < + typename KeysInputIteratorT, + typename ValuesInputIteratorT, + typename ValuesOutputIteratorT, + typename ScanOpT, + typename EqualityOpT = Equality> + CUB_RUNTIME_FUNCTION + static cudaError_t InclusiveScanByKey( + void *d_temp_storage, ///< [in] %Device-accessible allocation of temporary storage. When NULL, the required allocation size is written to \p temp_storage_bytes and no work is done. + size_t &temp_storage_bytes, ///< [in,out] Reference to size in bytes of \p d_temp_storage allocation + KeysInputIteratorT d_keys_in, ///< [in] Random-access input iterator to the input sequence of key items + ValuesInputIteratorT d_values_in, ///< [in] Random-access input iterator to the input sequence of value items + ValuesOutputIteratorT d_values_out, ///< [out] Random-access output iterator to the output sequence of value items + ScanOpT scan_op, ///< [in] Binary scan functor + int num_items, ///< [in] Total number of input items (i.e., the length of \p d_keys_in and \p d_values_in) + EqualityOpT equality_op = EqualityOpT(), ///< [in] Binary functor that defines the equality of keys. Default is cub::Equality(). + cudaStream_t stream=0, ///< [in] [optional] CUDA stream to launch kernels within. Default is stream0. + bool debug_synchronous=false) ///< [in] [optional] Whether or not to synchronize the stream after every kernel launch to check for errors. May cause significant slowdown. Default is \p false. + { + // Signed integer type for global offsets + typedef int OffsetT; + + return DispatchScanByKey< + KeysInputIteratorT, ValuesInputIteratorT, ValuesOutputIteratorT, EqualityOpT, ScanOpT, NullType, OffsetT> + ::Dispatch( + d_temp_storage, + temp_storage_bytes, + d_keys_in, + d_values_in, + d_values_out, + equality_op, + scan_op, + NullType(), + num_items, + stream, + debug_synchronous); + } + //@} end member group }; diff --git a/cub/device/dispatch/dispatch_scan.cuh b/cub/device/dispatch/dispatch_scan.cuh index 1100dfbf1b..2a5f78b113 100644 --- a/cub/device/dispatch/dispatch_scan.cuh +++ b/cub/device/dispatch/dispatch_scan.cuh @@ -34,7 +34,6 @@ #pragma once -#include #include #include "../../agent/agent_scan.cuh" @@ -149,7 +148,7 @@ struct DeviceScanPolicy LargeValues ? BLOCK_STORE_WARP_TRANSPOSE_TIMESLICED : BLOCK_STORE_WARP_TRANSPOSE; - /// SM35 + /// SM350 struct Policy350 : ChainedPolicy<350, Policy350, Policy350> { // GTX Titan: 29.5B items/s (232.4 GB/s) @ 48M 32-bit T @@ -237,8 +236,8 @@ struct DispatchScan: void* d_temp_storage; ///< [in] Device-accessible allocation of temporary storage. When NULL, the required allocation size is written to \p temp_storage_bytes and no work is done. size_t& temp_storage_bytes; ///< [in,out] Reference to size in bytes of \p d_temp_storage allocation - InputIteratorT d_in; ///< [in] Pointer to the input sequence of data items - OutputIteratorT d_out; ///< [out] Pointer to the output sequence of data items + InputIteratorT d_in; ///< [in] Iterator to the input sequence of data items + OutputIteratorT d_out; ///< [out] Iterator to the output sequence of data items ScanOpT scan_op; ///< [in] Binary scan functor InitValueT init_value; ///< [in] Initial value to seed the exclusive scan OffsetT num_items; ///< [in] Total number of input items (i.e., the length of \p d_in) @@ -250,8 +249,8 @@ struct DispatchScan: DispatchScan( void* d_temp_storage, ///< [in] Device-accessible allocation of temporary storage. When NULL, the required allocation size is written to \p temp_storage_bytes and no work is done. size_t& temp_storage_bytes, ///< [in,out] Reference to size in bytes of \p d_temp_storage allocation - InputIteratorT d_in, ///< [in] Pointer to the input sequence of data items - OutputIteratorT d_out, ///< [out] Pointer to the output sequence of data items + InputIteratorT d_in, ///< [in] Iterator to the input sequence of data items + OutputIteratorT d_out, ///< [out] Iterator to the output sequence of data items OffsetT num_items, ///< [in] Total number of input items (i.e., the length of \p d_in) ScanOpT scan_op, ///< [in] Binary scan functor InitValueT init_value, ///< [in] Initial value to seed the exclusive scan @@ -259,16 +258,16 @@ struct DispatchScan: bool debug_synchronous, int ptx_version ): - d_temp_storage(d_temp_storage), - temp_storage_bytes(temp_storage_bytes), - d_in(d_in), - d_out(d_out), - num_items(num_items), - scan_op(scan_op), - init_value(init_value), - stream(stream), - debug_synchronous(debug_synchronous), - ptx_version(ptx_version) + d_temp_storage(d_temp_storage), + temp_storage_bytes(temp_storage_bytes), + d_in(d_in), + d_out(d_out), + scan_op(scan_op), + init_value(init_value), + num_items(num_items), + stream(stream), + debug_synchronous(debug_synchronous), + ptx_version(ptx_version) {} template @@ -404,8 +403,8 @@ struct DispatchScan: static cudaError_t Dispatch( void* d_temp_storage, ///< [in] Device-accessible allocation of temporary storage. When NULL, the required allocation size is written to \p temp_storage_bytes and no work is done. size_t& temp_storage_bytes, ///< [in,out] Reference to size in bytes of \p d_temp_storage allocation - InputIteratorT d_in, ///< [in] Pointer to the input sequence of data items - OutputIteratorT d_out, ///< [out] Pointer to the output sequence of data items + InputIteratorT d_in, ///< [in] Iterator to the input sequence of data items + OutputIteratorT d_out, ///< [out] Iterator to the output sequence of data items ScanOpT scan_op, ///< [in] Binary scan functor InitValueT init_value, ///< [in] Initial value to seed the exclusive scan OffsetT num_items, ///< [in] Total number of input items (i.e., the length of \p d_in) @@ -423,16 +422,16 @@ struct DispatchScan: // Create dispatch functor DispatchScan dispatch( - d_temp_storage, - temp_storage_bytes, - d_in, - d_out, - num_items, - scan_op, - init_value, - stream, - debug_synchronous, - ptx_version + d_temp_storage, + temp_storage_bytes, + d_in, + d_out, + num_items, + scan_op, + init_value, + stream, + debug_synchronous, + ptx_version ); // Dispatch to chained policy if (CubDebug(error = MaxPolicyT::Invoke(ptx_version, dispatch))) break; diff --git a/cub/device/dispatch/dispatch_scan_by_key.cuh b/cub/device/dispatch/dispatch_scan_by_key.cuh new file mode 100644 index 0000000000..245e45820f --- /dev/null +++ b/cub/device/dispatch/dispatch_scan_by_key.cuh @@ -0,0 +1,434 @@ +/****************************************************************************** + * Copyright (c) 2021, NVIDIA CORPORATION. All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * * Redistributions of source code must retain the above copyright + * notice, this list of conditions and the following disclaimer. + * * Redistributions in binary form must reproduce the above copyright + * notice, this list of conditions and the following disclaimer in the + * documentation and/or other materials provided with the distribution. + * * Neither the name of the NVIDIA CORPORATION nor the + * names of its contributors may be used to endorse or promote products + * derived from this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND + * ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED + * WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL NVIDIA CORPORATION BE LIABLE FOR ANY + * DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES + * (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; + * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND + * ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS + * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + ******************************************************************************/ + +/** + * \file + * DeviceScan provides device-wide, parallel operations for computing a prefix scan across a sequence of data items residing within device-accessible memory. + */ + +#pragma once + +#include +#include + +#include "../../agent/agent_scan_by_key.cuh" +#include "../../thread/thread_operators.cuh" +#include "../../config.cuh" +#include "../../util_debug.cuh" +#include "../../util_device.cuh" +#include "../../util_math.cuh" +#include "dispatch_scan.cuh" + +#include + +CUB_NAMESPACE_BEGIN + + +/****************************************************************************** +* Kernel entry points +*****************************************************************************/ + +/** + * Scan kernel entry point (multi-block) + */ +template < + typename ChainedPolicyT, ///< Chained tuning policy + typename KeysInputIteratorT, ///< Random-access input iterator type + typename ValuesInputIteratorT, ///< Random-access input iterator type + typename ValuesOutputIteratorT, ///< Random-access output iterator type + typename ScanByKeyTileStateT, ///< Tile status interface type + typename EqualityOp, ///< Equality functor type + typename ScanOpT, ///< Scan functor type + typename InitValueT, ///< The init_value element for ScanOpT type (cub::NullType for inclusive scan) + typename OffsetT> ///< Signed integer type for global offsets +__launch_bounds__ (int(ChainedPolicyT::ActivePolicy::ScanByKeyPolicyT::BLOCK_THREADS)) +__global__ void DeviceScanByKeyKernel( + KeysInputIteratorT d_keys_in, ///< Input keys data + ValuesInputIteratorT d_values_in, ///< Input values data + ValuesOutputIteratorT d_values_out, ///< Output values data + ScanByKeyTileStateT tile_state, ///< Tile status interface + int start_tile, ///< The starting tile for the current grid + EqualityOp equality_op, ///< Binary equality functor + ScanOpT scan_op, ///< Binary scan functor + InitValueT init_value, ///< Initial value to seed the exclusive scan + OffsetT num_items) ///< Total number of scan items for the entire problem +{ + typedef typename ChainedPolicyT::ActivePolicy::ScanByKeyPolicyT ScanByKeyPolicyT; + + // Thread block type for scanning input tiles + typedef AgentScanByKey< + ScanByKeyPolicyT, + KeysInputIteratorT, + ValuesInputIteratorT, + ValuesOutputIteratorT, + EqualityOp, + ScanOpT, + InitValueT, + OffsetT> AgentScanByKeyT; + + // Shared memory for AgentScanByKey + __shared__ typename AgentScanByKeyT::TempStorage temp_storage; + + // Process tiles + AgentScanByKeyT( + temp_storage, + d_keys_in, + d_values_in, + d_values_out, + equality_op, + scan_op, + init_value + ).ConsumeRange( + num_items, + tile_state, + start_tile); +} + + +/****************************************************************************** + * Policy + ******************************************************************************/ + +template +struct DeviceScanByKeyPolicy +{ + using KeyT = typename std::iterator_traits::value_type; + using ValueT = typename If< + Equals::VALUE, + typename std::iterator_traits::value_type, + InitValueT>::Type; + static constexpr size_t MaxInputBytes = (sizeof(KeyT) > sizeof(ValueT) ? sizeof(KeyT) : sizeof(ValueT)); + static constexpr size_t CombinedInputBytes = sizeof(KeyT) + sizeof(ValueT); + + // SM350 + struct Policy350 : ChainedPolicy<350, Policy350, Policy350> + { + enum + { + NOMINAL_4B_ITEMS_PER_THREAD = 6, + ITEMS_PER_THREAD = ((MaxInputBytes <= 8) ? 6 : + Nominal4BItemsToItemsCombined(NOMINAL_4B_ITEMS_PER_THREAD, CombinedInputBytes)), + }; + + typedef AgentScanByKeyPolicy< + 128, ITEMS_PER_THREAD, + BLOCK_LOAD_WARP_TRANSPOSE, + LOAD_LDG, + BLOCK_SCAN_WARP_SCANS, + BLOCK_STORE_WARP_TRANSPOSE> + ScanByKeyPolicyT; + }; + + // SM520 + struct Policy520 : ChainedPolicy<520, Policy520, Policy350> + { + enum + { + NOMINAL_4B_ITEMS_PER_THREAD = 9, + + ITEMS_PER_THREAD = ((MaxInputBytes <= 8) ? 9 : + Nominal4BItemsToItemsCombined(NOMINAL_4B_ITEMS_PER_THREAD, CombinedInputBytes)), + }; + + typedef AgentScanByKeyPolicy< + 256, ITEMS_PER_THREAD, + BLOCK_LOAD_WARP_TRANSPOSE, + LOAD_LDG, + BLOCK_SCAN_WARP_SCANS, + BLOCK_STORE_WARP_TRANSPOSE> + ScanByKeyPolicyT; + }; + + /// MaxPolicy + typedef Policy520 MaxPolicy; +}; + + +/****************************************************************************** + * Dispatch + ******************************************************************************/ + + +/** + * Utility class for dispatching the appropriately-tuned kernels for DeviceScan + */ +template < + typename KeysInputIteratorT, ///< Random-access input iterator type + typename ValuesInputIteratorT, ///< Random-access input iterator type + typename ValuesOutputIteratorT, ///< Random-access output iterator type + typename EqualityOp, ///< Equality functor type + typename ScanOpT, ///< Scan functor type + typename InitValueT, ///< The init_value element for ScanOpT type (cub::NullType for inclusive scan) + typename OffsetT, ///< Signed integer type for global offsets + typename SelectedPolicy = DeviceScanByKeyPolicy +> +struct DispatchScanByKey: + SelectedPolicy +{ + //--------------------------------------------------------------------- + // Constants and Types + //--------------------------------------------------------------------- + + enum + { + INIT_KERNEL_THREADS = 128 + }; + + // The input key type + using KeyT = typename std::iterator_traits::value_type; + + // The input value type + using InputT = typename std::iterator_traits::value_type; + + // The output value type -- used as the intermediate accumulator + // Per https://wg21.link/P0571, use InitValueT if provided, otherwise the + // input iterator's value type. + using OutputT = + typename If::VALUE, InputT, InitValueT>::Type; + + void* d_temp_storage; ///< [in] %Device-accessible allocation of temporary storage. When NULL, the required allocation size is written to \p temp_storage_bytes and no work is done. + size_t& temp_storage_bytes; ///< [in,out] Reference to size in bytes of \p d_temp_storage allocation + KeysInputIteratorT d_keys_in; ///< [in] Iterator to the input sequence of key items + ValuesInputIteratorT d_values_in; ///< [in] Iterator to the input sequence of value items + ValuesOutputIteratorT d_values_out; ///< [out] Iterator to the input sequence of value items + EqualityOp equality_op; ///< [in]Binary equality functor + ScanOpT scan_op; ///< [in] Binary scan functor + InitValueT init_value; ///< [in] Initial value to seed the exclusive scan + OffsetT num_items; ///< [in] Total number of input items (i.e., the length of \p d_in) + cudaStream_t stream; ///< [in] [optional] CUDA stream to launch kernels within. Default is stream0. + bool debug_synchronous; + int ptx_version; + + CUB_RUNTIME_FUNCTION __forceinline__ + DispatchScanByKey( + void* d_temp_storage, ///< [in] %Device-accessible allocation of temporary storage. When NULL, the required allocation size is written to \p temp_storage_bytes and no work is done. + size_t& temp_storage_bytes, ///< [in,out] Reference to size in bytes of \p d_temp_storage allocation + KeysInputIteratorT d_keys_in, ///< [in] Iterator to the input sequence of key items + ValuesInputIteratorT d_values_in, ///< [in] Iterator to the input sequence of value items + ValuesOutputIteratorT d_values_out, ///< [out] Iterator to the input sequence of value items + EqualityOp equality_op, ///< [in] Binary equality functor + ScanOpT scan_op, ///< [in] Binary scan functor + InitValueT init_value, ///< [in] Initial value to seed the exclusive scan + OffsetT num_items, ///< [in] Total number of input items (i.e., the length of \p d_in) + cudaStream_t stream, ///< [in] [optional] CUDA stream to launch kernels within. Default is stream0. + bool debug_synchronous, + int ptx_version + ): + d_temp_storage(d_temp_storage), + temp_storage_bytes(temp_storage_bytes), + d_keys_in(d_keys_in), + d_values_in(d_values_in), + d_values_out(d_values_out), + equality_op(equality_op), + scan_op(scan_op), + init_value(init_value), + num_items(num_items), + stream(stream), + debug_synchronous(debug_synchronous), + ptx_version(ptx_version) + {} + + template + CUB_RUNTIME_FUNCTION __host__ __forceinline__ + cudaError_t Invoke(InitKernel init_kernel, ScanKernel scan_kernel) + { +#ifndef CUB_RUNTIME_ENABLED + + (void)init_kernel; + (void)scan_kernel; + + // Kernel launch not supported from this device + return CubDebug(cudaErrorNotSupported); + +#else + typedef typename ActivePolicyT::ScanByKeyPolicyT Policy; + typedef ReduceByKeyScanTileState ScanByKeyTileStateT; + + cudaError error = cudaSuccess; + do + { + // Get device ordinal + int device_ordinal; + if (CubDebug(error = cudaGetDevice(&device_ordinal))) break; + + // Number of input tiles + int tile_size = Policy::BLOCK_THREADS * Policy::ITEMS_PER_THREAD; + int num_tiles = static_cast(cub::DivideAndRoundUp(num_items, tile_size)); + + // Specify temporary storage allocation requirements + size_t allocation_sizes[1]; + if (CubDebug(error = ScanByKeyTileStateT::AllocationSize(num_tiles, allocation_sizes[0]))) break; // bytes needed for tile status descriptors + + // Compute allocation pointers into the single storage blob (or compute the necessary size of the blob) + void* allocations[1] = {}; + if (CubDebug(error = AliasTemporaries(d_temp_storage, temp_storage_bytes, allocations, allocation_sizes))) break; + if (d_temp_storage == NULL) + { + // Return if the caller is simply requesting the size of the storage allocation + break; + } + + // Return if empty problem + if (num_items == 0) + break; + + // Construct the tile status interface + ScanByKeyTileStateT tile_state; + if (CubDebug(error = tile_state.Init(num_tiles, allocations[0], allocation_sizes[0]))) break; + + // Log init_kernel configuration + int init_grid_size = cub::DivideAndRoundUp(num_tiles, INIT_KERNEL_THREADS); + if (debug_synchronous) _CubLog("Invoking init_kernel<<<%d, %d, 0, %lld>>>()\n", init_grid_size, INIT_KERNEL_THREADS, (long long) stream); + + // Invoke init_kernel to initialize tile descriptors + THRUST_NS_QUALIFIER::cuda_cub::launcher::triple_chevron( + init_grid_size, INIT_KERNEL_THREADS, 0, stream + ).doit(init_kernel, tile_state, num_tiles); + + // Check for failure to launch + if (CubDebug(error = cudaPeekAtLastError())) break; + + // Sync the stream if specified to flush runtime errors + if (debug_synchronous && (CubDebug(error = SyncStream(stream)))) break; + + + // Get SM occupancy for scan_kernel + int scan_sm_occupancy; + if (CubDebug(error = MaxSmOccupancy( + scan_sm_occupancy, // out + scan_kernel, + Policy::BLOCK_THREADS))) break; + + // Get max x-dimension of grid + int max_dim_x; + if (CubDebug(error = cudaDeviceGetAttribute(&max_dim_x, cudaDevAttrMaxGridDimX, device_ordinal))) break;; + + // Run grids in epochs (in case number of tiles exceeds max x-dimension + int scan_grid_size = CUB_MIN(num_tiles, max_dim_x); + for (int start_tile = 0; start_tile < num_tiles; start_tile += scan_grid_size) + { + // Log scan_kernel configuration + if (debug_synchronous) _CubLog("Invoking %d scan_kernel<<<%d, %d, 0, %lld>>>(), %d items per thread, %d SM occupancy\n", + start_tile, scan_grid_size, Policy::BLOCK_THREADS, (long long) stream, Policy::ITEMS_PER_THREAD, scan_sm_occupancy); + + // Invoke scan_kernel + THRUST_NS_QUALIFIER::cuda_cub::launcher::triple_chevron( + scan_grid_size, Policy::BLOCK_THREADS, 0, stream + ).doit( + scan_kernel, + d_keys_in, + d_values_in, + d_values_out, + tile_state, + start_tile, + equality_op, + scan_op, + init_value, + num_items); + + // Check for failure to launch + if (CubDebug(error = cudaPeekAtLastError())) break; + + // Sync the stream if specified to flush runtime errors + if (debug_synchronous && (CubDebug(error = SyncStream(stream)))) break; + } + } + while (0); + + return error; + +#endif // CUB_RUNTIME_ENABLED + } + + template + CUB_RUNTIME_FUNCTION __host__ __forceinline__ + cudaError_t Invoke() + { + typedef typename DispatchScanByKey::MaxPolicy MaxPolicyT; + typedef ReduceByKeyScanTileState ScanByKeyTileStateT; + // Ensure kernels are instantiated. + return Invoke( + DeviceScanInitKernel, + DeviceScanByKeyKernel< + MaxPolicyT, KeysInputIteratorT, ValuesInputIteratorT, ValuesOutputIteratorT, + ScanByKeyTileStateT, EqualityOp, ScanOpT, InitValueT, OffsetT> + ); + } + + + /** + * Internal dispatch routine + */ + CUB_RUNTIME_FUNCTION __forceinline__ + static cudaError_t Dispatch( + void* d_temp_storage, ///< [in] %Device-accessible allocation of temporary storage. When NULL, the required allocation size is written to \p temp_storage_bytes and no work is done. + size_t& temp_storage_bytes, ///< [in,out] Reference to size in bytes of \p d_temp_storage allocation + KeysInputIteratorT d_keys_in, ///< [in] Iterator to the input sequence of key items + ValuesInputIteratorT d_values_in, ///< [in] Iterator to the input sequence of value items + ValuesOutputIteratorT d_values_out, ///< [out] Iterator to the input sequence of value items + EqualityOp equality_op, ///< [in]Binary equality functor + ScanOpT scan_op, ///< [in] Binary scan functor + InitValueT init_value, ///< [in] Initial value to seed the exclusive scan + OffsetT num_items, ///< [in] Total number of input items (i.e., the length of \p d_in) + cudaStream_t stream, ///< [in] [optional] CUDA stream to launch kernels within. Default is stream0. + bool debug_synchronous) + { + typedef typename DispatchScanByKey::MaxPolicy MaxPolicyT; + + cudaError_t error; + do + { + // Get PTX version + int ptx_version = 0; + if (CubDebug(error = PtxVersion(ptx_version))) break; + + // Create dispatch functor + DispatchScanByKey dispatch( + d_temp_storage, + temp_storage_bytes, + d_keys_in, + d_values_in, + d_values_out, + equality_op, + scan_op, + init_value, + num_items, + stream, + debug_synchronous, + ptx_version + ); + // Dispatch to chained policy + if (CubDebug(error = MaxPolicyT::Invoke(ptx_version, dispatch))) break; + } + while (0); + + return error; + } +}; + + +CUB_NAMESPACE_END diff --git a/cub/util_math.cuh b/cub/util_math.cuh index 27a0d43622..3c5406fa46 100644 --- a/cub/util_math.cuh +++ b/cub/util_math.cuh @@ -67,6 +67,15 @@ DivideAndRoundUp(NumeratorT n, DenominatorT d) return static_cast(n / d + (n % d != 0 ? 1 : 0)); } +constexpr __device__ __host__ int +Nominal4BItemsToItemsCombined(int nominal_4b_items_per_thread, int combined_bytes) +{ + return (cub::min)(nominal_4b_items_per_thread, + (cub::max)(1, + nominal_4b_items_per_thread * 8 / + combined_bytes)); +} + template constexpr __device__ __host__ int Nominal4BItemsToItems(int nominal_4b_items_per_thread) diff --git a/test/test_device_scan_by_key.cu b/test/test_device_scan_by_key.cu new file mode 100644 index 0000000000..cb91814647 --- /dev/null +++ b/test/test_device_scan_by_key.cu @@ -0,0 +1,1244 @@ +/****************************************************************************** + * Copyright (c) 2021 NVIDIA CORPORATION. All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * * Redistributions of source code must retain the above copyright + * notice, this list of conditions and the following disclaimer. + * * Redistributions in binary form must reproduce the above copyright + * notice, this list of conditions and the following disclaimer in the + * documentation and/or other materials provided with the distribution. + * * Neither the name of the NVIDIA CORPORATION nor the + * names of its contributors may be used to endorse or promote products + * derived from this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND + * ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED + * WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL NVIDIA CORPORATION BE LIABLE FOR ANY + * DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES + * (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; + * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND + * ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS + * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + ******************************************************************************/ + +/****************************************************************************** + * Test of DeviceScan utilities + ******************************************************************************/ + +// Ensure printing of CUDA runtime errors to console +#define CUB_STDERR + +#include +#include + +#include +#include + +#include +#include +#include +#include +#include +#include + +#include "test_util.h" + +using namespace cub; + + +//--------------------------------------------------------------------- +// Globals, constants and typedefs +//--------------------------------------------------------------------- + +bool g_verbose = false; +int g_timing_iterations = 0; +int g_repeat = 0; +double g_device_giga_bandwidth; +CachingDeviceAllocator g_allocator(true); + +// Dispatch types +enum Backend +{ + CUB, // CUB method + THRUST, // Thrust method + CDP, // GPU-based (dynamic parallelism) dispatch to CUB method +}; + + +/** + * \brief WrapperFunctor (for precluding test-specialized dispatch to *Sum variants) + */ +template +struct WrapperFunctor +{ + OpT op; + + WrapperFunctor(OpT op) : op(op) {} + + template + __host__ __device__ __forceinline__ T operator()(const T &a, const T &b) const + { + return op(a, b); + } +}; + +/** + * \brief DivideByFiveFunctor (used by TestIterator) + */ +template +struct DivideByFiveFunctor +{ + template + __host__ __device__ __forceinline__ OutputT operator()(const T &a) const + { + return static_cast(a / 5); + } +}; + +/** + * \brief Mod2Equality (used for non-bool keys to make keys more likely to equal each other) + */ +struct Mod2Equality +{ + template + __host__ __device__ __forceinline__ T operator()(const T &a, const T &b) const + { + return (a % 2) == (b % 2); + } +}; + + +//--------------------------------------------------------------------- +// Dispatch to different CUB DeviceScan entrypoints +//--------------------------------------------------------------------- + +/** + * Dispatch to exclusive scan entrypoint + */ +template +CUB_RUNTIME_FUNCTION __forceinline__ +cudaError_t Dispatch( + Int2Type /*dispatch_to*/, + IsPrimitiveT /*is_primitive*/, + int timing_timing_iterations, + size_t */*d_temp_storage_bytes*/, + cudaError_t */*d_cdp_error*/, + + void* d_temp_storage, + size_t& temp_storage_bytes, + KeysInputIteratorT d_keys_in, + ValuesInputIteratorT d_values_in, + ValuesOutputIteratorT d_values_out, + ScanOpT scan_op, + InitialValueT initial_value, + OffsetT num_items, + EqualityOpT equality_op, + cudaStream_t stream, + bool debug_synchronous) +{ + cudaError_t error = cudaSuccess; + for (int i = 0; i < timing_timing_iterations; ++i) + { + error = DeviceScan::ExclusiveScanByKey(d_temp_storage, temp_storage_bytes, d_keys_in, d_values_in, d_values_out, scan_op, initial_value, num_items, equality_op, stream, debug_synchronous); + } + return error; +} + + +/** + * Dispatch to exclusive sum entrypoint + */ +template +CUB_RUNTIME_FUNCTION __forceinline__ +cudaError_t Dispatch( + Int2Type /*dispatch_to*/, + Int2Type /*is_primitive*/, + int timing_timing_iterations, + size_t */*d_temp_storage_bytes*/, + cudaError_t */*d_cdp_error*/, + + void* d_temp_storage, + size_t& temp_storage_bytes, + KeysInputIteratorT d_keys_in, + ValuesInputIteratorT d_values_in, + ValuesOutputIteratorT d_values_out, + Sum /*scan_op*/, + InitialValueT /*initial_value*/, + OffsetT num_items, + EqualityOpT equality_op, + cudaStream_t stream, + bool debug_synchronous) +{ + cudaError_t error = cudaSuccess; + for (int i = 0; i < timing_timing_iterations; ++i) + { + error = DeviceScan::ExclusiveSumByKey(d_temp_storage, temp_storage_bytes, d_keys_in, d_values_in, d_values_out, num_items, equality_op, stream, debug_synchronous); + } + return error; +} + + +/** + * Dispatch to inclusive scan entrypoint + */ +template +CUB_RUNTIME_FUNCTION __forceinline__ +cudaError_t Dispatch( + Int2Type /*dispatch_to*/, + IsPrimitiveT /*is_primitive*/, + int timing_timing_iterations, + size_t */*d_temp_storage_bytes*/, + cudaError_t */*d_cdp_error*/, + + void* d_temp_storage, + size_t& temp_storage_bytes, + KeysInputIteratorT d_keys_in, + ValuesInputIteratorT d_values_in, + ValuesOutputIteratorT d_values_out, + ScanOpT scan_op, + NullType /*initial_value*/, + OffsetT num_items, + EqualityOpT equality_op, + cudaStream_t stream, + bool debug_synchronous) +{ + cudaError_t error = cudaSuccess; + for (int i = 0; i < timing_timing_iterations; ++i) + { + error = DeviceScan::InclusiveScanByKey(d_temp_storage, temp_storage_bytes, d_keys_in, d_values_in, d_values_out, scan_op, num_items, equality_op, stream, debug_synchronous); + } + return error; +} + + +/** + * Dispatch to inclusive sum entrypoint + */ +template +CUB_RUNTIME_FUNCTION __forceinline__ +cudaError_t Dispatch( + Int2Type /*dispatch_to*/, + Int2Type /*is_primitive*/, + int timing_timing_iterations, + size_t */*d_temp_storage_bytes*/, + cudaError_t */*d_cdp_error*/, + + void* d_temp_storage, + size_t& temp_storage_bytes, + KeysInputIteratorT d_keys_in, + ValuesInputIteratorT d_values_in, + ValuesOutputIteratorT d_values_out, + Sum /*scan_op*/, + NullType /*initial_value*/, + OffsetT num_items, + EqualityOpT equality_op, + cudaStream_t stream, + bool debug_synchronous) +{ + cudaError_t error = cudaSuccess; + for (int i = 0; i < timing_timing_iterations; ++i) + { + error = DeviceScan::InclusiveSumByKey(d_temp_storage, temp_storage_bytes, d_keys_in, d_values_in, d_values_out, num_items, equality_op, stream, debug_synchronous); + } + return error; +} + +//--------------------------------------------------------------------- +// Dispatch to different Thrust entrypoints +//--------------------------------------------------------------------- + +/** + * Dispatch to exclusive scan entrypoint + */ +template +cudaError_t Dispatch( + Int2Type /*dispatch_to*/, + IsPrimitiveT /*is_primitive*/, + int timing_timing_iterations, + size_t */*d_temp_storage_bytes*/, + cudaError_t */*d_cdp_error*/, + + void* d_temp_storage, + size_t& temp_storage_bytes, + KeysInputIteratorT d_keys_in, + ValuesInputIteratorT d_values_in, + ValuesOutputIteratorT d_values_out, + ScanOpT scan_op, + InitialValueT initial_value, + OffsetT num_items, + EqualityOpT equality_op, + cudaStream_t /*stream*/, + bool /*debug_synchronous*/) +{ + // The input key type + typedef typename std::iterator_traits::value_type KeyT; + + // The input value type + typedef typename std::iterator_traits::value_type InputT; + + // The output value type + typedef typename If<(Equals::value_type, void>::VALUE), // OutputT = (if output iterator's value type is void) ? + typename std::iterator_traits::value_type, // ... then the input iterator's value type, + typename std::iterator_traits::value_type>::Type OutputT; // ... else the output iterator's value type + + if (d_temp_storage == 0) + { + temp_storage_bytes = 1; + } + else + { + THRUST_NS_QUALIFIER::device_ptr d_keys_in_wrapper(d_keys_in); + THRUST_NS_QUALIFIER::device_ptr d_values_in_wrapper(d_values_in); + THRUST_NS_QUALIFIER::device_ptr d_values_out_wrapper(d_values_out); + for (int i = 0; i < timing_timing_iterations; ++i) + { + THRUST_NS_QUALIFIER::exclusive_scan_by_key(d_keys_in_wrapper, d_keys_in_wrapper + num_items, d_values_in_wrapper, d_values_out_wrapper, initial_value, equality_op, scan_op); + } + } + + return cudaSuccess; +} + + +/** + * Dispatch to exclusive sum entrypoint + */ +template +cudaError_t Dispatch( + Int2Type /*dispatch_to*/, + Int2Type /*is_primitive*/, + int timing_timing_iterations, + size_t */*d_temp_storage_bytes*/, + cudaError_t */*d_cdp_error*/, + + void* d_temp_storage, + size_t& temp_storage_bytes, + KeysInputIteratorT d_keys_in, + ValuesInputIteratorT d_values_in, + ValuesOutputIteratorT d_values_out, + Sum /*scan_op*/, + InitialValueT /*initial_value*/, + OffsetT num_items, + EqualityOpT /*equality_op*/, + cudaStream_t /*stream*/, + bool /*debug_synchronous*/) +{ + // The input key type + typedef typename std::iterator_traits::value_type KeyT; + + // The input value type + typedef typename std::iterator_traits::value_type InputT; + + // The output value type + typedef typename If<(Equals::value_type, void>::VALUE), // OutputT = (if output iterator's value type is void) ? + typename std::iterator_traits::value_type, // ... then the input iterator's value type, + typename std::iterator_traits::value_type>::Type OutputT; // ... else the output iterator's value type + + if (d_temp_storage == 0) + { + temp_storage_bytes = 1; + } + else + { + THRUST_NS_QUALIFIER::device_ptr d_keys_in_wrapper(d_keys_in); + THRUST_NS_QUALIFIER::device_ptr d_values_in_wrapper(d_values_in); + THRUST_NS_QUALIFIER::device_ptr d_values_out_wrapper(d_values_out); + for (int i = 0; i < timing_timing_iterations; ++i) + { + THRUST_NS_QUALIFIER::exclusive_scan_by_key(d_keys_in_wrapper, d_keys_in_wrapper + num_items, d_values_in_wrapper, d_values_out_wrapper); + } + } + + return cudaSuccess; +} + + +/** + * Dispatch to inclusive scan entrypoint + */ +template +cudaError_t Dispatch( + Int2Type /*dispatch_to*/, + IsPrimitiveT /*is_primitive*/, + int timing_timing_iterations, + size_t */*d_temp_storage_bytes*/, + cudaError_t */*d_cdp_error*/, + + void* d_temp_storage, + size_t& temp_storage_bytes, + KeysInputIteratorT d_keys_in, + ValuesInputIteratorT d_values_in, + ValuesOutputIteratorT d_values_out, + ScanOpT scan_op, + NullType /*initial_value*/, + OffsetT num_items, + EqualityOpT equality_op, + cudaStream_t /*stream*/, + bool /*debug_synchronous*/) +{ + // The input key type + typedef typename std::iterator_traits::value_type KeyT; + + // The input value type + typedef typename std::iterator_traits::value_type InputT; + + // The output value type + typedef typename If<(Equals::value_type, void>::VALUE), // OutputT = (if output iterator's value type is void) ? + typename std::iterator_traits::value_type, // ... then the input iterator's value type, + typename std::iterator_traits::value_type>::Type OutputT; // ... else the output iterator's value type + + if (d_temp_storage == 0) + { + temp_storage_bytes = 1; + } + else + { + THRUST_NS_QUALIFIER::device_ptr d_keys_in_wrapper(d_keys_in); + THRUST_NS_QUALIFIER::device_ptr d_values_in_wrapper(d_values_in); + THRUST_NS_QUALIFIER::device_ptr d_values_out_wrapper(d_values_out); + for (int i = 0; i < timing_timing_iterations; ++i) + { + THRUST_NS_QUALIFIER::inclusive_scan(d_keys_in_wrapper, d_keys_in_wrapper + num_items, d_values_in_wrapper, d_values_out_wrapper, scan_op); + } + } + + return cudaSuccess; +} + +template +cudaError_t Dispatch( + Int2Type /*dispatch_to*/, + Int2Type /*is_primitive*/, + int timing_timing_iterations, + size_t */*d_temp_storage_bytes*/, + cudaError_t */*d_cdp_error*/, + + void* d_temp_storage, + size_t& temp_storage_bytes, + KeysInputIteratorT d_keys_in, + ValuesInputIteratorT d_values_in, + ValuesOutputIteratorT d_values_out, + Sum /*scan_op*/, + NullType /*initial_value*/, + OffsetT num_items, + EqualityOpT equality_op, + cudaStream_t /*stream*/, + bool /*debug_synchronous*/) +{ + // The input key type + typedef typename std::iterator_traits::value_type KeyT; + + // The input value type + typedef typename std::iterator_traits::value_type InputT; + + // The output value type + typedef typename If<(Equals::value_type, void>::VALUE), // OutputT = (if output iterator's value type is void) ? + typename std::iterator_traits::value_type, // ... then the input iterator's value type, + typename std::iterator_traits::value_type>::Type OutputT; // ... else the output iterator's value type + + if (d_temp_storage == 0) + { + temp_storage_bytes = 1; + } + else + { + THRUST_NS_QUALIFIER::device_ptr d_keys_in_wrapper(d_keys_in); + THRUST_NS_QUALIFIER::device_ptr d_values_in_wrapper(d_values_in); + THRUST_NS_QUALIFIER::device_ptr d_values_out_wrapper(d_values_out); + for (int i = 0; i < timing_timing_iterations; ++i) + { + THRUST_NS_QUALIFIER::inclusive_scan(d_keys_in_wrapper, d_keys_in_wrapper + num_items, d_values_in_wrapper, d_values_out_wrapper); + } + } + + return cudaSuccess; +} + + + +//--------------------------------------------------------------------- +// CUDA Nested Parallelism Test Kernel +//--------------------------------------------------------------------- + +/** + * Simple wrapper kernel to invoke DeviceScan + */ +template +__global__ void CnpDispatchKernel( + IsPrimitiveT is_primitive, + int timing_timing_iterations, + size_t *d_temp_storage_bytes, + cudaError_t *d_cdp_error, + + void* d_temp_storage, + size_t temp_storage_bytes, + KeysInputIteratorT d_keys_in, + ValuesInputIteratorT d_values_in, + ValuesOutputIteratorT d_values_out, + ScanOpT scan_op, + InitialValueT initial_value, + OffsetT num_items, + EqualityOpT equality_op, + bool debug_synchronous) +{ +#ifndef CUB_CDP + (void)is_primitive; + (void)timing_timing_iterations; + (void)d_temp_storage_bytes; + (void)d_cdp_error; + (void)d_temp_storage; + (void)temp_storage_bytes; + (void)d_keys_in; + (void)d_values_in; + (void)d_values_out; + (void)scan_op; + (void)initial_value; + (void)num_items; + (void)equality_op; + (void)debug_synchronous; + *d_cdp_error = cudaErrorNotSupported; +#else + *d_cdp_error = Dispatch( + Int2Type(), + is_primitive, + timing_timing_iterations, + d_temp_storage_bytes, + d_cdp_error, + d_temp_storage, + temp_storage_bytes, + d_in, + d_out, + scan_op, + initial_value, + num_items, + 0, + debug_synchronous); + + *d_temp_storage_bytes = temp_storage_bytes; +#endif +} + + +/** + * Dispatch to CDP kernel + */ +template +cudaError_t Dispatch( + Int2Type dispatch_to, + IsPrimitiveT is_primitive, + int timing_timing_iterations, + size_t *d_temp_storage_bytes, + cudaError_t *d_cdp_error, + + void* d_temp_storage, + size_t& temp_storage_bytes, + KeysInputIteratorT d_keys_in, + ValuesInputIteratorT d_values_in, + ValuesOutputIteratorT d_values_out, + ScanOpT scan_op, + InitialValueT initial_value, + OffsetT num_items, + EqualityOpT equality_op, + cudaStream_t stream, + bool debug_synchronous) +{ + // Invoke kernel to invoke device-side dispatch + CnpDispatchKernel<<<1,1>>>( + is_primitive, + timing_timing_iterations, + d_temp_storage_bytes, + d_cdp_error, + d_temp_storage, + temp_storage_bytes, + d_keys_in, + d_values_in, + d_values_out, + scan_op, + initial_value, + equality_op, + num_items, + debug_synchronous); + + // Copy out temp_storage_bytes + CubDebugExit(cudaMemcpy(&temp_storage_bytes, d_temp_storage_bytes, sizeof(size_t) * 1, cudaMemcpyDeviceToHost)); + + // Copy out error + cudaError_t retval; + CubDebugExit(cudaMemcpy(&retval, d_cdp_error, sizeof(cudaError_t) * 1, cudaMemcpyDeviceToHost)); + return retval; +} + + +//--------------------------------------------------------------------- +// Test generation +//--------------------------------------------------------------------- + + +/** + * Initialize problem + */ +template +void Initialize( + GenMode gen_mode, + T *h_in, + int num_items) +{ + for (int i = 0; i < num_items; ++i) + { + InitValue(gen_mode, h_in[i], i); + } + + if (g_verbose) + { + printf("Input:\n"); + DisplayResults(h_in, num_items); + printf("\n\n"); + } +} + +/** + * Solve exclusive-scan problem + */ +template < + typename KeysInputIteratorT, + typename ValuesInputIteratorT, + typename OutputT, + typename ScanOpT, + typename InitialValueT, + typename EqualityOpT> +void Solve( + KeysInputIteratorT h_keys_in, + ValuesInputIteratorT h_values_in, + OutputT *h_reference, + int num_items, + ScanOpT scan_op, + InitialValueT initial_value, + EqualityOpT equality_op) +{ + // Use the initial value type for accumulation per P0571 + using AccumT = InitialValueT; + + if (num_items > 0) + { + for (int i = 0; i < num_items;) { + AccumT val = static_cast(h_values_in[i]); + h_reference[i] = initial_value; + AccumT inclusive = scan_op(initial_value, val); + + ++i; + + for (; i < num_items && equality_op(h_keys_in[i - 1], h_keys_in[i]); ++i) + { + val = static_cast(h_values_in[i]); + h_reference[i] = static_cast(inclusive); + inclusive = scan_op(inclusive, val); + } + } + } +} + + +/** + * Solve inclusive-scan problem + */ +template < + typename KeysInputIteratorT, + typename ValuesInputIteratorT, + typename OutputT, + typename ScanOpT, + typename EqualityOpT> +void Solve( + KeysInputIteratorT h_keys_in, + ValuesInputIteratorT h_values_in, + OutputT *h_reference, + int num_items, + ScanOpT scan_op, + NullType /*initial_value*/, + EqualityOpT equality_op) +{ + // When no initial value type is supplied, use InputT for accumulation + // per P0571 + using AccumT = typename std::iterator_traits::value_type; + + if (num_items > 0) + { + for (int i = 0; i < num_items;) { + AccumT inclusive = h_values_in[i]; + h_reference[i] = static_cast(inclusive); + + ++i; + + for (; i < num_items && equality_op(h_keys_in[i - 1], h_keys_in[i]); ++i) + { + AccumT val = h_values_in[i]; + inclusive = scan_op(inclusive, val); + h_reference[i] = static_cast(inclusive); + } + } + } +} + +template +struct AllocateOutput { + static void run(OutputT *&d_out, DeviceInputIteratorT, int num_items) { + CubDebugExit(g_allocator.DeviceAllocate((void**)&d_out, sizeof(OutputT) * num_items)); + } +}; + +template +struct AllocateOutput { + static void run(OutputT *&d_out, OutputT *d_in, int num_items) { + d_out = d_in; + } +}; + +/** + * Test DeviceScan for a given problem input + */ +template < + Backend BACKEND, + typename KeysInputIteratorT, + typename ValuesInputIteratorT, + typename OutputT, + typename ScanOpT, + typename InitialValueT, + typename EqualityOpT, + bool InPlace=false> +void Test( + KeysInputIteratorT d_keys_in, + ValuesInputIteratorT d_values_in, + OutputT *h_reference, + int num_items, + ScanOpT scan_op, + InitialValueT initial_value, + EqualityOpT equality_op) +{ + typedef typename std::iterator_traits::value_type KeyT; + typedef typename std::iterator_traits::value_type InputT; + + // Allocate device output array + OutputT *d_values_out = NULL; + AllocateOutput::run(d_values_out, d_values_in, num_items); + + // Allocate CDP device arrays + size_t *d_temp_storage_bytes = NULL; + cudaError_t *d_cdp_error = NULL; + CubDebugExit(g_allocator.DeviceAllocate((void**)&d_temp_storage_bytes, sizeof(size_t) * 1)); + CubDebugExit(g_allocator.DeviceAllocate((void**)&d_cdp_error, sizeof(cudaError_t) * 1)); + + // Allocate temporary storage + void *d_temp_storage = NULL; + size_t temp_storage_bytes = 0; + CubDebugExit(Dispatch( + Int2Type(), + Int2Type::PRIMITIVE>(), + 1, + d_temp_storage_bytes, + d_cdp_error, + d_temp_storage, + temp_storage_bytes, + d_keys_in, + d_values_in, + d_values_out, + scan_op, + initial_value, + num_items, + equality_op, + 0, + true)); + CubDebugExit(g_allocator.DeviceAllocate(&d_temp_storage, temp_storage_bytes)); + + // Clear device output array + CubDebugExit(cudaMemset(d_values_out, 0, sizeof(OutputT) * num_items)); + + // Run warmup/correctness iteration + CubDebugExit(Dispatch( + Int2Type(), + Int2Type::PRIMITIVE>(), + 1, + d_temp_storage_bytes, + d_cdp_error, + d_temp_storage, + temp_storage_bytes, + d_keys_in, + d_values_in, + d_values_out, + scan_op, + initial_value, + num_items, + equality_op, + 0, + true)); + + // Check for correctness (and display results, if specified) + int compare = CompareDeviceResults(h_reference, d_values_out, num_items, true, g_verbose); + printf("\t%s", compare ? "FAIL" : "PASS"); + + // Flush any stdout/stderr + fflush(stdout); + fflush(stderr); + + // Performance + GpuTimer gpu_timer; + gpu_timer.Start(); + CubDebugExit(Dispatch(Int2Type(), + Int2Type::PRIMITIVE>(), + g_timing_iterations, + d_temp_storage_bytes, + d_cdp_error, + d_temp_storage, + temp_storage_bytes, + d_keys_in, + d_values_in, + d_values_out, + scan_op, + initial_value, + num_items, + equality_op, + 0, + false)); + gpu_timer.Stop(); + float elapsed_millis = gpu_timer.ElapsedMillis(); + + // Display performance + if (g_timing_iterations > 0) + { + float avg_millis = elapsed_millis / g_timing_iterations; + float giga_rate = float(num_items) / avg_millis / 1000.0f / 1000.0f; + float giga_bandwidth = giga_rate * (sizeof(InputT) + sizeof(OutputT)); + printf(", %.3f avg ms, %.3f billion items/s, %.3f logical GB/s, %.1f%% peak", + avg_millis, giga_rate, giga_bandwidth, giga_bandwidth / g_device_giga_bandwidth * 100.0); + } + + printf("\n\n"); + + // Cleanup + if (d_values_out) CubDebugExit(g_allocator.DeviceFree(d_values_out)); + if (d_temp_storage_bytes) CubDebugExit(g_allocator.DeviceFree(d_temp_storage_bytes)); + if (d_cdp_error) CubDebugExit(g_allocator.DeviceFree(d_cdp_error)); + if (d_temp_storage) CubDebugExit(g_allocator.DeviceFree(d_temp_storage)); + + // Correctness asserts + AssertEquals(0, compare); +} + +template < + Backend BACKEND, + typename KeysInputIteratorT, + typename ValuesInputIteratorT, + typename OutputT, + typename ScanOpT, + typename InitialValueT, + typename EqualityOpT> +auto TestInplace( + KeysInputIteratorT d_keys_in, + ValuesInputIteratorT d_values_in, + OutputT *h_reference, + int num_items, + ScanOpT scan_op, + InitialValueT initial_value, + EqualityOpT equality_op) -> typename std::enable_if::value>::type +{ + Test(d_keys_in, d_values_in, h_reference, num_items, scan_op, initial_value, equality_op); +} + +template < + Backend BACKEND, + typename KeysInputIteratorT, + typename ValuesInputIteratorT, + typename OutputT, + typename ScanOpT, + typename InitialValueT, + typename EqualityOpT> +auto TestInplace( + KeysInputIteratorT, + ValuesInputIteratorT d_values_in, + OutputT *, + int, + ScanOpT, + InitialValueT, + EqualityOpT) -> typename std::enable_if::value>::type +{ + (void)d_values_in; +} + +/** + * Test DeviceScan on pointer type + */ +template < + Backend BACKEND, + typename KeyT, + typename InputT, + typename OutputT, + typename ScanOpT, + typename InitialValueT, + typename EqualityOpT> +void TestPointer( + int num_items, + GenMode gen_mode, + ScanOpT scan_op, + InitialValueT initial_value, + EqualityOpT equality_op) +{ + printf("\nPointer %s %s cub::DeviceScan::%s %d items, %s->%s (%d->%d bytes) , gen-mode %s\n", + (BACKEND == CDP) ? "CDP CUB" : (BACKEND == THRUST) ? "Thrust" : "CUB", + (Equals::VALUE) ? "Inclusive" : "Exclusive", + (Equals::VALUE) ? "Sum" : "Scan", + num_items, + typeid(InputT).name(), typeid(OutputT).name(), (int) sizeof(InputT), (int) sizeof(OutputT), + (gen_mode == RANDOM) ? "RANDOM" : (gen_mode == INTEGER_SEED) ? "SEQUENTIAL" : "HOMOGENOUS"); + fflush(stdout); + + // Allocate host arrays + KeyT* h_keys_in = new KeyT[num_items]; + InputT* h_values_in = new InputT[num_items]; + OutputT* h_reference = new OutputT[num_items]; + + // Initialize problem and solution + Initialize(gen_mode, h_keys_in, num_items); + Initialize(gen_mode, h_values_in, num_items); + + // If the output type is primitive and the operator is cub::Sum, the test + // dispatcher throws away scan_op and initial_value for exclusive scan. + // Without an initial_value arg, the accumulator switches to the input value + // type. + // Do the same thing here: + if (Traits::PRIMITIVE && + Equals::VALUE && + !Equals::VALUE) + { + if (BACKEND == THRUST) { + Solve(h_keys_in, h_values_in, h_reference, num_items, cub::Sum{}, InputT{}, Equality{}); + } else { + Solve(h_keys_in, h_values_in, h_reference, num_items, cub::Sum{}, InputT{}, equality_op); + } + } + else + { + Solve(h_keys_in, h_values_in, h_reference, num_items, scan_op, initial_value, equality_op); + } + + // Allocate problem device arrays + KeyT *d_keys_in = NULL; + CubDebugExit(g_allocator.DeviceAllocate((void**)&d_keys_in, sizeof(KeyT) * num_items)); + InputT *d_values_in = NULL; + CubDebugExit(g_allocator.DeviceAllocate((void**)&d_values_in, sizeof(InputT) * num_items)); + + // Initialize device input + CubDebugExit(cudaMemcpy(d_keys_in, h_keys_in, sizeof(KeyT) * num_items, cudaMemcpyHostToDevice)); + CubDebugExit(cudaMemcpy(d_values_in, h_values_in, sizeof(InputT) * num_items, cudaMemcpyHostToDevice)); + + // Run Test + Test(d_keys_in, d_values_in, h_reference, num_items, scan_op, initial_value, equality_op); + TestInplace(d_keys_in, d_values_in, h_reference, num_items, scan_op, initial_value, equality_op); + + // Cleanup + if (h_keys_in) delete[] h_keys_in; + if (h_values_in) delete[] h_values_in; + if (h_reference) delete[] h_reference; + if (d_keys_in) CubDebugExit(g_allocator.DeviceFree(d_keys_in)); + if (d_values_in) CubDebugExit(g_allocator.DeviceFree(d_values_in)); +} + + +/** + * Test DeviceScan on iterator type + */ +template < + Backend BACKEND, + typename KeyT, + typename InputT, + typename OutputT, + typename ScanOpT, + typename InitialValueT, + typename EqualityOpT> +void TestIterator( + int num_items, + ScanOpT scan_op, + InitialValueT initial_value, + EqualityOpT equality_op) +{ + printf("\nIterator %s %s cub::DeviceScan::%s %d items, %s->%s (%d->%d bytes)\n", + (BACKEND == CDP) ? "CDP CUB" : (BACKEND == THRUST) ? "Thrust" : "CUB", + (Equals::VALUE) ? "Inclusive" : "Exclusive", + (Equals::VALUE) ? "Sum" : "Scan", + num_items, + typeid(InputT).name(), typeid(OutputT).name(), (int) sizeof(InputT), (int) sizeof(OutputT)); + fflush(stdout); + + // Use a counting iterator followed by div as the keys + using CountingIterT = CountingInputIterator; + CountingIterT h_keys_in_helper(0); + TransformInputIterator, CountingIterT> h_keys_in(h_keys_in_helper, DivideByFiveFunctor()); + + // Use a constant iterator as the input + InputT val = InputT(); + ConstantInputIterator h_values_in(val); + + // Allocate host arrays + OutputT* h_reference = new OutputT[num_items]; + + // Initialize problem and solution + Solve(h_keys_in, h_values_in, h_reference, num_items, scan_op, initial_value, equality_op); + + // Run Test + Test(h_keys_in, h_values_in, h_reference, num_items, scan_op, initial_value, equality_op); + + // Cleanup + if (h_reference) delete[] h_reference; +} + + +/** + * Test different gen modes + */ +template < + Backend BACKEND, + typename KeyT, + typename InputT, + typename OutputT, + typename ScanOpT, + typename InitialValueT, + typename EqualityOpT> +void Test( + int num_items, + ScanOpT scan_op, + InitialValueT initial_value, + EqualityOpT equality_op) +{ + TestPointer( num_items, UNIFORM, scan_op, initial_value, equality_op); + TestPointer( num_items, RANDOM, scan_op, initial_value, equality_op); + TestIterator( num_items, scan_op, initial_value, equality_op); +} + + +/** + * Test different dispatch + */ +template < + typename KeyT, + typename InputT, + typename OutputT, + typename ScanOpT, + typename InitialValueT, + typename EqualityOpT> +void Test( + int num_items, + ScanOpT scan_op, + InitialValueT initial_value, + EqualityOpT equality_op) +{ + Test(num_items, scan_op, initial_value, equality_op); +#ifdef CUB_CDP + Test(num_items, scan_op, initial_value, equality_op); +#endif +} + + +/** + * Test different operators + */ +template +void TestOp( + int num_items, + OutputT identity, + OutputT initial_value, + EqualityOpT equality_op) +{ + // Exclusive (use identity as initial value because it will dispatch to *Sum variants that don't take initial values) + Test(num_items, cub::Sum(), identity, equality_op); + Test(num_items, cub::Max(), identity, equality_op); + + // Exclusive (non-specialized, so we can test initial-value) + Test(num_items, WrapperFunctor(cub::Sum()), initial_value, equality_op); + Test(num_items, WrapperFunctor(cub::Max()), initial_value, equality_op); + + // Inclusive (no initial value) + Test(num_items, cub::Sum(), NullType(), equality_op); + Test(num_items, cub::Max(), NullType(), equality_op); +} + +/** + * Test different key type and equality operator + */ +template +void TestKeyTAndEqualityOp( + int num_items, + OutputT identity, + OutputT initial_value) +{ + TestOp(num_items, identity, initial_value, Equality()); + TestOp( num_items, identity, initial_value, Mod2Equality()); +} + +/** + * Test different input sizes + */ +template < + typename InputT, + typename OutputT> +void TestSize( + int num_items, + OutputT identity, + OutputT initial_value) +{ + if (num_items < 0) + { + TestKeyTAndEqualityOp(0, identity, initial_value); + TestKeyTAndEqualityOp(1, identity, initial_value); + TestKeyTAndEqualityOp(100, identity, initial_value); + TestKeyTAndEqualityOp(10000, identity, initial_value); + TestKeyTAndEqualityOp(1000000, identity, initial_value); + + // Randomly select problem size between 1:10,000,000 + unsigned int max_int = (unsigned int) -1; + for (int i = 0; i < 10; ++i) + { + unsigned int num; + RandomBits(num); + num = static_cast((double(num) * double(10000000)) / double(max_int)); + num = CUB_MAX(1, num); + TestKeyTAndEqualityOp(num, identity, initial_value); + } + } + else + { + TestKeyTAndEqualityOp(num_items, identity, initial_value); + } +} + + + +//--------------------------------------------------------------------- +// Main +//--------------------------------------------------------------------- + +/** + * Main + */ +int main(int argc, char** argv) +{ + int num_items = -1; + + // Initialize command line + CommandLineArgs args(argc, argv); + g_verbose = args.CheckCmdLineFlag("v"); + args.GetCmdLineArgument("n", num_items); + args.GetCmdLineArgument("i", g_timing_iterations); + args.GetCmdLineArgument("repeat", g_repeat); + + // Print usage + if (args.CheckCmdLineFlag("help")) + { + printf("%s " + "[--n= " + "[--i= " + "[--device=] " + "[--repeat=]" + "[--v] " + "[--cdp]" + "\n", argv[0]); + exit(0); + } + + // Initialize device + CubDebugExit(args.DeviceInit()); + g_device_giga_bandwidth = args.device_giga_bandwidth; + printf("\n"); + +#ifdef CUB_TEST_MINIMAL + + // Compile/run basic CUB test + if (num_items < 0) num_items = 32000000; + + TestPointer( num_items , RANDOM_BIT, Sum(), (int) (0), Equality()); + TestPointer( num_items , RANDOM_BIT, Sum(), (int) (0), Equality()); + + printf("----------------------------\n"); + + TestPointer( num_items , RANDOM_BIT, Sum(), (int) (0), Equality()); + TestPointer( num_items , RANDOM_BIT, Sum(), (long long) (0), Equality()); + + printf("----------------------------\n"); + + TestPointer( num_items , RANDOM_BIT, Sum(), (float) (0), Equality()); + TestPointer( num_items , RANDOM_BIT, Sum(), (double) (0), Equality()); + + +#elif defined(CUB_TEST_BENCHMARK) + + // Get device ordinal + int device_ordinal; + CubDebugExit(cudaGetDevice(&device_ordinal)); + + // Get device SM version + int sm_version = 0; + CubDebugExit(SmVersion(sm_version, device_ordinal)); + + // Compile/run quick tests + if (num_items < 0) num_items = 32000000; + + TestPointer( num_items * ((sm_version <= 130) ? 1 : 4), UNIFORM, Sum(), char(0), Equality()); + TestPointer( num_items * ((sm_version <= 130) ? 1 : 4), UNIFORM, Sum(), char(0), Equality()); + + printf("----------------------------\n"); + TestPointer( num_items * ((sm_version <= 130) ? 1 : 2), UNIFORM, Sum(), short(0), Equality()); + TestPointer( num_items * ((sm_version <= 130) ? 1 : 2), UNIFORM, Sum(), short(0), Equality()); + + printf("----------------------------\n"); + TestPointer( num_items , UNIFORM, Sum(), (int) (0), Equality()); + TestPointer( num_items , UNIFORM, Sum(), (int) (0), Equality()); + + printf("----------------------------\n"); + TestPointer( num_items / 2, UNIFORM, Sum(), (long long) (0), Equality()); + TestPointer(num_items / 2, UNIFORM, Sum(), (long long) (0), Equality()); + + printf("----------------------------\n"); + TestPointer( num_items / 4, UNIFORM, Sum(), TestBar(), Equality()); + TestPointer( num_items / 4, UNIFORM, Sum(), TestBar(), Equality()); + +#else + // Compile/run thorough tests + for (int i = 0; i <= g_repeat; ++i) + { + // Test different input+output data types + TestSize(num_items, (int) 0, (int) 99); + + // Test same intput+output data types + TestSize(num_items, (unsigned char) 0, (unsigned char) 99); + TestSize(num_items, (char) 0, (char) 99); + TestSize(num_items, (unsigned short) 0, (unsigned short)99); + TestSize(num_items, (unsigned int) 0, (unsigned int) 99); + TestSize(num_items, (unsigned long long) 0, (unsigned long long) 99); + + TestSize(num_items, make_uchar2(0, 0), make_uchar2(17, 21)); + TestSize(num_items, make_char2(0, 0), make_char2(17, 21)); + TestSize(num_items, make_ushort2(0, 0), make_ushort2(17, 21)); + TestSize(num_items, make_uint2(0, 0), make_uint2(17, 21)); + TestSize(num_items, make_ulonglong2(0, 0), make_ulonglong2(17, 21)); + TestSize(num_items, make_uchar4(0, 0, 0, 0), make_uchar4(17, 21, 32, 85)); + TestSize(num_items, make_char4(0, 0, 0, 0), make_char4(17, 21, 32, 85)); + + TestSize(num_items, make_ushort4(0, 0, 0, 0), make_ushort4(17, 21, 32, 85)); + TestSize(num_items, make_uint4(0, 0, 0, 0), make_uint4(17, 21, 32, 85)); + TestSize(num_items, make_ulonglong4(0, 0, 0, 0), make_ulonglong4(17, 21, 32, 85)); + + TestSize(num_items, + TestFoo::MakeTestFoo(0, 0, 0, 0), + TestFoo::MakeTestFoo(1ll << 63, 1 << 31, static_cast(1 << 15), static_cast(1 << 7))); + + TestSize(num_items, + TestBar(0, 0), + TestBar(1ll << 63, 1 << 31)); + } +#endif + + return 0; +} + + +