layout: global title: Structured Streaming + Kafka Integration Guide (Kafka broker version 0.10.0 or higher) license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the “License”); you may not use this file except in compliance with the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Structured Streaming integration for Kafka 0.10 to read data from and write data to Kafka.
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}}
Please note that to use the headers functionality, your Kafka client version should be version 0.11.0.0 or up.
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.
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)”)
df = spark
.readStream
.format(“kafka”)
.option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”)
.option(“subscribe”, “topic1”)
.option(“includeHeaders”, “true”)
.load() df.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”, “headers”)
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)”)
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 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 1 topic, with headers val df = spark .readStream .format(“kafka”) .option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”) .option(“subscribe”, “topic1”) .option(“includeHeaders”, “true”) .load() df.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”, “headers”) .as[(String, String, Array[(String, Array[Byte])])]
// 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 1 topic, with headers Dataset df = spark .readStream() .format(“kafka”) .option(“kafka.bootstrap.servers”, “host1:port1,host2:port2”) .option(“subscribe”, “topic1”) .option(“includeHeaders”, “true”) .load() df.selectExpr(“CAST(key AS STRING)”, “CAST(value AS STRING)”, “headers”);
// 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 %}
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.
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)”)
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)”)
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 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 %}
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:
The returned offset for each partition is the earliest offset whose timestamp is greater than or equal to the given timestamp in the corresponding partition. The behavior varies across options if Kafka doesn't return the matched offset - check the description of each option.
Spark simply passes the timestamp information to KafkaConsumer.offsetsForTimes, and doesn't interpret or reason about the value. For more details on KafkaConsumer.offsetsForTimes, please refer javadoc for details. Also, the meaning of timestamp here can be vary according to Kafka configuration (log.message.timestamp.type): please refer Kafka documentation for further details.
Timestamp offset options require Kafka 0.10.1.0 or higher.
In Spark 3.0 and before Spark uses KafkaConsumer for offset fetching which could cause infinite wait in the driver. In Spark 3.1 a new configuration option added spark.sql.streaming.kafka.useDeprecatedOffsetFetching (default: false) which allows Spark to use new offset fetching mechanism using AdminClient. (Set this to true
to use old offset fetching with KafkaConsumer.)
When the new mechanism used the following applies.
First of all the new approach supports Kafka brokers 0.11.0.0+
.
In Spark 3.0 and below, secure Kafka processing needed the following ACLs from driver perspective:
Since Spark 3.1, offsets can be obtained with AdminClient instead of KafkaConsumer and for that the following ACLs needed from driver perspective:
Since AdminClient in driver is not connecting to consumer group, group.id based authorization will not work anymore (executors never done group based authorization). Worth to mention executor side is behaving the exact same way like before (group prefix and override works).
It's time-consuming to initialize Kafka consumers, especially in streaming scenarios where processing time is a key factor. Because of this, Spark pools Kafka consumers on executors, by leveraging Apache Commons Pool.
The caching key is built up from the following information:
The following properties are available to configure the consumer pool:
The size of the pool is limited by spark.kafka.consumer.cache.capacity, but it works as “soft-limit” to not block Spark tasks.
Idle eviction thread periodically removes consumers which are not used longer than given timeout. If this threshold is reached when borrowing, it tries to remove the least-used entry that is currently not in use.
If it cannot be removed, then the pool will keep growing. In the worst case, the pool will grow to the max number of concurrent tasks that can run in the executor (that is, number of task slots).
If a task fails for any reason, the new task is executed with a newly created Kafka consumer for safety reasons. At the same time, we invalidate all consumers in pool which have same caching key, to remove consumer which was used in failed execution. Consumers which any other tasks are using will not be closed, but will be invalidated as well when they are returned into pool.
Along with consumers, Spark pools the records fetched from Kafka separately, to let Kafka consumers stateless in point of Spark‘s view, and maximize the efficiency of pooling. It leverages same cache key with Kafka consumers pool. Note that it doesn’t leverage Apache Commons Pool due to the difference of characteristics.
The following properties are available to configure the fetched data pool:
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. If a “partition” column is not specified (or its value is null
) then the partition is calculated by the Kafka producer. A Kafka partitioner can be specified in Spark by setting the kafka.partitioner.class
option. If not present, Kafka default partitioner will be used.
The following options must be set for the Kafka sink for both batch and streaming queries.
The following configurations are optional:
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()
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 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 %}
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()
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 %}
Given Kafka producer instance is designed to be thread-safe, Spark initializes a Kafka producer instance and co-use across tasks for same caching key.
The caching key is built up from the following information:
This includes configuration for authorization, which Spark will automatically include when delegation token is being used. Even we take authorization into account, you can expect same Kafka producer instance will be used among same Kafka producer configuration. It will use different Kafka producer when delegation token is renewed; Kafka producer instance for old delegation token will be evicted according to the cache policy.
The following properties are available to configure the producer pool:
Idle eviction thread periodically removes producers which are not used longer than given timeout. Note that the producer is shared and used concurrently, so the last used timestamp is determined by the moment the producer instance is returned and reference count is 0.
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:
groupIdPrefix
, default value is “spark-kafka-source”. You can also set “kafka.group.id” to force Spark to use a special group id, however, please read warnings for this option and use it with caution.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. Note that when the offsets consumed by a streaming application no longer exist in Kafka (e.g., topics are deleted, offsets are out of range, or offsets are removed after retention period), the offsets will not be reset and the streaming application will see data loss. In extreme cases, for example the throughput of the streaming application cannot catch up the retention speed of Kafka, the input rows of a batch might be gradually reduced until zero when the offset ranges of the batch are completely not in Kafka. Enabling failOnDataLoss
option can ask Structured Streaming to fail the query for such cases.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.
Kafka 0.9.0.0 introduced several features that increases security in a cluster. For detailed description about these possibilities, see Kafka security docs.
It's worth noting that security is optional and turned off by default.
Spark supports the following ways to authenticate against Kafka cluster:
This way the application can be configured via Spark parameters and may not need JAAS login configuration (Spark can use Kafka's dynamic JAAS configuration feature). For further information about delegation tokens, see Kafka delegation token docs.
The process is initiated by Spark's Kafka delegation token provider. When spark.kafka.clusters.${cluster}.auth.bootstrap.servers
is set, Spark considers the following log in options, in order of preference:
JAAS login configuration, please see example below.
Keytab file, such as,
./bin/spark-submit \ --keytab <KEYTAB_FILE> \ --principal <PRINCIPAL> \ --conf spark.kafka.clusters.${cluster}.auth.bootstrap.servers=<KAFKA_SERVERS> \ ...
Kerberos credential cache, such as,
./bin/spark-submit \ --conf spark.kafka.clusters.${cluster}.auth.bootstrap.servers=<KAFKA_SERVERS> \ ...
The Kafka delegation token provider can be turned off by setting spark.security.credentials.kafka.enabled
to false
(default: true
).
Spark can be configured to use the following authentication protocols to obtain token (it must match with Kafka broker configuration):
After obtaining delegation token successfully, Spark distributes it across nodes and renews it accordingly. Delegation token uses SCRAM
login module for authentication and because of that the appropriate spark.kafka.clusters.${cluster}.sasl.token.mechanism
(default: SCRAM-SHA-512
) has to be configured. Also, this parameter must match with Kafka broker configuration.
When delegation token is available on an executor Spark considers the following log in options, in order of preference:
When none of the above applies then unsecure connection assumed.
Delegation tokens can be obtained from multiple clusters and ${cluster} is an arbitrary unique identifier which helps to group different configurations.
Kafka's own configurations can be set with kafka.
prefix, e.g, --conf spark.kafka.clusters.${cluster}.kafka.retries=1
. For possible Kafka parameters, see Kafka adminclient config docs.
JAAS login configuration must placed on all nodes where Spark tries to access Kafka cluster. This provides the possibility to apply any custom authentication logic with a higher cost to maintain. This can be done several ways. One possibility is to provide additional JVM parameters, such as,
./bin/spark-submit \ --driver-java-options "-Djava.security.auth.login.config=/path/to/custom_jaas.conf" \ --conf spark.executor.extraJavaOptions=-Djava.security.auth.login.config=/path/to/custom_jaas.conf \ ...