forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[SPARK-41387][SS] Assert current end offset from Kafka data source fo…
…r Trigger.AvailableNow ### What changes were proposed in this pull request? This PR proposes to assert the end offset of current batch for Kafka data source with Trigger.AvailableNow, via fetching the latest offset from known topic partitions and compare with end offset. Kafka data source will throw error when the latest information from topic-partition is a loss e.g. topic-partition is missing, latest offset is less than the offset in end offset. This way we construct a safeguard for Trigger.AvailableNow to avoid hanging, although it won't prevent 100% of cases for future bugs. ### Why are the changes needed? This change will let Kafka data source do its best effort to fail fast when the state of the query is not able to reach the end state of Trigger.AvailableNow. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New test cases. Closes apache#38911 from HeartSaVioR/SPARK-41387. Authored-by: Jungtaek Lim <[email protected]> Signed-off-by: Jungtaek Lim <[email protected]>
- Loading branch information
1 parent
5aca8b4
commit bc6351d
Showing
5 changed files
with
307 additions
and
1 deletion.
There are no files selected for viewing
80 changes: 80 additions & 0 deletions
80
connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaExceptions.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,80 @@ | ||
/* | ||
* 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 org.apache.kafka.common.TopicPartition | ||
|
||
import org.apache.spark.{ErrorClassesJsonReader, SparkException} | ||
|
||
object KafkaExceptions { | ||
private val errorClassesJsonReader: ErrorClassesJsonReader = | ||
new ErrorClassesJsonReader( | ||
Seq(getClass.getClassLoader.getResource("error/kafka-error-classes.json"))) | ||
|
||
def mismatchedTopicPartitionsBetweenEndOffsetAndPrefetched( | ||
tpsForPrefetched: Set[TopicPartition], | ||
tpsForEndOffset: Set[TopicPartition]): SparkException = { | ||
val errMsg = errorClassesJsonReader.getErrorMessage( | ||
"MISMATCHED_TOPIC_PARTITIONS_BETWEEN_END_OFFSET_AND_PREFETCHED", | ||
Map( | ||
"tpsForPrefetched" -> tpsForPrefetched.toString(), | ||
"tpsForEndOffset" -> tpsForEndOffset.toString() | ||
) | ||
) | ||
new SparkException(errMsg) | ||
} | ||
|
||
def endOffsetHasGreaterOffsetForTopicPartitionThanPrefetched( | ||
prefetchedOffset: Map[TopicPartition, Long], | ||
endOffset: Map[TopicPartition, Long]): SparkException = { | ||
val errMsg = errorClassesJsonReader.getErrorMessage( | ||
"END_OFFSET_HAS_GREATER_OFFSET_FOR_TOPIC_PARTITION_THAN_PREFETCHED", | ||
Map( | ||
"prefetchedOffset" -> prefetchedOffset.toString(), | ||
"endOffset" -> endOffset.toString() | ||
) | ||
) | ||
new SparkException(errMsg) | ||
} | ||
|
||
def lostTopicPartitionsInEndOffsetWithTriggerAvailableNow( | ||
tpsForLatestOffset: Set[TopicPartition], | ||
tpsForEndOffset: Set[TopicPartition]): SparkException = { | ||
val errMsg = errorClassesJsonReader.getErrorMessage( | ||
"LOST_TOPIC_PARTITIONS_IN_END_OFFSET_WITH_TRIGGER_AVAILABLENOW", | ||
Map( | ||
"tpsForLatestOffset" -> tpsForLatestOffset.toString(), | ||
"tpsForEndOffset" -> tpsForEndOffset.toString() | ||
) | ||
) | ||
new SparkException(errMsg) | ||
} | ||
|
||
def endOffsetHasGreaterOffsetForTopicPartitionThanLatestWithTriggerAvailableNow( | ||
latestOffset: Map[TopicPartition, Long], | ||
endOffset: Map[TopicPartition, Long]): SparkException = { | ||
val errMsg = errorClassesJsonReader.getErrorMessage( | ||
"END_OFFSET_HAS_GREATER_OFFSET_FOR_TOPIC_PARTITION_THAN_LATEST_WITH_TRIGGER_AVAILABLENOW", | ||
Map( | ||
"latestOffset" -> latestOffset.toString(), | ||
"endOffset" -> endOffset.toString() | ||
) | ||
) | ||
new SparkException(errMsg) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
26 changes: 26 additions & 0 deletions
26
connector/kafka-0-10-sql/src/test/resources/error/kafka-error-classes.json
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,26 @@ | ||
{ | ||
"MISMATCHED_TOPIC_PARTITIONS_BETWEEN_END_OFFSET_AND_PREFETCHED" : { | ||
"message" : [ | ||
"Kafka data source in Trigger.AvailableNow should provide the same topic partitions in pre-fetched offset to end offset for each microbatch. The error could be transient - restart your query, and report if you still see the same issue.", | ||
"topic-partitions for pre-fetched offset: <tpsForPrefetched>, topic-partitions for end offset: <tpsForEndOffset>." | ||
] | ||
}, | ||
"END_OFFSET_HAS_GREATER_OFFSET_FOR_TOPIC_PARTITION_THAN_PREFETCHED" : { | ||
"message" : [ | ||
"For Kafka data source with Trigger.AvailableNow, end offset should have lower or equal offset per each topic partition than pre-fetched offset. The error could be transient - restart your query, and report if you still see the same issue.", | ||
"pre-fetched offset: <prefetchedOffset>, end offset: <endOffset>." | ||
] | ||
}, | ||
"LOST_TOPIC_PARTITIONS_IN_END_OFFSET_WITH_TRIGGER_AVAILABLENOW" : { | ||
"message" : [ | ||
"Some of partitions in Kafka topic(s) have been lost during running query with Trigger.AvailableNow. The error could be transient - restart your query, and report if you still see the same issue.", | ||
"topic-partitions for latest offset: <tpsForLatestOffset>, topic-partitions for end offset: <tpsForEndOffset>" | ||
] | ||
}, | ||
"END_OFFSET_HAS_GREATER_OFFSET_FOR_TOPIC_PARTITION_THAN_LATEST_WITH_TRIGGER_AVAILABLENOW" : { | ||
"message" : [ | ||
"Some of partitions in Kafka topic(s) report available offset which is less than end offset during running query with Trigger.AvailableNow. The error could be transient - restart your query, and report if you still see the same issue.", | ||
"latest offset: <latestOffset>, end offset: <endOffset>" | ||
] | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters