apache_beam.io.kafka module

Unbounded source and sink transforms for

Kafka.

These transforms are currently supported by Beam portable runners (for example, portable Flink and Spark) as well as Dataflow runner.

Setup

Transforms provided in this module are cross-language transforms implemented in the Beam Java SDK. During the pipeline construction, Python SDK will connect to a Java expansion service to expand these transforms. To facilitate this, a small amount of setup is needed before using these transforms in a Beam Python pipeline.

There are several ways to setup cross-language Kafka transforms.

  • Option 1: use the default expansion service

  • Option 2: specify a custom expansion service

See below for details regarding each of these options.

Option 1: Use the default expansion service

This is the recommended and easiest setup option for using Python Kafka transforms. This option is only available for Beam 2.22.0 and later.

This option requires following pre-requisites before running the Beam pipeline.

  • Install Java runtime in the computer from where the pipeline is constructed and make sure that ‘java’ command is available.

In this option, Python SDK will either download (for released Beam version) or build (when running from a Beam Git clone) an expansion service jar and use that to expand transforms. Currently Kafka transforms use the ‘beam-sdks-java-io-expansion-service’ jar for this purpose.

Note that the KafkaIO read transform can be compiled in two modes

  • ReadFromKafkaViaUnbounded (legacy)

  • ReadFromKafkaViaSDF (default)

To use the legacy mode, the use_deprecated_read flag should be specified within the IO expansion service. For example,

kafka.default_io_expansion_service(

append_args=[”–experiments=use_deprecated_read”]

)

Option 2: specify a custom expansion service

In this option, you startup your own expansion service and provide that as a parameter when using the transforms provided in this module.

This option requires following pre-requisites before running the Beam pipeline.

  • Startup your own expansion service.

  • Update your pipeline to provide the expansion service address when initiating Kafka transforms provided in this module.

Flink Users can use the built-in Expansion Service of the Flink Runner’s Job Server. If you start Flink’s Job Server, the expansion service will be started on port 8097. For a different address, please set the expansion_service parameter.

More information

For more information regarding cross-language transforms see: - https://beam.apache.org/roadmap/portability/

For more information specific to Flink runner see: - https://beam.apache.org/documentation/runners/flink/

class apache_beam.io.kafka.ReadFromKafkaSchema(consumer_config, topics, key_deserializer, value_deserializer, start_read_time, max_num_records, max_read_time, commit_offset_in_finalize, timestamp_policy, consumer_polling_timeout, redistribute, redistribute_num_keys, allow_duplicates)

Bases: tuple

Create new instance of ReadFromKafkaSchema(consumer_config, topics, key_deserializer, value_deserializer, start_read_time, max_num_records, max_read_time, commit_offset_in_finalize, timestamp_policy, consumer_polling_timeout, redistribute, redistribute_num_keys, allow_duplicates)

allow_duplicates: bool | None

Alias for field number 12

commit_offset_in_finalize: bool

Alias for field number 7

consumer_config: Mapping[str, str]

Alias for field number 0

consumer_polling_timeout: int | None

Alias for field number 9

key_deserializer: str

Alias for field number 2

max_num_records: int | None

Alias for field number 5

max_read_time: int | None

Alias for field number 6

redistribute: bool | None

Alias for field number 10

redistribute_num_keys: int32 | None

Alias for field number 11

start_read_time: int | None

Alias for field number 4

timestamp_policy: str

Alias for field number 8

topics: List[str]

Alias for field number 1

value_deserializer: str

Alias for field number 3

apache_beam.io.kafka.default_io_expansion_service(append_args=None)[source]
class apache_beam.io.kafka.ReadFromKafka(consumer_config, topics, key_deserializer='org.apache.kafka.common.serialization.ByteArrayDeserializer', value_deserializer='org.apache.kafka.common.serialization.ByteArrayDeserializer', start_read_time=None, max_num_records=None, max_read_time=None, commit_offset_in_finalize=False, timestamp_policy='ProcessingTime', consumer_polling_timeout=2, with_metadata=False, expansion_service=None, redistribute=False, redistribute_num_keys=0, allow_duplicates=False)[source]

Bases: ExternalTransform

An external PTransform which reads from Kafka and returns a KV pair for each item in the specified Kafka topics. If no Kafka Deserializer for key/value is provided, then the data will be returned as a raw byte array.

Experimental; no backwards compatibility guarantees.

Initializes a read operation from Kafka.

Parameters:
  • consumer_config – A dictionary containing the consumer configuration.

  • topics – A list of topic strings.

  • key_deserializer – A fully-qualified Java class name of a Kafka Deserializer for the topic’s key, e.g. ‘org.apache.kafka.common.serialization.LongDeserializer’. Default: ‘org.apache.kafka.common.serialization.ByteArrayDeserializer’.

  • value_deserializer – A fully-qualified Java class name of a Kafka Deserializer for the topic’s value, e.g. ‘org.apache.kafka.common.serialization.LongDeserializer’. Default: ‘org.apache.kafka.common.serialization.ByteArrayDeserializer’.

  • start_read_time – Use timestamp to set up start offset in milliseconds epoch.

  • max_num_records – Maximum amount of records to be read. Mainly used for tests and demo applications.

  • max_read_time – Maximum amount of time in seconds the transform executes. Mainly used for tests and demo applications.

  • commit_offset_in_finalize – Whether to commit offsets when finalizing.

  • timestamp_policy – The built-in timestamp policy which is used for extracting timestamp from KafkaRecord.

  • consumer_polling_timeout – Kafka client polling request timeout time in seconds. A lower timeout optimizes for latency. Increase the timeout if the consumer is not fetching any records. Default is 2 seconds.

  • with_metadata – whether the returned PCollection should contain Kafka related metadata or not. If False (default), elements of the returned PCollection will be of type ‘bytes’ if True, elements of the returned PCollection will be of the type ‘Row’. Note that, currently this only works when using default key and value deserializers where Java Kafka Reader reads keys and values as ‘byte[]’.

  • expansion_service – The address (host:port) of the ExpansionService.

  • redistribute – whether a Redistribute transform should be applied immediately after the read.

  • redistribute_num_keys – Configures how many keys the Redistribute spreads the data across.

  • allow_duplicates – whether the Redistribute transform allows for duplicates (this serves solely as a hint to the underlying runner).

byte_array_deserializer = 'org.apache.kafka.common.serialization.ByteArrayDeserializer'
processing_time_policy = 'ProcessingTime'
create_time_policy = 'CreateTime'
log_append_time = 'LogAppendTime'
URN_WITH_METADATA = 'beam:transform:org.apache.beam:kafka_read_with_metadata:v1'
URN_WITHOUT_METADATA = 'beam:transform:org.apache.beam:kafka_read_without_metadata:v1'
class apache_beam.io.kafka.WriteToKafkaSchema(producer_config, topic, key_serializer, value_serializer)

Bases: tuple

Create new instance of WriteToKafkaSchema(producer_config, topic, key_serializer, value_serializer)

key_serializer: str

Alias for field number 2

producer_config: Mapping[str, str]

Alias for field number 0

topic: str

Alias for field number 1

value_serializer: str

Alias for field number 3

class apache_beam.io.kafka.WriteToKafka(producer_config, topic, key_serializer='org.apache.kafka.common.serialization.ByteArraySerializer', value_serializer='org.apache.kafka.common.serialization.ByteArraySerializer', expansion_service=None)[source]

Bases: ExternalTransform

An external PTransform which writes KV data to a specified Kafka topic. If no Kafka Serializer for key/value is provided, then key/value are assumed to be byte arrays.

Experimental; no backwards compatibility guarantees.

Initializes a write operation to Kafka.

Parameters:
  • producer_config – A dictionary containing the producer configuration.

  • topic – A Kafka topic name.

  • key_deserializer – A fully-qualified Java class name of a Kafka Serializer for the topic’s key, e.g. ‘org.apache.kafka.common.serialization.LongSerializer’. Default: ‘org.apache.kafka.common.serialization.ByteArraySerializer’.

  • value_deserializer – A fully-qualified Java class name of a Kafka Serializer for the topic’s value, e.g. ‘org.apache.kafka.common.serialization.LongSerializer’. Default: ‘org.apache.kafka.common.serialization.ByteArraySerializer’.

  • expansion_service – The address (host:port) of the ExpansionService.

byte_array_serializer = 'org.apache.kafka.common.serialization.ByteArraySerializer'
URN = 'beam:transform:org.apache.beam:kafka_write:v1'