[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:
Shixiong Zhu 2019-01-14 13:37:24 -08:00
parent 115fecfd84
commit bafc7ac025
No known key found for this signature in database
GPG key ID: ECB1BBEA55295E39
8 changed files with 106 additions and 16 deletions

View file

@ -379,7 +379,25 @@ The following configurations are optional:
<td>string</td> <td>string</td>
<td>spark-kafka-source</td> <td>spark-kafka-source</td>
<td>streaming and batch</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> </tr>
</table> </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: 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 - **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 set the prefix of the automatically generated group.id's via the optional source option `groupIdPrefix`,
is "spark-kafka-source". 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 - **auto.offset.reset**: Set the source option `startingOffsets` to specify
where to start instead. Structured Streaming manages which offsets are consumed internally, rather 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 than rely on the kafka Consumer to do it. This will ensure that no data is missed when new

View file

@ -20,7 +20,7 @@ package org.apache.spark.sql.kafka010
import java.{util => ju} import java.{util => ju}
import java.util.concurrent.TimeoutException 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.kafka.common.TopicPartition
import org.apache.spark.TaskContext import org.apache.spark.TaskContext
@ -167,7 +167,13 @@ class KafkaContinuousScanConfigBuilder(
val deletedPartitions = oldStartPartitionOffsets.keySet.diff(currentPartitionSet) val deletedPartitions = oldStartPartitionOffsets.keySet.diff(currentPartitionSet)
if (deletedPartitions.nonEmpty) { 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 ++ val startOffsets = newPartitionOffsets ++

View file

@ -22,6 +22,7 @@ import java.io._
import java.nio.charset.StandardCharsets import java.nio.charset.StandardCharsets
import org.apache.commons.io.IOUtils import org.apache.commons.io.IOUtils
import org.apache.kafka.clients.consumer.ConsumerConfig
import org.apache.spark.SparkEnv import org.apache.spark.SparkEnv
import org.apache.spark.internal.Logging import org.apache.spark.internal.Logging
@ -122,7 +123,13 @@ private[kafka010] class KafkaMicroBatchReadSupport(
// Find deleted partitions, and report data loss if required // Find deleted partitions, and report data loss if required
val deletedPartitions = startPartitionOffsets.keySet.diff(endPartitionOffsets.keySet) val deletedPartitions = startPartitionOffsets.keySet.diff(endPartitionOffsets.keySet)
if (deletedPartitions.nonEmpty) { 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 // Use the end partitions to calculate offset ranges to ignore partitions that have

View file

@ -46,7 +46,7 @@ import org.apache.spark.util.{ThreadUtils, UninterruptibleThread}
*/ */
private[kafka010] class KafkaOffsetReader( private[kafka010] class KafkaOffsetReader(
consumerStrategy: ConsumerStrategy, consumerStrategy: ConsumerStrategy,
driverKafkaParams: ju.Map[String, Object], val driverKafkaParams: ju.Map[String, Object],
readerOptions: Map[String, String], readerOptions: Map[String, String],
driverGroupIdPrefix: String) extends Logging { driverGroupIdPrefix: String) extends Logging {
/** /**
@ -82,7 +82,9 @@ private[kafka010] class KafkaOffsetReader(
assert(Thread.currentThread().isInstanceOf[UninterruptibleThread]) assert(Thread.currentThread().isInstanceOf[UninterruptibleThread])
if (_consumer == null) { if (_consumer == null) {
val newKafkaParams = new ju.HashMap[String, Object](driverKafkaParams) 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 = consumerStrategy.createConsumer(newKafkaParams)
} }
_consumer _consumer

View file

@ -22,6 +22,7 @@ import java.io._
import java.nio.charset.StandardCharsets import java.nio.charset.StandardCharsets
import org.apache.commons.io.IOUtils import org.apache.commons.io.IOUtils
import org.apache.kafka.clients.consumer.ConsumerConfig
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.spark.SparkContext import org.apache.spark.SparkContext
@ -242,7 +243,12 @@ private[kafka010] class KafkaSource(
val deletedPartitions = fromPartitionOffsets.keySet.diff(untilPartitionOffsets.keySet) val deletedPartitions = fromPartitionOffsets.keySet.diff(untilPartitionOffsets.keySet)
if (deletedPartitions.nonEmpty) { 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 // Use the until partitions to calculate offset ranges to ignore partitions that have

View file

@ -335,9 +335,11 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
// Validate user-specified Kafka options // Validate user-specified Kafka options
if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.GROUP_ID_CONFIG}")) { if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.GROUP_ID_CONFIG}")) {
throw new IllegalArgumentException( logWarning(CUSTOM_GROUP_ID_ERROR_MESSAGE)
s"Kafka option '${ConsumerConfig.GROUP_ID_CONFIG}' is not supported as " + if (caseInsensitiveParams.contains(GROUP_ID_PREFIX)) {
s"user-specified consumer groups are not used to track offsets.") 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}")) { 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[kafka010] val ENDING_OFFSETS_OPTION_KEY = "endingoffsets"
private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss" private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss"
private val MIN_PARTITIONS_OPTION_KEY = "minpartitions" private val MIN_PARTITIONS_OPTION_KEY = "minpartitions"
private val GROUP_ID_PREFIX = "groupidprefix"
val TOPIC_OPTION_KEY = "topic" val TOPIC_OPTION_KEY = "topic"
@ -459,7 +462,16 @@ private[kafka010] object KafkaSourceProvider extends Logging {
| source option "failOnDataLoss" to "false". | source option "failOnDataLoss" to "false".
""".stripMargin """.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 serClassName = classOf[ByteArraySerializer].getName
private val deserClassName = classOf[ByteArrayDeserializer].getName private val deserClassName = classOf[ByteArrayDeserializer].getName
@ -510,7 +522,7 @@ private[kafka010] object KafkaSourceProvider extends Logging {
.set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none")
// So that consumers in executors do not mess with any existing group id // 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 // So that consumers in executors does not commit offsets unnecessarily
.set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false")
@ -529,7 +541,7 @@ private[kafka010] object KafkaSourceProvider extends Logging {
parameters: Map[String, String], parameters: Map[String, String],
metadataPath: String): String = { metadataPath: String): String = {
val groupIdPrefix = parameters val groupIdPrefix = parameters
.getOrElse("groupIdPrefix", "spark-kafka-source") .getOrElse(GROUP_ID_PREFIX, "spark-kafka-source")
s"${groupIdPrefix}-${UUID.randomUUID}-${metadataPath.hashCode}" s"${groupIdPrefix}-${UUID.randomUUID}-${metadataPath.hashCode}"
} }

View file

@ -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") { test("ensure stream-stream self-join generates only one offset in log and correct metrics") {
val topic = newTopic() val topic = newTopic()
testUtils.createTopic(topic, partitions = 2) testUtils.createTopic(topic, partitions = 2)
@ -1233,7 +1260,6 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest {
assert(ex.getMessage.toLowerCase(Locale.ROOT).contains("not supported")) assert(ex.getMessage.toLowerCase(Locale.ROOT).contains("not supported"))
} }
testUnsupportedConfig("kafka.group.id")
testUnsupportedConfig("kafka.auto.offset.reset") testUnsupportedConfig("kafka.auto.offset.reset")
testUnsupportedConfig("kafka.enable.auto.commit") testUnsupportedConfig("kafka.enable.auto.commit")
testUnsupportedConfig("kafka.interceptor.classes") testUnsupportedConfig("kafka.interceptor.classes")

View file

@ -239,6 +239,18 @@ class KafkaRelationSuite extends QueryTest with SharedSQLContext with KafkaTest
testBadOptions("subscribePattern" -> "")("pattern to subscribe is empty") 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") { test("read Kafka transactional messages: read_committed") {
val topic = newTopic() val topic = newTopic()
testUtils.createTopic(topic) testUtils.createTopic(topic)