Input / output#

Interfaces#

classFileInterface:publicstd::enable_shared_from_this<FileInterface>#

Subclassed byarrow::io::InputStream,arrow::io::OutputStream

Public Functions

virtualStatusClose()=0#

Close the stream cleanly.

For writable streams, this will attempt to flush any pending data before releasing the underlying resource.

AfterClose() is called,closed() returns true and the stream is not available for further operations.

virtualFutureCloseAsync()#

Close the stream asynchronously.

By default, this will just submit the synchronousClose() to the default I/O thread pool. Subclasses may implement this in a more efficient manner.

virtualStatusAbort()#

Close the stream abruptly.

This method does not guarantee that any pending data is flushed. It merely releases any underlying resource used by the stream for its operation.

AfterAbort() is called,closed() returns true and the stream is not available for further operations.

virtualResult<int64_t>Tell()const=0#

Return the position in this stream.

virtualboolclosed()const=0#

Return whether the stream is closed.

classReadable#

Subclassed byarrow::io::InputStream

Public Functions

virtualResult<int64_t>Read(int64_tnbytes,void*out)=0#

Read data from current file position.

Read at mostnbytes from the current file position intoout. The number of bytes read is returned.

virtualResult<std::shared_ptr<Buffer>>Read(int64_tnbytes)=0#

Read data from current file position.

Read at mostnbytes from the current file position. Less bytes may be read if EOF is reached. This method updates the current file position.

In some cases (e.g. a memory-mapped file), this method may avoid a memory copy.

virtualconstIOContext&io_context()const#

EXPERIMENTAL: The IOContext associated with this file.

By default, this is the same as default_io_context(), but it may be overridden by subclasses.

classSeekable#

Subclassed byarrow::io::RandomAccessFile, arrow::io::WritableFile

classWritable#

Subclassed byarrow::io::OutputStream

Public Functions

virtualStatusWrite(constvoid*data,int64_tnbytes)=0#

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 ownedBuffer.

virtualStatusWrite(conststd::shared_ptr<Buffer>&data)#

Write the given data to the stream.

Since theBuffer owns its memory, this method can avoid a copy if buffering is required. SeeWrite(const void*, int64_t) for details.

virtualStatusFlush()#

Flush buffered bytes, if any.

classInputStream:publicvirtualarrow::io::FileInterface,publicvirtualarrow::io::Readable#

Subclassed by arrow::io::SlowInputStreamBase< InputStream >, arrow::io::internal::InputStreamConcurrencyWrapper< BufferedInputStream >, arrow::io::internal::InputStreamConcurrencyWrapper< CompressedInputStream >,arrow::io::RandomAccessFile, arrow::io::StdinStream,arrow::io::TransformInputStream, arrow::io::internal::InputStreamConcurrencyWrapper< Derived >

Public Functions

StatusAdvance(int64_tnbytes)#

Advance or skip stream indicated number of bytes.

Parameters:

nbytes[in] the number to move forward

Returns:

Status

virtualResult<std::string_view>Peek(int64_tnbytes)#

Return zero-copy string_view to upcoming bytes.

Do not modify the stream position. The view becomes invalid after any operation on the stream. May trigger buffering if the requested size is larger than the number of buffered bytes.

May return NotImplemented on streams that don’t support it.

Parameters:

nbytes[in] the maximum number of bytes to see

virtualboolsupports_zero_copy()const#

Return true ifInputStream is capable of zero copyBuffer reads.

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

virtualResult<std::shared_ptr<constKeyValueMetadata>>ReadMetadata()#

Read and return stream metadata.

If the stream implementation doesn’t support metadata, empty metadata is returned. Note that it is allowed to return a null pointer rather than an allocated empty metadata.

virtualFuture<std::shared_ptr<constKeyValueMetadata>>ReadMetadataAsync(constIOContext&io_context)#

Read stream metadata asynchronously.

classRandomAccessFile:publicarrow::io::InputStream,publicarrow::io::Seekable#

Subclassed by arrow::io::SlowInputStreamBase< RandomAccessFile >, arrow::io::internal::RandomAccessFileConcurrencyWrapper< CudaBufferReader >, arrow::io::internal::RandomAccessFileConcurrencyWrapper< BufferReader >, arrow::io::internal::RandomAccessFileConcurrencyWrapper< ReadableFile >, arrow::io::HdfsReadableFile,arrow::io::ReadWriteFileInterface, arrow::io::internal::RandomAccessFileConcurrencyWrapper< Derived >

Public Functions

~RandomAccessFile()override#

Necessary because we hold a std::unique_ptr.

virtualResult<int64_t>GetSize()=0#

Return the total file size in bytes.

This method does not read or move the current file position, so is safe to call concurrently with e.g.ReadAt().

virtualResult<int64_t>ReadAt(int64_tposition,int64_tnbytes,void*out)#

Read data from given file position.

At mostnbytes bytes are read. The number of bytes read is returned (it can be less thannbytes if EOF is reached).

This method can be safely called from multiple threads concurrently. It is unspecified whether this method updates the file position or not.

The default RandomAccessFile-provided implementation uses Seek() andRead(), but subclasses may override it with a more efficient implementation that doesn’t depend on implicit file positioning.

Parameters:
  • position[in] Where to read bytes from

  • nbytes[in] The number of bytes to read

  • out[out] The buffer to read bytes into

Returns:

The number of bytes read, or an error

virtualResult<std::shared_ptr<Buffer>>ReadAt(int64_tposition,int64_tnbytes)#

Read data from given file position.

At mostnbytes bytes are read, but it can be less if EOF is reached.

Parameters:
  • position[in] Where to read bytes from

  • nbytes[in] The number of bytes to read

Returns:

A buffer containing the bytes read, or an error

virtualFuture<std::shared_ptr<Buffer>>ReadAsync(constIOContext&,int64_tposition,int64_tnbytes)#

EXPERIMENTAL: Read data asynchronously.

Future<std::shared_ptr<Buffer>>ReadAsync(int64_tposition,int64_tnbytes)#

EXPERIMENTAL: Read data asynchronously, using the file’s IOContext.

virtualstd::vector<Future<std::shared_ptr<Buffer>>>ReadManyAsync(constIOContext&,conststd::vector<ReadRange>&ranges)#

EXPERIMENTAL: Explicit multi-read.

Request multiple reads at once

The underlying filesystem may optimize these reads by coalescing small reads into large reads or by breaking up large reads into multiple parallel smaller reads. The reads should be issued in parallel if it makes sense for the filesystem.

One future will be returned for each input read range. Multiple returned futures may correspond to a single read. Or, a single returned future may be a combined result of several individual reads.

Parameters:

ranges[in] The ranges to read

Returns:

A future that will complete with the data from the requested range is available

std::vector<Future<std::shared_ptr<Buffer>>>ReadManyAsync(conststd::vector<ReadRange>&ranges)#

EXPERIMENTAL: Explicit multi-read, using the file’s IOContext.

virtualStatusWillNeed(conststd::vector<ReadRange>&ranges)#

EXPERIMENTAL: Inform that the given ranges may be read soon.

Some implementations might arrange to prefetch some of the data. However, no guarantee is made and the default implementation does nothing. For robust prefetching, useReadAt() orReadAsync().

Public Static Functions

staticResult<std::shared_ptr<InputStream>>GetStream(std::shared_ptr<RandomAccessFile>file,int64_tfile_offset,int64_tnbytes)#

Create an isolatedInputStream that reads a segment of aRandomAccessFile.

Multiple such stream can be created and used independently without interference

Parameters:
  • file[in] a file instance

  • file_offset[in] the starting position in the file

  • nbytes[in] the extent of bytes to read. The file should have sufficient bytes available

classOutputStream:publicvirtualarrow::io::FileInterface,publicarrow::io::Writable#

Subclassed byarrow::io::BufferOutputStream,arrow::io::BufferedOutputStream,arrow::io::CompressedOutputStream,arrow::io::FileOutputStream, arrow::io::HdfsOutputStream,arrow::io::MockOutputStream, arrow::io::StderrStream, arrow::io::StdoutStream, arrow::io::WritableFile

classReadWriteFileInterface:publicarrow::io::RandomAccessFile,publicarrow::io::WritableFile#

Subclassed byarrow::io::MemoryMappedFile

Concrete implementations#

In-memory streams#

classBufferReader:publicarrow::io::internal::RandomAccessFileConcurrencyWrapper<BufferReader>#

Random access zero-copy reads on anarrow::Buffer.

Public Functions

explicitBufferReader(std::shared_ptr<Buffer>buffer)#

Instantiate from std::shared_ptr<Buffer>.

This is a zero-copy constructor.

virtualboolclosed()constoverride#

Return whether the stream is closed.

virtualboolsupports_zero_copy()constoverride#

Return true ifInputStream is capable of zero copyBuffer reads.

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

virtualFuture<std::shared_ptr<Buffer>>ReadAsync(constIOContext&,int64_tposition,int64_tnbytes)override#

EXPERIMENTAL: Read data asynchronously.

virtualStatusWillNeed(conststd::vector<ReadRange>&ranges)override#

EXPERIMENTAL: Inform that the given ranges may be read soon.

Some implementations might arrange to prefetch some of the data. However, no guarantee is made and the default implementation does nothing. For robust prefetching, useReadAt() orReadAsync().

Public Static Functions

staticstd::unique_ptr<BufferReader>FromString(std::stringdata)#

Instantiate from std::string. Owns data.

classMockOutputStream:publicarrow::io::OutputStream#

A helper class to track the size of allocations.

Writes to this stream do not copy or retain any data, they just bump a size counter that can be later used to know exactly which data size needs to be allocated for actual writing.

Public Functions

virtualStatusClose()override#

Close the stream cleanly.

For writable streams, this will attempt to flush any pending data before releasing the underlying resource.

AfterClose() is called,closed() returns true and the stream is not available for further operations.

virtualboolclosed()constoverride#

Return whether the stream is closed.

virtualResult<int64_t>Tell()constoverride#

Return the position in this stream.

virtualStatusWrite(constvoid*data,int64_tnbytes)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 ownedBuffer.

classBufferOutputStream:publicarrow::io::OutputStream#

An output stream that writes to a resizable buffer.

Public Functions

virtualStatusClose()override#

Close the stream, preserving the buffer (retrieve it withFinish()).

virtualboolclosed()constoverride#

Return whether the stream is closed.

virtualResult<int64_t>Tell()constoverride#

Return the position in this stream.

virtualStatusWrite(constvoid*data,int64_tnbytes)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 ownedBuffer.

Result<std::shared_ptr<Buffer>>Finish()#

Close the stream and return the buffer.

StatusReset(int64_tinitial_capacity=1024,MemoryPool*pool=default_memory_pool())#

Initialize state ofOutputStream with newly allocated memory and set position to 0.

Parameters:
  • initial_capacity[in] the starting allocated capacity

  • pool[inout] the memory pool to use for allocations

Returns:

Status

Public Static Functions

staticResult<std::shared_ptr<BufferOutputStream>>Create(int64_tinitial_capacity=4096,MemoryPool*pool=default_memory_pool())#

Create in-memory output stream with indicated capacity using a memory pool.

Parameters:
  • initial_capacity[in] the initial allocated internal capacity of theOutputStream

  • pool[inout] aMemoryPool to use for allocations

Returns:

the created stream

classFixedSizeBufferWriter:publicarrow::io::WritableFile#

An output stream that writes into a fixed-size mutable buffer.

Public Functions

explicitFixedSizeBufferWriter(conststd::shared_ptr<Buffer>&buffer)#

Input buffer must be mutable, will abort if not.

virtualStatusClose()override#

Close the stream cleanly.

For writable streams, this will attempt to flush any pending data before releasing the underlying resource.

AfterClose() is called,closed() returns true and the stream is not available for further operations.

virtualboolclosed()constoverride#

Return whether the stream is closed.

virtualResult<int64_t>Tell()constoverride#

Return the position in this stream.

virtualStatusWrite(constvoid*data,int64_tnbytes)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 ownedBuffer.

Local files#

classReadableFile:publicarrow::io::internal::RandomAccessFileConcurrencyWrapper<ReadableFile>#

An operating system file open in read-only mode.

Reads through this implementation are unbuffered. If many small reads need to be issued, it is recommended to use a buffering layer for good performance.

Public Functions

virtualboolclosed()constoverride#

Return whether the stream is closed.

virtualStatusWillNeed(conststd::vector<ReadRange>&ranges)override#

EXPERIMENTAL: Inform that the given ranges may be read soon.

Some implementations might arrange to prefetch some of the data. However, no guarantee is made and the default implementation does nothing. For robust prefetching, useReadAt() orReadAsync().

Public Static Functions

staticResult<std::shared_ptr<ReadableFile>>Open(conststd::string&path,MemoryPool*pool=default_memory_pool())#

Open a local file for reading.

Parameters:
  • path[in] with UTF8 encoding

  • pool[in] aMemoryPool for memory allocations

Returns:

ReadableFile instance

staticResult<std::shared_ptr<ReadableFile>>Open(intfd,MemoryPool*pool=default_memory_pool())#

Open a local file for reading.

The file descriptor becomes owned by theReadableFile, and will be closed onClose() or destruction.

Parameters:
  • fd[in] file descriptor

  • pool[in] aMemoryPool for memory allocations

Returns:

ReadableFile instance

classFileOutputStream:publicarrow::io::OutputStream#

An operating system file open in write-only mode.

Public Functions

virtualStatusClose()override#

Close the stream cleanly.

For writable streams, this will attempt to flush any pending data before releasing the underlying resource.

AfterClose() is called,closed() returns true and the stream is not available for further operations.

virtualboolclosed()constoverride#

Return whether the stream is closed.

virtualResult<int64_t>Tell()constoverride#

Return the position in this stream.

virtualStatusWrite(constvoid*data,int64_tnbytes)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 ownedBuffer.

Public Static Functions

staticResult<std::shared_ptr<FileOutputStream>>Open(conststd::string&path,boolappend=false)#

Open a local file for writing, truncating any existing file.

When opening a new file, any existing file with the indicated path is truncated to 0 bytes, deleting any existing data

Parameters:
  • path[in] with UTF8 encoding

  • append[in] append to existing file, otherwise truncate to 0 bytes

Returns:

an openFileOutputStream

staticResult<std::shared_ptr<FileOutputStream>>Open(intfd)#

Open a file descriptor for writing.

The underlying file isn’t truncated.

The file descriptor becomes owned by the

OutputStream, and will be closed onClose() or destruction.

Parameters:

fd[in] file descriptor

Returns:

an openFileOutputStream

classMemoryMappedFile:publicarrow::io::ReadWriteFileInterface#

A file interface that uses memory-mapped files for memory interactions.

This implementation supports zero-copy reads. The same class is used for both reading and writing.

If opening a file in a writable mode, it is not truncated first as withFileOutputStream.

Public Functions

virtualStatusClose()override#

Close the stream cleanly.

For writable streams, this will attempt to flush any pending data before releasing the underlying resource.

AfterClose() is called,closed() returns true and the stream is not available for further operations.

virtualboolclosed()constoverride#

Return whether the stream is closed.

virtualResult<int64_t>Tell()constoverride#

Return the position in this stream.

virtualResult<int64_t>Read(int64_tnbytes,void*out)override#

Read data from current file position.

Read at mostnbytes from the current file position intoout. The number of bytes read is returned.

virtualResult<std::shared_ptr<Buffer>>Read(int64_tnbytes)override#

Read data from current file position.

Read at mostnbytes from the current file position. Less bytes may be read if EOF is reached. This method updates the current file position.

In some cases (e.g. a memory-mapped file), this method may avoid a memory copy.

virtualResult<std::shared_ptr<Buffer>>ReadAt(int64_tposition,int64_tnbytes)override#

Read data from given file position.

At mostnbytes bytes are read, but it can be less if EOF is reached.

Parameters:
  • position[in] Where to read bytes from

  • nbytes[in] The number of bytes to read

Returns:

A buffer containing the bytes read, or an error

virtualResult<int64_t>ReadAt(int64_tposition,int64_tnbytes,void*out)override#

Read data from given file position.

At mostnbytes bytes are read. The number of bytes read is returned (it can be less thannbytes if EOF is reached).

This method can be safely called from multiple threads concurrently. It is unspecified whether this method updates the file position or not.

The default RandomAccessFile-provided implementation uses Seek() andRead(), but subclasses may override it with a more efficient implementation that doesn’t depend on implicit file positioning.

Parameters:
  • position[in] Where to read bytes from

  • nbytes[in] The number of bytes to read

  • out[out] The buffer to read bytes into

Returns:

The number of bytes read, or an error

virtualFuture<std::shared_ptr<Buffer>>ReadAsync(constIOContext&,int64_tposition,int64_tnbytes)override#

EXPERIMENTAL: Read data asynchronously.

virtualStatusWillNeed(conststd::vector<ReadRange>&ranges)override#

EXPERIMENTAL: Inform that the given ranges may be read soon.

Some implementations might arrange to prefetch some of the data. However, no guarantee is made and the default implementation does nothing. For robust prefetching, useReadAt() orReadAsync().

virtualboolsupports_zero_copy()constoverride#

Return true ifInputStream is capable of zero copyBuffer reads.

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

virtualStatusWrite(constvoid*data,int64_tnbytes)override#

Write data at the current position in the file. Thread-safe.

StatusResize(int64_tnew_size)#

Set the size of the map to new_size.

virtualStatusWriteAt(int64_tposition,constvoid*data,int64_tnbytes)override#

Write data at a particular position in the file. Thread-safe.

virtualResult<int64_t>GetSize()override#

Return the total file size in bytes.

This method does not read or move the current file position, so is safe to call concurrently with e.g.ReadAt().

Public Static Functions

staticResult<std::shared_ptr<MemoryMappedFile>>Create(conststd::string&path,int64_tsize)#

Create new file with indicated size, return in read/write mode.

Buffering input / output wrappers#

classBufferedInputStream:publicarrow::io::internal::InputStreamConcurrencyWrapper<BufferedInputStream>#

AnInputStream that performs buffered reads from an unbufferedInputStream, which can mitigate the overhead of many small reads in some cases.

Public Functions

StatusSetBufferSize(int64_tnew_buffer_size)#

Resize internal read buffer; calls to Read(…) will read at least this many bytes from the rawInputStream if possible.

Parameters:

new_buffer_size[in] the new read buffer size

Returns:

Status

int64_tbytes_buffered()const#

Return the number of remaining bytes in the read buffer.

int64_tbuffer_size()const#

Return the current size of the internal buffer.

std::shared_ptr<InputStream>Detach()#

Release the rawInputStream.

Any data buffered will be discarded. Further operations on this object are invalid

Returns:

raw the underlyingInputStream

std::shared_ptr<InputStream>raw()const#

Return the unbufferedInputStream.

virtualboolclosed()constoverride#

Return whether the stream is closed.

virtualResult<std::shared_ptr<constKeyValueMetadata>>ReadMetadata()override#

Read and return stream metadata.

If the stream implementation doesn’t support metadata, empty metadata is returned. Note that it is allowed to return a null pointer rather than an allocated empty metadata.

virtualFuture<std::shared_ptr<constKeyValueMetadata>>ReadMetadataAsync(constIOContext&io_context)override#

Read stream metadata asynchronously.

Public Static Functions

staticResult<std::shared_ptr<BufferedInputStream>>Create(int64_tbuffer_size,MemoryPool*pool,std::shared_ptr<InputStream>raw,int64_traw_read_bound=-1)#

Create aBufferedInputStream from a rawInputStream.

Parameters:
  • buffer_size[in] the size of the temporary read buffer

  • pool[in] aMemoryPool to use for allocations

  • raw[in] a rawInputStream

  • raw_read_bound[in] a bound on the maximum number of bytes to read from the raw input stream. The default -1 indicates that it is unbounded

Returns:

the createdBufferedInputStream

classBufferedOutputStream:publicarrow::io::OutputStream#

Public Functions

StatusSetBufferSize(int64_tnew_buffer_size)#

Resize internal buffer.

Parameters:

new_buffer_size[in] the new buffer size

Returns:

Status

int64_tbuffer_size()const#

Return the current size of the internal buffer.

int64_tbytes_buffered()const#

Return the number of remaining bytes that have not been flushed to the rawOutputStream.

Result<std::shared_ptr<OutputStream>>Detach()#

Flush any buffered writes and release the rawOutputStream.

Further operations on this object are invalid

Returns:

the underlyingOutputStream

virtualStatusClose()override#

Close the buffered output stream.

This implicitly closes the underlying raw output stream.

virtualStatusAbort()override#

Close the stream abruptly.

This method does not guarantee that any pending data is flushed. It merely releases any underlying resource used by the stream for its operation.

AfterAbort() is called,closed() returns true and the stream is not available for further operations.

virtualboolclosed()constoverride#

Return whether the stream is closed.

virtualResult<int64_t>Tell()constoverride#

Return the position in this stream.

virtualStatusWrite(constvoid*data,int64_tnbytes)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 ownedBuffer.

virtualStatusWrite(conststd::shared_ptr<Buffer>&data)override#

Write the given data to the stream.

Since theBuffer owns its memory, this method can avoid a copy if buffering is required. SeeWrite(const void*, int64_t) for details.

virtualStatusFlush()override#

Flush buffered bytes, if any.

std::shared_ptr<OutputStream>raw()const#

Return the underlying raw output stream.

Public Static Functions

staticResult<std::shared_ptr<BufferedOutputStream>>Create(int64_tbuffer_size,MemoryPool*pool,std::shared_ptr<OutputStream>raw)#

Create a buffered output stream wrapping the given output stream.

Parameters:
  • buffer_size[in] the size of the temporary write buffer

  • pool[in] aMemoryPool to use for allocations

  • raw[in] anotherOutputStream

Returns:

the createdBufferedOutputStream

Compressed input / output wrappers#

classCompressedInputStream:publicarrow::io::internal::InputStreamConcurrencyWrapper<CompressedInputStream>#

Public Functions

virtualboolclosed()constoverride#

Return whether the stream is closed.

virtualResult<std::shared_ptr<constKeyValueMetadata>>ReadMetadata()override#

Read and return stream metadata.

If the stream implementation doesn’t support metadata, empty metadata is returned. Note that it is allowed to return a null pointer rather than an allocated empty metadata.

virtualFuture<std::shared_ptr<constKeyValueMetadata>>ReadMetadataAsync(constIOContext&io_context)override#

Read stream metadata asynchronously.

std::shared_ptr<InputStream>raw()const#

Return the underlying raw input stream.

Public Static Functions

staticResult<std::shared_ptr<CompressedInputStream>>Make(util::Codec*codec,conststd::shared_ptr<InputStream>&raw,MemoryPool*pool=default_memory_pool())#

Create a compressed input stream wrapping the given input stream.

The codec must be capable of streaming decompression. Some codecs, like Snappy, are not able to do so.

classCompressedOutputStream:publicarrow::io::OutputStream#

Public Functions

virtualStatusClose()override#

Close the compressed output stream.

This implicitly closes the underlying raw output stream.

virtualStatusAbort()override#

Close the stream abruptly.

This method does not guarantee that any pending data is flushed. It merely releases any underlying resource used by the stream for its operation.

AfterAbort() is called,closed() returns true and the stream is not available for further operations.

virtualboolclosed()constoverride#

Return whether the stream is closed.

virtualResult<int64_t>Tell()constoverride#

Return the position in this stream.

virtualStatusWrite(constvoid*data,int64_tnbytes)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 ownedBuffer.

virtualStatusFlush()override#

Flush buffered bytes, if any.

std::shared_ptr<OutputStream>raw()const#

Return the underlying raw output stream.

Public Static Functions

staticResult<std::shared_ptr<CompressedOutputStream>>Make(util::Codec*codec,conststd::shared_ptr<OutputStream>&raw,MemoryPool*pool=default_memory_pool())#

Create a compressed output stream wrapping the given output stream.

The codec must be capable of streaming compression. Some codecs, like Snappy, are not able to do so.

Transforming input wrapper#

classTransformInputStream:publicarrow::io::InputStream#

Public Functions

virtualStatusClose()override#

Close the stream cleanly.

For writable streams, this will attempt to flush any pending data before releasing the underlying resource.

AfterClose() is called,closed() returns true and the stream is not available for further operations.

virtualStatusAbort()override#

Close the stream abruptly.

This method does not guarantee that any pending data is flushed. It merely releases any underlying resource used by the stream for its operation.

AfterAbort() is called,closed() returns true and the stream is not available for further operations.

virtualboolclosed()constoverride#

Return whether the stream is closed.

virtualResult<int64_t>Read(int64_tnbytes,void*out)override#

Read data from current file position.

Read at mostnbytes from the current file position intoout. The number of bytes read is returned.

virtualResult<std::shared_ptr<Buffer>>Read(int64_tnbytes)override#

Read data from current file position.

Read at mostnbytes from the current file position. Less bytes may be read if EOF is reached. This method updates the current file position.

In some cases (e.g. a memory-mapped file), this method may avoid a memory copy.

virtualResult<std::shared_ptr<constKeyValueMetadata>>ReadMetadata()override#

Read and return stream metadata.

If the stream implementation doesn’t support metadata, empty metadata is returned. Note that it is allowed to return a null pointer rather than an allocated empty metadata.

virtualFuture<std::shared_ptr<constKeyValueMetadata>>ReadMetadataAsync(constIOContext&io_context)override#

Read stream metadata asynchronously.

virtualResult<int64_t>Tell()constoverride#

Return the position in this stream.

On this page