layout: global title: Structured Streaming + Kafka Integration Guide (Kafka broker version 0.10.0 or higher)

Structured Streaming integration for Kafka 0.10 to read data from and write data to Kafka.

Linking

For Scala/Java applications using SBT/Maven project definitions, link your application with the following artifact:

groupId = org.apache.spark
artifactId = spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}}
version = {{site.SPARK_VERSION_SHORT}}

For Python applications, you need to add this above library and its dependencies when deploying your application. See the Deploying subsection below.

For experimenting on spark-shell, you need to add this above library and its dependencies too when invoking spark-shell. Also, see the Deploying subsection below.

Reading Data from Kafka

Creating a Kafka Source for Streaming Queries

// Subscribe to 1 topic val df = spark .readStream .format(“kafka”) .option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”) .option(“subscribe”, “topic1”) .load() df.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”) .as[(String, String)]

// Subscribe to multiple topics val df = spark .readStream .format(“kafka”) .option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”) .option(“subscribe”, “topic1,topic2”) .load() df.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”) .as[(String, String)]

// Subscribe to a pattern val df = spark .readStream .format(“kafka”) .option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”) .option(“subscribePattern”, “topic.*”) .load() df.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”) .as[(String, String)]

{% endhighlight %}

// Subscribe to 1 topic Dataset df = spark .readStream() .format(“kafka”) .option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”) .option(“subscribe”, “topic1”) .load() df.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”)

// Subscribe to multiple topics Dataset df = spark .readStream() .format(“kafka”) .option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”) .option(“subscribe”, “topic1,topic2”) .load() df.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”)

// Subscribe to a pattern Dataset df = spark .readStream() .format(“kafka”) .option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”) .option(“subscribePattern”, “topic.*”) .load() df.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”)

{% endhighlight %}

Subscribe to 1 topic

df = spark
.readStream
.format(“kafka”)
.option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”)
.option(“subscribe”, “topic1”)
.load() df.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”)

Subscribe to multiple topics

df = spark
.readStream
.format(“kafka”)
.option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”)
.option(“subscribe”, “topic1,topic2”)
.load() df.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”)

Subscribe to a pattern

df = spark
.readStream
.format(“kafka”)
.option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”)
.option(“subscribePattern”, “topic.*”)
.load() df.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”)

{% endhighlight %}

Creating a Kafka Source for Batch Queries

If you have a use case that is better suited to batch processing, you can create a Dataset/DataFrame for a defined range of offsets.

// Subscribe to 1 topic defaults to the earliest and latest offsets val df = spark .read .format(“kafka”) .option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”) .option(“subscribe”, “topic1”) .load() df.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”) .as[(String, String)]

// Subscribe to multiple topics, specifying explicit Kafka offsets val df = spark .read .format(“kafka”) .option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”) .option(“subscribe”, “topic1,topic2”) .option(“startingOffsets”, “““{“topic1”:{“0”:23,“1”:-2},“topic2”:{“0”:-2}}”””) .option(“endingOffsets”, “““{“topic1”:{“0”:50,“1”:-1},“topic2”:{“0”:-1}}”””) .load() df.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”) .as[(String, String)]

// Subscribe to a pattern, at the earliest and latest offsets val df = spark .read .format(“kafka”) .option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”) .option(“subscribePattern”, “topic.*”) .option(“startingOffsets”, “earliest”) .option(“endingOffsets”, “latest”) .load() df.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”) .as[(String, String)]

{% endhighlight %}

// Subscribe to 1 topic defaults to the earliest and latest offsets Dataset df = spark .read() .format(“kafka”) .option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”) .option(“subscribe”, “topic1”) .load(); df.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”);

// Subscribe to multiple topics, specifying explicit Kafka offsets Dataset df = spark .read() .format(“kafka”) .option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”) .option(“subscribe”, “topic1,topic2”) .option(“startingOffsets”, “{"topic1":{"0":23,"1":-2},"topic2":{"0":-2}}”) .option(“endingOffsets”, “{"topic1":{"0":50,"1":-1},"topic2":{"0":-1}}”) .load(); df.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”);

// Subscribe to a pattern, at the earliest and latest offsets Dataset df = spark .read() .format(“kafka”) .option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”) .option(“subscribePattern”, “topic.*”) .option(“startingOffsets”, “earliest”) .option(“endingOffsets”, “latest”) .load(); df.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”);

{% endhighlight %}

Subscribe to 1 topic defaults to the earliest and latest offsets

df = spark
.read
.format(“kafka”)
.option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”)
.option(“subscribe”, “topic1”)
.load() df.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”)

Subscribe to multiple topics, specifying explicit Kafka offsets

df = spark
.read
.format(“kafka”)
.option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”)
.option(“subscribe”, “topic1,topic2”)
.option(“startingOffsets”, “““{“topic1”:{“0”:23,“1”:-2},“topic2”:{“0”:-2}}”””)
.option(“endingOffsets”, “““{“topic1”:{“0”:50,“1”:-1},“topic2”:{“0”:-1}}”””)
.load() df.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”)

Subscribe to a pattern, at the earliest and latest offsets

df = spark
.read
.format(“kafka”)
.option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”)
.option(“subscribePattern”, “topic.*”)
.option(“startingOffsets”, “earliest”)
.option(“endingOffsets”, “latest”)
.load() df.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”) {% endhighlight %}

Each row in the source has the following schema:

The following options must be set for the Kafka source for both batch and streaming queries.

The following configurations are optional:

Writing Data to Kafka

Here, we describe the support for writing Streaming Queries and Batch Queries to Apache Kafka. Take note that Apache Kafka only supports at least once write semantics. Consequently, when writing---either Streaming Queries or Batch Queries---to Kafka, some records may be duplicated; this can happen, for example, if Kafka needs to retry a message that was not acknowledged by a Broker, even though that Broker received and wrote the message record. Structured Streaming cannot prevent such duplicates from occurring due to these Kafka write semantics. However, if writing the query is successful, then you can assume that the query output was written at least once. A possible solution to remove duplicates when reading the written data could be to introduce a primary (unique) key that can be used to perform de-duplication when reading.

The Dataframe being written to Kafka should have the following columns in schema:

The value column is the only required option. If a key column is not specified then a null valued key column will be automatically added (see Kafka semantics on how null valued key values are handled). If a topic column exists then its value is used as the topic when writing the given row to Kafka, unless the “topic” configuration option is set i.e., the “topic” configuration option overrides the topic column.

The following options must be set for the Kafka sink for both batch and streaming queries.

The following configurations are optional:

Creating a Kafka Sink for Streaming Queries

// Write key-value data from a DataFrame to a specific Kafka topic specified in an option val ds = df .selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”) .writeStream .format(“kafka”) .option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”) .option(“topic”, “topic1”) .start()

// Write key-value data from a DataFrame to Kafka using a topic specified in the data val ds = df .selectExpr(“topic”, “CAST(key AS STRING)”, “CAST(value AS STRING)”) .writeStream .format(“kafka”) .option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”) .start()

{% endhighlight %}

// Write key-value data from a DataFrame to a specific Kafka topic specified in an option StreamingQuery ds = df .selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”) .writeStream() .format(“kafka”) .option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”) .option(“topic”, “topic1”) .start()

// Write key-value data from a DataFrame to Kafka using a topic specified in the data StreamingQuery ds = df .selectExpr(“topic”, “CAST(key AS STRING)”, “CAST(value AS STRING)”) .writeStream() .format(“kafka”) .option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”) .start()

{% endhighlight %}

Write key-value data from a DataFrame to a specific Kafka topic specified in an option

ds = df
.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”)
.writeStream
.format(“kafka”)
.option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”)
.option(“topic”, “topic1”)
.start()

Write key-value data from a DataFrame to Kafka using a topic specified in the data

ds = df
.selectExpr(“topic”, “CAST(key AS STRING)”, “CAST(value AS STRING)”)
.writeStream
.format(“kafka”)
.option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”)
.start()

{% endhighlight %}

Writing the output of Batch Queries to Kafka

// Write key-value data from a DataFrame to a specific Kafka topic specified in an option df.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”) .write .format(“kafka”) .option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”) .option(“topic”, “topic1”) .save()

// Write key-value data from a DataFrame to Kafka using a topic specified in the data df.selectExpr(“topic”, “CAST(key AS STRING)”, “CAST(value AS STRING)”) .write .format(“kafka”) .option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”) .save()

{% endhighlight %}

// Write key-value data from a DataFrame to a specific Kafka topic specified in an option df.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”) .write() .format(“kafka”) .option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”) .option(“topic”, “topic1”) .save()

// Write key-value data from a DataFrame to Kafka using a topic specified in the data df.selectExpr(“topic”, “CAST(key AS STRING)”, “CAST(value AS STRING)”) .write() .format(“kafka”) .option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”) .save()

{% endhighlight %}

Write key-value data from a DataFrame to a specific Kafka topic specified in an option

df.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”)
.write
.format(“kafka”)
.option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”)
.option(“topic”, “topic1”)
.save()

Write key-value data from a DataFrame to Kafka using a topic specified in the data

df.selectExpr(“topic”, “CAST(key AS STRING)”, “CAST(value AS STRING)”)
.write
.format(“kafka”)
.option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”)
.save()

{% endhighlight %}

Kafka Specific Configurations

Kafka's own configurations can be set via DataStreamReader.option with kafka. prefix, e.g, stream.option("kafka.bootstrap.servers", "host:port"). For possible kafka parameters, see Kafka consumer config docs for parameters related to reading data, and Kafka producer config docs for parameters related to writing data.

Note that the following Kafka params cannot be set and the Kafka source or sink will throw an exception:

  • group.id: Kafka source will create a unique group id for each query automatically.
  • auto.offset.reset: Set the source option startingOffsets to specify where to start instead. Structured Streaming manages which offsets are consumed internally, rather than rely on the kafka Consumer to do it. This will ensure that no data is missed when new topics/partitions are dynamically subscribed. Note that startingOffsets only applies when a new streaming query is started, and that resuming will always pick up from where the query left off.
  • key.deserializer: Keys are always deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame operations to explicitly deserialize the keys.
  • value.deserializer: Values are always deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame operations to explicitly deserialize the values.
  • key.serializer: Keys are always serialized with ByteArraySerializer or StringSerializer. Use DataFrame operations to explicitly serialize the keys into either strings or byte arrays.
  • value.serializer: values are always serialized with ByteArraySerializer or StringSerializer. Use DataFrame operations to explicitly serialize the values into either strings or byte arrays.
  • enable.auto.commit: Kafka source doesn't commit any offset.
  • interceptor.classes: Kafka source always read keys and values as byte arrays. It's not safe to use ConsumerInterceptor as it may break the query.

Deploying

As with any Spark applications, spark-submit is used to launch your application. spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}} and its dependencies can be directly added to spark-submit using --packages, such as,

./bin/spark-submit --packages org.apache.spark:spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ...

For experimenting on spark-shell, you can also use --packages to add spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}} and its dependencies directly,

./bin/spark-shell --packages org.apache.spark:spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ...

See Application Submission Guide for more details about submitting applications with external dependencies.