Skip to content
Snippets Groups Projects
Commit 8c460804 authored by uncleGen's avatar uncleGen Committed by Shixiong Zhu
Browse files

[SPARK-19853][SS] uppercase kafka topics fail when startingOffsets are SpecificOffsets


When using the KafkaSource with Structured Streaming, consumer assignments are not what the user expects if startingOffsets is set to an explicit set of topics/partitions in JSON where the topic(s) happen to have uppercase characters. When StartingOffsets is constructed, the original string value from options is transformed toLowerCase to make matching on "earliest" and "latest" case insensitive. However, the toLowerCase JSON is passed to SpecificOffsets for the terminal condition, so topic names may not be what the user intended by the time assignments are made with the underlying KafkaConsumer.

KafkaSourceProvider.scala:
```
val startingOffsets = caseInsensitiveParams.get(STARTING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match {
    case Some("latest") => LatestOffsets
    case Some("earliest") => EarliestOffsets
    case Some(json) => SpecificOffsets(JsonUtils.partitionOffsets(json))
    case None => LatestOffsets
  }
```

Thank cbowden for reporting.

Jenkins

Author: uncleGen <hustyugm@gmail.com>

Closes #17209 from uncleGen/SPARK-19853.

(cherry picked from commit 0a4d06a7)
Signed-off-by: default avatarShixiong Zhu <shixiong@databricks.com>
parent f9833c66
No related branches found
No related tags found
No related merge requests found
...@@ -82,13 +82,8 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister ...@@ -82,13 +82,8 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
.map { k => k.drop(6).toString -> parameters(k) } .map { k => k.drop(6).toString -> parameters(k) }
.toMap .toMap
val startingStreamOffsets = val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit(caseInsensitiveParams,
caseInsensitiveParams.get(STARTING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit)
case Some("latest") => LatestOffsetRangeLimit
case Some("earliest") => EarliestOffsetRangeLimit
case Some(json) => SpecificOffsetRangeLimit(JsonUtils.partitionOffsets(json))
case None => LatestOffsetRangeLimit
}
val kafkaOffsetReader = new KafkaOffsetReader( val kafkaOffsetReader = new KafkaOffsetReader(
strategy(caseInsensitiveParams), strategy(caseInsensitiveParams),
...@@ -128,19 +123,13 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister ...@@ -128,19 +123,13 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
.map { k => k.drop(6).toString -> parameters(k) } .map { k => k.drop(6).toString -> parameters(k) }
.toMap .toMap
val startingRelationOffsets = val startingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit(
caseInsensitiveParams.get(STARTING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { caseInsensitiveParams, STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit)
case Some("earliest") => EarliestOffsetRangeLimit assert(startingRelationOffsets != LatestOffsetRangeLimit)
case Some(json) => SpecificOffsetRangeLimit(JsonUtils.partitionOffsets(json))
case None => EarliestOffsetRangeLimit
}
val endingRelationOffsets = val endingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit(caseInsensitiveParams,
caseInsensitiveParams.get(ENDING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit)
case Some("latest") => LatestOffsetRangeLimit assert(endingRelationOffsets != EarliestOffsetRangeLimit)
case Some(json) => SpecificOffsetRangeLimit(JsonUtils.partitionOffsets(json))
case None => LatestOffsetRangeLimit
}
val kafkaOffsetReader = new KafkaOffsetReader( val kafkaOffsetReader = new KafkaOffsetReader(
strategy(caseInsensitiveParams), strategy(caseInsensitiveParams),
...@@ -388,34 +377,34 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister ...@@ -388,34 +377,34 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
private def validateBatchOptions(caseInsensitiveParams: Map[String, String]) = { private def validateBatchOptions(caseInsensitiveParams: Map[String, String]) = {
// Batch specific options // Batch specific options
caseInsensitiveParams.get(STARTING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { KafkaSourceProvider.getKafkaOffsetRangeLimit(
case Some("earliest") => // good to go caseInsensitiveParams, STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) match {
case Some("latest") => case EarliestOffsetRangeLimit => // good to go
case LatestOffsetRangeLimit =>
throw new IllegalArgumentException("starting offset can't be latest " + throw new IllegalArgumentException("starting offset can't be latest " +
"for batch queries on Kafka") "for batch queries on Kafka")
case Some(json) => (SpecificOffsetRangeLimit(JsonUtils.partitionOffsets(json))) case SpecificOffsetRangeLimit(partitionOffsets) =>
.partitionOffsets.foreach { partitionOffsets.foreach {
case (tp, off) if off == KafkaOffsetRangeLimit.LATEST => case (tp, off) if off == KafkaOffsetRangeLimit.LATEST =>
throw new IllegalArgumentException(s"startingOffsets for $tp can't " + throw new IllegalArgumentException(s"startingOffsets for $tp can't " +
"be latest for batch queries on Kafka") "be latest for batch queries on Kafka")
case _ => // ignore case _ => // ignore
} }
case _ => // default to earliest
} }
caseInsensitiveParams.get(ENDING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { KafkaSourceProvider.getKafkaOffsetRangeLimit(
case Some("earliest") => caseInsensitiveParams, ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) match {
case EarliestOffsetRangeLimit =>
throw new IllegalArgumentException("ending offset can't be earliest " + throw new IllegalArgumentException("ending offset can't be earliest " +
"for batch queries on Kafka") "for batch queries on Kafka")
case Some("latest") => // good to go case LatestOffsetRangeLimit => // good to go
case Some(json) => (SpecificOffsetRangeLimit(JsonUtils.partitionOffsets(json))) case SpecificOffsetRangeLimit(partitionOffsets) =>
.partitionOffsets.foreach { partitionOffsets.foreach {
case (tp, off) if off == KafkaOffsetRangeLimit.EARLIEST => case (tp, off) if off == KafkaOffsetRangeLimit.EARLIEST =>
throw new IllegalArgumentException(s"ending offset for $tp can't be " + throw new IllegalArgumentException(s"ending offset for $tp can't be " +
"earliest for batch queries on Kafka") "earliest for batch queries on Kafka")
case _ => // ignore case _ => // ignore
} }
case _ => // default to latest
} }
validateGeneralOptions(caseInsensitiveParams) validateGeneralOptions(caseInsensitiveParams)
...@@ -432,7 +421,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister ...@@ -432,7 +421,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
def set(key: String, value: Object): this.type = { def set(key: String, value: Object): this.type = {
map.put(key, value) map.put(key, value)
logInfo(s"$module: Set $key to $value, earlier value: ${kafkaParams.get(key).getOrElse("")}") logInfo(s"$module: Set $key to $value, earlier value: ${kafkaParams.getOrElse(key, "")}")
this this
} }
...@@ -450,10 +439,22 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister ...@@ -450,10 +439,22 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
private[kafka010] object KafkaSourceProvider { private[kafka010] object KafkaSourceProvider {
private val STRATEGY_OPTION_KEYS = Set("subscribe", "subscribepattern", "assign") private val STRATEGY_OPTION_KEYS = Set("subscribe", "subscribepattern", "assign")
private val STARTING_OFFSETS_OPTION_KEY = "startingoffsets" private[kafka010] val STARTING_OFFSETS_OPTION_KEY = "startingoffsets"
private val ENDING_OFFSETS_OPTION_KEY = "endingoffsets" private[kafka010] val ENDING_OFFSETS_OPTION_KEY = "endingoffsets"
private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss" private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss"
val TOPIC_OPTION_KEY = "topic" val TOPIC_OPTION_KEY = "topic"
private val deserClassName = classOf[ByteArrayDeserializer].getName private val deserClassName = classOf[ByteArrayDeserializer].getName
def getKafkaOffsetRangeLimit(
params: Map[String, String],
offsetOptionKey: String,
defaultOffsets: KafkaOffsetRangeLimit): KafkaOffsetRangeLimit = {
params.get(offsetOptionKey).map(_.trim) match {
case Some(offset) if offset.toLowerCase == "latest" => LatestOffsetRangeLimit
case Some(offset) if offset.toLowerCase == "earliest" => EarliestOffsetRangeLimit
case Some(json) => SpecificOffsetRangeLimit(JsonUtils.partitionOffsets(json))
case None => defaultOffsets
}
}
} }
...@@ -37,6 +37,7 @@ import org.apache.spark.SparkContext ...@@ -37,6 +37,7 @@ import org.apache.spark.SparkContext
import org.apache.spark.sql.ForeachWriter import org.apache.spark.sql.ForeachWriter
import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.functions.{count, window} import org.apache.spark.sql.functions.{count, window}
import org.apache.spark.sql.kafka010.KafkaSourceProvider._
import org.apache.spark.sql.streaming.{ProcessingTime, StreamTest} import org.apache.spark.sql.streaming.{ProcessingTime, StreamTest}
import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession} import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession}
...@@ -604,6 +605,24 @@ class KafkaSourceSuite extends KafkaSourceTest { ...@@ -604,6 +605,24 @@ class KafkaSourceSuite extends KafkaSourceTest {
assert(query.exception.isEmpty) assert(query.exception.isEmpty)
} }
test("get offsets from case insensitive parameters") {
for ((optionKey, optionValue, answer) <- Seq(
(STARTING_OFFSETS_OPTION_KEY, "earLiEst", EarliestOffsetRangeLimit),
(ENDING_OFFSETS_OPTION_KEY, "laTest", LatestOffsetRangeLimit),
(STARTING_OFFSETS_OPTION_KEY, """{"topic-A":{"0":23}}""",
SpecificOffsetRangeLimit(Map(new TopicPartition("topic-A", 0) -> 23))))) {
val offset = getKafkaOffsetRangeLimit(Map(optionKey -> optionValue), optionKey, answer)
assert(offset === answer)
}
for ((optionKey, answer) <- Seq(
(STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit),
(ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit))) {
val offset = getKafkaOffsetRangeLimit(Map.empty, optionKey, answer)
assert(offset === answer)
}
}
private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" private def newTopic(): String = s"topic-${topicId.getAndIncrement()}"
private def assignString(topic: String, partitions: Iterable[Int]): String = { private def assignString(topic: String, partitions: Iterable[Int]): String = {
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment