-
Notifications
You must be signed in to change notification settings - Fork 28.6k
[SPARK-23539][SS] Add support for Kafka headers in Structured Streaming #22282
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
2935dae
0ce84b3
f580431
2330679
9949c4e
1d926c4
e458a60
de02de4
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 |
---|---|---|
|
@@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.InternalRow | |
import org.apache.spark.sql.catalyst.expressions.UnsafeRow | ||
import org.apache.spark.sql.connector.read.InputPartition | ||
import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReader, ContinuousPartitionReaderFactory, ContinuousStream, Offset, PartitionOffset} | ||
import org.apache.spark.sql.kafka010.KafkaSourceProvider.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE} | ||
import org.apache.spark.sql.kafka010.KafkaSourceProvider._ | ||
import org.apache.spark.sql.util.CaseInsensitiveStringMap | ||
|
||
/** | ||
|
@@ -56,6 +56,7 @@ class KafkaContinuousStream( | |
|
||
private[kafka010] val pollTimeoutMs = | ||
options.getLong(KafkaSourceProvider.CONSUMER_POLL_TIMEOUT, 512) | ||
private val includeHeaders = options.getBoolean(INCLUDE_HEADERS, false) | ||
|
||
// Initialized when creating reader factories. If this diverges from the partitions at the latest | ||
// offsets, we need to reconfigure. | ||
|
@@ -88,7 +89,7 @@ class KafkaContinuousStream( | |
if (deletedPartitions.nonEmpty) { | ||
val message = if ( | ||
offsetReader.driverKafkaParams.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) { | ||
s"$deletedPartitions are gone. ${KafkaSourceProvider.CUSTOM_GROUP_ID_ERROR_MESSAGE}" | ||
s"$deletedPartitions are gone. ${CUSTOM_GROUP_ID_ERROR_MESSAGE}" | ||
} else { | ||
s"$deletedPartitions are gone. Some data may have been missed." | ||
} | ||
|
@@ -102,7 +103,7 @@ class KafkaContinuousStream( | |
startOffsets.toSeq.map { | ||
case (topicPartition, start) => | ||
KafkaContinuousInputPartition( | ||
topicPartition, start, kafkaParams, pollTimeoutMs, failOnDataLoss) | ||
topicPartition, start, kafkaParams, pollTimeoutMs, failOnDataLoss, includeHeaders) | ||
}.toArray | ||
} | ||
|
||
|
@@ -153,19 +154,22 @@ class KafkaContinuousStream( | |
* @param pollTimeoutMs The timeout for Kafka consumer polling. | ||
* @param failOnDataLoss Flag indicating whether data reader should fail if some offsets | ||
* are skipped. | ||
* @param includeHeaders Flag indicating whether to include Kafka records' headers. | ||
*/ | ||
case class KafkaContinuousInputPartition( | ||
topicPartition: TopicPartition, | ||
startOffset: Long, | ||
kafkaParams: ju.Map[String, Object], | ||
pollTimeoutMs: Long, | ||
failOnDataLoss: Boolean) extends InputPartition | ||
topicPartition: TopicPartition, | ||
startOffset: Long, | ||
kafkaParams: ju.Map[String, Object], | ||
pollTimeoutMs: Long, | ||
failOnDataLoss: Boolean, | ||
includeHeaders: Boolean) extends InputPartition | ||
|
||
object KafkaContinuousReaderFactory extends ContinuousPartitionReaderFactory { | ||
override def createReader(partition: InputPartition): ContinuousPartitionReader[InternalRow] = { | ||
val p = partition.asInstanceOf[KafkaContinuousInputPartition] | ||
new KafkaContinuousPartitionReader( | ||
p.topicPartition, p.startOffset, p.kafkaParams, p.pollTimeoutMs, p.failOnDataLoss) | ||
p.topicPartition, p.startOffset, p.kafkaParams, p.pollTimeoutMs, | ||
p.failOnDataLoss, p.includeHeaders) | ||
} | ||
} | ||
|
||
|
@@ -184,9 +188,11 @@ class KafkaContinuousPartitionReader( | |
startOffset: Long, | ||
kafkaParams: ju.Map[String, Object], | ||
pollTimeoutMs: Long, | ||
failOnDataLoss: Boolean) extends ContinuousPartitionReader[InternalRow] { | ||
failOnDataLoss: Boolean, | ||
includeHeaders: Boolean) extends ContinuousPartitionReader[InternalRow] { | ||
private val consumer = KafkaDataConsumer.acquire(topicPartition, kafkaParams) | ||
private val converter = new KafkaRecordToUnsafeRowConverter | ||
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'd have 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. So +1 on your proposal. The proposed name is just 2 cents, and I'm not sure which name fits best. 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. @HeartSaVioR Great. Let's continue on discussing which name would be the best. |
||
private val unsafeRowProjector = new KafkaRecordToRowConverter() | ||
.toUnsafeRowProjector(includeHeaders) | ||
|
||
private var nextKafkaOffset = startOffset | ||
private var currentRecord: ConsumerRecord[Array[Byte], Array[Byte]] = _ | ||
|
@@ -225,7 +231,7 @@ class KafkaContinuousPartitionReader( | |
} | ||
|
||
override def get(): UnsafeRow = { | ||
converter.toUnsafeRow(currentRecord) | ||
unsafeRowProjector(currentRecord) | ||
} | ||
|
||
override def getOffset(): KafkaSourcePartitionOffset = { | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,93 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.spark.sql.kafka010 | ||
|
||
import java.sql.Timestamp | ||
|
||
import scala.collection.JavaConverters._ | ||
|
||
import org.apache.kafka.clients.consumer.ConsumerRecord | ||
|
||
import org.apache.spark.sql.catalyst.InternalRow | ||
import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow} | ||
import org.apache.spark.sql.catalyst.util.{DateTimeUtils, GenericArrayData} | ||
import org.apache.spark.sql.types._ | ||
import org.apache.spark.unsafe.types.UTF8String | ||
|
||
/** A simple class for converting Kafka ConsumerRecord to InternalRow/UnsafeRow */ | ||
private[kafka010] class KafkaRecordToRowConverter { | ||
import KafkaRecordToRowConverter._ | ||
|
||
private val toUnsafeRowWithoutHeaders = UnsafeProjection.create(schemaWithoutHeaders) | ||
private val toUnsafeRowWithHeaders = UnsafeProjection.create(schemaWithHeaders) | ||
|
||
val toInternalRowWithoutHeaders: Record => InternalRow = | ||
(cr: Record) => InternalRow( | ||
cr.key, cr.value, UTF8String.fromString(cr.topic), cr.partition, cr.offset, | ||
DateTimeUtils.fromJavaTimestamp(new Timestamp(cr.timestamp)), cr.timestampType.id | ||
) | ||
|
||
val toInternalRowWithHeaders: Record => InternalRow = | ||
(cr: Record) => InternalRow( | ||
cr.key, cr.value, UTF8String.fromString(cr.topic), cr.partition, cr.offset, | ||
DateTimeUtils.fromJavaTimestamp(new Timestamp(cr.timestamp)), cr.timestampType.id, | ||
if (cr.headers.iterator().hasNext) { | ||
new GenericArrayData(cr.headers.iterator().asScala | ||
.map(header => | ||
InternalRow(UTF8String.fromString(header.key()), header.value()) | ||
).toArray) | ||
} else { | ||
null | ||
} | ||
) | ||
|
||
def toUnsafeRowWithoutHeadersProjector: Record => UnsafeRow = | ||
(cr: Record) => toUnsafeRowWithoutHeaders(toInternalRowWithoutHeaders(cr)) | ||
|
||
def toUnsafeRowWithHeadersProjector: Record => UnsafeRow = | ||
(cr: Record) => toUnsafeRowWithHeaders(toInternalRowWithHeaders(cr)) | ||
|
||
def toUnsafeRowProjector(includeHeaders: Boolean): Record => UnsafeRow = { | ||
if (includeHeaders) toUnsafeRowWithHeadersProjector else toUnsafeRowWithoutHeadersProjector | ||
} | ||
} | ||
|
||
private[kafka010] object KafkaRecordToRowConverter { | ||
type Record = ConsumerRecord[Array[Byte], Array[Byte]] | ||
|
||
val headersType = ArrayType(StructType(Array( | ||
StructField("key", StringType), | ||
StructField("value", BinaryType)))) | ||
|
||
private val schemaWithoutHeaders = new StructType(Array( | ||
StructField("key", BinaryType), | ||
StructField("value", BinaryType), | ||
StructField("topic", StringType), | ||
StructField("partition", IntegerType), | ||
StructField("offset", LongType), | ||
StructField("timestamp", TimestampType), | ||
StructField("timestampType", IntegerType) | ||
)) | ||
|
||
private val schemaWithHeaders = | ||
new StructType(schemaWithoutHeaders.fields :+ StructField("headers", headersType)) | ||
|
||
def kafkaSchema(includeHeaders: Boolean): StructType = { | ||
if (includeHeaders) schemaWithHeaders else schemaWithoutHeaders | ||
} | ||
} |
This file was deleted.
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -19,9 +19,13 @@ package org.apache.spark.sql.kafka010 | |
|
||
import java.{util => ju} | ||
|
||
import scala.collection.JavaConverters._ | ||
|
||
import org.apache.kafka.clients.producer.{Callback, KafkaProducer, ProducerRecord, RecordMetadata} | ||
import org.apache.kafka.common.header.Header | ||
import org.apache.kafka.common.header.internals.RecordHeader | ||
|
||
import org.apache.spark.sql.catalyst.InternalRow | ||
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} | ||
|
||
|
@@ -88,7 +92,17 @@ 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 record = new ProducerRecord[Array[Byte], Array[Byte]](topic.toString, key, value) | ||
val record = if (projectedRow.isNullAt(3)) { | ||
new ProducerRecord[Array[Byte], Array[Byte]](topic.toString, null, 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) | ||
} | ||
producer.send(record, callback) | ||
} | ||
|
||
|
@@ -131,9 +145,26 @@ private[kafka010] abstract class KafkaRowWriter( | |
throw new IllegalStateException(s"${KafkaWriter.VALUE_ATTRIBUTE_NAME} " + | ||
s"attribute unsupported type ${t.catalogString}") | ||
} | ||
val headersExpression = inputSchema | ||
.find(_.name == KafkaWriter.HEADERS_ATTRIBUTE_NAME).getOrElse( | ||
Literal(CatalystTypeConverters.convertToCatalyst(null), | ||
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. Should this be indented further? 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 tried, but the formatter reverts the indention to the current status. 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. The style checker or something else? 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. The code formatter of the IDE. Also, it passes the style checker of mvn. |
||
KafkaRecordToRowConverter.headersType) | ||
) | ||
headersExpression.dataType match { | ||
case KafkaRecordToRowConverter.headersType => // good | ||
case t => | ||
throw new IllegalStateException(s"${KafkaWriter.HEADERS_ATTRIBUTE_NAME} " + | ||
dongjinleekr marked this conversation as resolved.
Show resolved
Hide resolved
|
||
s"attribute unsupported type ${t.catalogString}") | ||
} | ||
UnsafeProjection.create( | ||
Seq(topicExpression, Cast(keyExpression, BinaryType), | ||
Cast(valueExpression, BinaryType)), inputSchema) | ||
Seq( | ||
topicExpression, | ||
Cast(keyExpression, BinaryType), | ||
Cast(valueExpression, BinaryType), | ||
headersExpression | ||
), | ||
inputSchema | ||
) | ||
} | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,6 +17,7 @@ | |
|
||
package org.apache.spark.sql.kafka010 | ||
|
||
import java.nio.charset.StandardCharsets.UTF_8 | ||
import java.util.Locale | ||
import java.util.concurrent.atomic.AtomicInteger | ||
|
||
|
@@ -32,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} | ||
import org.apache.spark.sql.types.{BinaryType, DataType, StringType, StructField, StructType} | ||
|
||
abstract class KafkaSinkSuiteBase extends QueryTest with SharedSparkSession with KafkaTest { | ||
protected var testUtils: KafkaTestUtils = _ | ||
|
@@ -59,13 +60,14 @@ abstract class KafkaSinkSuiteBase extends QueryTest with SharedSparkSession with | |
|
||
protected def newTopic(): String = s"topic-${topicId.getAndIncrement()}" | ||
|
||
protected def createKafkaReader(topic: String): DataFrame = { | ||
protected def createKafkaReader(topic: String, includeHeaders: Boolean = false): DataFrame = { | ||
spark.read | ||
.format("kafka") | ||
.option("kafka.bootstrap.servers", testUtils.brokerAddress) | ||
.option("startingOffsets", "earliest") | ||
.option("endingOffsets", "latest") | ||
.option("subscribe", topic) | ||
.option("includeHeaders", includeHeaders.toString) | ||
.load() | ||
} | ||
} | ||
|
@@ -368,15 +370,51 @@ abstract class KafkaSinkBatchSuiteBase extends KafkaSinkSuiteBase { | |
test("batch - write to kafka") { | ||
val topic = newTopic() | ||
testUtils.createTopic(topic) | ||
val df = Seq("1", "2", "3", "4", "5").map(v => (topic, v)).toDF("topic", "value") | ||
val data = Seq( | ||
Row(topic, "1", Seq( | ||
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. ditto: |
||
Row("a", "b".getBytes(UTF_8)) | ||
)), | ||
Row(topic, "2", Seq( | ||
Row("c", "d".getBytes(UTF_8)), | ||
Row("e", "f".getBytes(UTF_8)) | ||
)), | ||
Row(topic, "3", Seq( | ||
Row("g", "h".getBytes(UTF_8)), | ||
Row("g", "i".getBytes(UTF_8)) | ||
)), | ||
Row(topic, "4", null), | ||
Row(topic, "5", Seq( | ||
Row("j", "k".getBytes(UTF_8)), | ||
Row("j", "l".getBytes(UTF_8)), | ||
Row("m", "n".getBytes(UTF_8)) | ||
)) | ||
) | ||
|
||
val df = spark.createDataFrame( | ||
spark.sparkContext.parallelize(data), | ||
StructType(Seq(StructField("topic", StringType), StructField("value", StringType), | ||
StructField("headers", KafkaRecordToRowConverter.headersType))) | ||
) | ||
|
||
df.write | ||
.format("kafka") | ||
.option("kafka.bootstrap.servers", testUtils.brokerAddress) | ||
.option("topic", topic) | ||
.save() | ||
checkAnswer( | ||
createKafkaReader(topic).selectExpr("CAST(value as STRING) value"), | ||
Row("1") :: Row("2") :: Row("3") :: Row("4") :: Row("5") :: Nil) | ||
createKafkaReader(topic, includeHeaders = true).selectExpr( | ||
"CAST(value as STRING) value", "headers" | ||
), | ||
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("5", Seq( | ||
Row("j", "k".getBytes(UTF_8)), | ||
Row("j", "l".getBytes(UTF_8)), | ||
Row("m", "n".getBytes(UTF_8)))) :: | ||
Nil | ||
) | ||
} | ||
|
||
test("batch - null topic field value, and no topic option") { | ||
|
Uh oh!
There was an error while loading. Please reload this page.