apache_beam.io.fileio module

PTransforms for manipulating files in Apache Beam.

Provides reading PTransforms, MatchFiles, MatchAll, that produces a PCollection of records representing a file and its metadata; and ReadMatches, which takes in a PCollection of file metadata records, and produces a PCollection of ReadableFile objects. These transforms currently do not support splitting by themselves.

Writing to Files

The transforms in this file include WriteToFiles, which allows you to write a beam.PCollection to files, and gives you many options to customize how to do this.

The WriteToFiles transform supports bounded and unbounded PCollections (i.e. it can be used both batch and streaming pipelines). For streaming pipelines, it currently does not have support for multiple trigger firings on the same window.

File Naming

One of the parameters received by WriteToFiles is a function specifying how to name the files that are written. This is a function that takes in the following parameters:

  • window
  • pane
  • shard_index
  • total_shards
  • compression
  • destination

It should return a file name that is unique for a combination of these parameters.

The default naming strategy is to name files in the format $prefix-$start-$end-$pane-$shard-of-$numShards$suffix$compressionSuffix, where:

  • $prefix is, by default, “output”.
  • $start and $end are the boundaries of the window for the data being written. These are omitted if we’re using the Global window.
  • $pane is the index for the number of firing for a window.
  • $shard and $numShards are the current shard number, and the total number of shards for this window firing.
  • $suffix is, by default, an empty string, but it can be set by the user via default_file_naming.

Dynamic Destinations

If the elements in the input beam.PCollection can be partitioned into groups that should be treated differently (e.g. some events are to be stored as CSV, while some others are to be stored as Avro files), it is possible to do this by passing a destination parameter to WriteToFiles. Something like the following:

my_pcollection | beam.io.fileio.WriteToFiles(
      path='/my/file/path',
      destination=lambda record: 'avro' if record['type'] == 'A' else 'csv',
      sink=lambda dest: AvroSink() if dest == 'avro' else CsvSink(),
      file_naming=beam.io.fileio.destination_prefix_naming())

In this transform, depending on the type of a record, it will be written down to a destination named ‘avro’, or ‘csv’. The value returned by the destination call is then passed to the sink call, to determine what sort of sink will be used for each destination. The return type of the destination parameter can be anything, as long as elements can be grouped by it.

class apache_beam.io.fileio.EmptyMatchTreatment[source]

Bases: object

How to treat empty matches in MatchAll and MatchFiles transforms.

If empty matches are disallowed, an error will be thrown if a pattern does not match any files.

ALLOW = 'ALLOW'
DISALLOW = 'DISALLOW'
ALLOW_IF_WILDCARD = 'ALLOW_IF_WILDCARD'
static allow_empty_match(pattern, setting)[source]
class apache_beam.io.fileio.MatchFiles(file_pattern: str, empty_match_treatment='ALLOW_IF_WILDCARD')[source]

Bases: apache_beam.transforms.ptransform.PTransform

Matches a file pattern using FileSystems.match.

This PTransform returns a PCollection of matching files in the form of FileMetadata objects.

expand(pcoll) → apache_beam.pvalue.PCollection[apache_beam.io.filesystem.FileMetadata][apache_beam.io.filesystem.FileMetadata][source]
class apache_beam.io.fileio.MatchAll(empty_match_treatment='ALLOW')[source]

Bases: apache_beam.transforms.ptransform.PTransform

Matches file patterns from the input PCollection via FileSystems.match.

This PTransform returns a PCollection of matching files in the form of FileMetadata objects.

expand(pcoll: apache_beam.pvalue.PCollection) → apache_beam.pvalue.PCollection[apache_beam.io.filesystem.FileMetadata][apache_beam.io.filesystem.FileMetadata][source]
class apache_beam.io.fileio.ReadableFile(metadata, compression=None)[source]

Bases: object

A utility class for accessing files.

open(mime_type='text/plain', compression_type=None)[source]
read(mime_type='application/octet-stream')[source]
read_utf8()[source]
class apache_beam.io.fileio.ReadMatches(compression=None, skip_directories=True)[source]

Bases: apache_beam.transforms.ptransform.PTransform

Converts each result of MatchFiles() or MatchAll() to a ReadableFile.

This helps read in a file’s contents or obtain a file descriptor.

expand(pcoll: apache_beam.pvalue.PCollection[typing.Union[str, apache_beam.io.filesystem.FileMetadata]][Union[str, apache_beam.io.filesystem.FileMetadata]]) → apache_beam.pvalue.PCollection[apache_beam.io.fileio.ReadableFile][apache_beam.io.fileio.ReadableFile][source]