Skip to content
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

[SPARK-51440] classify the NPE when null topic field value is in kafka message data and there is no topic option #50214

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,11 @@
"Specified: <specifiedPartitions> Assigned: <assignedPartitions>"
]
},
"KAFKA_NULL_TOPIC_IN_DATA": {
"message" : [
"The Kafka message data sent to the producer contains a null topic. Use the `topic` option for setting a default topic."
]
},
"KAFKA_DATA_LOSS" : {
"message" : [
"Some data may have been lost because they are not available in Kafka any more;",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -177,6 +177,12 @@ object KafkaExceptions {
"specifiedPartitions" -> specifiedPartitions.toString,
"assignedPartitions" -> assignedPartitions.toString))
}

def nullTopicInData(): KafkaIllegalStateException = {
new KafkaIllegalStateException(
errorClass = "KAFKA_NULL_TOPIC_IN_DATA",
messageParameters = Map.empty)
}
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import org.apache.kafka.common.header.internals.RecordHeader

import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, UnsafeProjection}
import org.apache.spark.sql.kafka010.KafkaExceptions.nullTopicInData
import org.apache.spark.sql.kafka010.producer.{CachedKafkaProducer, InternalKafkaProducerPool}
import org.apache.spark.sql.types.BinaryType

Expand Down Expand Up @@ -95,8 +96,7 @@ private[kafka010] abstract class KafkaRowWriter(
val key = projectedRow.getBinary(1)
val value = projectedRow.getBinary(2)
if (topic == null) {
throw new NullPointerException(s"null topic present in the data. Use the " +
s"${KafkaSourceProvider.TOPIC_OPTION_KEY} option for setting a default topic.")
throw nullTopicInData()
}
val partition: Integer =
if (projectedRow.isNullAt(4)) null else projectedRow.getInt(4)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ import org.apache.kafka.common.Cluster
import org.apache.kafka.common.serialization.ByteArraySerializer
import org.scalatest.time.SpanSugar._

import org.apache.spark.{SparkConf, SparkContext, SparkException, TestUtils}
import org.apache.spark.{SparkConf, SparkContext, TestUtils}
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, SpecificInternalRow, UnsafeProjection}
import org.apache.spark.sql.execution.streaming.{MemoryStream, MemoryStreamBase}
Expand Down Expand Up @@ -491,14 +491,17 @@ abstract class KafkaSinkBatchSuiteBase extends KafkaSinkSuiteBase {

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] {
val ex = intercept[KafkaIllegalStateException] {
df.write
.format("kafka")
.option("kafka.bootstrap.servers", testUtils.brokerAddress)
.mode("append")
.save()
}
TestUtils.assertExceptionMsg(ex, "null topic present in the data")
checkError(
exception = ex,
condition = "KAFKA_NULL_TOPIC_IN_DATA"
)
}

protected def testUnsupportedSaveModes(msg: (SaveMode) => Seq[String]): Unit = {
Expand Down