File Formats

CSV reader

struct ConvertOptions

Public Functions

Status Validate() const

Test that all set options are valid.

Public Members

bool check_utf8 = true

Whether to check UTF8 validity of string columns.

std::unordered_map<std::string, std::shared_ptr<DataType>> column_types

Optional per-column types (disabling type inference on those columns)

std::vector<std::string> null_values

Recognized spellings for null values.

std::vector<std::string> true_values

Recognized spellings for boolean true values.

std::vector<std::string> false_values

Recognized spellings for boolean false values.

bool strings_can_be_null = false

Whether string / binary columns can have null values.

If true, then strings in “null_values” are considered null for string columns. If false, then all strings are valid string values.

bool quoted_strings_can_be_null = true

Whether quoted values can be null.

If true, then strings in “null_values” are also considered null when they appear quoted in the CSV file. Otherwise, quoted values are never considered null.

bool auto_dict_encode = false

Whether to try to automatically dict-encode string / binary data.

If true, then when type inference detects a string or binary column, it is dict-encoded up to auto_dict_max_cardinality distinct values (per chunk), after which it switches to regular encoding.

This setting is ignored for non-inferred columns (those in column_types).

char decimal_point = '.'

Decimal point character for floating-point and decimal data.

std::vector<std::string> include_columns

If non-empty, indicates the names of columns from the CSV file that should be actually read and converted (in the vector’s order).

Columns not in this vector will be ignored.

bool include_missing_columns = false

If false, columns in include_columns but not in the CSV file will error out.

If true, columns in include_columns but not in the CSV file will produce a column of nulls (whose type is selected using column_types, or null by default) This option is ignored if include_columns is empty.

std::vector<std::shared_ptr<TimestampParser>> timestamp_parsers

User-defined timestamp parsers, using the virtual parser interface in arrow/util/value_parsing.h.

More than one parser can be specified, and the CSV conversion logic will try parsing values starting from the beginning of this vector. If no parsers are specified, we use the default built-in ISO-8601 parser.

Public Static Functions

static ConvertOptions Defaults()

Create conversion options with default values, including conventional values for null_values, true_values and false_values

struct ParseOptions

Public Functions

Status Validate() const

Test that all set options are valid.

Public Members

char delimiter = ','

Field delimiter.

bool quoting = true

Whether quoting is used.

char quote_char = '"'

Quoting character (if quoting is true)

bool double_quote = true

Whether a quote inside a value is double-quoted.

bool escaping = false

Whether escaping is used.

char escape_char = kDefaultEscapeChar

Escaping character (if escaping is true)

bool newlines_in_values = false

Whether values are allowed to contain CR (0x0d) and LF (0x0a) characters.

bool ignore_empty_lines = true

Whether empty lines are ignored.

If false, an empty line represents a single empty value (assuming a one-column CSV file).

InvalidRowHandler invalid_row_handler

A handler function for rows which do not have the correct number of columns.

Public Static Functions

static ParseOptions Defaults()

Create parsing options with default values.

struct ReadOptions

Public Functions

Status Validate() const

Test that all set options are valid.

Public Members

bool use_threads = true

Whether to use the global CPU thread pool.

int32_t block_size = 1 << 20

Block size we request from the IO layer.

This will determine multi-threading granularity as well as the size of individual record batches. Minimum valid value for block size is 1

int32_t skip_rows = 0

Number of header rows to skip (not including the row of column names, if any)

int32_t skip_rows_after_names = 0

Number of rows to skip after the column names are read, if any.

std::vector<std::string> column_names

Column names for the target table.

If empty, fall back on autogenerate_column_names.

bool autogenerate_column_names = false

Whether to autogenerate column names if column_names is empty.

If true, column names will be of the form “f0”, “f1”… If false, column names will be read from the first CSV row after skip_rows.

Public Static Functions

static ReadOptions Defaults()

Create read options with default values.

class TableReader

A class that reads an entire CSV file into a Arrow Table.

Public Functions

virtual Result<std::shared_ptr<Table>> Read() = 0

Read the entire CSV file and convert it to a Arrow Table.

virtual Future<std::shared_ptr<Table>> ReadAsync() = 0

Read the entire CSV file and convert it to a Arrow Table.

Public Static Functions

static Result<std::shared_ptr<TableReader>> Make(io::IOContext io_context, std::shared_ptr<io::InputStream> input, const ReadOptions&, const ParseOptions&, const ConvertOptions&)

Create a TableReader instance.

class StreamingReader : public arrow::RecordBatchReader

A class that reads a CSV file incrementally.

Caveats:

Public Functions

virtual int64_t bytes_read() const = 0

Return the number of bytes which have been read and processed.

The returned number includes CSV bytes which the StreamingReader has finished processing, but not bytes for which some processing (e.g. CSV parsing or conversion to Arrow layout) is still ongoing.

Furthermore, the following rules apply:

  • bytes skipped by ReadOptions.skip_rows are counted as being read before any records are returned.

  • bytes read while parsing the header are counted as being read before any records are returned.

  • bytes skipped by ReadOptions.skip_rows_after_names are counted after the first batch is returned.

Public Static Functions

static Future<std::shared_ptr<StreamingReader>> MakeAsync(io::IOContext io_context, std::shared_ptr<io::InputStream> input, arrow::internal::Executor *cpu_executor, const ReadOptions&, const ParseOptions&, const ConvertOptions&)

Create a StreamingReader instance.

This involves some I/O as the first batch must be loaded during the creation process so it is returned as a future

Currently, the StreamingReader is not async-reentrant and does not do any fan-out parsing (see ARROW-11889)

CSV writer

struct WriteOptions

Public Functions

Status Validate() const

Test that all set options are valid.

Public Members

bool include_header = true

Whether to write an initial header line with column names.

int32_t batch_size = 1024

Maximum number of rows processed at a time.

The CSV writer converts and writes data in batches of N rows. This number can impact performance.

char delimiter = ','

Field delimiter.

std::string null_string

The string to write for null values. Quotes are not allowed in this string.

io::IOContext io_context

IO context for writing.

std::string eol = "\n"

The end of line character to use for ending rows.

QuotingStyle quoting_style = QuotingStyle::Needed

Quoting style.

Public Static Functions

static WriteOptions Defaults()

Create write options with default values.

Status WriteCSV(const Table &table, const WriteOptions &options, arrow::io::OutputStream *output)

Convert table to CSV and write the result to output.

Experimental

Status WriteCSV(const RecordBatch &batch, const WriteOptions &options, arrow::io::OutputStream *output)

Convert batch to CSV and write the result to output.

Experimental

Status WriteCSV(const std::shared_ptr<RecordBatchReader> &reader, const WriteOptions &options, arrow::io::OutputStream *output)

Convert batches read through a RecordBatchReader to CSV and write the results to output.

Experimental

Result<std::shared_ptr<ipc::RecordBatchWriter>> MakeCSVWriter(std::shared_ptr<io::OutputStream> sink, const std::shared_ptr<Schema> &schema, const WriteOptions &options = WriteOptions::Defaults())

Create a new CSV writer.

User is responsible for closing the actual OutputStream.

Parameters:
  • sink[in] output stream to write to

  • schema[in] the schema of the record batches to be written

  • options[in] options for serialization

Returns:

Result<std::shared_ptr<RecordBatchWriter>>

Result<std::shared_ptr<ipc::RecordBatchWriter>> MakeCSVWriter(io::OutputStream *sink, const std::shared_ptr<Schema> &schema, const WriteOptions &options = WriteOptions::Defaults())

Create a new CSV writer.

Parameters:
  • sink[in] output stream to write to (does not take ownership)

  • schema[in] the schema of the record batches to be written

  • options[in] options for serialization

Returns:

Result<std::shared_ptr<RecordBatchWriter>>

Line-separated JSON

enum class arrow::json::UnexpectedFieldBehavior : char

Values:

enumerator Ignore

Unexpected JSON fields are ignored.

enumerator Error

Unexpected JSON fields error out.

enumerator InferType

Unexpected JSON fields are type-inferred and included in the output.

struct ReadOptions

Public Members

bool use_threads = true

Whether to use the global CPU thread pool.

int32_t block_size = 1 << 20

Block size we request from the IO layer; also determines the size of chunks when use_threads is true.

Public Static Functions

static ReadOptions Defaults()

Create read options with default values.

struct ParseOptions

Public Members

std::shared_ptr<Schema> explicit_schema

Optional explicit schema (disables type inference on those fields)

bool newlines_in_values = false

Whether objects may be printed across multiple lines (for example pretty-printed)

If true, parsing may be slower.

UnexpectedFieldBehavior unexpected_field_behavior = UnexpectedFieldBehavior::InferType

How JSON fields outside of explicit_schema (if given) are treated.

Public Static Functions

static ParseOptions Defaults()

Create parsing options with default values.

class TableReader

A class that reads an entire JSON file into a Arrow Table.

The file is expected to consist of individual line-separated JSON objects

Public Functions

virtual Result<std::shared_ptr<Table>> Read() = 0

Read the entire JSON file and convert it to a Arrow Table.

Public Static Functions

static Result<std::shared_ptr<TableReader>> Make(MemoryPool *pool, std::shared_ptr<io::InputStream> input, const ReadOptions&, const ParseOptions&)

Create a TableReader instance.

Parquet reader

class ReaderProperties

Public Functions

inline bool is_buffered_stream_enabled() const

Buffered stream reading allows the user to control the memory usage of parquet readers.

This ensure that all RandomAccessFile::ReadAt calls are wrapped in a buffered reader that uses a fix sized buffer (of size buffer_size()) instead of the full size of the ReadAt.

The primary reason for this control knobs is for resource control and not performance.

class ArrowReaderProperties

EXPERIMENTAL: Properties for configuring FileReader behavior.

Public Functions

inline void set_pre_buffer(bool pre_buffer)

Enable read coalescing.

When enabled, the Arrow reader will pre-buffer necessary regions of the file in-memory. This is intended to improve performance on high-latency filesystems (e.g. Amazon S3).

inline void set_cache_options(::arrow::io::CacheOptions options)

Set options for read coalescing.

This can be used to tune the implementation for characteristics of different filesystems.

inline void set_io_context(const ::arrow::io::IOContext &ctx)

Set execution context for read coalescing.

inline void set_coerce_int96_timestamp_unit(::arrow::TimeUnit::type unit)

Set timestamp unit to use for deprecated INT96-encoded timestamps (default is NANO).

class ParquetFileReader

Public Functions

void PreBuffer(const std::vector<int> &row_groups, const std::vector<int> &column_indices, const ::arrow::io::IOContext &ctx, const ::arrow::io::CacheOptions &options)

Pre-buffer the specified column indices in all row groups.

Readers can optionally call this to cache the necessary slices of the file in-memory before deserialization. Arrow readers can automatically do this via an option. This is intended to increase performance when reading from high-latency filesystems (e.g. Amazon S3).

After calling this, creating readers for row groups/column indices that were not buffered may fail. Creating multiple readers for the a subset of the buffered regions is acceptable. This may be called again to buffer a different set of row groups/columns.

If memory usage is a concern, note that data will remain buffered in memory until either PreBuffer() is called again, or the reader itself is destructed. Reading - and buffering - only one row group at a time may be useful.

This method may throw.

::arrow::Future WhenBuffered(const std::vector<int> &row_groups, const std::vector<int> &column_indices) const

Wait for the specified row groups and column indices to be pre-buffered.

After the returned Future completes, reading the specified row groups/columns will not block.

PreBuffer must be called first. This method does not throw.

struct Contents
class FileReader

Arrow read adapter class for deserializing Parquet files as Arrow row batches.

This interfaces caters for different use cases and thus provides different interfaces. In its most simplistic form, we cater for a user that wants to read the whole Parquet at once with the FileReader::ReadTable method.

More advanced users that also want to implement parallelism on top of each single Parquet files should do this on the RowGroup level. For this, they can call FileReader::RowGroup(i)->ReadTable to receive only the specified RowGroup as a table.

In the most advanced situation, where a consumer wants to independently read RowGroups in parallel and consume each column individually, they can call FileReader::RowGroup(i)->Column(j)->Read and receive an arrow::Column instance.

The parquet format supports an optional integer field_id which can be assigned to a field. Arrow will convert these field IDs to a metadata key named PARQUET:field_id on the appropriate field.

Public Functions

virtual ::arrow::Status GetSchema(std::shared_ptr<::arrow::Schema> *out) = 0

Return arrow schema for all the columns.

virtual ::arrow::Status ReadColumn(int i, std::shared_ptr<::arrow::ChunkedArray> *out) = 0

Read column as a whole into a chunked array.

The indicated column index is relative to the schema

virtual ::arrow::Status GetRecordBatchReader(const std::vector<int> &row_group_indices, std::unique_ptr<::arrow::RecordBatchReader> *out) = 0

Return a RecordBatchReader of row groups selected from row_group_indices.

Note that the ordering in row_group_indices matters. FileReaders must outlive their RecordBatchReaders.

Returns:

error Status if row_group_indices contains an invalid index

virtual ::arrow::Status GetRecordBatchReader(const std::vector<int> &row_group_indices, const std::vector<int> &column_indices, std::unique_ptr<::arrow::RecordBatchReader> *out) = 0

Return a RecordBatchReader of row groups selected from row_group_indices, whose columns are selected by column_indices.

Note that the ordering in row_group_indices and column_indices matter. FileReaders must outlive their RecordBatchReaders.

Returns:

error Status if either row_group_indices or column_indices contains an invalid index

virtual ::arrow::Result< std::function<::arrow::Future< std::shared_ptr<::arrow::RecordBatch > >)> > GetRecordBatchGenerator (std::shared_ptr< FileReader > reader, const std::vector< int > row_group_indices, const std::vector< int > column_indices, ::arrow::internal::Executor *cpu_executor=NULLPTR, int64_t rows_to_readahead=0)=0

Return a generator of record batches.

The FileReader must outlive the generator, so this requires that you pass in a shared_ptr.

Returns:

error Result if either row_group_indices or column_indices contains an invalid index

virtual ::arrow::Status ReadTable(std::shared_ptr<::arrow::Table> *out) = 0

Read all columns into a Table.

virtual ::arrow::Status ReadTable(const std::vector<int> &column_indices, std::shared_ptr<::arrow::Table> *out) = 0

Read the given columns into a Table.

The indicated column indices are relative to the schema

virtual ::arrow::Status ScanContents(std::vector<int> columns, const int32_t column_batch_size, int64_t *num_rows) = 0

Scan file contents with one thread, return number of rows.

virtual std::shared_ptr<RowGroupReader> RowGroup(int row_group_index) = 0

Return a reader for the RowGroup, this object must not outlive the FileReader.

virtual int num_row_groups() const = 0

The number of row groups in the file.

virtual void set_use_threads(bool use_threads) = 0

Set whether to use multiple threads during reads of multiple columns.

By default only one thread is used.

virtual void set_batch_size(int64_t batch_size) = 0

Set number of records to read per batch for the RecordBatchReader.

Public Static Functions

static ::arrow::Status Make(::arrow::MemoryPool *pool, std::unique_ptr<ParquetFileReader> reader, const ArrowReaderProperties &properties, std::unique_ptr<FileReader> *out)

Factory function to create a FileReader from a ParquetFileReader and properties.

static ::arrow::Status Make(::arrow::MemoryPool *pool, std::unique_ptr<ParquetFileReader> reader, std::unique_ptr<FileReader> *out)

Factory function to create a FileReader from a ParquetFileReader.

class FileReaderBuilder

Experimental helper class for bindings (like Python) that struggle either with std::move or C++ exceptions.

Public Functions

::arrow::Status Open(std::shared_ptr<::arrow::io::RandomAccessFile> file, const ReaderProperties &properties = default_reader_properties(), std::shared_ptr<FileMetaData> metadata = NULLPTR)

Create FileReaderBuilder from Arrow file and optional properties / metadata.

FileReaderBuilder *memory_pool(::arrow::MemoryPool *pool)

Set Arrow MemoryPool for memory allocation.

FileReaderBuilder *properties(const ArrowReaderProperties &arg_properties)

Set Arrow reader properties.

::arrow::Status Build(std::unique_ptr<FileReader> *out)

Build FileReader instance.

::arrow::Status OpenFile(std::shared_ptr<::arrow::io::RandomAccessFile>, ::arrow::MemoryPool *allocator, std::unique_ptr<FileReader> *reader)

Build FileReader from Arrow file and MemoryPool.

Advanced settings are supported through the FileReaderBuilder class.

class StreamReader

A class for reading Parquet files using an output stream type API.

The values given must be of the correct type i.e. the type must match the file schema exactly otherwise a ParquetException will be thrown.

The user must explicitly advance to the next row using the EndRow() function or EndRow input manipulator.

Required and optional fields are supported:

  • Required fields are read using operator>>(T)

  • Optional fields are read with operator>>(arrow::util::optional<T>)

Note that operator>>(arrow::util::optional<T>) can be used to read required fields.

Similarly operator>>(T) can be used to read optional fields. However, if the value is not present then a ParquetException will be raised.

Currently there is no support for repeated fields.

Public Functions

void EndRow()

Terminate current row and advance to next one.

Throws:

ParquetException – if all columns in the row were not read or skipped.

int64_t SkipColumns(int64_t num_columns_to_skip)

Skip the data in the next columns.

If the number of columns exceeds the columns remaining on the current row then skipping is terminated - it does not continue skipping columns on the next row. Skipping of columns still requires the use ‘EndRow’ even if all remaining columns were skipped.

Returns:

Number of columns actually skipped.

int64_t SkipRows(int64_t num_rows_to_skip)

Skip the data in the next rows.

Skipping of rows is not allowed if reading of data for the current row is not finished. Skipping of rows will be terminated if the end of file is reached.

Returns:

Number of rows actually skipped.

Parquet writer

class WriterProperties
class Builder

Public Functions

inline Builder *memory_pool(MemoryPool *pool)

Specify the memory pool for the writer. Default default_memory_pool.

inline Builder *enable_dictionary()

Enable dictionary encoding in general for all columns. Default enabled.

inline Builder *disable_dictionary()

Disable dictionary encoding in general for all columns. Default enabled.

inline Builder *enable_dictionary(const std::string &path)

Enable dictionary encoding for column specified by path. Default enabled.

inline Builder *enable_dictionary(const std::shared_ptr<schema::ColumnPath> &path)

Enable dictionary encoding for column specified by path. Default enabled.

inline Builder *disable_dictionary(const std::string &path)

Disable dictionary encoding for column specified by path. Default enabled.

inline Builder *disable_dictionary(const std::shared_ptr<schema::ColumnPath> &path)

Disable dictionary encoding for column specified by path. Default enabled.

inline Builder *dictionary_pagesize_limit(int64_t dictionary_psize_limit)

Specify the dictionary page size limit per row group. Default 1MB.

inline Builder *write_batch_size(int64_t write_batch_size)

Specify the write batch size while writing batches of Arrow values into Parquet.

Default 1024.

inline Builder *max_row_group_length(int64_t max_row_group_length)

Specify the max row group length.

Default 64M.

inline Builder *data_pagesize(int64_t pg_size)

Specify the data page size.

Default 1MB.

inline Builder *data_page_version(ParquetDataPageVersion data_page_version)

Specify the data page version.

Default V1.

inline Builder *version(ParquetVersion::type version)

Specify the Parquet file version.

Default PARQUET_2_4.

inline Builder *encoding(Encoding::type encoding_type)

Define the encoding that is used when we don’t utilise dictionary encoding.

This either apply if dictionary encoding is disabled or if we fallback as the dictionary grew too large.

inline Builder *encoding(const std::string &path, Encoding::type encoding_type)

Define the encoding that is used when we don’t utilise dictionary encoding.

This either apply if dictionary encoding is disabled or if we fallback as the dictionary grew too large.

inline Builder *encoding(const std::shared_ptr<schema::ColumnPath> &path, Encoding::type encoding_type)

Define the encoding that is used when we don’t utilise dictionary encoding.

This either apply if dictionary encoding is disabled or if we fallback as the dictionary grew too large.

inline Builder *compression(Compression::type codec)

Specify compression codec in general for all columns.

Default UNCOMPRESSED.

inline Builder *max_statistics_size(size_t max_stats_sz)

Specify max statistics size to store min max value.

Default 4KB.

inline Builder *compression(const std::string &path, Compression::type codec)

Specify compression codec for the column specified by path.

Default UNCOMPRESSED.

inline Builder *compression(const std::shared_ptr<schema::ColumnPath> &path, Compression::type codec)

Specify compression codec for the column specified by path.

Default UNCOMPRESSED.

inline Builder *compression_level(int compression_level)

Specify the default compression level for the compressor in every column.

In case a column does not have an explicitly specified compression level, the default one would be used.

The provided compression level is compressor specific. The user would have to familiarize oneself with the available levels for the selected compressor. If the compressor does not allow for selecting different compression levels, calling this function would not have any effect. Parquet and Arrow do not validate the passed compression level. If no level is selected by the user or if the special std::numeric_limits<int>::min() value is passed, then Arrow selects the compression level.

inline Builder *compression_level(const std::string &path, int compression_level)

Specify a compression level for the compressor for the column described by path.

The provided compression level is compressor specific. The user would have to familiarize oneself with the available levels for the selected compressor. If the compressor does not allow for selecting different compression levels, calling this function would not have any effect. Parquet and Arrow do not validate the passed compression level. If no level is selected by the user or if the special std::numeric_limits<int>::min() value is passed, then Arrow selects the compression level.

inline Builder *compression_level(const std::shared_ptr<schema::ColumnPath> &path, int compression_level)

Specify a compression level for the compressor for the column described by path.

The provided compression level is compressor specific. The user would have to familiarize oneself with the available levels for the selected compressor. If the compressor does not allow for selecting different compression levels, calling this function would not have any effect. Parquet and Arrow do not validate the passed compression level. If no level is selected by the user or if the special std::numeric_limits<int>::min() value is passed, then Arrow selects the compression level.

inline Builder *encryption(std::shared_ptr<FileEncryptionProperties> file_encryption_properties)

Define the file encryption properties.

Default NULL.

inline Builder *enable_statistics()

Enable statistics in general.

Default enabled.

inline Builder *disable_statistics()

Disable statistics in general.

Default enabled.

inline Builder *enable_statistics(const std::string &path)

Enable statistics for the column specified by path.

Default enabled.

inline Builder *enable_statistics(const std::shared_ptr<schema::ColumnPath> &path)

Enable statistics for the column specified by path.

Default enabled.

inline Builder *disable_statistics(const std::string &path)

Disable statistics for the column specified by path.

Default enabled.

inline Builder *disable_statistics(const std::shared_ptr<schema::ColumnPath> &path)

Disable statistics for the column specified by path.

Default enabled.

inline std::shared_ptr<WriterProperties> build()

Build the WriterProperties with the builder parameters.

Returns:

The WriterProperties defined by the builder.

class ArrowWriterProperties

Public Functions

inline bool compliant_nested_types() const

Enable nested type naming according to the parquet specification.

Older versions of arrow wrote out field names for nested lists based on the name of the field. According to the parquet specification they should always be “element”.

inline EngineVersion engine_version() const

The underlying engine version to use when writing Arrow data.

V2 is currently the latest V1 is considered deprecated but left in place in case there are bugs detected in V2.

class Builder

Public Functions

inline Builder *store_schema()

EXPERIMENTAL: Write binary serialized Arrow schema to the file, to enable certain read options (like “read_dictionary”) to be set automatically.

class FileWriter

Iterative FileWriter class.

Start a new RowGroup or Chunk with NewRowGroup. Write column-by-column the whole column chunk.

If PARQUET:field_id is present as a metadata key on a field, and the corresponding value is a nonnegative integer, then it will be used as the field_id in the parquet file.

Public Functions

virtual ::arrow::Status WriteTable(const ::arrow::Table &table, int64_t chunk_size) = 0

Write a Table to Parquet.

virtual ::arrow::Status WriteColumnChunk(const std::shared_ptr<::arrow::ChunkedArray> &data, int64_t offset, int64_t size) = 0

Write ColumnChunk in row group using slice of a ChunkedArray.

::arrow::Status parquet::arrow::WriteTable(const ::arrow::Table &table, MemoryPool *pool, std::shared_ptr<::arrow::io::OutputStream> sink, int64_t chunk_size, std::shared_ptr<WriterProperties> properties = default_writer_properties(), std::shared_ptr<ArrowWriterProperties> arrow_properties = default_arrow_writer_properties())

Write a Table to Parquet.

class StreamWriter

A class for writing Parquet files using an output stream type API.

The values given must be of the correct type i.e. the type must match the file schema exactly otherwise a ParquetException will be thrown.

The user must explicitly indicate the end of the row using the EndRow() function or EndRow output manipulator.

A maximum row group size can be configured, the default size is 512MB. Alternatively the row group size can be set to zero and the user can create new row groups by calling the EndRowGroup() function or using the EndRowGroup output manipulator.

Required and optional fields are supported:

  • Required fields are written using operator<<(T)

  • Optional fields are written using operator<<(arrow::util::optional<T>).

Note that operator<<(T) can be used to write optional fields.

Similarly, operator<<(arrow::util::optional<T>) can be used to write required fields. However if the optional parameter does not have a value (i.e. it is nullopt) then a ParquetException will be raised.

Currently there is no support for repeated fields.

Public Functions

StreamWriter &operator<<(bool v)

Output operators for required fields.

These can also be used for optional fields when a value must be set.

template<int N>
inline StreamWriter &operator<<(const char (&v)[N])

Output operators for fixed length strings.

StreamWriter &operator<<(const char *v)

Output operators for variable length strings.

template<typename T>
inline StreamWriter &operator<<(const optional<T> &v)

Output operator for optional fields.

int64_t SkipColumns(int num_columns_to_skip)

Skip the next N columns of optional data.

If there are less than N columns remaining then the excess columns are ignored.

Throws:

ParquetException – if there is an attempt to skip any required column.

Returns:

Number of columns actually skipped.

void EndRow()

Terminate the current row and advance to next one.

Throws:

ParquetException – if all columns in the row were not written or skipped.

void EndRowGroup()

Terminate the current row group and create new one.

struct FixedStringView

Helper class to write fixed length strings.

This is useful as the standard string view (such as arrow::util::string_view) is for variable length data.

ORC

class ORCFileReader

Read an Arrow Table or RecordBatch from an ORC file.

Public Functions

Result<std::shared_ptr<Schema>> ReadSchema()

Return the schema read from the ORC file.

Returns:

the returned Schema object

Result<std::shared_ptr<Table>> Read()

Read the file as a Table.

The table will be composed of one record batch per stripe.

Returns:

the returned Table

Result<std::shared_ptr<Table>> Read(const std::shared_ptr<Schema> &schema)

Read the file as a Table.

The table will be composed of one record batch per stripe.

Parameters:

schema[in] the Table schema

Returns:

the returned Table

Result<std::shared_ptr<Table>> Read(const std::vector<int> &include_indices)

Read the file as a Table.

The table will be composed of one record batch per stripe.

Parameters:

include_indices[in] the selected field indices to read

Returns:

the returned Table

Result<std::shared_ptr<Table>> Read(const std::vector<std::string> &include_names)

Read the file as a Table.

The table will be composed of one record batch per stripe.

Parameters:

include_names[in] the selected field names to read

Returns:

the returned Table

Result<std::shared_ptr<Table>> Read(const std::shared_ptr<Schema> &schema, const std::vector<int> &include_indices)

Read the file as a Table.

The table will be composed of one record batch per stripe.

Parameters:
  • schema[in] the Table schema

  • include_indices[in] the selected field indices to read

Returns:

the returned Table

Result<std::shared_ptr<RecordBatch>> ReadStripe(int64_t stripe)

Read a single stripe as a RecordBatch.

Parameters:

stripe[in] the stripe index

Returns:

the returned RecordBatch

Result<std::shared_ptr<RecordBatch>> ReadStripe(int64_t stripe, const std::vector<int> &include_indices)

Read a single stripe as a RecordBatch.

Parameters:
  • stripe[in] the stripe index

  • include_indices[in] the selected field indices to read

Returns:

the returned RecordBatch

Result<std::shared_ptr<RecordBatch>> ReadStripe(int64_t stripe, const std::vector<std::string> &include_names)

Read a single stripe as a RecordBatch.

Parameters:
  • stripe[in] the stripe index

  • include_names[in] the selected field names to read

Returns:

the returned RecordBatch

Status Seek(int64_t row_number)

Seek to designated row.

Invoke NextStripeReader() after seek will return stripe reader starting from designated row.

Parameters:

row_number[in] the rows number to seek

Result<std::shared_ptr<RecordBatchReader>> NextStripeReader(int64_t batch_size)

Get a stripe level record batch iterator.

Each record batch will have up to batch_size rows. NextStripeReader serves as a fine grained alternative to ReadStripe which may cause OOM issues by loading the whole stripe into memory.

Note this will only read rows for the current stripe, not the entire file.

Parameters:

batch_size[in] the maximum number of rows in each record batch

Returns:

the returned stripe reader

Result<std::shared_ptr<RecordBatchReader>> NextStripeReader(int64_t batch_size, const std::vector<int> &include_indices)

Get a stripe level record batch iterator.

Each record batch will have up to batch_size rows. NextStripeReader serves as a fine grained alternative to ReadStripe which may cause OOM issues by loading the whole stripe into memory.

Note this will only read rows for the current stripe, not the entire file.

Parameters:
  • batch_size[in] the maximum number of rows in each record batch

  • include_indices[in] the selected field indices to read

Returns:

the stripe reader

Result<std::shared_ptr<RecordBatchReader>> GetRecordBatchReader(int64_t batch_size, const std::vector<std::string> &include_names)

Get a record batch iterator for the entire file.

Each record batch will have up to batch_size rows.

Parameters:
  • batch_size[in] the maximum number of rows in each record batch

  • include_names[in] the selected field names to read, if not empty (otherwise all fields are read)

Returns:

the record batch iterator

int64_t NumberOfStripes()

The number of stripes in the file.

int64_t NumberOfRows()

The number of rows in the file.

FileVersion GetFileVersion()

Get the format version of the file.

Currently known values are 0.11 and 0.12.

Returns:

The FileVersion of the ORC file.

std::string GetSoftwareVersion()

Get the software instance and version that wrote this file.

Returns:

a user-facing string that specifies the software version

Result<Compression::type> GetCompression()

Get the compression kind of the file.

Returns:

The kind of compression in the ORC file.

int64_t GetCompressionSize()

Get the buffer size for the compression.

Returns:

Number of bytes to buffer for the compression codec.

int64_t GetRowIndexStride()

Get the number of rows per an entry in the row index.

Returns:

the number of rows per an entry in the row index or 0 if there is no row index.

WriterId GetWriterId()

Get ID of writer that generated the file.

Returns:

UNKNOWN_WRITER if the writer ID is undefined

int32_t GetWriterIdValue()

Get the writer id value when getWriterId() returns an unknown writer.

Returns:

the integer value of the writer ID.

WriterVersion GetWriterVersion()

Get the version of the writer.

Returns:

the version of the writer.

int64_t GetNumberOfStripeStatistics()

Get the number of stripe statistics in the file.

Returns:

the number of stripe statistics

int64_t GetContentLength()

Get the length of the data stripes in the file.

Returns:

return the number of bytes in stripes

int64_t GetStripeStatisticsLength()

Get the length of the file stripe statistics.

Returns:

the number of compressed bytes in the file stripe statistics

int64_t GetFileFooterLength()

Get the length of the file footer.

Returns:

the number of compressed bytes in the file footer

int64_t GetFilePostscriptLength()

Get the length of the file postscript.

Returns:

the number of bytes in the file postscript

int64_t GetFileLength()

Get the total length of the file.

Returns:

the number of bytes in the file

std::string GetSerializedFileTail()

Get the serialized file tail.

Usefull if another reader of the same file wants to avoid re-reading the file tail. See ReadOptions.SetSerializedFileTail().

Returns:

a string of bytes with the file tail

Result<std::shared_ptr<const KeyValueMetadata>> ReadMetadata()

Return the metadata read from the ORC file.

Returns:

A KeyValueMetadata object containing the ORC metadata

Public Static Functions

static Result<std::unique_ptr<ORCFileReader>> Open(const std::shared_ptr<io::RandomAccessFile> &file, MemoryPool *pool)

Creates a new ORC reader.

Parameters:
  • file[in] the data source

  • pool[in] a MemoryPool to use for buffer allocations

Returns:

the returned reader object

struct WriteOptions

Options for the ORC Writer.

Public Members

int64_t batch_size = 1024

Number of rows the ORC writer writes at a time, default 1024.

FileVersion file_version = FileVersion(0, 12)

Which ORC file version to use, default FileVersion(0, 12)

int64_t stripe_size = 64 * 1024 * 1024

Size of each ORC stripe in bytes, default 64 MiB.

Compression::type compression = Compression::UNCOMPRESSED

The compression codec of the ORC file, there is no compression by default.

int64_t compression_block_size = 64 * 1024

The size of each compression block in bytes, default 64 KiB.

CompressionStrategy compression_strategy = CompressionStrategy::kSpeed

The compression strategy i.e.

speed vs size reduction, default CompressionStrategy::kSpeed

int64_t row_index_stride = 10000

The number of rows per an entry in the row index, default 10000.

double padding_tolerance = 0.0

The padding tolerance, default 0.0.

double dictionary_key_size_threshold = 0.0

The dictionary key size threshold.

0 to disable dictionary encoding. 1 to always enable dictionary encoding, default 0.0

std::vector<int64_t> bloom_filter_columns

The array of columns that use the bloom filter, default empty.

double bloom_filter_fpp = 0.05

The upper limit of the false-positive rate of the bloom filter, default 0.05.

class ORCFileWriter

Write an Arrow Table or RecordBatch to an ORC file.

Public Functions

Status Write(const Table &table)

Write a table.

Parameters:

table[in] the Arrow table from which data is extracted

Returns:

Status

Status Close()

Close an ORC writer (orc::Writer)

Returns:

Status

Public Static Functions

static Result<std::unique_ptr<ORCFileWriter>> Open(io::OutputStream *output_stream, const WriteOptions &write_options = WriteOptions())

Creates a new ORC writer.

Parameters:
  • output_stream[in] a pointer to the io::OutputStream to write into

  • write_options[in] the ORC writer options for Arrow

Returns:

the returned writer object