-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-29500][SQL][SS] Support partition column when writing to Kafka #26153
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 all commits
0ac0640
1172d79
5b64872
10308ae
021d2b7
57f7863
96a8e9f
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 |
|---|---|---|
|
|
@@ -22,6 +22,8 @@ import java.util.Locale | |
| import java.util.concurrent.atomic.AtomicInteger | ||
|
|
||
| import org.apache.kafka.clients.producer.ProducerConfig | ||
| import org.apache.kafka.clients.producer.internals.DefaultPartitioner | ||
| import org.apache.kafka.common.Cluster | ||
| import org.apache.kafka.common.serialization.ByteArraySerializer | ||
| import org.scalatest.time.SpanSugar._ | ||
|
|
||
|
|
@@ -33,7 +35,7 @@ import org.apache.spark.sql.functions._ | |
| import org.apache.spark.sql.internal.SQLConf | ||
| import org.apache.spark.sql.streaming._ | ||
| import org.apache.spark.sql.test.SharedSparkSession | ||
| import org.apache.spark.sql.types.{BinaryType, DataType, StringType, StructField, StructType} | ||
| import org.apache.spark.sql.types.{BinaryType, DataType, IntegerType, StringType, StructField, StructType} | ||
|
|
||
| abstract class KafkaSinkSuiteBase extends QueryTest with SharedSparkSession with KafkaTest { | ||
| protected var testUtils: KafkaTestUtils = _ | ||
|
|
@@ -293,6 +295,21 @@ class KafkaSinkStreamingSuite extends KafkaSinkSuiteBase with StreamTest { | |
| } | ||
| assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( | ||
| "key attribute type must be a string or binary")) | ||
|
|
||
| try { | ||
| ex = intercept[StreamingQueryException] { | ||
| /* partition field wrong type */ | ||
| writer = createKafkaWriter(input.toDF())( | ||
| withSelectExpr = s"'$topic' as topic", "value", "value as partition" | ||
| ) | ||
| input.addData("1", "2", "3", "4", "5") | ||
| writer.processAllAvailable() | ||
| } | ||
| } finally { | ||
| writer.stop() | ||
| } | ||
| assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( | ||
| "partition attribute type must be an int")) | ||
| } | ||
|
|
||
| test("streaming - write to non-existing topic") { | ||
|
|
@@ -418,6 +435,65 @@ abstract class KafkaSinkBatchSuiteBase extends KafkaSinkSuiteBase { | |
| ) | ||
| } | ||
|
|
||
| def writeToKafka(df: DataFrame, topic: String, options: Map[String, String] = Map.empty): Unit = { | ||
| df | ||
| .write | ||
| .format("kafka") | ||
| .option("kafka.bootstrap.servers", testUtils.brokerAddress) | ||
| .option("topic", topic) | ||
| .options(options) | ||
| .mode("append") | ||
| .save() | ||
| } | ||
|
|
||
| def partitionsInTopic(topic: String): Set[Int] = { | ||
| createKafkaReader(topic) | ||
| .select("partition") | ||
| .map(_.getInt(0)) | ||
| .collect() | ||
| .toSet | ||
| } | ||
|
|
||
| test("batch - partition column and partitioner priorities") { | ||
|
Contributor
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. I was thinking a bit different test split but this is also fine. It covers all the priority scenarios, good job. In this case I agree with your view about not having
Contributor
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. Looks like this test covers everything. Great. |
||
| val nrPartitions = 4 | ||
| val topic1 = newTopic() | ||
| val topic2 = newTopic() | ||
| val topic3 = newTopic() | ||
| val topic4 = newTopic() | ||
| testUtils.createTopic(topic1, nrPartitions) | ||
| testUtils.createTopic(topic2, nrPartitions) | ||
| testUtils.createTopic(topic3, nrPartitions) | ||
| testUtils.createTopic(topic4, nrPartitions) | ||
| val customKafkaPartitionerConf = Map( | ||
| "kafka.partitioner.class" -> "org.apache.spark.sql.kafka010.TestKafkaPartitioner" | ||
| ) | ||
|
|
||
| val df = (0 until 5).map(n => (topic1, s"$n", s"$n")).toDF("topic", "key", "value") | ||
|
|
||
| // default kafka partitioner | ||
| writeToKafka(df, topic1) | ||
| val partitionsInTopic1 = partitionsInTopic(topic1) | ||
| assert(partitionsInTopic1.size > 1) | ||
|
|
||
| // custom partitioner (always returns 0) overrides default partitioner | ||
| writeToKafka(df, topic2, customKafkaPartitionerConf) | ||
| val partitionsInTopic2 = partitionsInTopic(topic2) | ||
| assert(partitionsInTopic2.size == 1) | ||
| assert(partitionsInTopic2.head == 0) | ||
|
|
||
| // partition column overrides custom partitioner | ||
| val dfWithCustomPartition = df.withColumn("partition", lit(2)) | ||
| writeToKafka(dfWithCustomPartition, topic3, customKafkaPartitionerConf) | ||
| val partitionsInTopic3 = partitionsInTopic(topic3) | ||
| assert(partitionsInTopic3.size == 1) | ||
| assert(partitionsInTopic3.head == 2) | ||
|
|
||
| // when the partition column value is null, it is ignored | ||
| val dfWithNullPartitions = df.withColumn("partition", lit(null).cast(IntegerType)) | ||
| writeToKafka(dfWithNullPartitions, topic4) | ||
| assert(partitionsInTopic(topic4) == partitionsInTopic1) | ||
| } | ||
|
|
||
| test("batch - null topic field value, and no topic option") { | ||
| val df = Seq[(String, String)](null.asInstanceOf[String] -> "1").toDF("topic", "value") | ||
| val ex = intercept[SparkException] { | ||
|
|
@@ -515,3 +591,13 @@ class KafkaSinkBatchSuiteV2 extends KafkaSinkBatchSuiteBase { | |
| } | ||
| } | ||
| } | ||
|
|
||
| class TestKafkaPartitioner extends DefaultPartitioner { | ||
| override def partition( | ||
| topic: String, | ||
| key: Any, | ||
| keyBytes: Array[Byte], | ||
| value: Any, | ||
| valueBytes: Array[Byte], | ||
| cluster: Cluster): Int = 0 | ||
| } | ||
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.
I know this copy-paste started earlier but it's a good opportunity to reduce it.
As I see only the select expression and the message is changing here, right?
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.
Actually I just submitted a patch #26158 to handle refactor to old ones: depending on which one to merge first, we can apply it.
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.
OK, started to review it...