apache_beam.yaml.yaml_io module

This module contains the Python implementations for the builtin IOs.

They are referenced from standard_io.py.

Note that in the case that they overlap with other (likely Java) implementations of the same transforms, the configs must be kept in sync.

apache_beam.yaml.yaml_io.read_from_text(path: str)[source]

Reads lines from a text files.

The resulting PCollection consists of rows with a single string filed named “line.”

Parameters:path (str) – The file path to read from. The path can contain glob characters such as * and ?.
apache_beam.yaml.yaml_io.write_to_text(path: str)[source]

Writes a PCollection to a (set of) text files(s).

The input must be a PCollection whose schema has exactly one field.

Parameters:path (str) – The file path to write to. The files written will begin with this prefix, followed by a shard identifier.
apache_beam.yaml.yaml_io.read_from_bigquery(*, table: Optional[str] = None, query: Optional[str] = None, row_restriction: Optional[str] = None, fields: Optional[Iterable[str]] = None)[source]

Reads data from BigQuery.

Exactly one of table or query must be set. If query is set, neither row_restriction nor fields should be set.

Parameters:
  • table (str) – The table to read from, specified as DATASET.TABLE or PROJECT:DATASET.TABLE.
  • query (str) – A query to be used instead of the table argument.
  • row_restriction (str) – Optional SQL text filtering statement, similar to a WHERE clause in a query. Aggregates are not supported. Restricted to a maximum length for 1 MB.
  • selected_fields (List[str]) – Optional List of names of the fields in the table that should be read. If empty, all fields will be read. If the specified field is a nested field, all the sub-fields in the field will be selected. The output field order is unrelated to the order of fields given here.
apache_beam.yaml.yaml_io.write_to_bigquery(table: str, *, create_disposition: Optional[str] = 'CREATE_IF_NEEDED', write_disposition: Optional[str] = 'WRITE_APPEND', error_handling=None)[source]
apache_beam.yaml.yaml_io.read_from_pubsub(*, topic: Optional[str] = None, subscription: Optional[str] = None, format: str, schema: Optional[Any] = None, attributes: Optional[Iterable[str]] = None, attributes_map: Optional[str] = None, id_attribute: Optional[str] = None, timestamp_attribute: Optional[str] = None, error_handling: apache_beam.yaml.yaml_mapping.ErrorHandlingConfig = None)[source]

Reads messages from Cloud Pub/Sub.

Parameters:
  • topic – Cloud Pub/Sub topic in the form “projects/<project>/topics/<topic>”. If provided, subscription must be None.
  • subscription – Existing Cloud Pub/Sub subscription to use in the form “projects/<project>/subscriptions/<subscription>”. If not specified, a temporary subscription will be created from the specified topic. If provided, topic must be None.
  • format

    The expected format of the message payload. Currently suported formats are

    • RAW: Produces records with a single payload field whose contents
      are the raw bytes of the pubsub message.
    • AVRO: Parses records with a given Avro schema.
    • JSON: Parses records with a given JSON schema.
  • schema – Schema specification for the given format.
  • attributes – List of attribute keys whose values will be flattened into the output message as additional fields. For example, if the format is raw and attributes is [“a”, “b”] then this read will produce elements of the form Row(payload=…, a=…, b=…).
  • attribute_map – Name of a field in which to store the full set of attributes associated with this message. For example, if the format is raw and attribute_map is set to “attrs” then this read will produce elements of the form Row(payload=…, attrs=…) where attrs is a Map type of string to string. If both attributes and attribute_map are set, the overlapping attribute values will be present in both the flattened structure and the attribute map.
  • id_attribute – The attribute on incoming Pub/Sub messages to use as a unique record identifier. When specified, the value of this attribute (which can be any string that uniquely identifies the record) will be used for deduplication of messages. If not provided, we cannot guarantee that no duplicate data will be delivered on the Pub/Sub stream. In this case, deduplication of the stream will be strictly best effort.
  • timestamp_attribute

    Message value to use as element timestamp. If None, uses message publishing time as the timestamp.

    Timestamp values should be in one of two formats:

    • A numerical value representing the number of milliseconds since the Unix epoch.
    • A string in RFC 3339 format, UTC timezone. Example: 2015-10-29T23:41:41.123Z. The sub-second component of the timestamp is optional, and digits beyond the first three (i.e., time units smaller than milliseconds) may be ignored.
apache_beam.yaml.yaml_io.write_to_pubsub(*, topic: str, format: str, schema: Optional[Any] = None, attributes: Optional[Iterable[str]] = None, attributes_map: Optional[str] = None, id_attribute: Optional[str] = None, timestamp_attribute: Optional[str] = None, error_handling: apache_beam.yaml.yaml_mapping.ErrorHandlingConfig = None)[source]

Writes messages to Cloud Pub/Sub.

Parameters:
  • topic – Cloud Pub/Sub topic in the form “/topics/<project>/<topic>”.
  • format

    How to format the message payload. Currently suported formats are

    • RAW: Expects a message with a single field (excluding
      attribute-related fields) whose contents are used as the raw bytes of the pubsub message.
    • AVRO: Encodes records with a given Avro schema, which may be inferred
      from the input PCollection schema.
    • JSON: Formats records with a given JSON schema, which may be inferred
      from the input PCollection schema.
  • schema – Schema specification for the given format.
  • attributes – List of attribute keys whose values will be pulled out as PubSub message attributes. For example, if the format is raw and attributes is [“a”, “b”] then elements of the form Row(any_field=…, a=…, b=…) will result in PubSub messages whose payload has the contents of any_field and whose attribute will be populated with the values of a and b.
  • attribute_map – Name of a string-to-string map field in which to pull a set of attributes associated with this message. For example, if the format is raw and attribute_map is set to “attrs” then elements of the form Row(any_field=…, attrs=…) will result in PubSub messages whose payload has the contents of any_field and whose attribute will be populated with the values from attrs. If both attributes and attribute_map are set, the union of attributes from these two sources will be used to populate the PubSub message attributes.
  • id_attribute – If set, will set an attribute for each Cloud Pub/Sub message with the given name and a unique value. This attribute can then be used in a ReadFromPubSub PTransform to deduplicate messages.
  • timestamp_attribute – If set, will set an attribute for each Cloud Pub/Sub message with the given name and the message’s publish time as the value.
apache_beam.yaml.yaml_io.io_providers()[source]