[SPARK-26350][SS] Allow to override group id of the Kafka consumer
## What changes were proposed in this pull request? This PR allows the user to override `kafka.group.id` for better monitoring or security. The user needs to make sure there are not multiple queries or sources using the same group id. It also fixes a bug that the `groupIdPrefix` option cannot be retrieved. ## How was this patch tested? The new added unit tests. Closes #23301 from zsxwing/SPARK-26350. Authored-by: Shixiong Zhu <zsxwing@gmail.com> Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
This commit is contained in:
parent
115fecfd84
commit
bafc7ac025
|
@ -379,7 +379,25 @@ The following configurations are optional:
|
|||
<td>string</td>
|
||||
<td>spark-kafka-source</td>
|
||||
<td>streaming and batch</td>
|
||||
<td>Prefix of consumer group identifiers (`group.id`) that are generated by structured streaming queries</td>
|
||||
<td>Prefix of consumer group identifiers (`group.id`) that are generated by structured streaming
|
||||
queries. If "kafka.group.id" is set, this option will be ignored. </td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>kafka.group.id</td>
|
||||
<td>string</td>
|
||||
<td>none</td>
|
||||
<td>streaming and batch</td>
|
||||
<td>The Kafka group id to use in Kafka consumer while reading from Kafka. Use this with caution.
|
||||
By default, each query generates a unique group id for reading data. This ensures that each Kafka
|
||||
source has its own consumer group that does not face interference from any other consumer, and
|
||||
therefore can read all of the partitions of its subscribed topics. In some scenarios (for example,
|
||||
Kafka group-based authorization), you may want to use a specific authorized group id to read data.
|
||||
You can optionally set the group id. However, do this with extreme caution as it can cause
|
||||
unexpected behavior. Concurrently running queries (both, batch and streaming) or sources with the
|
||||
same group id are likely interfere with each other causing each query to read only part of the
|
||||
data. This may also occur when queries are started/restarted in quick succession. To minimize such
|
||||
issues, set the Kafka consumer session timeout (by setting option "kafka.session.timeout.ms") to
|
||||
be very small. When this is set, option "groupIdPrefix" will be ignored. </td>
|
||||
</tr>
|
||||
</table>
|
||||
|
||||
|
@ -592,8 +610,9 @@ 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. The user can
|
||||
set the prefix of the automatically generated group.id's via the optional source option `groupIdPrefix`, default value
|
||||
is "spark-kafka-source".
|
||||
set the prefix of the automatically generated group.id's via the optional source option `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.
|
||||
- **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
|
||||
|
|
|
@ -20,7 +20,7 @@ package org.apache.spark.sql.kafka010
|
|||
import java.{util => ju}
|
||||
import java.util.concurrent.TimeoutException
|
||||
|
||||
import org.apache.kafka.clients.consumer.{ConsumerRecord, OffsetOutOfRangeException}
|
||||
import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, OffsetOutOfRangeException}
|
||||
import org.apache.kafka.common.TopicPartition
|
||||
|
||||
import org.apache.spark.TaskContext
|
||||
|
@ -167,7 +167,13 @@ class KafkaContinuousScanConfigBuilder(
|
|||
|
||||
val deletedPartitions = oldStartPartitionOffsets.keySet.diff(currentPartitionSet)
|
||||
if (deletedPartitions.nonEmpty) {
|
||||
reportDataLoss(s"Some partitions were deleted: $deletedPartitions")
|
||||
val message = if (
|
||||
offsetReader.driverKafkaParams.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) {
|
||||
s"$deletedPartitions are gone. ${KafkaSourceProvider.CUSTOM_GROUP_ID_ERROR_MESSAGE}"
|
||||
} else {
|
||||
s"$deletedPartitions are gone. Some data may have been missed."
|
||||
}
|
||||
reportDataLoss(message)
|
||||
}
|
||||
|
||||
val startOffsets = newPartitionOffsets ++
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.io._
|
|||
import java.nio.charset.StandardCharsets
|
||||
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.kafka.clients.consumer.ConsumerConfig
|
||||
|
||||
import org.apache.spark.SparkEnv
|
||||
import org.apache.spark.internal.Logging
|
||||
|
@ -122,7 +123,13 @@ private[kafka010] class KafkaMicroBatchReadSupport(
|
|||
// Find deleted partitions, and report data loss if required
|
||||
val deletedPartitions = startPartitionOffsets.keySet.diff(endPartitionOffsets.keySet)
|
||||
if (deletedPartitions.nonEmpty) {
|
||||
reportDataLoss(s"$deletedPartitions are gone. Some data may have been missed")
|
||||
val message =
|
||||
if (kafkaOffsetReader.driverKafkaParams.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) {
|
||||
s"$deletedPartitions are gone. ${KafkaSourceProvider.CUSTOM_GROUP_ID_ERROR_MESSAGE}"
|
||||
} else {
|
||||
s"$deletedPartitions are gone. Some data may have been missed."
|
||||
}
|
||||
reportDataLoss(message)
|
||||
}
|
||||
|
||||
// Use the end partitions to calculate offset ranges to ignore partitions that have
|
||||
|
|
|
@ -46,7 +46,7 @@ import org.apache.spark.util.{ThreadUtils, UninterruptibleThread}
|
|||
*/
|
||||
private[kafka010] class KafkaOffsetReader(
|
||||
consumerStrategy: ConsumerStrategy,
|
||||
driverKafkaParams: ju.Map[String, Object],
|
||||
val driverKafkaParams: ju.Map[String, Object],
|
||||
readerOptions: Map[String, String],
|
||||
driverGroupIdPrefix: String) extends Logging {
|
||||
/**
|
||||
|
@ -82,7 +82,9 @@ private[kafka010] class KafkaOffsetReader(
|
|||
assert(Thread.currentThread().isInstanceOf[UninterruptibleThread])
|
||||
if (_consumer == null) {
|
||||
val newKafkaParams = new ju.HashMap[String, Object](driverKafkaParams)
|
||||
newKafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, nextGroupId())
|
||||
if (driverKafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG) == null) {
|
||||
newKafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, nextGroupId())
|
||||
}
|
||||
_consumer = consumerStrategy.createConsumer(newKafkaParams)
|
||||
}
|
||||
_consumer
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.io._
|
|||
import java.nio.charset.StandardCharsets
|
||||
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.kafka.clients.consumer.ConsumerConfig
|
||||
import org.apache.kafka.common.TopicPartition
|
||||
|
||||
import org.apache.spark.SparkContext
|
||||
|
@ -242,7 +243,12 @@ private[kafka010] class KafkaSource(
|
|||
|
||||
val deletedPartitions = fromPartitionOffsets.keySet.diff(untilPartitionOffsets.keySet)
|
||||
if (deletedPartitions.nonEmpty) {
|
||||
reportDataLoss(s"$deletedPartitions are gone. Some data may have been missed")
|
||||
val message = if (kafkaReader.driverKafkaParams.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) {
|
||||
s"$deletedPartitions are gone. ${KafkaSourceProvider.CUSTOM_GROUP_ID_ERROR_MESSAGE}"
|
||||
} else {
|
||||
s"$deletedPartitions are gone. Some data may have been missed."
|
||||
}
|
||||
reportDataLoss(message)
|
||||
}
|
||||
|
||||
// Use the until partitions to calculate offset ranges to ignore partitions that have
|
||||
|
|
|
@ -335,9 +335,11 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
|
|||
// Validate user-specified Kafka options
|
||||
|
||||
if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.GROUP_ID_CONFIG}")) {
|
||||
throw new IllegalArgumentException(
|
||||
s"Kafka option '${ConsumerConfig.GROUP_ID_CONFIG}' is not supported as " +
|
||||
s"user-specified consumer groups are not used to track offsets.")
|
||||
logWarning(CUSTOM_GROUP_ID_ERROR_MESSAGE)
|
||||
if (caseInsensitiveParams.contains(GROUP_ID_PREFIX)) {
|
||||
logWarning("Option 'groupIdPrefix' will be ignored as " +
|
||||
s"option 'kafka.${ConsumerConfig.GROUP_ID_CONFIG}' has been set.")
|
||||
}
|
||||
}
|
||||
|
||||
if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}")) {
|
||||
|
@ -440,6 +442,7 @@ private[kafka010] object KafkaSourceProvider extends Logging {
|
|||
private[kafka010] val ENDING_OFFSETS_OPTION_KEY = "endingoffsets"
|
||||
private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss"
|
||||
private val MIN_PARTITIONS_OPTION_KEY = "minpartitions"
|
||||
private val GROUP_ID_PREFIX = "groupidprefix"
|
||||
|
||||
val TOPIC_OPTION_KEY = "topic"
|
||||
|
||||
|
@ -459,7 +462,16 @@ private[kafka010] object KafkaSourceProvider extends Logging {
|
|||
| source option "failOnDataLoss" to "false".
|
||||
""".stripMargin
|
||||
|
||||
|
||||
val CUSTOM_GROUP_ID_ERROR_MESSAGE =
|
||||
s"""Kafka option 'kafka.${ConsumerConfig.GROUP_ID_CONFIG}' has been set on this query, it is
|
||||
| not recommended to set this option. This option is unsafe to use since multiple concurrent
|
||||
| queries or sources using the same group id will interfere with each other as they are part
|
||||
| of the same consumer group. Restarted queries may also suffer interference from the
|
||||
| previous run having the same group id. The user should have only one query per group id,
|
||||
| and/or set the option 'kafka.session.timeout.ms' to be very small so that the Kafka
|
||||
| consumers from the previous query are marked dead by the Kafka group coordinator before the
|
||||
| restarted query starts running.
|
||||
""".stripMargin
|
||||
|
||||
private val serClassName = classOf[ByteArraySerializer].getName
|
||||
private val deserClassName = classOf[ByteArrayDeserializer].getName
|
||||
|
@ -510,7 +522,7 @@ private[kafka010] object KafkaSourceProvider extends Logging {
|
|||
.set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none")
|
||||
|
||||
// So that consumers in executors do not mess with any existing group id
|
||||
.set(ConsumerConfig.GROUP_ID_CONFIG, s"$uniqueGroupId-executor")
|
||||
.setIfUnset(ConsumerConfig.GROUP_ID_CONFIG, s"$uniqueGroupId-executor")
|
||||
|
||||
// So that consumers in executors does not commit offsets unnecessarily
|
||||
.set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false")
|
||||
|
@ -529,7 +541,7 @@ private[kafka010] object KafkaSourceProvider extends Logging {
|
|||
parameters: Map[String, String],
|
||||
metadataPath: String): String = {
|
||||
val groupIdPrefix = parameters
|
||||
.getOrElse("groupIdPrefix", "spark-kafka-source")
|
||||
.getOrElse(GROUP_ID_PREFIX, "spark-kafka-source")
|
||||
s"${groupIdPrefix}-${UUID.randomUUID}-${metadataPath.hashCode}"
|
||||
}
|
||||
|
||||
|
|
|
@ -629,6 +629,33 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
|
|||
)
|
||||
}
|
||||
|
||||
test("allow group.id override") {
|
||||
// Tests code path KafkaSourceProvider.{sourceSchema(.), createSource(.)}
|
||||
// as well as KafkaOffsetReader.createConsumer(.)
|
||||
val topic = newTopic()
|
||||
testUtils.createTopic(topic, partitions = 3)
|
||||
testUtils.sendMessages(topic, (1 to 10).map(_.toString).toArray, Some(0))
|
||||
testUtils.sendMessages(topic, (11 to 20).map(_.toString).toArray, Some(1))
|
||||
testUtils.sendMessages(topic, (21 to 30).map(_.toString).toArray, Some(2))
|
||||
|
||||
val dsKafka = spark
|
||||
.readStream
|
||||
.format("kafka")
|
||||
.option("kafka.group.id", "id-" + Random.nextInt())
|
||||
.option("kafka.bootstrap.servers", testUtils.brokerAddress)
|
||||
.option("subscribe", topic)
|
||||
.option("startingOffsets", "earliest")
|
||||
.load()
|
||||
.selectExpr("CAST(value AS STRING)")
|
||||
.as[String]
|
||||
.map(_.toInt)
|
||||
|
||||
testStream(dsKafka)(
|
||||
makeSureGetOffsetCalled,
|
||||
CheckAnswer(1 to 30: _*)
|
||||
)
|
||||
}
|
||||
|
||||
test("ensure stream-stream self-join generates only one offset in log and correct metrics") {
|
||||
val topic = newTopic()
|
||||
testUtils.createTopic(topic, partitions = 2)
|
||||
|
@ -1233,7 +1260,6 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest {
|
|||
assert(ex.getMessage.toLowerCase(Locale.ROOT).contains("not supported"))
|
||||
}
|
||||
|
||||
testUnsupportedConfig("kafka.group.id")
|
||||
testUnsupportedConfig("kafka.auto.offset.reset")
|
||||
testUnsupportedConfig("kafka.enable.auto.commit")
|
||||
testUnsupportedConfig("kafka.interceptor.classes")
|
||||
|
|
|
@ -239,6 +239,18 @@ class KafkaRelationSuite extends QueryTest with SharedSQLContext with KafkaTest
|
|||
testBadOptions("subscribePattern" -> "")("pattern to subscribe is empty")
|
||||
}
|
||||
|
||||
test("allow group.id overriding") {
|
||||
// Tests code path KafkaSourceProvider.createRelation(.)
|
||||
val topic = newTopic()
|
||||
testUtils.createTopic(topic, partitions = 3)
|
||||
testUtils.sendMessages(topic, (1 to 10).map(_.toString).toArray, Some(0))
|
||||
testUtils.sendMessages(topic, (11 to 20).map(_.toString).toArray, Some(1))
|
||||
testUtils.sendMessages(topic, (21 to 30).map(_.toString).toArray, Some(2))
|
||||
|
||||
val df = createDF(topic, withOptions = Map("kafka.group.id" -> "custom"))
|
||||
checkAnswer(df, (1 to 30).map(_.toString).toDF())
|
||||
}
|
||||
|
||||
test("read Kafka transactional messages: read_committed") {
|
||||
val topic = newTopic()
|
||||
testUtils.createTopic(topic)
|
||||
|
|
Loading…
Reference in a new issue