redsk commented on a change in pull request #26153: [SPARK-29500][SQL][SS]
Support partition column when writing to Kafka
URL: https://github.com/apache/spark/pull/26153#discussion_r336382437
##########
File path:
external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala
##########
@@ -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") {
Review comment:
My main question was: "what happens if I specify partitions that don't
exist?" And the answer is that Kafka throws an exception with msg: `Topic
$topic not present in metadata after $producerTimeout ms` which is not
particularly clear.
I could have called `producer.partitionsFor(topic)` to check if the
specified partition is in range (and throw a more meaningful exception if not)
but it's an expensive call for each message. I could cache the result but the
number of partitions could change in between calls and I didn't want to
complicate the implementation. Hence the need to see what the standard Kafka
producer does (which is "not much").
And the rationale is: if Kafka implementation changes, this test will fail
and we'll have a chance to provide a more meaningful exception.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]