CUDA support

Contexts

class arrow::cuda::CudaDeviceManager

Public Functions

Result<std::shared_ptr<CudaDevice>> GetDevice(int device_number)

Get a CudaDevice instance for a particular device.

Parameters

[in] device_number – the CUDA device number

Result<std::shared_ptr<CudaContext>> GetContext(int device_number)

Get the CUDA driver context for a particular device.

Parameters

[in] device_number – the CUDA device number

Returns

cached context

Result<std::shared_ptr<CudaContext>> GetSharedContext(int device_number, void *handle)

Get the shared CUDA driver context for a particular device.

Parameters
  • [in] device_number – the CUDA device number

  • [in] handle – CUDA context handle created by another library

Returns

shared context

Result<std::shared_ptr<CudaHostBuffer>> AllocateHost(int device_number, int64_t nbytes)

Allocate host memory with fast access to given GPU device.

Parameters
  • [in] device_number – the CUDA device number

  • [in] nbytes – number of bytes

Returns

Host buffer or Status

Status FreeHost(void *data, int64_t nbytes)

Free host memory.

The given memory pointer must have been allocated with AllocateHost.

class arrow::cuda::CudaContext : public std::enable_shared_from_this<CudaContext>

Object-oriented interface to the low-level CUDA driver API.

Public Functions

Result<std::shared_ptr<CudaBuffer>> Allocate(int64_t nbytes)

Allocate CUDA memory on GPU device for this context.

Parameters

[in] nbytes – number of bytes

Returns

the allocated buffer

Status Free(void *device_ptr, int64_t nbytes)

Release CUDA memory on GPU device for this context.

Parameters
  • [in] device_ptr – the buffer address

  • [in] nbytes – number of bytes

Returns

Status

Result<std::shared_ptr<CudaBuffer>> View(uint8_t *data, int64_t nbytes)

Create a view of CUDA memory on GPU device of this context.

Note

The caller is responsible for allocating and freeing the memory as well as ensuring that the memory belongs to the CUDA context that this CudaContext instance holds.

Parameters
  • [in] data – the starting device address

  • [in] nbytes – number of bytes

Returns

the view buffer

Result<std::shared_ptr<CudaBuffer>> OpenIpcBuffer(const CudaIpcMemHandle &ipc_handle)

Open existing CUDA IPC memory handle.

Parameters

[in] ipc_handle – opaque pointer to CUipcMemHandle (driver API)

Returns

a CudaBuffer referencing the IPC segment

Status CloseIpcBuffer(CudaBuffer *buffer)

Close memory mapped with IPC buffer.

Parameters

[in] buffer – a CudaBuffer referencing

Returns

Status

Status Synchronize(void)

Block until the all device tasks are completed.

void *handle() const

Expose CUDA context handle to other libraries.

std::shared_ptr<CudaMemoryManager> memory_manager() const

Return the default memory manager tied to this context’s device.

std::shared_ptr<CudaDevice> device() const

Return the device instance associated with this context.

int device_number() const

Return the logical device number.

Result<uintptr_t> GetDeviceAddress(uint8_t *addr)

Return the device address that is reachable from kernels running in the context.

The device address is defined as a memory address accessible by device. While it is often a device memory address, it can be also a host memory address, for instance, when the memory is allocated as host memory (using cudaMallocHost or cudaHostAlloc) or as managed memory (using cudaMallocManaged) or the host memory is page-locked (using cudaHostRegister).

Parameters

[in] addr – device or host memory address

Returns

the device address

Devices

class arrow::cuda::CudaDevice : public arrow::Device

Device implementation for CUDA.

Each CudaDevice instance is tied to a particular CUDA device (identified by its logical device number).

Public Functions

virtual const char *type_name() const override

A shorthand for this device’s type.

The returned value is different for each device class, but is the same for all instances of a given class. It can be used as a replacement for RTTI.

virtual std::string ToString() const override

A human-readable description of the device.

The returned value should be detailed enough to distinguish between different instances, where necessary.

virtual bool Equals(const Device&) const override

Whether this instance points to the same device as another one.

virtual std::shared_ptr<MemoryManager> default_memory_manager() override

Return a MemoryManager instance tied to this device.

The returned instance uses default parameters for this device type’s MemoryManager implementation. Some devices also allow constructing MemoryManager instances with non-default parameters.

int device_number() const

Return the device logical number.

std::string device_name() const

Return the GPU model name.

int64_t total_memory() const

Return total memory on this device.

int handle() const

Return a raw CUDA device handle.

The returned value can be used to expose this device to other libraries. It should be interpreted as CUdevice.

Result<std::shared_ptr<CudaContext>> GetContext()

Get a CUDA driver context for this device.

The returned context is associated with the primary CUDA context for the device. This is the recommended way of getting a context for a device, as it allows interoperating transparently with any library using the primary CUDA context API.

Result<std::shared_ptr<CudaContext>> GetSharedContext(void *handle)

Get a CUDA driver context for this device, using an existing handle.

The handle is not owned: it will not be released when the CudaContext is destroyed. This function should only be used if you need interoperation with a library that uses a non-primary context.

Parameters

[in] handle – CUDA context handle created by another library

Result<std::shared_ptr<CudaHostBuffer>> AllocateHostBuffer(int64_t size)

Allocate a host-residing, GPU-accessible buffer.

The buffer is allocated using this device’s primary context.

Parameters

[in] size – The buffer size in bytes

Public Static Functions

static Result<std::shared_ptr<CudaDevice>> Make(int device_number)

Return a CudaDevice instance for a particular device.

Parameters

[in] device_number – the CUDA device number

class arrow::cuda::CudaMemoryManager : public arrow::MemoryManager

MemoryManager implementation for CUDA.

Public Functions

virtual Result<std::shared_ptr<io::RandomAccessFile>> GetBufferReader(std::shared_ptr<Buffer> buf) override

Create a RandomAccessFile to read a particular buffer.

The given buffer must be tied to this MemoryManager.

See also the Buffer::GetReader shorthand.

virtual Result<std::shared_ptr<io::OutputStream>> GetBufferWriter(std::shared_ptr<Buffer> buf) override

Create a OutputStream to write to a particular buffer.

The given buffer must be mutable and tied to this MemoryManager. The returned stream object writes into the buffer’s underlying memory (but it won’t resize it).

See also the Buffer::GetWriter shorthand.

virtual Result<std::shared_ptr<Buffer>> AllocateBuffer(int64_t size) override

Allocate a (mutable) Buffer.

The buffer will be allocated in the device’s memory.

std::shared_ptr<CudaDevice> cuda_device() const

The CudaDevice instance tied to this MemoryManager.

This is a useful shorthand returning a concrete-typed pointer, avoiding having to cast the device() result.

Buffers

class arrow::cuda::CudaBuffer : public arrow::Buffer

An Arrow buffer located on a GPU device.

Be careful using this in any Arrow code which may not be GPU-aware

Public Functions

Status CopyToHost(const int64_t position, const int64_t nbytes, void *out) const

Copy memory from GPU device to CPU host.

Parameters
  • [in] position – start position inside buffer to copy bytes from

  • [in] nbytes – number of bytes to copy

  • [out] out – start address of the host memory area to copy to

Returns

Status

Status CopyFromHost(const int64_t position, const void *data, int64_t nbytes)

Copy memory to device at position.

Parameters
  • [in] position – start position to copy bytes to

  • [in] data – the host data to copy

  • [in] nbytes – number of bytes to copy

Returns

Status

Status CopyFromDevice(const int64_t position, const void *data, int64_t nbytes)

Copy memory from device to device at position.

Note

It is assumed that both source and destination device memories have been allocated within the same context.

Parameters
  • [in] position – start position inside buffer to copy bytes to

  • [in] data – start address of the device memory area to copy from

  • [in] nbytes – number of bytes to copy

Returns

Status

Status CopyFromAnotherDevice(const std::shared_ptr<CudaContext> &src_ctx, const int64_t position, const void *data, int64_t nbytes)

Copy memory from another device to device at position.

Parameters
  • [in] src_ctx – context of the source device memory

  • [in] position – start position inside buffer to copy bytes to

  • [in] data – start address of the another device memory area to copy from

  • [in] nbytes – number of bytes to copy

Returns

Status

virtual Result<std::shared_ptr<CudaIpcMemHandle>> ExportForIpc()

Expose this device buffer as IPC memory which can be used in other processes.

Note

After calling this function, this device memory will not be freed when the CudaBuffer is destructed

Returns

Handle or Status

Public Static Functions

static Result<std::shared_ptr<CudaBuffer>> FromBuffer(std::shared_ptr<Buffer> buffer)

Convert back generic buffer into CudaBuffer.

Note

This function returns an error if the buffer isn’t backed by GPU memory

Parameters

[in] buffer – buffer to convert

Returns

CudaBuffer or Status

class arrow::cuda::CudaHostBuffer : public arrow::MutableBuffer

Device-accessible CPU memory created using cudaHostAlloc.

Public Functions

Result<uintptr_t> GetDeviceAddress(const std::shared_ptr<CudaContext> &ctx)

Return a device address the GPU can read this memory from.

Memory Input / Output

class arrow::cuda::CudaBufferReader : public arrow::io::internal::RandomAccessFileConcurrencyWrapper<CudaBufferReader>

File interface for zero-copy read from CUDA buffers.

CAUTION: reading to a Buffer returns a Buffer pointing to device memory. It will generally not be compatible with Arrow code expecting a buffer pointing to CPU memory. Reading to a raw pointer, though, copies device memory into the host memory pointed to.

Public Functions

virtual bool closed() const override

Return whether the stream is closed.

virtual bool supports_zero_copy() const override

Return true if InputStream is capable of zero copy Buffer reads.

Zero copy reads imply the use of Buffer-returning Read() overloads.

class arrow::cuda::CudaBufferWriter : public arrow::io::WritableFile

File interface for writing to CUDA buffers, with optional buffering.

Public Functions

virtual Status Close() override

Close writer and flush buffered bytes to GPU.

virtual bool closed() const override

Return whether the stream is closed.

virtual Status Flush() override

Flush buffered bytes to GPU.

virtual Status Write(const void *data, int64_t nbytes) override

Write the given data to the stream.

This method always processes the bytes in full. Depending on the semantics of the stream, the data may be written out immediately, held in a buffer, or written asynchronously. In the case where the stream buffers the data, it will be copied. To avoid potentially large copies, use the Write variant that takes an owned Buffer.

virtual Result<int64_t> Tell() const override

Return the position in this stream.

Status SetBufferSize(const int64_t buffer_size)

Set CPU buffer size to limit calls to cudaMemcpy.

By default writes are unbuffered

Parameters

[in] buffer_size – the size of CPU buffer to allocate

Returns

Status

int64_t buffer_size() const

Returns size of host (CPU) buffer, 0 for unbuffered.

int64_t num_bytes_buffered() const

Returns number of bytes buffered on host.

IPC

class arrow::cuda::CudaIpcMemHandle

Public Functions

Result<std::shared_ptr<Buffer>> Serialize(MemoryPool *pool = default_memory_pool()) const

Write CudaIpcMemHandle to a Buffer.

Parameters

[in] pool – a MemoryPool to allocate memory from

Returns

Buffer or Status

Public Static Functions

static Result<std::shared_ptr<CudaIpcMemHandle>> FromBuffer(const void *opaque_handle)

Create CudaIpcMemHandle from opaque buffer (e.g.

from another process)

Parameters

[in] opaque_handle – a CUipcMemHandle as a const void*

Returns

Handle or Status

Result<std::shared_ptr<CudaBuffer>> SerializeRecordBatch(const RecordBatch &batch, CudaContext *ctx)

Write record batch message to GPU device memory.

Parameters
  • [in] batch – record batch to write

  • [in] ctxCudaContext to allocate device memory from

Returns

CudaBuffer or Status

Result<std::shared_ptr<RecordBatch>> ReadRecordBatch(const std::shared_ptr<Schema> &schema, const ipc::DictionaryMemo *dictionary_memo, const std::shared_ptr<CudaBuffer> &buffer, MemoryPool *pool = default_memory_pool())

ReadRecordBatch specialized to handle metadata on CUDA device.

Parameters
  • [in] schema – the Schema for the record batch

  • [in] dictionary_memo – DictionaryMemo which has any dictionaries. Can be nullptr if you are sure there are no dictionary-encoded fields

  • [in] buffer – a CudaBuffer containing the complete IPC message

  • [in] pool – a MemoryPool to use for allocating space for the metadata

Returns

RecordBatch or Status