-
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 3 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 |
|---|---|---|
|
|
@@ -614,6 +614,10 @@ The Dataframe being written to Kafka should have the following columns in schema | |
| <td>topic (*optional)</td> | ||
| <td>string</td> | ||
| </tr> | ||
| <tr> | ||
| <td>partition (optional)</td> | ||
| <td>int</td> | ||
| </tr> | ||
| </table> | ||
| \* The topic column is required if the "topic" configuration option is not specified.<br> | ||
|
|
||
|
|
@@ -622,6 +626,10 @@ a ```null``` valued key column will be automatically added (see Kafka semantics | |
| 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 then the partition is calculated by the Kafka producer | ||
| (using ```org.apache.kafka.clients.producer.internals.DefaultPartitioner```). | ||
| This can be overridden in Spark by setting the ```kafka.partitioner.class``` option. | ||
|
||
|
|
||
|
|
||
| The following options must be set for the Kafka sink | ||
| for both batch and streaming queries. | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -27,7 +27,7 @@ import org.apache.kafka.common.header.internals.RecordHeader | |
|
|
||
| import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} | ||
| import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal, UnsafeProjection} | ||
| import org.apache.spark.sql.types.{BinaryType, StringType} | ||
| import org.apache.spark.sql.types.{BinaryType, IntegerType, StringType} | ||
|
|
||
| /** | ||
| * Writes out data in a single Spark task, without any concerns about how | ||
|
|
@@ -92,16 +92,19 @@ private[kafka010] abstract class KafkaRowWriter( | |
| throw new NullPointerException(s"null topic present in the data. Use the " + | ||
| s"${KafkaSourceProvider.TOPIC_OPTION_KEY} option for setting a default topic.") | ||
| } | ||
| val partition: java.lang.Integer = | ||
|
||
| if (projectedRow.isNullAt(4)) null else projectedRow.getInt(4) | ||
| val record = if (projectedRow.isNullAt(3)) { | ||
| new ProducerRecord[Array[Byte], Array[Byte]](topic.toString, null, key, value) | ||
| new ProducerRecord[Array[Byte], Array[Byte]](topic.toString, partition, key, value) | ||
| } else { | ||
| val headerArray = projectedRow.getArray(3) | ||
| val headers = (0 until headerArray.numElements()).map { i => | ||
| val struct = headerArray.getStruct(i, 2) | ||
| new RecordHeader(struct.getUTF8String(0).toString, struct.getBinary(1)) | ||
| .asInstanceOf[Header] | ||
| } | ||
| new ProducerRecord[Array[Byte], Array[Byte]](topic.toString, null, key, value, headers.asJava) | ||
| new ProducerRecord[Array[Byte], Array[Byte]]( | ||
| topic.toString, partition, key, value, headers.asJava) | ||
| } | ||
| producer.send(record, callback) | ||
| } | ||
|
|
@@ -156,12 +159,23 @@ private[kafka010] abstract class KafkaRowWriter( | |
| throw new IllegalStateException(s"${KafkaWriter.HEADERS_ATTRIBUTE_NAME} " + | ||
| s"attribute unsupported type ${t.catalogString}") | ||
| } | ||
| val partitionExpression = | ||
| inputSchema.find(_.name == KafkaWriter.PARTITION_ATTRIBUTE_NAME) | ||
| .getOrElse(Literal(null, IntegerType)) | ||
| partitionExpression.dataType match { | ||
| case IntegerType => // good | ||
| case t => | ||
| throw new IllegalStateException(s"${KafkaWriter.PARTITION_ATTRIBUTE_NAME} " + | ||
| s"attribute unsupported type $t. ${KafkaWriter.PARTITION_ATTRIBUTE_NAME} " + | ||
| s"must be a ${IntegerType.catalogString}") | ||
| } | ||
| UnsafeProjection.create( | ||
| Seq( | ||
| topicExpression, | ||
| Cast(keyExpression, BinaryType), | ||
| Cast(valueExpression, BinaryType), | ||
| headersExpression | ||
| headersExpression, | ||
| partitionExpression | ||
| ), | ||
| inputSchema | ||
| ) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -33,7 +33,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 +293,21 @@ class KafkaSinkStreamingSuite extends KafkaSinkSuiteBase with StreamTest { | |
| } | ||
| assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( | ||
| "key attribute type must be a string or binary")) | ||
|
|
||
| try { | ||
|
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 know this copy-paste started earlier but it's a good opportunity to reduce it.
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. Actually I just submitted a patch #26158 to handle refactor to old ones: depending on which one to merge first, we can apply it.
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. OK, started to review it... |
||
| 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") { | ||
|
|
@@ -369,31 +384,32 @@ abstract class KafkaSinkBatchSuiteBase extends KafkaSinkSuiteBase { | |
|
|
||
| test("batch - write to kafka") { | ||
| val topic = newTopic() | ||
| testUtils.createTopic(topic) | ||
| testUtils.createTopic(topic, 4) | ||
| val data = Seq( | ||
| Row(topic, "1", Seq( | ||
| Row("a", "b".getBytes(UTF_8)) | ||
| )), | ||
| ), 0), | ||
| Row(topic, "2", Seq( | ||
| Row("c", "d".getBytes(UTF_8)), | ||
| Row("e", "f".getBytes(UTF_8)) | ||
| )), | ||
| ), 1), | ||
| Row(topic, "3", Seq( | ||
| Row("g", "h".getBytes(UTF_8)), | ||
| Row("g", "i".getBytes(UTF_8)) | ||
| )), | ||
| Row(topic, "4", null), | ||
| ), 2), | ||
| Row(topic, "4", null, 3), | ||
| Row(topic, "5", Seq( | ||
| Row("j", "k".getBytes(UTF_8)), | ||
| Row("j", "l".getBytes(UTF_8)), | ||
| Row("m", "n".getBytes(UTF_8)) | ||
| )) | ||
| ), 0) | ||
| ) | ||
|
|
||
| val df = spark.createDataFrame( | ||
| spark.sparkContext.parallelize(data), | ||
| StructType(Seq(StructField("topic", StringType), StructField("value", StringType), | ||
| StructField("headers", KafkaRecordToRowConverter.headersType))) | ||
| StructField("headers", KafkaRecordToRowConverter.headersType), | ||
| StructField("partition", IntegerType))) | ||
| ) | ||
|
|
||
| df.write | ||
|
|
@@ -404,20 +420,85 @@ abstract class KafkaSinkBatchSuiteBase extends KafkaSinkSuiteBase { | |
| .save() | ||
| checkAnswer( | ||
| createKafkaReader(topic, includeHeaders = true).selectExpr( | ||
| "CAST(value as STRING) value", "headers" | ||
| "CAST(value as STRING) value", "headers", "partition" | ||
| ), | ||
| Row("1", Seq(Row("a", "b".getBytes(UTF_8)))) :: | ||
| Row("2", Seq(Row("c", "d".getBytes(UTF_8)), Row("e", "f".getBytes(UTF_8)))) :: | ||
| Row("3", Seq(Row("g", "h".getBytes(UTF_8)), Row("g", "i".getBytes(UTF_8)))) :: | ||
| Row("4", null) :: | ||
| Row("1", Seq(Row("a", "b".getBytes(UTF_8))), 0) :: | ||
| Row("2", Seq(Row("c", "d".getBytes(UTF_8)), Row("e", "f".getBytes(UTF_8))), 1) :: | ||
| Row("3", Seq(Row("g", "h".getBytes(UTF_8)), Row("g", "i".getBytes(UTF_8))), 2) :: | ||
| Row("4", null, 3) :: | ||
| Row("5", Seq( | ||
| Row("j", "k".getBytes(UTF_8)), | ||
| Row("j", "l".getBytes(UTF_8)), | ||
| Row("m", "n".getBytes(UTF_8)))) :: | ||
| Row("m", "n".getBytes(UTF_8))), 0) :: | ||
| Nil | ||
| ) | ||
| } | ||
|
|
||
| test("batch - partition column vs default Kafka partitioner") { | ||
|
||
| val fixedKey = "fixed_key" | ||
| val nrPartitions = 100 | ||
|
||
|
|
||
| // default Kafka partitioner calculate partition deterministically based on the key | ||
| val keyTopic = newTopic() | ||
| testUtils.createTopic(keyTopic, nrPartitions) | ||
|
|
||
| Seq((keyTopic, fixedKey, "value")) | ||
| .toDF("topic", "key", "value") | ||
| .write | ||
| .format("kafka") | ||
| .option("kafka.bootstrap.servers", testUtils.brokerAddress) | ||
| .option("topic", keyTopic) | ||
| .mode("append") | ||
| .save() | ||
|
|
||
| // getting the partition corresponding to the fixed key | ||
| val keyPartition = createKafkaReader(keyTopic).select("partition") | ||
| .map(_.getInt(0)).collect().toList.head | ||
|
|
||
| val topic = newTopic() | ||
| testUtils.createTopic(topic, nrPartitions) | ||
|
|
||
| // even values use default kafka partitioner, odd use 'n' | ||
| val df = (0 until 100) | ||
| .map(n => (topic, fixedKey, s"$n", if (n % 2 == 0) None else Some(n))) | ||
| .toDF("topic", "key", "value", "partition") | ||
|
|
||
| df.write | ||
| .format("kafka") | ||
| .option("kafka.bootstrap.servers", testUtils.brokerAddress) | ||
| .option("topic", topic) | ||
| .mode("append") | ||
| .save() | ||
|
|
||
| checkAnswer( | ||
| createKafkaReader(topic).selectExpr( | ||
| "CAST(key as STRING) key", "CAST(value as STRING) value", "partition" | ||
| ), | ||
| (0 until 100) | ||
| .map(n => (fixedKey, s"$n", if (n % 2 == 0) keyPartition else n)) | ||
| .toDF("key", "value", "partition") | ||
| ) | ||
| } | ||
|
|
||
| test("batch - non-existing partitions trigger standard Kafka exception") { | ||
|
||
| val topic = newTopic() | ||
| val producerTimeout = "1000" | ||
| testUtils.createTopic(topic, 4) | ||
| val df = Seq((topic, "1", 5)).toDF("topic", "value", "partition") | ||
|
|
||
| val ex = intercept[SparkException] { | ||
| df.write | ||
| .format("kafka") | ||
| .option("kafka.bootstrap.servers", testUtils.brokerAddress) | ||
| .option("kafka.max.block.ms", producerTimeout) // default (60000 ms) would be too slow | ||
| .option("topic", topic) | ||
| .mode("append") | ||
| .save() | ||
| } | ||
| TestUtils.assertExceptionMsg( | ||
| ex, s"Topic $topic not present in metadata after $producerTimeout ms") | ||
| } | ||
|
|
||
| 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] { | ||
|
|
||
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.
It is better to avoid mentioning default of Kafka directly, as Kafka could change it and the information becomes stale then. I feel that's OK to just mention it will follow the configuration of Kafka.
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.
+1 on not mentioning the Kafka internal class, just saying default partitioner.
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.
Sure, will do.