Skip to content

Commit

Permalink
Merge pull request NVIDIA#675 from mfbalin/fancy_device_memcpy
Browse files Browse the repository at this point in the history
Initial implementation for DeviceCopy::Batched
  • Loading branch information
elstehle authored Apr 21, 2023
2 parents 76471de + 5523b9d commit 63fc662
Show file tree
Hide file tree
Showing 6 changed files with 874 additions and 62 deletions.
127 changes: 106 additions & 21 deletions cub/agent/agent_batch_memcpy.cuh
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,8 @@
#include <cub/util_ptx.cuh>
#include <cub/util_type.cuh>

#include <cuda/std/type_traits>

#include <cstdint>

CUB_NAMESPACE_BEGIN
Expand Down Expand Up @@ -287,6 +289,82 @@ VectorizedCopy(int32_t thread_rank, void *dest, ByteOffsetT num_bytes, const voi
}
}

template <bool IsMemcpy,
uint32_t LOGICAL_WARP_SIZE,
typename InputBufferT,
typename OutputBufferT,
typename OffsetT,
typename ::cuda::std::enable_if<IsMemcpy, int>::type = 0>
__device__ __forceinline__ void copy_items(InputBufferT input_buffer,
OutputBufferT output_buffer,
OffsetT num_bytes,
OffsetT offset = 0)
{
VectorizedCopy<LOGICAL_WARP_SIZE, uint4>(threadIdx.x % LOGICAL_WARP_SIZE,
&reinterpret_cast<char *>(output_buffer)[offset],
num_bytes,
&reinterpret_cast<const char *>(input_buffer)[offset]);
}

template <bool IsMemcpy,
uint32_t LOGICAL_WARP_SIZE,
typename InputBufferT,
typename OutputBufferT,
typename OffsetT,
typename ::cuda::std::enable_if<!IsMemcpy, int>::type = 0>
__device__ __forceinline__ void copy_items(InputBufferT input_buffer,
OutputBufferT output_buffer,
OffsetT num_items,
OffsetT offset = 0)
{
output_buffer += offset;
input_buffer += offset;
for (OffsetT i = threadIdx.x % LOGICAL_WARP_SIZE; i < num_items; i += LOGICAL_WARP_SIZE)
{
*(output_buffer + i) = *(input_buffer + i);
}
}

template <bool IsMemcpy,
typename AliasT,
typename InputIt,
typename OffsetT,
typename ::cuda::std::enable_if<IsMemcpy, int>::type = 0>
__device__ __forceinline__ AliasT read_item(InputIt buffer_src, OffsetT offset)
{
return *(reinterpret_cast<const AliasT *>(buffer_src) + offset);
}

template <bool IsMemcpy,
typename AliasT,
typename InputIt,
typename OffsetT,
typename ::cuda::std::enable_if<!IsMemcpy, int>::type = 0>
__device__ __forceinline__ AliasT read_item(InputIt buffer_src, OffsetT offset)
{
return *(buffer_src + offset);
}

template <bool IsMemcpy,
typename AliasT,
typename OutputIt,
typename OffsetT,
typename ::cuda::std::enable_if<IsMemcpy, int>::type = 0>
__device__ __forceinline__ void write_item(OutputIt buffer_dst, OffsetT offset, AliasT value)
{
*(reinterpret_cast<AliasT *>(buffer_dst) + offset) = value;
}

template <bool IsMemcpy,
typename AliasT,
typename OutputIt,
typename OffsetT,
typename ::cuda::std::enable_if<!IsMemcpy, int>::type = 0>
__device__ __forceinline__ void write_item(OutputIt buffer_dst, OffsetT offset, AliasT value)
{
*(buffer_dst + offset) = value;
}

/**
* @brief A helper class that allows threads to maintain multiple counters, where the counter that
* shall be incremented can be addressed dynamically without incurring register spillage.
Expand Down Expand Up @@ -431,7 +509,8 @@ template <typename AgentMemcpySmallBuffersPolicyT,
typename BlevBufferTileOffsetsOutItT,
typename BlockOffsetT,
typename BLevBufferOffsetTileState,
typename BLevBlockOffsetTileState>
typename BLevBlockOffsetTileState,
bool IsMemcpy>
class AgentBatchMemcpy
{
private:
Expand Down Expand Up @@ -470,7 +549,14 @@ private:
// TYPE DECLARATIONS
//---------------------------------------------------------------------
/// Internal load/store type. For byte-wise memcpy, a single-byte type
using AliasT = char;
using AliasT = typename ::cuda::std::conditional<
IsMemcpy,
std::iterator_traits<char *>,
std::iterator_traits<cub::detail::value_t<InputBufferIt>>>::type::value_type;

/// Types of the input and output buffers
using InputBufferT = cub::detail::value_t<InputBufferIt>;
using OutputBufferT = cub::detail::value_t<OutputBufferIt>;

/// Type that has to be sufficiently large to hold any of the buffers' sizes.
/// The BufferSizeIteratorT's value type must be convertible to this type.
Expand Down Expand Up @@ -775,17 +861,16 @@ private:
BlockBufferOffsetT num_wlev_buffers)
{
const int32_t warp_id = threadIdx.x / CUB_PTX_WARP_THREADS;
const int32_t warp_lane = threadIdx.x % CUB_PTX_WARP_THREADS;
constexpr uint32_t WARPS_PER_BLOCK = BLOCK_THREADS / CUB_PTX_WARP_THREADS;

for (BlockBufferOffsetT buffer_offset = warp_id; buffer_offset < num_wlev_buffers;
buffer_offset += WARPS_PER_BLOCK)
{
const auto buffer_id = buffers_by_size_class[buffer_offset].buffer_id;
detail::VectorizedCopy<CUB_PTX_WARP_THREADS, uint4>(warp_lane,
tile_buffer_dsts[buffer_id],
tile_buffer_sizes[buffer_id],
tile_buffer_srcs[buffer_id]);
copy_items<IsMemcpy, CUB_PTX_WARP_THREADS, InputBufferT, OutputBufferT, BufferSizeT>(
tile_buffer_srcs[buffer_id],
tile_buffer_dsts[buffer_id],
tile_buffer_sizes[buffer_id]);
}
}

Expand Down Expand Up @@ -875,18 +960,18 @@ private:
#pragma unroll
for (int32_t i = 0; i < TLEV_BYTES_PER_THREAD; i++)
{
src_byte[i] = reinterpret_cast<const AliasT *>(
tile_buffer_srcs[zipped_byte_assignment[i].tile_buffer_id])[zipped_byte_assignment[i]
.buffer_byte_offset];
src_byte[i] = read_item<IsMemcpy, AliasT, InputBufferT>(
tile_buffer_srcs[zipped_byte_assignment[i].tile_buffer_id],
zipped_byte_assignment[i].buffer_byte_offset);
absolute_tlev_byte_offset += BLOCK_THREADS;
}
#pragma unroll
for (int32_t i = 0; i < TLEV_BYTES_PER_THREAD; i++)
{
reinterpret_cast<AliasT *>(
tile_buffer_dsts[zipped_byte_assignment[i].tile_buffer_id])[zipped_byte_assignment[i]
.buffer_byte_offset] =
src_byte[i];
write_item<IsMemcpy, AliasT, OutputBufferT>(
tile_buffer_dsts[zipped_byte_assignment[i].tile_buffer_id],
zipped_byte_assignment[i].buffer_byte_offset,
src_byte[i]);
}
}
else
Expand All @@ -897,13 +982,13 @@ private:
{
if (absolute_tlev_byte_offset < num_total_tlev_bytes)
{
const AliasT src_byte = reinterpret_cast<const AliasT *>(
tile_buffer_srcs[zipped_byte_assignment[i].tile_buffer_id])[zipped_byte_assignment[i]
.buffer_byte_offset];
reinterpret_cast<AliasT *>(
tile_buffer_dsts[zipped_byte_assignment[i].tile_buffer_id])[zipped_byte_assignment[i]
.buffer_byte_offset] =
src_byte;
const AliasT src_byte = read_item<IsMemcpy, AliasT, InputBufferT>(
tile_buffer_srcs[zipped_byte_assignment[i].tile_buffer_id],
zipped_byte_assignment[i].buffer_byte_offset);
write_item<IsMemcpy, AliasT, OutputBufferT>(
tile_buffer_dsts[zipped_byte_assignment[i].tile_buffer_id],
zipped_byte_assignment[i].buffer_byte_offset,
src_byte);
}
absolute_tlev_byte_offset += BLOCK_THREADS;
}
Expand Down
1 change: 1 addition & 0 deletions cub/cub.cuh
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@

// Device
#include "device/device_adjacent_difference.cuh"
#include "device/device_copy.cuh"
#include "device/device_histogram.cuh"
#include "device/device_memcpy.cuh"
#include "device/device_merge_sort.cuh"
Expand Down
172 changes: 172 additions & 0 deletions cub/device/device_copy.cuh
Original file line number Diff line number Diff line change
@@ -0,0 +1,172 @@
/******************************************************************************
* Copyright (c) 2023, 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
* cub::DeviceCopy provides device-wide, parallel operations for copying data.
*/

#pragma once

#include <cub/config.cuh>
#include <cub/device/dispatch/dispatch_batch_memcpy.cuh>

#include <thrust/system/cuda/detail/core/triple_chevron_launch.h>

#include <cstdint>

CUB_NAMESPACE_BEGIN

/**
* @brief cub::DeviceCopy provides device-wide, parallel operations for copying data.
* \ingroup SingleModule
*/
struct DeviceCopy
{
/**
* @brief Copies data from a batch of given source ranges to their corresponding destination
* ranges.
* @note If any input range aliases any output range the behavior is undefined. If
* any output range aliases another output range the behavior is undefined. Input
* ranges can alias one another.
*
* @par Snippet
* The code snippet below illustrates usage of DeviceCopy::Batched to perform a DeviceRunLength
* Decode operation.
* @par
* @code
* struct GetIteratorToRange
* {
* __host__ __device__ __forceinline__ auto operator()(uint32_t index)
* {
* return thrust::make_constant_iterator(d_data_in[index]);
* }
* int32_t *d_data_in;
* };
*
* struct GetPtrToRange
* {
* __host__ __device__ __forceinline__ auto operator()(uint32_t index)
* {
* return d_data_out + d_offsets[index];
* }
* int32_t *d_data_out;
* uint32_t *d_offsets;
* };
*
* struct GetRunLength
* {
* __host__ __device__ __forceinline__ uint32_t operator()(uint32_t index)
* {
* return d_offsets[index + 1] - d_offsets[index];
* }
* uint32_t *d_offsets;
* };
*
* uint32_t num_ranges = 5;
* int32_t *d_data_in; // e.g., [4, 2, 7, 3, 1]
* int32_t *d_data_out; // e.g., [0, ... ]
* uint32_t *d_offsets; // e.g., [0, 2, 5, 6, 9, 14]
*
* // Returns a constant iterator to the element of the i-th run
* thrust::counting_iterator<uint32_t> iota(0);
* auto iterators_in = thrust::make_transform_iterator(iota, GetIteratorToRange{d_data_in});
*
* // Returns the run length of the i-th run
* auto sizes = thrust::make_transform_iterator(iota, GetRunLength{d_offsets});
*
* // Returns pointers to the output range for each run
* auto ptrs_out = thrust::make_transform_iterator(iota, GetPtrToRange{d_data_out, d_offsets});
*
* // Determine temporary device storage requirements
* void *d_temp_storage = nullptr;
* size_t temp_storage_bytes = 0;
* cub::DeviceCopy::Batched(d_temp_storage, temp_storage_bytes, iterators_in, ptrs_out, sizes,
* num_ranges);
*
* // Allocate temporary storage
* cudaMalloc(&d_temp_storage, temp_storage_bytes);
*
* // Run batched copy algorithm (used to perform runlength decoding)
* cub::DeviceCopy::Batched(d_temp_storage, temp_storage_bytes, iterators_in, ptrs_out, sizes,
* num_ranges);
*
* // d_data_out <-- [4, 4, 2, 2, 2, 7, 3, 3, 3, 1, 1, 1, 1, 1]
* @endcode
* @tparam InputIt <b>[inferred]</b> Device-accessible random-access input iterator type
* providing the iterators to the source ranges
* @tparam OutputIt <b>[inferred]</b> Device-accessible random-access input iterator type
* providing the iterators to the destination ranges
* @tparam SizeIteratorT <b>[inferred]</b> Device-accessible random-access input iterator
* type providing the number of items to be copied for each pair of ranges
* @param 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.
* @param temp_storage_bytes [in,out] Reference to size in bytes of \p d_temp_storage allocation
* @param input_it [in] Device-accessible iterator providing the iterators to the source
* ranges
* @param output_it [in] Device-accessible iterator providing the iterators to the
* destination ranges
* @param sizes [in] Device-accessible iterator providing the number of elements to be copied
* for each pair of ranges
* @param num_ranges [in] The total number of range pairs
* @param stream [in] <b>[optional]</b> CUDA stream to launch kernels within. Default is
* stream<sub>0</sub>.
*/
template <typename InputIt, typename OutputIt, typename SizeIteratorT>
CUB_RUNTIME_FUNCTION static cudaError_t Batched(void *d_temp_storage,
size_t &temp_storage_bytes,
InputIt input_it,
OutputIt output_it,
SizeIteratorT sizes,
uint32_t num_ranges,
cudaStream_t stream = 0)
{
// Integer type large enough to hold any offset in [0, num_ranges)
using RangeOffsetT = uint32_t;

// Integer type large enough to hold any offset in [0, num_thread_blocks_launched), where a safe
// uppper bound on num_thread_blocks_launched can be assumed to be given by
// IDIV_CEIL(num_ranges, 64)
using BlockOffsetT = uint32_t;

return detail::DispatchBatchMemcpy<InputIt,
OutputIt,
SizeIteratorT,
RangeOffsetT,
BlockOffsetT,
detail::DeviceBatchMemcpyPolicy,
false>::Dispatch(d_temp_storage,
temp_storage_bytes,
input_it,
output_it,
sizes,
num_ranges,
stream);
}
};

CUB_NAMESPACE_END
Loading

0 comments on commit 63fc662

Please sign in to comment.