-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-18682][SS] Batch Source for Kafka #16686
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 1 commit
d371758
b6c3055
4c81812
ab02a4c
e6b57ed
ff94ed8
f8fd34c
41271e2
74d96fc
d31fc81
1db1649
3b0d48b
a5b0269
c08c01f
79d335e
b597cf1
2487a72
789d3af
5b48fc6
5776009
aef89bc
4e56f8c
3bc7c4c
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -42,6 +42,11 @@ private[kafka010] trait KafkaOffsetReader { | |
| */ | ||
| def close() | ||
|
|
||
| /** | ||
| * @return The Set of TopicPartitions for a given topic | ||
| */ | ||
| def fetchTopicPartitions(): Set[TopicPartition] | ||
|
|
||
| /** | ||
| * Set consumer position to specified offsets, making sure all assignments are set. | ||
|
||
| */ | ||
|
|
@@ -108,7 +113,14 @@ private[kafka010] class KafkaOffsetReaderImpl( | |
|
|
||
| def close(): Unit = consumer.close() | ||
|
|
||
| def fetchSpecificStartingOffsets( | ||
| override def fetchTopicPartitions(): Set[TopicPartition] = { | ||
| assert(Thread.currentThread().isInstanceOf[UninterruptibleThread]) | ||
| // Poll to get the latest assigned partitions | ||
| consumer.poll(0) | ||
| consumer.assignment().asScala.toSet | ||
|
||
| } | ||
|
|
||
| override def fetchSpecificStartingOffsets( | ||
| partitionOffsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] = | ||
| withRetriesWithoutInterrupt { | ||
| // Poll to get the latest assigned partitions | ||
|
|
@@ -131,7 +143,7 @@ private[kafka010] class KafkaOffsetReaderImpl( | |
| } | ||
| } | ||
|
|
||
| def fetchEarliestOffsets(): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { | ||
| override def fetchEarliestOffsets(): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { | ||
| // Poll to get the latest assigned partitions | ||
| consumer.poll(0) | ||
| val partitions = consumer.assignment() | ||
|
|
@@ -144,7 +156,7 @@ private[kafka010] class KafkaOffsetReaderImpl( | |
| partitionOffsets | ||
| } | ||
|
|
||
| def fetchLatestOffsets(): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { | ||
| override def fetchLatestOffsets(): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { | ||
| // Poll to get the latest assigned partitions | ||
| consumer.poll(0) | ||
| val partitions = consumer.assignment() | ||
|
|
@@ -157,7 +169,7 @@ private[kafka010] class KafkaOffsetReaderImpl( | |
| partitionOffsets | ||
| } | ||
|
|
||
| def fetchNewPartitionEarliestOffsets( | ||
| override def fetchNewPartitionEarliestOffsets( | ||
| newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = { | ||
| if (newPartitions.isEmpty) { | ||
| Map.empty[TopicPartition, Long] | ||
|
|
@@ -283,6 +295,13 @@ private[kafka010] class UninterruptibleKafkaOffsetReader(kafkaOffsetReader: Kafk | |
| kafkaReaderThread.shutdownNow() | ||
| } | ||
|
|
||
| override def fetchTopicPartitions(): Set[TopicPartition] = { | ||
| val future = Future { | ||
| kafkaOffsetReader.fetchTopicPartitions() | ||
| }(execContext) | ||
| ThreadUtils.awaitResult(future, Duration.Inf) | ||
| } | ||
|
|
||
| override def fetchSpecificStartingOffsets( | ||
| partitionOffsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] = { | ||
| val future = Future { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -23,7 +23,7 @@ import org.apache.kafka.common.TopicPartition | |
| import org.scalatest.BeforeAndAfter | ||
|
|
||
| import org.apache.spark.sql.QueryTest | ||
| import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession} | ||
| import org.apache.spark.sql.test.SharedSQLContext | ||
|
|
||
| class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLContext { | ||
|
|
||
|
|
@@ -61,20 +61,19 @@ class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLCon | |
| testUtils.sendMessages(topic, Array("20"), Some(2)) | ||
|
|
||
| // Specify explicit earliest and latest offset values | ||
| val reader = spark | ||
| val df = spark | ||
| .read | ||
| .format("kafka") | ||
| .option("kafka.bootstrap.servers", testUtils.brokerAddress) | ||
| .option("subscribe", topic) | ||
| .option("startingOffsets", "earliest") | ||
| .option("endingOffsets", "latest") | ||
| .load() | ||
| var df = reader.selectExpr("CAST(value AS STRING)") | ||
| .selectExpr("CAST(value AS STRING)") | ||
| checkAnswer(df, (0 to 20).map(_.toString).toDF) | ||
|
|
||
| // "latest" should late bind to the current (latest) offset in the reader | ||
| // "latest" should late bind to the current (latest) offset in the df | ||
| testUtils.sendMessages(topic, (21 to 29).map(_.toString).toArray, Some(2)) | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: could you add the following test below this line to make the semantics clear?
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This no longer holds now that we're binding in the executor, right? |
||
| df = reader.selectExpr("CAST(value AS STRING)") | ||
| checkAnswer(df, (0 to 29).map(_.toString).toDF) | ||
| } | ||
|
|
||
|
|
@@ -118,25 +117,23 @@ class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLCon | |
| new TopicPartition(topic, 2) -> 1L // explicit offset happens to = the latest | ||
| ) | ||
| val endingOffsets = JsonUtils.partitionOffsets(endPartitionOffsets) | ||
| val reader = spark | ||
| val df = spark | ||
|
||
| .read | ||
| .format("kafka") | ||
| .option("kafka.bootstrap.servers", testUtils.brokerAddress) | ||
| .option("subscribe", topic) | ||
| .option("startingOffsets", startingOffsets) | ||
| .option("endingOffsets", endingOffsets) | ||
| .load() | ||
| var df = reader.selectExpr("CAST(value AS STRING)") | ||
| .selectExpr("CAST(value as STRING)") | ||
| checkAnswer(df, (0 to 20).map(_.toString).toDF) | ||
|
|
||
| // static offset partition 2, nothing should change | ||
| testUtils.sendMessages(topic, (31 to 39).map(_.toString).toArray, Some(2)) | ||
| df = reader.selectExpr("CAST(value AS STRING)") | ||
| checkAnswer(df, (0 to 20).map(_.toString).toDF) | ||
|
|
||
| // latest offset partition 1, should change | ||
| testUtils.sendMessages(topic, (21 to 30).map(_.toString).toArray, Some(1)) | ||
| df = reader.selectExpr("CAST(value AS STRING)") | ||
| checkAnswer(df, (0 to 30).map(_.toString).toDF) | ||
| } | ||
|
|
||
|
|
@@ -147,18 +144,64 @@ class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLCon | |
| testUtils.sendMessages(topic, (0 to 10).map(_.toString).toArray, Some(0)) | ||
|
|
||
| // Specify explicit earliest and latest offset values | ||
| val reader = spark | ||
| val df = spark | ||
| .read | ||
| .format("kafka") | ||
| .option("kafka.bootstrap.servers", testUtils.brokerAddress) | ||
| .option("subscribe", topic) | ||
| .option("startingOffsets", "earliest") | ||
| .option("endingOffsets", "latest") | ||
| .load() | ||
| var df = reader.selectExpr("CAST(value AS STRING)") | ||
| .selectExpr("CAST(value AS STRING)") | ||
| checkAnswer(df.union(df), ((0 to 10) ++ (0 to 10)).map(_.toString).toDF) | ||
| } | ||
|
|
||
| test("test late binding start offsets") { | ||
| try { | ||
| // First, establish a new KafkaUtils instance that will clear | ||
| // all messages when cleanupLogs is called. | ||
| if (testUtils != null) { | ||
| testUtils.teardown() | ||
| testUtils = null | ||
| } | ||
| // The following settings will ensure that all log entries | ||
| // are removed following a call to cleanupLogs | ||
| val brokerProps = Map[String, Object]( | ||
| "log.retention.bytes" -> 1.asInstanceOf[AnyRef], // retain nothing | ||
| "log.retention.ms" -> 1.asInstanceOf[AnyRef] // no wait time | ||
| ) | ||
| testUtils = new KafkaTestUtils(withBrokerProps = Some(brokerProps)) | ||
| testUtils.setup() | ||
|
|
||
| val topic = newTopic() | ||
| testUtils.createTopic(topic, partitions = 1) | ||
| testUtils.sendMessages(topic, (0 to 9).map(_.toString).toArray, Some(0)) | ||
| // Specify explicit earliest and latest offset values | ||
| val df = spark | ||
| .read | ||
| .format("kafka") | ||
| .option("kafka.bootstrap.servers", testUtils.brokerAddress) | ||
| .option("subscribe", topic) | ||
| .option("startingOffsets", "earliest") | ||
| .option("endingOffsets", "latest") | ||
| .load() | ||
| .selectExpr("CAST(value AS STRING)") | ||
| checkAnswer(df, (0 to 9).map(_.toString).toDF) | ||
| // Blow away current set of messages. | ||
| testUtils.cleanupLogs() | ||
| // Add some more data, but do not call cleanup | ||
| testUtils.sendMessages(topic, (10 to 19).map(_.toString).toArray, Some(0)) | ||
| // Ensure that we late bind to the new starting position | ||
| checkAnswer(df, (10 to 19).map(_.toString).toDF) | ||
| } finally { | ||
| if (testUtils != null) { | ||
| testUtils.teardown() | ||
| } | ||
| testUtils = new KafkaTestUtils | ||
| testUtils.setup() | ||
| } | ||
| } | ||
|
|
||
| test("bad source options") { | ||
| def testBadOptions(options: (String, String)*)(expectedMsgs: String*): Unit = { | ||
| val ex = intercept[IllegalArgumentException] { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -50,7 +50,7 @@ import org.apache.spark.SparkConf | |
| * | ||
| * The reason to put Kafka test utility class in src is to test Python related Kafka APIs. | ||
| */ | ||
| class KafkaTestUtils extends Logging { | ||
| class KafkaTestUtils(withBrokerProps: Option[Map[String, Object]] = None) extends Logging { | ||
|
|
||
| // Zookeeper related configurations | ||
| private val zkHost = "localhost" | ||
|
|
@@ -249,6 +249,24 @@ class KafkaTestUtils extends Logging { | |
| offsets | ||
| } | ||
|
|
||
| def cleanupLogs(): Unit = { | ||
| server.logManager.cleanupLogs() | ||
| } | ||
|
|
||
| def getEarliestOffsets(topics: Set[String]): Map[TopicPartition, Long] = { | ||
| val kc = new KafkaConsumer[String, String](consumerConfiguration) | ||
| logInfo("Created consumer to get latest offsets") | ||
|
||
| kc.subscribe(topics.asJavaCollection) | ||
| kc.poll(0) | ||
| val partitions = kc.assignment() | ||
| kc.pause(partitions) | ||
| kc.seekToBeginning(partitions) | ||
| val offsets = partitions.asScala.map(p => p -> kc.position(p)).toMap | ||
| kc.close() | ||
| logInfo("Closed consumer to get latest offsets") | ||
|
||
| offsets | ||
| } | ||
|
|
||
| def getLatestOffsets(topics: Set[String]): Map[TopicPartition, Long] = { | ||
| val kc = new KafkaConsumer[String, String](consumerConfiguration) | ||
| logInfo("Created consumer to get latest offsets") | ||
|
|
@@ -274,6 +292,11 @@ class KafkaTestUtils extends Logging { | |
| props.put("log.flush.interval.messages", "1") | ||
| props.put("replica.socket.timeout.ms", "1500") | ||
| props.put("delete.topic.enable", "true") | ||
| withBrokerProps.map { p => | ||
|
||
| p.foreach { | ||
| case (key, value) => props.put(key, value) | ||
| } | ||
| } | ||
| props | ||
| } | ||
|
|
||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
exposing internal var is generally not a good idea. A better approach is be to add the necessary methods (for which you need the consumer) in the class CachedKafkaConsumer.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
and why renamed?