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

Structured Streaming integration for Kafka 0.10 to poll data from 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.

Creating a Kafka Source Stream

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

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

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

{% endhighlight %}

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

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

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

{% endhighlight %}

Subscribe to 1 topic

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

Subscribe to multiple topics

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

Subscribe to a pattern

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

{% endhighlight %}

Creating a Kafka Source Batch

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

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

// Subscribe to multiple topics, specifying explicit Kafka offsets val ds2 = 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() ds2.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”) .as[(String, String)]

// Subscribe to a pattern, at the earliest and latest offsets val ds3 = spark .read .format(“kafka”) .option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”) .option(“subscribePattern”, “topic.*”) .option(“startingOffsets”, “earliest”) .option(“endingOffsets”, “latest”) .load() ds3.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 ds1 = spark .read() .format(“kafka”) .option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”) .option(“subscribe”, “topic1”) .load(); ds1.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”);

// Subscribe to multiple topics, specifying explicit Kafka offsets Dataset ds2 = 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(); ds2.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”);

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

{% endhighlight %}

Subscribe to 1 topic defaults to the earliest and latest offsets

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

Subscribe to multiple topics, specifying explicit Kafka offsets

ds2 = 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() ds2.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”)

Subscribe to a pattern, at the earliest and latest offsets

ds3 = spark
.read
.format(“kafka”)
.option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”)
.option(“subscribePattern”, “topic.*”)
.option(“startingOffsets”, “earliest”)
.option(“endingOffsets”, “latest”)
.load() ds3.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:

Kafka's own configurations can be set via DataStreamReader.option with kafka. prefix, e.g, stream.option("kafka.bootstrap.servers", "host:port"). For possible kafkaParams, see Kafka consumer config docs.

Note that the following Kafka params cannot be set and the Kafka source 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 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.
  • 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}} ...

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