Dataset

Interface

class Fragment : public std::enable_shared_from_this<Fragment>

A granular piece of a Dataset, such as an individual file.

A Fragment can be read/scanned separately from other fragments. It yields a collection of RecordBatches when scanned, encapsulated in one or more ScanTasks.

Note that Fragments have well defined physical schemas which are reconciled by the Datasets which contain them; these physical schemas may differ from a parent Dataset’s schema and the physical schemas of sibling Fragments.

Subclassed by arrow::dataset::FileFragment, arrow::dataset::InMemoryFragment

Public Functions

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

Return the physical schema of the Fragment.

The physical schema is also called the writer schema. This method is blocking and may suffer from high latency filesystem. The schema is cached after being read once, or may be specified at construction.

virtual Result<RecordBatchGenerator> ScanBatchesAsync(const std::shared_ptr<ScanOptions> &options) = 0

An asynchronous version of Scan.

virtual Future<util::optional<int64_t>> CountRows(compute::Expression predicate, const std::shared_ptr<ScanOptions> &options)

Count the number of rows in this fragment matching the filter using metadata only.

That is, this method may perform I/O, but will not load data.

If this is not possible, resolve with an empty optional. The fragment can perform I/O (e.g. to read metadata) before it deciding whether it can satisfy the request.

inline const compute::Expression &partition_expression() const

An expression which evaluates to true for all data viewed by this Fragment.

class Dataset : public std::enable_shared_from_this<Dataset>

A container of zero or more Fragments.

A Dataset acts as a union of Fragments, e.g. files deeply nested in a directory. A Dataset has a schema to which Fragments must align during a scan operation. This is analogous to Avro’s reader and writer schema.

Subclassed by arrow::dataset::FileSystemDataset, arrow::dataset::InMemoryDataset, arrow::dataset::UnionDataset

Public Functions

Result<std::shared_ptr<ScannerBuilder>> NewScan()

Begin to build a new Scan operation against this Dataset.

Result<FragmentIterator> GetFragments(compute::Expression predicate)

GetFragments returns an iterator of Fragments given a predicate.

inline const compute::Expression &partition_expression() const

An expression which evaluates to true for all data viewed by this Dataset.

May be null, which indicates no information is available.

virtual std::string type_name() const = 0

The name identifying the kind of Dataset.

virtual Result<std::shared_ptr<Dataset>> ReplaceSchema(std::shared_ptr<Schema> schema) const = 0

Return a copy of this Dataset with a different schema.

The copy will view the same Fragments. If the new schema is not compatible with the original dataset’s schema then an error will be raised.

Partitioning

enum class SegmentEncoding : int8_t

The encoding of partition segments.

Values:

enumerator None

No encoding.

enumerator Uri

Segment values are URL-encoded.

static constexpr char kDefaultHiveNullFallback[] = "__HIVE_DEFAULT_PARTITION__"

The default fallback used for null values in a Hive-style partitioning.

std::ostream &operator<<(std::ostream &os, SegmentEncoding segment_encoding)
std::string StripPrefix(const std::string &path, const std::string &prefix)
std::string StripPrefixAndFilename(const std::string &path, const std::string &prefix)

Extracts the directory and filename and removes the prefix of a path.

e.g., StripPrefixAndFilename("/data/year=2019/c.txt", "/data") -> {"year=2019","c.txt"}

std::vector<std::string> StripPrefixAndFilename(const std::vector<std::string> &paths, const std::string &prefix)

Vector version of StripPrefixAndFilename.

std::vector<std::string> StripPrefixAndFilename(const std::vector<fs::FileInfo> &files, const std::string &prefix)

Vector version of StripPrefixAndFilename.

class Partitioning : public arrow::util::EqualityComparable<Partitioning>
#include <arrow/dataset/partition.h>

Interface for parsing partition expressions from string partition identifiers.

For example, the identifier “foo=5” might be parsed to an equality expression between the “foo” field and the value 5.

Some partitionings may store the field names in a metadata store instead of in file paths, for example dataset_root/2009/11/… could be used when the partition fields are “year” and “month”

Paths are consumed from left to right. Paths must be relative to the root of a partition; path prefixes must be removed before passing the path to a partitioning for parsing.

Subclassed by arrow::dataset::FunctionPartitioning, arrow::dataset::KeyValuePartitioning

Public Functions

virtual std::string type_name() const = 0

The name identifying the kind of partitioning.

virtual Result<compute::Expression> Parse(const std::string &path) const = 0

Parse a path into a partition expression.

inline const std::shared_ptr<Schema> &schema() const

The partition schema.

Public Static Functions

static std::shared_ptr<Partitioning> Default()

A default Partitioning which always yields scalar(true)

struct PartitionedBatches
#include <arrow/dataset/partition.h>

If the input batch shares any fields with this partitioning, produce sub-batches which satisfy mutually exclusive Expressions.

struct KeyValuePartitioningOptions
#include <arrow/dataset/partition.h>

Options for key-value based partitioning (hive/directory).

Subclassed by arrow::dataset::HivePartitioningOptions

Public Members

SegmentEncoding segment_encoding = SegmentEncoding::Uri

After splitting a path into components, decode the path components before parsing according to this scheme.

struct PartitioningFactoryOptions
#include <arrow/dataset/partition.h>

Options for inferring a partitioning.

Subclassed by arrow::dataset::HivePartitioningFactoryOptions

Public Members

bool infer_dictionary = false

When inferring a schema for partition fields, yield dictionary encoded types instead of plain.

This can be more efficient when materializing virtual columns, and Expressions parsed by the finished Partitioning will include dictionaries of all unique inspected values for each field.

std::shared_ptr<Schema> schema

Optionally, an expected schema can be provided, in which case inference will only check discovered fields against the schema and update internal state (such as dictionaries).

SegmentEncoding segment_encoding = SegmentEncoding::Uri

After splitting a path into components, decode the path components before parsing according to this scheme.

struct HivePartitioningFactoryOptions : public arrow::dataset::PartitioningFactoryOptions
#include <arrow/dataset/partition.h>

Options for inferring a hive-style partitioning.

Public Members

std::string null_fallback

The hive partitioning scheme maps null to a hard coded fallback string.

class PartitioningFactory
#include <arrow/dataset/partition.h>

PartitioningFactory provides creation of a partitioning when the specific schema must be inferred from available paths (no explicit schema is known).

Public Functions

virtual std::string type_name() const = 0

The name identifying the kind of partitioning.

virtual Result<std::shared_ptr<Schema>> Inspect(const std::vector<std::string> &paths) = 0

Get the schema for the resulting Partitioning.

This may reset internal state, for example dictionaries of unique representations.

virtual Result<std::shared_ptr<Partitioning>> Finish(const std::shared_ptr<Schema> &schema) const = 0

Create a partitioning using the provided schema (fields may be dropped).

class KeyValuePartitioning : public arrow::dataset::Partitioning
#include <arrow/dataset/partition.h>

Subclass for the common case of a partitioning which yields an equality expression for each segment.

Subclassed by arrow::dataset::DirectoryPartitioning, arrow::dataset::FilenamePartitioning, arrow::dataset::HivePartitioning

Public Functions

virtual Result<compute::Expression> Parse(const std::string &path) const override

Parse a path into a partition expression.

struct Key
#include <arrow/dataset/partition.h>

An unconverted equality expression consisting of a field name and the representation of a scalar value.

class DirectoryPartitioning : public arrow::dataset::KeyValuePartitioning
#include <arrow/dataset/partition.h>

DirectoryPartitioning parses one segment of a path for each field in its schema.

All fields are required, so paths passed to DirectoryPartitioning::Parse must contain segments for each field.

For example given schema<year:int16, month:int8> the path “/2009/11” would be parsed to (“year”_ == 2009 and “month”_ == 11)

Public Functions

explicit DirectoryPartitioning(std::shared_ptr<Schema> schema, ArrayVector dictionaries = {}, KeyValuePartitioningOptions options = {})

If a field in schema is of dictionary type, the corresponding element of dictionaries must be contain the dictionary of values for that field.

inline virtual std::string type_name() const override

The name identifying the kind of partitioning.

Public Static Functions

static std::shared_ptr<PartitioningFactory> MakeFactory(std::vector<std::string> field_names, PartitioningFactoryOptions = {})

Create a factory for a directory partitioning.

Parameters:

field_names[in] The names for the partition fields. Types will be inferred.

struct HivePartitioningOptions : public arrow::dataset::KeyValuePartitioningOptions
#include <arrow/dataset/partition.h>
class HivePartitioning : public arrow::dataset::KeyValuePartitioning
#include <arrow/dataset/partition.h>

Multi-level, directory based partitioning originating from Apache Hive with all data files stored in the leaf directories.

Data is partitioned by static values of a particular column in the schema. Partition keys are represented in the form $key=$value in directory names. Field order is ignored, as are missing or unrecognized field names.

For example given schema<year:int16, month:int8, day:int8> the path “/day=321/ignored=3.4/year=2009” parses to (“year”_ == 2009 and “day”_ == 321)

Public Functions

inline explicit HivePartitioning(std::shared_ptr<Schema> schema, ArrayVector dictionaries = {}, std::string null_fallback = kDefaultHiveNullFallback)

If a field in schema is of dictionary type, the corresponding element of dictionaries must be contain the dictionary of values for that field.

inline virtual std::string type_name() const override

The name identifying the kind of partitioning.

Public Static Functions

static std::shared_ptr<PartitioningFactory> MakeFactory(HivePartitioningFactoryOptions = {})

Create a factory for a hive partitioning.

class FunctionPartitioning : public arrow::dataset::Partitioning
#include <arrow/dataset/partition.h>

Implementation provided by lambda or other callable.

Public Functions

inline virtual std::string type_name() const override

The name identifying the kind of partitioning.

inline virtual Result<compute::Expression> Parse(const std::string &path) const override

Parse a path into a partition expression.

class FilenamePartitioning : public arrow::dataset::KeyValuePartitioning
#include <arrow/dataset/partition.h>

Public Functions

explicit FilenamePartitioning(std::shared_ptr<Schema> schema, ArrayVector dictionaries = {}, KeyValuePartitioningOptions options = {})

Construct a FilenamePartitioning from its components.

If a field in schema is of dictionary type, the corresponding element of dictionaries must be contain the dictionary of values for that field.

inline virtual std::string type_name() const override

The name identifying the kind of partitioning.

Public Static Functions

static std::shared_ptr<PartitioningFactory> MakeFactory(std::vector<std::string> field_names, PartitioningFactoryOptions = {})

Create a factory for a filename partitioning.

Parameters:

field_names[in] The names for the partition fields. Types will be inferred.

class PartitioningOrFactory
#include <arrow/dataset/partition.h>

Either a Partitioning or a PartitioningFactory.

Public Functions

inline const std::shared_ptr<Partitioning> &partitioning() const

The partitioning (if given).

inline const std::shared_ptr<PartitioningFactory> &factory() const

The partition factory (if given).

Result<std::shared_ptr<Schema>> GetOrInferSchema(const std::vector<std::string> &paths)

Get the partition schema, inferring it with the given factory if needed.

Dataset discovery/factories

struct InspectOptions
#include <arrow/dataset/discovery.h>

Public Members

int fragments = 1

Indicate how many fragments should be inspected to infer the unified dataset schema.

Limiting the number of fragments accessed improves the latency of the discovery process when dealing with a high number of fragments and/or high latency file systems.

The default value of 1 inspects the schema of the first (in no particular order) fragment only. If the dataset has a uniform schema for all fragments, this default is the optimal value. In order to inspect all fragments and robustly unify their potentially varying schemas, set this option to kInspectAllFragments. A value of 0 disables inspection of fragments altogether so only the partitioning schema will be inspected.

Public Static Attributes

static constexpr int kInspectAllFragments = -1

See fragments property.

struct FinishOptions
#include <arrow/dataset/discovery.h>

Public Members

std::shared_ptr<Schema> schema = NULLPTR

Finalize the dataset with this given schema.

If the schema is not provided, infer the schema via the Inspect, see the inspect_options property.

InspectOptions inspect_options = {}

If the schema is not provided, it will be discovered by passing the following options to DatasetDiscovery::Inspect.

bool validate_fragments = false

Indicate if the given Schema (when specified), should be validated against the fragments’ schemas.

inspect_options will control how many fragments are checked.

class DatasetFactory
#include <arrow/dataset/discovery.h>

DatasetFactory provides a way to inspect/discover a Dataset’s expected schema before materializing said Dataset.

Subclassed by arrow::dataset::FileSystemDatasetFactory, arrow::dataset::ParquetDatasetFactory, arrow::dataset::UnionDatasetFactory

Public Functions

virtual Result<std::vector<std::shared_ptr<Schema>>> InspectSchemas(InspectOptions options) = 0

Get the schemas of the Fragments and Partitioning.

Result<std::shared_ptr<Schema>> Inspect(InspectOptions options = {})

Get unified schema for the resulting Dataset.

Result<std::shared_ptr<Dataset>> Finish()

Create a Dataset.

Result<std::shared_ptr<Dataset>> Finish(std::shared_ptr<Schema> schema)

Create a Dataset with the given schema (see InspectOptions::schema)

virtual Result<std::shared_ptr<Dataset>> Finish(FinishOptions options) = 0

Create a Dataset with the given options.

inline const compute::Expression &root_partition() const

Optional root partition for the resulting Dataset.

inline Status SetRootPartition(compute::Expression partition)

Set the root partition for the resulting Dataset.

Scanning

using TaggedRecordBatchGenerator = std::function<Future<TaggedRecordBatch>()>
using TaggedRecordBatchIterator = Iterator<TaggedRecordBatch>
using EnumeratedRecordBatchGenerator = std::function<Future<EnumeratedRecordBatch>()>
using EnumeratedRecordBatchIterator = Iterator<EnumeratedRecordBatch>
constexpr int64_t kDefaultBatchSize = 1 << 17
constexpr int32_t kDefaultBatchReadahead = 16
constexpr int32_t kDefaultFragmentReadahead = 4
void SetProjection(ScanOptions *options, ProjectionDescr projection)

Utility method to set the projection expression and schema.

class FragmentScanOptions
#include <arrow/dataset/dataset.h>

Per-scan options for fragment(s) in a dataset.

These options are not intrinsic to the format or fragment itself, but do affect the results of a scan. These are options which make sense to change between repeated reads of the same dataset, such as format-specific conversion options (that do not affect the schema).

Subclassed by arrow::dataset::CsvFragmentScanOptions, arrow::dataset::IpcFragmentScanOptions, arrow::dataset::ParquetFragmentScanOptions

struct ScanOptions
#include <arrow/dataset/scanner.h>

Scan-specific options, which can be changed between scans of the same dataset.

Public Functions

std::vector<FieldRef> MaterializedFields() const

Return a vector of FieldRefs that require materialization.

This is usually the union of the fields referenced in the projection and the filter expression. Examples:

  • SELECT a, b WHERE a < 2 && c > 1 => [“a”, “b”, “a”, “c”]

  • SELECT a + b < 3 WHERE a > 1 => [“a”, “b”]

This is needed for expression where a field may not be directly used in the final projection but is still required to evaluate the expression.

This is used by Fragment implementations to apply the column sub-selection optimization.

Public Members

compute::Expression filter = compute::literal(true)

A row filter (which will be pushed down to partitioning/reading if supported).

compute::Expression projection

A projection expression (which can add/remove/rename columns).

std::shared_ptr<Schema> dataset_schema

Schema with which batches will be read from fragments.

This is also known as the “reader schema” it will be used (for example) in constructing CSV file readers to identify column types for parsing. Usually only a subset of its fields (see MaterializedFields) will be materialized during a scan.

std::shared_ptr<Schema> projected_schema

Schema of projected record batches.

This is independent of dataset_schema as its fields are derived from the projection. For example, let

dataset_schema = {“a”: int32, “b”: int32, “id”: utf8} projection = project({equal(field_ref(“a”), field_ref(“b”))}, {“a_plus_b”})

(no filter specified). In this case, the projected_schema would be

{“a_plus_b”: int32}

int64_t batch_size = kDefaultBatchSize

Maximum row count for scanned batches.

int32_t batch_readahead = kDefaultBatchReadahead

How many batches to read ahead within a file.

Set to 0 to disable batch readahead

Note: May not be supported by all formats Note: Will be ignored if use_threads is set to false

int32_t fragment_readahead = kDefaultFragmentReadahead

How many files to read ahead.

Set to 0 to disable fragment readahead

Note: May not be enforced by all scanners Note: Will be ignored if use_threads is set to false

MemoryPool *pool = arrow::default_memory_pool()

A pool from which materialized and scanned arrays will be allocated.

io::IOContext io_context

IOContext for any IO tasks.

Note: The IOContext executor will be ignored if use_threads is set to false

bool use_threads = false

If true the scanner will scan in parallel.

Note: If true, this will use threads from both the cpu_executor and the io_context.executor Note: This must be true in order for any readahead to happen

std::shared_ptr<FragmentScanOptions> fragment_scan_options

Fragment-specific scan options.

compute::BackpressureOptions backpressure = compute::BackpressureOptions::DefaultBackpressure()

Parameters which control when the plan should pause for a slow consumer.

struct ProjectionDescr
#include <arrow/dataset/scanner.h>

Describes a projection.

Public Members

compute::Expression expression

The projection expression itself This expression must be a call to make_struct.

std::shared_ptr<Schema> schema

The output schema of the projection.

This can be calculated from the input schema and the expression but it is cached here for convenience.

Public Static Functions

static Result<ProjectionDescr> FromStructExpression(const compute::Expression &expression, const Schema &dataset_schema)

Create a ProjectionDescr by binding an expression to the dataset schema.

expression must return a struct type

static Result<ProjectionDescr> FromExpressions(std::vector<compute::Expression> exprs, std::vector<std::string> names, const Schema &dataset_schema)

Create a ProjectionDescr from expressions/names for each field.

static Result<ProjectionDescr> FromNames(std::vector<std::string> names, const Schema &dataset_schema)

Create a default projection referencing fields in the dataset schema.

static Result<ProjectionDescr> Default(const Schema &dataset_schema)

Make a projection that projects every field in the dataset schema.

struct TaggedRecordBatch
#include <arrow/dataset/scanner.h>

Combines a record batch with the fragment that the record batch originated from.

Knowing the source fragment can be useful for debugging & understanding loaded data

struct EnumeratedRecordBatch
#include <arrow/dataset/scanner.h>

Combines a tagged batch with positional information.

This is returned when scanning batches in an unordered fashion. This information is needed if you ever want to reassemble the batches in order

class Scanner
#include <arrow/dataset/scanner.h>

A scanner glues together several dataset classes to load in data.

The dataset contains a collection of fragments and partitioning rules.

The fragments identify independently loadable units of data (i.e. each fragment has a potentially unique schema and possibly even format. It should be possible to read fragments in parallel if desired).

The fragment’s format contains the logic necessary to actually create a task to load the fragment into memory. That task may or may not support parallel execution of its own.

The scanner is then responsible for creating scan tasks from every fragment in the dataset and (potentially) sequencing the loaded record batches together.

The scanner should not buffer the entire dataset in memory (unless asked) instead yielding record batches as soon as they are ready to scan. Various readahead properties control how much data is allowed to be scanned before pausing to let a slow consumer catchup.

Today the scanner also handles projection & filtering although that may change in the future.

Public Functions

virtual Status Scan(std::function<Status(TaggedRecordBatch)> visitor) = 0

Apply a visitor to each RecordBatch as it is scanned.

If multiple threads are used (via use_threads), the visitor will be invoked from those threads and is responsible for any synchronization.

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

Convert a Scanner into a Table.

Use this convenience utility with care. This will serially materialize the Scan result in memory before creating the Table.

virtual Result<TaggedRecordBatchIterator> ScanBatches() = 0

Scan the dataset into a stream of record batches.

Each batch is tagged with the fragment it originated from. The batches will arrive in order. The order of fragments is determined by the dataset.

Note: The scanner will perform some readahead but will avoid materializing too much in memory (this is goverended by the readahead options and use_threads option). If the readahead queue fills up then I/O will pause until the calling thread catches up.

virtual Result<EnumeratedRecordBatchIterator> ScanBatchesUnordered() = 0

Scan the dataset into a stream of record batches.

Unlike ScanBatches this method may allow record batches to be returned out of order. This allows for more efficient scanning: some fragments may be accessed more quickly than others (e.g. may be cached in RAM or just happen to get scheduled earlier by the I/O)

To make up for the out-of-order iteration each batch is further tagged with positional information.

virtual Result<std::shared_ptr<Table>> TakeRows(const Array &indices) = 0

A convenience to synchronously load the given rows by index.

Will only consume as many batches as needed from ScanBatches().

virtual Result<std::shared_ptr<Table>> Head(int64_t num_rows) = 0

Get the first N rows.

virtual Result<int64_t> CountRows() = 0

Count rows matching a predicate.

This method will push down the predicate and compute the result based on fragment metadata if possible.

virtual Result<std::shared_ptr<RecordBatchReader>> ToRecordBatchReader() = 0

Convert the Scanner to a RecordBatchReader so it can be easily used with APIs that expect a reader.

inline const std::shared_ptr<ScanOptions> &options() const

Get the options for this scan.

virtual const std::shared_ptr<Dataset> &dataset() const = 0

Get the dataset that this scanner will scan.

class ScannerBuilder
#include <arrow/dataset/scanner.h>

ScannerBuilder is a factory class to construct a Scanner.

It is used to pass information, notably a potential filter expression and a subset of columns to materialize.

Public Functions

Status Project(std::vector<std::string> columns)

Set the subset of columns to materialize.

Columns which are not referenced may not be read from fragments.

Parameters:

columns[in] list of columns to project. Order and duplicates will be preserved.

Returns:

Failure if any column name does not exists in the dataset’s Schema.

Status Project(std::vector<compute::Expression> exprs, std::vector<std::string> names)

Set expressions which will be evaluated to produce the materialized columns.

Columns which are not referenced may not be read from fragments.

Parameters:
  • exprs[in] expressions to evaluate to produce columns.

  • names[in] list of names for the resulting columns.

Returns:

Failure if any referenced column does not exists in the dataset’s Schema.

Status Filter(const compute::Expression &filter)

Set the filter expression to return only rows matching the filter.

The predicate will be passed down to Sources and corresponding Fragments to exploit predicate pushdown if possible using partition information or Fragment internal metadata, e.g. Parquet statistics. Columns which are not referenced may not be read from fragments.

Parameters:

filter[in] expression to filter rows with.

Returns:

Failure if any referenced columns does not exist in the dataset’s Schema.

Status UseThreads(bool use_threads = true)

Indicate if the Scanner should make use of the available ThreadPool found in ScanOptions;.

Status FragmentReadahead(int fragment_readahead)

Limit how many fragments the scanner will read at once.

Status BatchSize(int64_t batch_size)

Set the maximum number of rows per RecordBatch.

This option provides a control limiting the memory owned by any RecordBatch.

Parameters:

batch_size[in] the maximum number of rows.

Returns:

An error if the number for batch is not greater than 0.

Status Pool(MemoryPool *pool)

Set the pool from which materialized and scanned arrays will be allocated.

Status FragmentScanOptions(std::shared_ptr<FragmentScanOptions> fragment_scan_options)

Set fragment-specific scan options.

Status Backpressure(compute::BackpressureOptions backpressure)

Override default backpressure configuration.

Result<std::shared_ptr<Scanner>> Finish()

Return the constructed now-immutable Scanner object.

Public Static Functions

static std::shared_ptr<ScannerBuilder> FromRecordBatchReader(std::shared_ptr<RecordBatchReader> reader)

Make a scanner from a record batch reader.

The resulting scanner can be scanned only once. This is intended to support writing data from streaming sources or other sources that can be iterated only once.

class ScanNodeOptions : public arrow::compute::ExecNodeOptions
#include <arrow/dataset/scanner.h>

Construct a source ExecNode which yields batches from a dataset scan.

Does not construct associated filter or project nodes. Yielded batches will be augmented with fragment/batch indices to enable stable ordering for simple ExecPlans.

Concrete implementations

class InMemoryFragment : public arrow::dataset::Fragment
#include <arrow/dataset/dataset.h>

A trivial Fragment that yields ScanTask out of a fixed set of RecordBatch.

Public Functions

virtual Result<RecordBatchGenerator> ScanBatchesAsync(const std::shared_ptr<ScanOptions> &options) override

An asynchronous version of Scan.

virtual Future<util::optional<int64_t>> CountRows(compute::Expression predicate, const std::shared_ptr<ScanOptions> &options) override

Count the number of rows in this fragment matching the filter using metadata only.

That is, this method may perform I/O, but will not load data.

If this is not possible, resolve with an empty optional. The fragment can perform I/O (e.g. to read metadata) before it deciding whether it can satisfy the request.

class InMemoryDataset : public arrow::dataset::Dataset
#include <arrow/dataset/dataset.h>

A Source which yields fragments wrapping a stream of record batches.

The record batches must match the schema provided to the source at construction.

Public Functions

inline InMemoryDataset(std::shared_ptr<Schema> schema, std::shared_ptr<RecordBatchGenerator> get_batches)

Construct a dataset from a schema and a factory of record batch iterators.

InMemoryDataset(std::shared_ptr<Schema> schema, RecordBatchVector batches)

Convenience constructor taking a fixed list of batches.

explicit InMemoryDataset(std::shared_ptr<Table> table)

Convenience constructor taking a Table.

inline virtual std::string type_name() const override

The name identifying the kind of Dataset.

virtual Result<std::shared_ptr<Dataset>> ReplaceSchema(std::shared_ptr<Schema> schema) const override

Return a copy of this Dataset with a different schema.

The copy will view the same Fragments. If the new schema is not compatible with the original dataset’s schema then an error will be raised.

class RecordBatchGenerator
#include <arrow/dataset/dataset.h>
class UnionDataset : public arrow::dataset::Dataset
#include <arrow/dataset/dataset.h>

A Dataset wrapping child Datasets.

Public Functions

inline virtual std::string type_name() const override

The name identifying the kind of Dataset.

virtual Result<std::shared_ptr<Dataset>> ReplaceSchema(std::shared_ptr<Schema> schema) const override

Return a copy of this Dataset with a different schema.

The copy will view the same Fragments. If the new schema is not compatible with the original dataset’s schema then an error will be raised.

Public Static Functions

static Result<std::shared_ptr<UnionDataset>> Make(std::shared_ptr<Schema> schema, DatasetVector children)

Construct a UnionDataset wrapping child Datasets.

Parameters:
  • schema[in] the schema of the resulting dataset.

  • children[in] one or more child Datasets. Their schemas must be identical to schema.

class UnionDatasetFactory : public arrow::dataset::DatasetFactory
#include <arrow/dataset/discovery.h>

DatasetFactory provides a way to inspect/discover a Dataset’s expected schema before materialization.

Public Functions

inline const std::vector<std::shared_ptr<DatasetFactory>> &factories() const

Return the list of child DatasetFactory.

virtual Result<std::vector<std::shared_ptr<Schema>>> InspectSchemas(InspectOptions options) override

Get the schemas of the Datasets.

Instead of applying options globally, it applies at each child factory. This will not respect options.fragments exactly, but will respect the spirit of peeking the first fragments or all of them.

virtual Result<std::shared_ptr<Dataset>> Finish(FinishOptions options) override

Create a Dataset.

File System Datasets

struct FileSystemFactoryOptions
#include <arrow/dataset/discovery.h>

Public Members

PartitioningOrFactory partitioning = {Partitioning::Default()}

Either an explicit Partitioning or a PartitioningFactory to discover one.

If a factory is provided, it will be used to infer a schema for partition fields based on file and directory paths then construct a Partitioning. The default is a Partitioning which will yield no partition information.

The (explicit or discovered) partitioning will be applied to discovered files and the resulting partition information embedded in the Dataset.

std::string partition_base_dir

For the purposes of applying the partitioning, paths will be stripped of the partition_base_dir.

Files not matching the partition_base_dir prefix will be skipped for partition discovery. The ignored files will still be part of the Dataset, but will not have partition information.

Example: partition_base_dir = “/dataset”;

  • “/dataset/US/sales.csv” -> “US/sales.csv” will be given to the partitioning

  • ”/home/john/late_sales.csv” -> Will be ignored for partition discovery.

This is useful for partitioning which parses directory when ordering is important, e.g. DirectoryPartitioning.

bool exclude_invalid_files = false

Invalid files (via selector or explicitly) will be excluded by checking with the FileFormat::IsSupported method.

This will incur IO for each files in a serial and single threaded fashion. Disabling this feature will skip the IO, but unsupported files may be present in the Dataset (resulting in an error at scan time).

std::vector<std::string> selector_ignore_prefixes = {".", "_",}

When discovering from a Selector (and not from an explicit file list), ignore files and directories matching any of these prefixes.

Example (with selector = “/dataset/&zwj;**”): selector_ignore_prefixes = {“_”, “.DS_STORE” };

  • “/dataset/data.csv” -> not ignored

  • ”/dataset/_metadata” -> ignored

  • ”/dataset/.DS_STORE” -> ignored

  • ”/dataset/_hidden/dat” -> ignored

  • ”/dataset/nested/.DS_STORE” -> ignored

class FileSystemDatasetFactory : public arrow::dataset::DatasetFactory
#include <arrow/dataset/discovery.h>

FileSystemDatasetFactory creates a Dataset from a vector of fs::FileInfo or a fs::FileSelector.

Public Functions

virtual Result<std::vector<std::shared_ptr<Schema>>> InspectSchemas(InspectOptions options) override

Get the schemas of the Fragments and Partitioning.

virtual Result<std::shared_ptr<Dataset>> Finish(FinishOptions options) override

Create a Dataset with the given options.

Public Static Functions

static Result<std::shared_ptr<DatasetFactory>> Make(std::shared_ptr<fs::FileSystem> filesystem, const std::vector<std::string> &paths, std::shared_ptr<FileFormat> format, FileSystemFactoryOptions options)

Build a FileSystemDatasetFactory from an explicit list of paths.

Parameters:
static Result<std::shared_ptr<DatasetFactory>> Make(std::shared_ptr<fs::FileSystem> filesystem, fs::FileSelector selector, std::shared_ptr<FileFormat> format, FileSystemFactoryOptions options)

Build a FileSystemDatasetFactory from a fs::FileSelector.

The selector will expand to a vector of FileInfo. The expansion/crawling is performed in this function call. Thus, the finalized Dataset is working with a snapshot of the filesystem. If options.partition_base_dir is not provided, it will be overwritten with selector.base_dir.

Parameters:
static Result<std::shared_ptr<DatasetFactory>> Make(std::string uri, std::shared_ptr<FileFormat> format, FileSystemFactoryOptions options)

Build a FileSystemDatasetFactory from an uri including filesystem information.

Parameters:
static Result<std::shared_ptr<DatasetFactory>> Make(std::shared_ptr<fs::FileSystem> filesystem, const std::vector<fs::FileInfo> &files, std::shared_ptr<FileFormat> format, FileSystemFactoryOptions options)

Build a FileSystemDatasetFactory from an explicit list of file information.

Parameters:
class FileSource : public arrow::util::EqualityComparable<FileSource>
#include <arrow/dataset/file_base.h>

The path and filesystem where an actual file is located or a buffer which can be read like a file.

Public Functions

inline Compression::type compression() const

Return the type of raw compression on the file, if any.

inline const std::string &path() const

Return the file path, if any. Only valid when file source wraps a path.

inline const std::shared_ptr<fs::FileSystem> &filesystem() const

Return the filesystem, if any. Otherwise returns nullptr.

inline const std::shared_ptr<Buffer> &buffer() const

Return the buffer containing the file, if any. Otherwise returns nullptr.

Result<std::shared_ptr<io::RandomAccessFile>> Open() const

Get a RandomAccessFile which views this file source.

Result<std::shared_ptr<io::InputStream>> OpenCompressed(util::optional<Compression::type> compression = util::nullopt) const

Get an InputStream which views this file source (and decompresses if needed)

Parameters:

compression[in] If nullopt, guess the compression scheme from the filename, else decompress with the given codec

bool Equals(const FileSource &other) const

equality comparison with another FileSource

class FileFormat : public std::enable_shared_from_this<FileFormat>
#include <arrow/dataset/file_base.h>

Base class for file format implementation.

Subclassed by arrow::dataset::CsvFileFormat, arrow::dataset::IpcFileFormat, arrow::dataset::OrcFileFormat, arrow::dataset::ParquetFileFormat, skyhook::SkyhookFileFormat

Public Functions

virtual std::string type_name() const = 0

The name identifying the kind of file format.

virtual Result<bool> IsSupported(const FileSource &source) const = 0

Indicate if the FileSource is supported/readable by this format.

virtual Result<std::shared_ptr<Schema>> Inspect(const FileSource &source) const = 0

Return the schema of the file if possible.

virtual Result<std::shared_ptr<FileFragment>> MakeFragment(FileSource source, compute::Expression partition_expression, std::shared_ptr<Schema> physical_schema)

Open a fragment.

Result<std::shared_ptr<FileFragment>> MakeFragment(FileSource source, compute::Expression partition_expression)

Create a FileFragment for a FileSource.

Result<std::shared_ptr<FileFragment>> MakeFragment(FileSource source, std::shared_ptr<Schema> physical_schema = NULLPTR)

Create a FileFragment for a FileSource.

virtual Result<std::shared_ptr<FileWriter>> MakeWriter(std::shared_ptr<io::OutputStream> destination, std::shared_ptr<Schema> schema, std::shared_ptr<FileWriteOptions> options, fs::FileLocator destination_locator) const = 0

Create a writer for this format.

virtual std::shared_ptr<FileWriteOptions> DefaultWriteOptions() = 0

Get default write options for this format.

Public Members

std::shared_ptr<FragmentScanOptions> default_fragment_scan_options

Options affecting how this format is scanned.

The options here can be overridden at scan time.

class FileFragment : public arrow::dataset::Fragment, public arrow::util::EqualityComparable<FileFragment>
#include <arrow/dataset/file_base.h>

A Fragment that is stored in a file with a known format.

Subclassed by arrow::dataset::ParquetFileFragment

Public Functions

virtual Result<RecordBatchGenerator> ScanBatchesAsync(const std::shared_ptr<ScanOptions> &options) override

An asynchronous version of Scan.

virtual Future<util::optional<int64_t>> CountRows(compute::Expression predicate, const std::shared_ptr<ScanOptions> &options) override

Count the number of rows in this fragment matching the filter using metadata only.

That is, this method may perform I/O, but will not load data.

If this is not possible, resolve with an empty optional. The fragment can perform I/O (e.g. to read metadata) before it deciding whether it can satisfy the request.

class FileSystemDataset : public arrow::dataset::Dataset
#include <arrow/dataset/file_base.h>

A Dataset of FileFragments.

A FileSystemDataset is composed of one or more FileFragment. The fragments are independent and don’t need to share the same format and/or filesystem.

Public Functions

inline virtual std::string type_name() const override

Return the type name of the dataset.

virtual Result<std::shared_ptr<Dataset>> ReplaceSchema(std::shared_ptr<Schema> schema) const override

Replace the schema of the dataset.

std::vector<std::string> files() const

Return the path of files.

inline const std::shared_ptr<FileFormat> &format() const

Return the format.

inline const std::shared_ptr<fs::FileSystem> &filesystem() const

Return the filesystem. May be nullptr if the fragments wrap buffers.

inline const std::shared_ptr<Partitioning> &partitioning() const

Return the partitioning.

May be nullptr if the dataset was not constructed with a partitioning.

Public Static Functions

static Result<std::shared_ptr<FileSystemDataset>> Make(std::shared_ptr<Schema> schema, compute::Expression root_partition, std::shared_ptr<FileFormat> format, std::shared_ptr<fs::FileSystem> filesystem, std::vector<std::shared_ptr<FileFragment>> fragments, std::shared_ptr<Partitioning> partitioning = NULLPTR)

Create a FileSystemDataset.

Note that fragments wrapping files resident in differing filesystems are not permitted; to work with multiple filesystems use a UnionDataset.

Parameters:
  • schema[in] the schema of the dataset

  • root_partition[in] the partition expression of the dataset

  • format[in] the format of each FileFragment.

  • filesystem[in] the filesystem of each FileFragment, or nullptr if the fragments wrap buffers.

  • fragments[in] list of fragments to create the dataset from.

  • partitioning[in] the Partitioning object in case the dataset is created with a known partitioning (e.g. from a discovered partitioning through a DatasetFactory), or nullptr if not known.

Returns:

A constructed dataset.

static Status Write(const FileSystemDatasetWriteOptions &write_options, std::shared_ptr<Scanner> scanner)

Write a dataset.

class FileWriteOptions
#include <arrow/dataset/file_base.h>

Options for writing a file of this format.

Subclassed by arrow::dataset::CsvFileWriteOptions, arrow::dataset::IpcFileWriteOptions, arrow::dataset::ParquetFileWriteOptions

class FileWriter
#include <arrow/dataset/file_base.h>

A writer for this format.

Subclassed by arrow::dataset::CsvFileWriter, arrow::dataset::IpcFileWriter, arrow::dataset::ParquetFileWriter

Public Functions

virtual Status Write(const std::shared_ptr<RecordBatch> &batch) = 0

Write the given batch.

Status Write(RecordBatchReader *batches)

Write all batches from the reader.

virtual Future Finish()

Indicate that writing is done.

Result<int64_t> GetBytesWritten() const

After Finish() is called, provides number of bytes written to file.

struct FileSystemDatasetWriteOptions
#include <arrow/dataset/file_base.h>

Options for writing a dataset.

Public Members

std::shared_ptr<FileWriteOptions> file_write_options

Options for individual fragment writing.

std::shared_ptr<fs::FileSystem> filesystem

FileSystem into which a dataset will be written.

std::string base_dir

Root directory into which the dataset will be written.

std::shared_ptr<Partitioning> partitioning

Partitioning used to generate fragment paths.

int max_partitions = 1024

Maximum number of partitions any batch may be written into, default is 1K.

std::string basename_template

Template string used to generate fragment basenames.

{i} will be replaced by an auto incremented integer.

uint32_t max_open_files = 900

If greater than 0 then this will limit the maximum number of files that can be left open.

If an attempt is made to open too many files then the least recently used file will be closed. If this setting is set too low you may end up fragmenting your data into many small files.

The default is 900 which also allows some # of files to be open by the scanner before hitting the default Linux limit of 1024

uint64_t max_rows_per_file = 0

If greater than 0 then this will limit how many rows are placed in any single file.

Otherwise there will be no limit and one file will be created in each output directory unless files need to be closed to respect max_open_files

uint64_t min_rows_per_group = 0

If greater than 0 then this will cause the dataset writer to batch incoming data and only write the row groups to the disk when sufficient rows have accumulated.

The final row group size may be less than this value and other options such as max_open_files or max_rows_per_file lead to smaller row group sizes.

uint64_t max_rows_per_group = 1 << 20

If greater than 0 then the dataset writer may split up large incoming batches into multiple row groups.

If this value is set then min_rows_per_group should also be set or else you may end up with very small row groups (e.g. if the incoming row group size is just barely larger than this value).

ExistingDataBehavior existing_data_behavior = ExistingDataBehavior::kError

Controls what happens if an output directory already exists.

bool create_dir = true

If false the dataset writer will not create directories This is mainly intended for filesystems that do not require directories such as S3.

std::function<Status(FileWriter*)> writer_pre_finish = [](FileWriter*) {returnStatus::OK();}

Callback to be invoked against all FileWriters before they are finalized with FileWriter::Finish().

std::function<Status(FileWriter*)> writer_post_finish = [](FileWriter*) {returnStatus::OK();}

Callback to be invoked against all FileWriters after they have called FileWriter::Finish().

class WriteNodeOptions : public arrow::compute::ExecNodeOptions
#include <arrow/dataset/file_base.h>

Wraps FileSystemDatasetWriteOptions for consumption as compute::ExecNodeOptions.

Public Members

FileSystemDatasetWriteOptions write_options

Options to control how to write the dataset.

std::shared_ptr<const KeyValueMetadata> custom_metadata

Optional metadata to attach to written batches.

File Formats

constexpr char kIpcTypeName[] = "ipc"
constexpr char kOrcTypeName[] = "orc"
constexpr char kParquetTypeName[] = "parquet"
class CsvFileFormat : public arrow::dataset::FileFormat
#include <arrow/dataset/file_csv.h>

A FileFormat implementation that reads from and writes to Csv files.

Public Functions

inline virtual std::string type_name() const override

The name identifying the kind of file format.

virtual Result<bool> IsSupported(const FileSource &source) const override

Indicate if the FileSource is supported/readable by this format.

virtual Result<std::shared_ptr<Schema>> Inspect(const FileSource &source) const override

Return the schema of the file if possible.

virtual Result<std::shared_ptr<FileWriter>> MakeWriter(std::shared_ptr<io::OutputStream> destination, std::shared_ptr<Schema> schema, std::shared_ptr<FileWriteOptions> options, fs::FileLocator destination_locator) const override

Create a writer for this format.

virtual std::shared_ptr<FileWriteOptions> DefaultWriteOptions() override

Get default write options for this format.

Public Members

csv::ParseOptions parse_options = csv::ParseOptions::Defaults()

Options affecting the parsing of CSV files.

struct CsvFragmentScanOptions : public arrow::dataset::FragmentScanOptions
#include <arrow/dataset/file_csv.h>

Per-scan options for CSV fragments.

Public Members

csv::ConvertOptions convert_options = csv::ConvertOptions::Defaults()

CSV conversion options.

csv::ReadOptions read_options = csv::ReadOptions::Defaults()

CSV reading options.

Note that use_threads is always ignored.

class CsvFileWriteOptions : public arrow::dataset::FileWriteOptions
#include <arrow/dataset/file_csv.h>

Public Members

std::shared_ptr<csv::WriteOptions> write_options

Options passed to csv::MakeCSVWriter.

class CsvFileWriter : public arrow::dataset::FileWriter
#include <arrow/dataset/file_csv.h>

Public Functions

virtual Status Write(const std::shared_ptr<RecordBatch> &batch) override

Write the given batch.

class IpcFileFormat : public arrow::dataset::FileFormat
#include <arrow/dataset/file_ipc.h>

A FileFormat implementation that reads from and writes to Ipc files.

Public Functions

inline virtual std::string type_name() const override

The name identifying the kind of file format.

virtual Result<bool> IsSupported(const FileSource &source) const override

Indicate if the FileSource is supported/readable by this format.

virtual Result<std::shared_ptr<Schema>> Inspect(const FileSource &source) const override

Return the schema of the file if possible.

virtual Result<std::shared_ptr<FileWriter>> MakeWriter(std::shared_ptr<io::OutputStream> destination, std::shared_ptr<Schema> schema, std::shared_ptr<FileWriteOptions> options, fs::FileLocator destination_locator) const override

Create a writer for this format.

virtual std::shared_ptr<FileWriteOptions> DefaultWriteOptions() override

Get default write options for this format.

class IpcFragmentScanOptions : public arrow::dataset::FragmentScanOptions
#include <arrow/dataset/file_ipc.h>

Per-scan options for IPC fragments.

Public Members

std::shared_ptr<ipc::IpcReadOptions> options

Options passed to the IPC file reader.

included_fields, memory_pool, and use_threads are ignored.

std::shared_ptr<io::CacheOptions> cache_options

If present, the async scanner will enable I/O coalescing.

This is ignored by the sync scanner.

class IpcFileWriteOptions : public arrow::dataset::FileWriteOptions
#include <arrow/dataset/file_ipc.h>

Public Members

std::shared_ptr<ipc::IpcWriteOptions> options

Options passed to ipc::MakeFileWriter. use_threads is ignored.

std::shared_ptr<const KeyValueMetadata> metadata

custom_metadata written to the file’s footer

class IpcFileWriter : public arrow::dataset::FileWriter
#include <arrow/dataset/file_ipc.h>

Public Functions

virtual Status Write(const std::shared_ptr<RecordBatch> &batch) override

Write the given batch.

class OrcFileFormat : public arrow::dataset::FileFormat
#include <arrow/dataset/file_orc.h>

A FileFormat implementation that reads from and writes to ORC files.

Public Functions

inline virtual std::string type_name() const override

The name identifying the kind of file format.

virtual Result<bool> IsSupported(const FileSource &source) const override

Indicate if the FileSource is supported/readable by this format.

virtual Result<std::shared_ptr<Schema>> Inspect(const FileSource &source) const override

Return the schema of the file if possible.

virtual Result<std::shared_ptr<FileWriter>> MakeWriter(std::shared_ptr<io::OutputStream> destination, std::shared_ptr<Schema> schema, std::shared_ptr<FileWriteOptions> options, fs::FileLocator destination_locator) const override

Create a writer for this format.

virtual std::shared_ptr<FileWriteOptions> DefaultWriteOptions() override

Get default write options for this format.

class ParquetFileFormat : public arrow::dataset::FileFormat
#include <arrow/dataset/file_parquet.h>

A FileFormat implementation that reads from Parquet files.

Public Functions

explicit ParquetFileFormat(const parquet::ReaderProperties &reader_properties)

Convenience constructor which copies properties from a parquet::ReaderProperties.

memory_pool will be ignored.

inline virtual std::string type_name() const override

The name identifying the kind of file format.

virtual Result<bool> IsSupported(const FileSource &source) const override

Indicate if the FileSource is supported/readable by this format.

virtual Result<std::shared_ptr<Schema>> Inspect(const FileSource &source) const override

Return the schema of the file if possible.

virtual Result<std::shared_ptr<FileFragment>> MakeFragment(FileSource source, compute::Expression partition_expression, std::shared_ptr<Schema> physical_schema) override

Create a Fragment targeting all RowGroups.

Result<std::shared_ptr<ParquetFileFragment>> MakeFragment(FileSource source, compute::Expression partition_expression, std::shared_ptr<Schema> physical_schema, std::vector<int> row_groups)

Create a Fragment, restricted to the specified row groups.

Result<std::shared_ptr<parquet::arrow::FileReader>> GetReader(const FileSource &source, const std::shared_ptr<ScanOptions> &options) const

Return a FileReader on the given source.

virtual Result<std::shared_ptr<FileWriter>> MakeWriter(std::shared_ptr<io::OutputStream> destination, std::shared_ptr<Schema> schema, std::shared_ptr<FileWriteOptions> options, fs::FileLocator destination_locator) const override

Create a writer for this format.

virtual std::shared_ptr<FileWriteOptions> DefaultWriteOptions() override

Get default write options for this format.

Result<std::shared_ptr<FileFragment>> MakeFragment(FileSource source, compute::Expression partition_expression, std::shared_ptr<Schema> physical_schema)

Open a fragment.

Result<std::shared_ptr<FileFragment>> MakeFragment(FileSource source, compute::Expression partition_expression)

Create a FileFragment for a FileSource.

Result<std::shared_ptr<FileFragment>> MakeFragment(FileSource source, std::shared_ptr<Schema> physical_schema = NULLPTR)

Create a FileFragment for a FileSource.

struct ReaderOptions
#include <arrow/dataset/file_parquet.h>
class ParquetFileFragment : public arrow::dataset::FileFragment
#include <arrow/dataset/file_parquet.h>

A FileFragment with parquet logic.

ParquetFileFragment provides a lazy (with respect to IO) interface to scan parquet files. Any heavy IO calls are deferred to the Scan() method.

The caller can provide an optional list of selected RowGroups to limit the number of scanned RowGroups, or to partition the scans across multiple threads.

Metadata can be explicitly provided, enabling pushdown predicate benefits without the potentially heavy IO of loading Metadata from the file system. This can induce significant performance boost when scanning high latency file systems.

Public Functions

inline const std::vector<int> &row_groups() const

Return the RowGroups selected by this fragment.

inline const std::shared_ptr<parquet::FileMetaData> &metadata() const

Return the FileMetaData associated with this fragment.

Status EnsureCompleteMetadata(parquet::arrow::FileReader *reader = NULLPTR)

Ensure this fragment’s FileMetaData is in memory.

Result<std::shared_ptr<Fragment>> Subset(compute::Expression predicate)

Return fragment which selects a filtered subset of this fragment’s RowGroups.

class ParquetFragmentScanOptions : public arrow::dataset::FragmentScanOptions
#include <arrow/dataset/file_parquet.h>

Per-scan options for Parquet fragments.

Public Members

std::shared_ptr<parquet::ReaderProperties> reader_properties

Reader properties.

Not all properties are respected: memory_pool comes from ScanOptions.

std::shared_ptr<parquet::ArrowReaderProperties> arrow_reader_properties

Arrow reader properties.

Not all properties are respected: batch_size comes from ScanOptions. Additionally, dictionary columns come from ParquetFileFormat::ReaderOptions::dict_columns.

class ParquetFileWriteOptions : public arrow::dataset::FileWriteOptions
#include <arrow/dataset/file_parquet.h>

Public Members

std::shared_ptr<parquet::WriterProperties> writer_properties

Parquet writer properties.

std::shared_ptr<parquet::ArrowWriterProperties> arrow_writer_properties

Parquet Arrow writer properties.

class ParquetFileWriter : public arrow::dataset::FileWriter
#include <arrow/dataset/file_parquet.h>

Public Functions

virtual Status Write(const std::shared_ptr<RecordBatch> &batch) override

Write the given batch.

struct ParquetFactoryOptions
#include <arrow/dataset/file_parquet.h>

Options for making a FileSystemDataset from a Parquet _metadata file.

Public Members

PartitioningOrFactory partitioning = {Partitioning::Default()}

Either an explicit Partitioning or a PartitioningFactory to discover one.

If a factory is provided, it will be used to infer a schema for partition fields based on file and directory paths then construct a Partitioning. The default is a Partitioning which will yield no partition information.

The (explicit or discovered) partitioning will be applied to discovered files and the resulting partition information embedded in the Dataset.

std::string partition_base_dir

For the purposes of applying the partitioning, paths will be stripped of the partition_base_dir.

Files not matching the partition_base_dir prefix will be skipped for partition discovery. The ignored files will still be part of the Dataset, but will not have partition information.

Example: partition_base_dir = “/dataset”;

  • “/dataset/US/sales.csv” -> “US/sales.csv” will be given to the partitioning

  • ”/home/john/late_sales.csv” -> Will be ignored for partition discovery.

This is useful for partitioning which parses directory when ordering is important, e.g. DirectoryPartitioning.

bool validate_column_chunk_paths = false

Assert that all ColumnChunk paths are consistent.

The parquet spec allows for ColumnChunk data to be stored in multiple files, but ParquetDatasetFactory supports only a single file with all ColumnChunk data. If this flag is set construction of a ParquetDatasetFactory will raise an error if ColumnChunk data is not resident in a single file.

class ParquetDatasetFactory : public arrow::dataset::DatasetFactory
#include <arrow/dataset/file_parquet.h>

Create FileSystemDataset from custom _metadata cache file.

Dask and other systems will generate a cache metadata file by concatenating the RowGroupMetaData of multiple parquet files into a single parquet file that only contains metadata and no ColumnChunk data.

ParquetDatasetFactory creates a FileSystemDataset composed of ParquetFileFragment where each fragment is pre-populated with the exact number of row groups and statistics for each columns.

Public Functions

virtual Result<std::vector<std::shared_ptr<Schema>>> InspectSchemas(InspectOptions options) override

Get the schemas of the Fragments and Partitioning.

virtual Result<std::shared_ptr<Dataset>> Finish(FinishOptions options) override

Create a Dataset with the given options.

Public Static Functions

static Result<std::shared_ptr<DatasetFactory>> Make(const std::string &metadata_path, std::shared_ptr<fs::FileSystem> filesystem, std::shared_ptr<ParquetFileFormat> format, ParquetFactoryOptions options)

Create a ParquetDatasetFactory from a metadata path.

The metadata_path will be read from filesystem. Each RowGroup contained in the metadata file will be relative to dirname(metadata_path).

Parameters:
  • metadata_path[in] path of the metadata parquet file

  • filesystem[in] from which to open/read the path

  • format[in] to read the file with.

  • options[in] see ParquetFactoryOptions

static Result<std::shared_ptr<DatasetFactory>> Make(const FileSource &metadata, const std::string &base_path, std::shared_ptr<fs::FileSystem> filesystem, std::shared_ptr<ParquetFileFormat> format, ParquetFactoryOptions options)

Create a ParquetDatasetFactory from a metadata source.

Similar to the previous Make definition, but the metadata can be a Buffer and the base_path is explicited instead of inferred from the metadata path.

Parameters:
  • metadata[in] source to open the metadata parquet file from

  • base_path[in] used as the prefix of every parquet files referenced

  • filesystem[in] from which to read the files referenced.

  • format[in] to read the file with.

  • options[in] see ParquetFactoryOptions

struct RadosConnCtx
#include <skyhook/client/file_skyhook.h>

A struct to hold the parameters required for connecting to a RADOS cluster.

class SkyhookFileFormat : public arrow::dataset::FileFormat
#include <skyhook/client/file_skyhook.h>

A FileFormat implementation that offloads fragment scan operations to the Ceph OSDs. For more details, see the Skyhook paper, https://arxiv.org/pdf/2105.09894.pdf.

Public Functions

inline virtual std::string type_name() const override

The name identifying the kind of file format.

inline virtual arrow::Result<bool> IsSupported(const arrow::dataset::FileSource &source) const override

Indicate if the FileSource is supported/readable by this format.

virtual arrow::Result<std::shared_ptr<arrow::Schema>> Inspect(const arrow::dataset::FileSource &source) const override

Return the schema of the file fragment.

Parameters:

source[in] The source of the file fragment.

Returns:

The schema of the file fragment.

arrow::Result<std::shared_ptr<arrow::dataset::FileWriter>> MakeWriter(std::shared_ptr<arrow::io::OutputStream> destination, std::shared_ptr<arrow::Schema> schema, std::shared_ptr<arrow::dataset::FileWriteOptions> options, arrow::fs::FileLocator destination_locator) const override

Create a writer for this format.

virtual std::shared_ptr<arrow::dataset::FileWriteOptions> DefaultWriteOptions() override

Get default write options for this format.