diff --git a/cub/agent/agent_reduce.cuh b/cub/agent/agent_reduce.cuh index aa18efa07e..1a0ed12e2c 100644 --- a/cub/agent/agent_reduce.cuh +++ b/cub/agent/agent_reduce.cuh @@ -80,6 +80,32 @@ struct AgentReducePolicy : ScalingType static constexpr CacheLoadModifier LOAD_MODIFIER = _LOAD_MODIFIER; }; +template +struct AgentWarpReducePolicy +{ + // TODO MemBoundScaling-like computation + static constexpr int ITEMS_PER_THREAD = NOMINAL_ITEMS_PER_THREAD_4B; + + static constexpr int WARP_THREADS = NOMINAL_WARP_THREADS_4B; + + /// Number of items per vectorized load + static constexpr int VECTOR_LOAD_LENGTH = _VECTOR_LOAD_LENGTH; + + /// Cache load modifier for reading input elements + static constexpr CacheLoadModifier LOAD_MODIFIER = _LOAD_MODIFIER; + + constexpr static int ITEMS_PER_TILE = ITEMS_PER_THREAD * WARP_THREADS; + + constexpr static int SEGMENTS_PER_BLOCK = BLOCK_THREADS / WARP_THREADS; + + static_assert((BLOCK_THREADS % WARP_THREADS) == 0, "Block should be multiple of warp"); +}; + /****************************************************************************** * Thread block abstractions ******************************************************************************/ @@ -116,8 +142,10 @@ template -struct AgentReduce + typename AccumT, + typename CollectiveReduceT, + int THREADS> +struct AgentReduceImpl { //--------------------------------------------------------------------- // Types and constants @@ -139,9 +167,8 @@ struct AgentReduce InputIteratorT>; /// Constants - static constexpr int BLOCK_THREADS = AgentReducePolicy::BLOCK_THREADS; static constexpr int ITEMS_PER_THREAD = AgentReducePolicy::ITEMS_PER_THREAD; - static constexpr int TILE_ITEMS = BLOCK_THREADS * ITEMS_PER_THREAD; + static constexpr int TILE_ITEMS = THREADS * ITEMS_PER_THREAD; static constexpr int VECTOR_LOAD_LENGTH = CUB_MIN(ITEMS_PER_THREAD, AgentReducePolicy::VECTOR_LOAD_LENGTH); @@ -155,17 +182,10 @@ struct AgentReduce static constexpr CacheLoadModifier LOAD_MODIFIER = AgentReducePolicy::LOAD_MODIFIER; - static constexpr BlockReduceAlgorithm BLOCK_ALGORITHM = - AgentReducePolicy::BLOCK_ALGORITHM; - - /// Parameterized BlockReduce primitive - using BlockReduceT = - BlockReduce; - /// Shared memory type required by this thread block struct _TempStorage { - typename BlockReduceT::TempStorage reduce; + typename CollectiveReduceT::TempStorage reduce; }; /// Alias wrapper allowing storage to be unioned @@ -177,9 +197,10 @@ struct AgentReduce //--------------------------------------------------------------------- _TempStorage &temp_storage; ///< Reference to temp_storage - InputIteratorT d_in; ///< Input data to reduce + unsigned int lane_id; WrappedInputIteratorT d_wrapped_in; ///< Wrapped input data to reduce ReductionOp reduction_op; ///< Binary reduction operator + InputIteratorT d_in; ///< Input data to reduce //--------------------------------------------------------------------- // Utility @@ -213,13 +234,15 @@ struct AgentReduce * @param d_in Input data to reduce * @param reduction_op Binary reduction operator */ - __device__ __forceinline__ AgentReduce(TempStorage &temp_storage, - InputIteratorT d_in, - ReductionOp reduction_op) + __device__ __forceinline__ AgentReduceImpl(TempStorage &temp_storage, + InputIteratorT d_in, + ReductionOp reduction_op, + unsigned int lane_id) : temp_storage(temp_storage.Alias()) , d_in(d_in) , d_wrapped_in(d_in) , reduction_op(reduction_op) + , lane_id(lane_id) {} //--------------------------------------------------------------------- @@ -243,9 +266,9 @@ struct AgentReduce AccumT items[ITEMS_PER_THREAD]; // Load items in striped fashion - LoadDirectStriped(threadIdx.x, - d_wrapped_in + block_offset, - items); + LoadDirectStriped(lane_id, + d_wrapped_in + block_offset, + items); // Reduce items within each thread stripe thread_aggregate = @@ -276,7 +299,7 @@ struct AgentReduce // Fabricate a vectorized input iterator InputT *d_in_unqualified = const_cast(d_in) + block_offset + - (threadIdx.x * VECTOR_LOAD_LENGTH); + (lane_id * VECTOR_LOAD_LENGTH); CacheModifiedInputIterator d_vec_in(reinterpret_cast(d_in_unqualified)); @@ -286,7 +309,7 @@ struct AgentReduce #pragma unroll for (int i = 0; i < WORDS; ++i) { - vec_items[i] = d_vec_in[BLOCK_THREADS * i]; + vec_items[i] = d_vec_in[THREADS * i]; } // Convert from input type to output type @@ -320,13 +343,13 @@ struct AgentReduce Int2Type /*can_vectorize*/) { // Partial tile - int thread_offset = threadIdx.x; + int thread_offset = lane_id; // Read first item if ((IS_FIRST_TILE) && (thread_offset < valid_items)) { thread_aggregate = d_wrapped_in[block_offset + thread_offset]; - thread_offset += BLOCK_THREADS; + thread_offset += THREADS; } // Continue reading items (block-striped) @@ -335,7 +358,7 @@ struct AgentReduce InputT item(d_wrapped_in[block_offset + thread_offset]); thread_aggregate = reduction_op(thread_aggregate, item); - thread_offset += BLOCK_THREADS; + thread_offset += THREADS; } } @@ -364,8 +387,12 @@ struct AgentReduce valid_items, Int2Type(), can_vectorize); - return BlockReduceT(temp_storage.reduce) - .Reduce(thread_aggregate, reduction_op, valid_items); + + // TODO Extract clamping into the SFINAE to keep block version as is + int num_valid = (THREADS <= valid_items) ? THREADS : valid_items; + + return CollectiveReduceT(temp_storage.reduce) + .Reduce(thread_aggregate, reduction_op, num_valid); } // At least one full block @@ -399,7 +426,7 @@ struct AgentReduce } // Compute block-wide reduction (all threads have valid items) - return BlockReduceT(temp_storage.reduce) + return CollectiveReduceT(temp_storage.reduce) .Reduce(thread_aggregate, reduction_op); } @@ -440,5 +467,76 @@ struct AgentReduce } }; +template +struct AgentReduce : AgentReduceImpl, + AgentReducePolicy::BLOCK_THREADS> +{ + using base_t = AgentReduceImpl, + AgentReducePolicy::BLOCK_THREADS>; + + __device__ __forceinline__ AgentReduce(typename base_t::TempStorage &temp_storage, + InputIteratorT d_in, + ReductionOp reduction_op) + : base_t(temp_storage, d_in, reduction_op, threadIdx.x) + { + } +}; + +template +struct AgentWarpReduce : AgentReduceImpl, + AgentReducePolicy::WARP_THREADS> +{ + using base_t = AgentReduceImpl, + AgentReducePolicy::WARP_THREADS>; + + __device__ __forceinline__ AgentWarpReduce(typename base_t::TempStorage &temp_storage, + InputIteratorT d_in, + ReductionOp reduction_op, + int lane_id) + : base_t(temp_storage, d_in, reduction_op, lane_id) + { + } +}; + CUB_NAMESPACE_END diff --git a/cub/device/dispatch/dispatch_reduce.cuh b/cub/device/dispatch/dispatch_reduce.cuh index ac434eb862..6c73cd9856 100644 --- a/cub/device/dispatch/dispatch_reduce.cuh +++ b/cub/device/dispatch/dispatch_reduce.cuh @@ -34,11 +34,10 @@ #pragma once -#include -#include - #include #include +#include +#include #include #include #include @@ -47,8 +46,15 @@ #include #include +#include +#include #include +#include + +#include + + CUB_NAMESPACE_BEGIN /****************************************************************************** @@ -352,6 +358,168 @@ __global__ void DeviceSegmentedReduceKernel( } } +template +__launch_bounds__(int(ChainedPolicyT::ActivePolicy::ReducePolicy::BLOCK_THREADS)) +__global__ void DeviceSegmentedReduceWithPartitioningKernel( + InputIteratorT d_in, + OutputIteratorT d_out, + BeginOffsetIteratorT d_begin_offsets, + EndOffsetIteratorT d_end_offsets, + unsigned int large_blocks_end, + unsigned int medium_blocks_end, + unsigned int medium_segments, + unsigned int small_segments, + const unsigned int *d_large_segments_indices, + const unsigned int *d_medium_segments_indices, + const unsigned int *d_small_segments_indices, + ReductionOpT reduction_op, + InitT init) +{ + using ActivePolicyT = typename ChainedPolicyT::ActivePolicy; + + // Thread block type for reducing input tiles + using AgentReduceT = + AgentReduce; + + using AgentMediumReduceT = + AgentWarpReduce; + + using AgentSmallReduceT = + AgentWarpReduce; + + constexpr auto segments_per_medium_block = + static_cast(ActivePolicyT::MediumReducePolicy::SEGMENTS_PER_BLOCK); + constexpr auto medium_threads_per_warp = + static_cast(ActivePolicyT::MediumReducePolicy::WARP_THREADS); + + constexpr auto segments_per_small_block = + static_cast(ActivePolicyT::SmallReducePolicy::SEGMENTS_PER_BLOCK); + constexpr auto small_threads_per_warp = + static_cast(ActivePolicyT::SmallReducePolicy::WARP_THREADS); + + using MediumWarpReduce = cub::WarpReduce; + using SmallWarpReduce = cub::WarpReduce; + + // Shared memory storage + __shared__ union { + typename AgentReduceT::TempStorage large_storage; + typename AgentMediumReduceT::TempStorage medium_storage[segments_per_medium_block]; + typename AgentSmallReduceT::TempStorage small_storage[segments_per_small_block]; + } temp_storage; + + const unsigned int bid = blockIdx.x; + const unsigned int tid = threadIdx.x; + + if (bid < large_blocks_end) + { + const unsigned int global_segment_id = d_large_segments_indices[bid]; + + OffsetT segment_begin = d_begin_offsets[global_segment_id]; + OffsetT segment_end = d_end_offsets[global_segment_id]; + + // Consume input tiles + AccumT block_aggregate = AgentReduceT(temp_storage.large_storage, d_in, reduction_op) + .ConsumeRange(segment_begin, segment_end); + + // Normalize as needed + NormalizeReductionOutput(block_aggregate, segment_begin, d_in); + + if (tid == 0) + { + d_out[global_segment_id] = reduction_op(init, block_aggregate); + } + } + else if (bid < medium_blocks_end) + { + const unsigned int sid_within_block = tid / medium_threads_per_warp; + const unsigned int medium_segment_id = (bid - large_blocks_end) * segments_per_medium_block + + sid_within_block; + + if (medium_segment_id < medium_segments) + { + const unsigned int lane_id = tid % medium_threads_per_warp; + const unsigned int global_segment_id = d_medium_segments_indices[medium_segment_id]; + + OffsetT segment_begin = d_begin_offsets[global_segment_id]; + OffsetT segment_end = d_end_offsets[global_segment_id]; + + // Consume input tiles + AccumT warp_aggregate = + AgentMediumReduceT(temp_storage.medium_storage[sid_within_block], d_in, reduction_op, lane_id) + .ConsumeRange(segment_begin, segment_end); + + // Normalize as needed + NormalizeReductionOutput(warp_aggregate, segment_begin, d_in); + + if (lane_id == 0) + { + d_out[global_segment_id] = reduction_op(init, warp_aggregate); + } + } + } + else + { + const unsigned int sid_within_block = tid / small_threads_per_warp; + const unsigned int small_segment_id = (bid - medium_blocks_end) * segments_per_small_block + + sid_within_block; + + if (small_segment_id < small_segments) + { + const unsigned int lane_id = tid % small_threads_per_warp; + const unsigned int global_segment_id = d_small_segments_indices[small_segment_id]; + + OffsetT segment_begin = d_begin_offsets[global_segment_id]; + OffsetT segment_end = d_end_offsets[global_segment_id]; + + // Check if empty problem + if (segment_begin == segment_end) + { + if (lane_id == 0) + { + d_out[global_segment_id] = init; + } + return; + } + + // Consume input tiles + AccumT warp_aggregate = + AgentSmallReduceT(temp_storage.small_storage[sid_within_block], d_in, reduction_op, lane_id) + .ConsumeRange(segment_begin, segment_end); + + // Normalize as needed + NormalizeReductionOutput(warp_aggregate, segment_begin, d_in); + + if (lane_id == 0) + { + d_out[global_segment_id] = reduction_op(init, warp_aggregate); + } + } + } +} + /****************************************************************************** * Policy ******************************************************************************/ @@ -377,34 +545,13 @@ struct DeviceReducePolicy // Architecture-specific tuning policies //--------------------------------------------------------------------------- - /// SM30 - struct Policy300 : ChainedPolicy<300, Policy300, Policy300> - { - static constexpr int threads_per_block = 256; - static constexpr int items_per_thread = 20; - static constexpr int items_per_vec_load = 2; - - // ReducePolicy (GTX670: 154.0 @ 48M 4B items) - using ReducePolicy = AgentReducePolicy; - - // SingleTilePolicy - using SingleTilePolicy = ReducePolicy; - - // SegmentedReducePolicy - using SegmentedReducePolicy = ReducePolicy; - }; - /// SM35 - struct Policy350 : ChainedPolicy<350, Policy350, Policy300> + struct Policy350 : ChainedPolicy<350, Policy350, Policy350> { static constexpr int threads_per_block = 256; static constexpr int items_per_thread = 20; static constexpr int items_per_vec_load = 4; + static constexpr int partitioning_threshold = 4096; // ReducePolicy (GTX Titan: 255.1 GB/s @ 48M 4B items; 228.7 GB/s @ 192M 1B // items) @@ -420,6 +567,22 @@ struct DeviceReducePolicy // SegmentedReducePolicy using SegmentedReducePolicy = ReducePolicy; + + // TODO Tune + using MediumReducePolicy = AgentWarpReducePolicy; + + // TODO Tune + using SmallReducePolicy = AgentWarpReducePolicy; }; /// SM60 @@ -428,6 +591,7 @@ struct DeviceReducePolicy static constexpr int threads_per_block = 256; static constexpr int items_per_thread = 16; static constexpr int items_per_vec_load = 4; + static constexpr int partitioning_threshold = 4096; // ReducePolicy (P100: 591 GB/s @ 64M 4B items; 583 GB/s @ 256M 1B items) using ReducePolicy = AgentReducePolicy; + + // TODO Tune + using SmallReducePolicy = AgentWarpReducePolicy; }; using MaxPolicy = Policy600; @@ -1003,6 +1183,54 @@ template < typename SelectedPolicy = DeviceReducePolicy> struct DispatchSegmentedReduce : SelectedPolicy { + struct LargeSegmentsSelectorT + { + OffsetT value{}; + BeginOffsetIteratorT d_offset_begin{}; + EndOffsetIteratorT d_offset_end{}; + + __host__ __device__ __forceinline__ + LargeSegmentsSelectorT(OffsetT value, + BeginOffsetIteratorT d_offset_begin, + EndOffsetIteratorT d_offset_end) + : value(value) + , d_offset_begin(d_offset_begin) + , d_offset_end(d_offset_end) + {} + + __host__ __device__ __forceinline__ bool + operator()(unsigned int segment_id) const + { + const OffsetT segment_size = d_offset_end[segment_id] - + d_offset_begin[segment_id]; + return segment_size > value; + } + }; + + struct SmallSegmentsSelectorT + { + OffsetT value{}; + BeginOffsetIteratorT d_offset_begin{}; + EndOffsetIteratorT d_offset_end{}; + + __host__ __device__ __forceinline__ + SmallSegmentsSelectorT(OffsetT value, + BeginOffsetIteratorT d_offset_begin, + EndOffsetIteratorT d_offset_end) + : value(value) + , d_offset_begin(d_offset_begin) + , d_offset_end(d_offset_end) + {} + + __host__ __device__ __forceinline__ bool + operator()(unsigned int segment_id) const + { + const OffsetT segment_size = d_offset_end[segment_id] - + d_offset_begin[segment_id]; + return segment_size < value; + } + }; + //--------------------------------------------------------------------------- // Problem state //--------------------------------------------------------------------------- @@ -1124,62 +1352,234 @@ struct DispatchSegmentedReduce : SelectedPolicy * Kernel function pointer to parameterization of * cub::DeviceSegmentedReduceKernel */ - template + template CUB_RUNTIME_FUNCTION __forceinline__ cudaError_t - InvokePasses(DeviceSegmentedReduceKernelT segmented_reduce_kernel) + InvokePasses(DeviceSegmentedReduceKernelT segmented_reduce_kernel, + DeviceSegmentedReduceWithPartitioningKernelT segmented_reduce_with_partitioning_kernel) { cudaError error = cudaSuccess; do { + cub::detail::temporary_storage::layout<4> temporary_storage_layout; + +#ifndef CUB_RDC_ENABLED + constexpr static int num_selected_groups = 2; + + bool partition_segments = num_segments > ActivePolicyT::partitioning_threshold; + + if (partition_segments) + { + NV_IF_TARGET(NV_IS_HOST, + (cudaStreamCaptureStatus status = cudaStreamCaptureStatusNone; + cudaStreamIsCapturing(stream, &status); + partition_segments = status == cudaStreamCaptureStatusNone;)); + } + + auto partition_storage_slot = temporary_storage_layout.get_slot(0); + auto large_and_medium_partitioning_slot = temporary_storage_layout.get_slot(1); + auto small_partitioning_slot = temporary_storage_layout.get_slot(2); + auto group_sizes_slot = temporary_storage_layout.get_slot(3); + + LargeSegmentsSelectorT large_segments_selector( + ActivePolicyT::MediumReducePolicy::ITEMS_PER_TILE, + d_begin_offsets, + d_end_offsets); + + SmallSegmentsSelectorT small_segments_selector( + ActivePolicyT::SmallReducePolicy::ITEMS_PER_TILE + 1, + d_begin_offsets, + d_end_offsets); + + auto device_partition_temp_storage = + partition_storage_slot->create_alias(); + auto large_and_medium_segments_indices = + large_and_medium_partitioning_slot->create_alias(); + auto small_segments_indices = + small_partitioning_slot->create_alias(); + auto group_sizes = group_sizes_slot->create_alias(); + + std::size_t three_way_partition_temp_storage_bytes {}; + + if (partition_segments) { + large_and_medium_segments_indices.grow(num_segments); + small_segments_indices.grow(num_segments); + group_sizes.grow(num_selected_groups); + + auto medium_indices_iterator = + THRUST_NS_QUALIFIER::make_reverse_iterator( + large_and_medium_segments_indices.get()); + + cub::DevicePartition::If( + nullptr, + three_way_partition_temp_storage_bytes, + THRUST_NS_QUALIFIER::counting_iterator(0), + large_and_medium_segments_indices.get(), + small_segments_indices.get(), + medium_indices_iterator, + group_sizes.get(), + num_segments, + large_segments_selector, + small_segments_selector, + stream); + + device_partition_temp_storage.grow( + three_way_partition_temp_storage_bytes); + } +#endif + // Return if the caller is simply requesting the size of the storage // allocation - if (d_temp_storage == NULL) + if (d_temp_storage == nullptr) { - temp_storage_bytes = 1; + temp_storage_bytes = temporary_storage_layout.get_size(); return cudaSuccess; } - // Init kernel configuration - KernelConfig segmented_reduce_config; if (CubDebug( - error = segmented_reduce_config - .Init( - segmented_reduce_kernel))) + error = temporary_storage_layout.map_to_buffer(d_temp_storage, + temp_storage_bytes))) { break; } - // Log device_reduce_sweep_kernel configuration - #ifdef CUB_DETAIL_DEBUG_ENABLE_LOG - _CubLog("Invoking SegmentedDeviceReduceKernel<<<%d, %d, 0, %lld>>>(), " - "%d items per thread, %d SM occupancy\n", - num_segments, - ActivePolicyT::SegmentedReducePolicy::BLOCK_THREADS, - (long long)stream, - ActivePolicyT::SegmentedReducePolicy::ITEMS_PER_THREAD, - segmented_reduce_config.sm_occupancy); - #endif - - // Invoke DeviceReduceKernel - THRUST_NS_QUALIFIER::cuda_cub::launcher::triple_chevron( - num_segments, - ActivePolicyT::SegmentedReducePolicy::BLOCK_THREADS, - 0, - stream) - .doit(segmented_reduce_kernel, - d_in, - d_out, - d_begin_offsets, - d_end_offsets, - num_segments, - reduction_op, - init); - - // Check for failure to launch - if (CubDebug(error = cudaPeekAtLastError())) +#ifndef CUB_RDC_ENABLED + if (partition_segments) { - break; + auto medium_indices_iterator = + THRUST_NS_QUALIFIER::make_reverse_iterator( + large_and_medium_segments_indices.get() + num_segments); + + error = cub::DevicePartition::If( + device_partition_temp_storage.get(), + three_way_partition_temp_storage_bytes, + THRUST_NS_QUALIFIER::counting_iterator(0), + large_and_medium_segments_indices.get(), + small_segments_indices.get(), + medium_indices_iterator, + group_sizes.get(), + num_segments, + large_segments_selector, + small_segments_selector, + stream); + + if (CubDebug(error)) + { + return error; + } + + unsigned int h_group_sizes[num_selected_groups]; + + if (CubDebug(error = cudaMemcpyAsync(h_group_sizes, + group_sizes.get(), + num_selected_groups * + sizeof(unsigned int), + cudaMemcpyDeviceToHost, + stream))) + { + return error; + } + + if (CubDebug(error = SyncStream(stream))) + { + return error; + } + + const unsigned int large_segments = h_group_sizes[0]; + const unsigned int small_segments = h_group_sizes[1]; + const unsigned int medium_segments = + static_cast(num_segments) - + (large_segments + small_segments); + + const unsigned int large_blocks = large_segments; + const unsigned int small_blocks = DivideAndRoundUp(small_segments, ActivePolicyT::SmallReducePolicy::SEGMENTS_PER_BLOCK); + const unsigned int medium_blocks = DivideAndRoundUp(medium_segments, ActivePolicyT::MediumReducePolicy::SEGMENTS_PER_BLOCK); + const unsigned int total_blocks = large_blocks + medium_blocks + small_blocks; + + #ifdef CUB_DETAIL_DEBUG_ENABLE_LOG + _CubLog("Invoking DeviceSegmentedReduceWithPartitioningKernel<<<%d, %d, 0, %lld>>>(), " + "%d large segments, %d medium segments, %d small segments\n", + num_segments, + ActivePolicyT::SegmentedReducePolicy::BLOCK_THREADS, + (long long)stream, + large_segments, + medium_segments, + small_segments); + #endif + + // Invoke DeviceReduceKernel + THRUST_NS_QUALIFIER::cuda_cub::launcher::triple_chevron( + total_blocks, + ActivePolicyT::SegmentedReducePolicy::BLOCK_THREADS, + 0, + stream) + .doit(segmented_reduce_with_partitioning_kernel, + d_in, + d_out, + d_begin_offsets, + d_end_offsets, + large_blocks, + large_blocks + medium_blocks, + medium_segments, + small_segments, + large_and_medium_segments_indices.get(), + large_and_medium_segments_indices.get() + num_segments - medium_segments, + small_segments_indices.get(), + reduction_op, + init); + + // Check for failure to launch + if (CubDebug(error = cudaPeekAtLastError())) + { + break; + } + } + else +#endif + { + // Log device_reduce_sweep_kernel configuration + #ifdef CUB_DETAIL_DEBUG_ENABLE_LOG + // Init kernel configuration + KernelConfig segmented_reduce_config; + if (CubDebug( + error = segmented_reduce_config + .Init( + segmented_reduce_kernel))) + { + break; + } + + _CubLog("Invoking SegmentedDeviceReduceKernel<<<%d, %d, 0, %lld>>>(), " + "%d items per thread, %d SM occupancy\n", + num_segments, + ActivePolicyT::SegmentedReducePolicy::BLOCK_THREADS, + (long long)stream, + ActivePolicyT::SegmentedReducePolicy::ITEMS_PER_THREAD, + segmented_reduce_config.sm_occupancy); + #endif + + // Invoke DeviceReduceKernel + THRUST_NS_QUALIFIER::cuda_cub::launcher::triple_chevron( + num_segments, + ActivePolicyT::SegmentedReducePolicy::BLOCK_THREADS, + 0, + stream) + .doit(segmented_reduce_kernel, + d_in, + d_out, + d_begin_offsets, + d_end_offsets, + num_segments, + reduction_op, + init); + + // Check for failure to launch + if (CubDebug(error = cudaPeekAtLastError())) + { + break; + } } // Sync the stream if specified to flush runtime errors @@ -1209,7 +1609,16 @@ struct DispatchSegmentedReduce : SelectedPolicy OffsetT, ReductionOpT, InitT, - AccumT>); + AccumT>, + DeviceSegmentedReduceWithPartitioningKernel); } //---------------------------------------------------------------------------