You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/12/03 19:20:57 UTC

[GitHub] [spark] wecharyu opened a new pull request, #38898: [SPARK-41375][SS] Avoid empty latest KafkaSourceOffset

wecharyu opened a new pull request, #38898:
URL: https://github.com/apache/spark/pull/38898

   ### What changes were proposed in this pull request?
   Add the empty offset filter in `latestOffset()` for Kafka Source.
   
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   To avoid the data duplication in the extreme cases where spark fetch empty latest Kafka source offset.
   
   
   ### Does this PR introduce _any_ user-facing change?
   No
   
   
   ### How was this patch tested?
   
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on pull request #38898: [SPARK-41375][SS] Avoid empty latest KafkaSourceOffset

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on PR #38898:
URL: https://github.com/apache/spark/pull/38898#issuecomment-1341979338

   I'm trying to understand the case - if my understanding is correct, the new test is just to trigger the same behavior rather than reproducing actual problem, right? In the new test, recognizing all topic partitions as new one and process all records in next microbatch is arguably NOT a wrong behavior for me, hence I really would like to understand the actual problem.
   
   According to the JIRA description, the actual problem is that Kafka can "transiently" give no topic partition as assignment when it performs reassignment among consumers, specifically here:
   
   ```
         consumer.poll(0)
         val partitions = consumer.assignment()
   ```
   
   which we expect Kafka to assign topic partitions to this consumer accordingly after calling poll.
   
   Do I understand correctly?


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR closed pull request #38898: [SPARK-41375][SS] Avoid empty latest KafkaSourceOffset

Posted by GitBox <gi...@apache.org>.
HeartSaVioR closed pull request #38898: [SPARK-41375][SS] Avoid empty latest KafkaSourceOffset
URL: https://github.com/apache/spark/pull/38898


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] jerrypeng commented on pull request #38898: [SPARK-41375][SS] Avoid empty latest KafkaSourceOffset

Posted by GitBox <gi...@apache.org>.
jerrypeng commented on PR #38898:
URL: https://github.com/apache/spark/pull/38898#issuecomment-1341350829

   > To avoid the data duplication in the extreme cases where spark fetch empty latest Kafka source offset.
   
   @wecharyu how does an empty latest Kafka source offset cause data duplication?


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] jerrypeng commented on pull request #38898: [SPARK-41375][SS] Avoid empty latest KafkaSourceOffset

Posted by GitBox <gi...@apache.org>.
jerrypeng commented on PR #38898:
URL: https://github.com/apache/spark/pull/38898#issuecomment-1340530153

   @wecharyu can you run one batch and then delete all the partitions?


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] wecharyu commented on pull request #38898: [SPARK-41375][SS] Avoid empty latest KafkaSourceOffset

Posted by GitBox <gi...@apache.org>.
wecharyu commented on PR #38898:
URL: https://github.com/apache/spark/pull/38898#issuecomment-1339648870

   > Can you write a unit test for this?
   
   It seems a bit difficult to write unit test to cover the case where fetching empty partitions from Kafka cluster, any idea will be appreciated.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #38898: [SPARK-41375][SS] Avoid empty latest KafkaSourceOffset

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on code in PR #38898:
URL: https://github.com/apache/spark/pull/38898#discussion_r1042910210


##########
connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala:
##########
@@ -627,6 +627,45 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
+  test("SPARK-41375: empty partitions should not record to latest offset") {
+    val topicPrefix = newTopic()

Review Comment:
   Please set `spark.sql.streaming.kafka.useDeprecatedOffsetFetching` to `true`. You can do this with leveraging `withSQLConf(...map of explicit config here...) { ...test code here... }`



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on pull request #38898: [SPARK-41375][SS] Avoid empty latest KafkaSourceOffset

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on PR #38898:
URL: https://github.com/apache/spark/pull/38898#issuecomment-1342243376

   Thanks! Merging to master.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #38898: [SPARK-41375][SS] Avoid empty latest KafkaSourceOffset

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on code in PR #38898:
URL: https://github.com/apache/spark/pull/38898#discussion_r1042911844


##########
connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala:
##########
@@ -627,6 +627,45 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
+  test("SPARK-41375: empty partitions should not record to latest offset") {
+    val topicPrefix = newTopic()

Review Comment:
   Well never mind. We're not reproducing the actual problem here, then it seems sufficient.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] wecharyu commented on pull request #38898: [SPARK-41375][SS] Avoid empty latest KafkaSourceOffset

Posted by GitBox <gi...@apache.org>.
wecharyu commented on PR #38898:
URL: https://github.com/apache/spark/pull/38898#issuecomment-1342415345

   @jerrypeng @HeartSaVioR thanks for your review!


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] wecharyu commented on pull request #38898: [SPARK-41375][SS] Avoid empty latest KafkaSourceOffset

Posted by GitBox <gi...@apache.org>.
wecharyu commented on PR #38898:
URL: https://github.com/apache/spark/pull/38898#issuecomment-1341944936

   @jerrypeng the empty offset will be stored in `committedOffsets`, when we run next batch, the following code will record an empty map startOffset in `newBatchesPlan`:
   https://github.com/apache/spark/blob/2179955b1e7fe9ed7ea44a4e0d794694d0f7133e/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala#L665
   Then while fetching partitions, all the partitions are considered as "new partitions" and will fetch the earliest offsets, which will produce dupicate data.
   https://github.com/apache/spark/blob/2179955b1e7fe9ed7ea44a4e0d794694d0f7133e/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderAdmin.scala#L443-L460


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #38898: [SPARK-41375][SS] Avoid empty latest KafkaSourceOffset

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on PR #38898:
URL: https://github.com/apache/spark/pull/38898#issuecomment-1336533569

   Can one of the admins verify this patch?


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on pull request #38898: [SPARK-41375][SS] Avoid empty latest KafkaSourceOffset

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on PR #38898:
URL: https://github.com/apache/spark/pull/38898#issuecomment-1341968071

   Could you please try to summarize the description of JIRA to PR template, especially the part of "Root Cause"? Also, is it "known" issue for Kafka consumer?
   
   Also please note that we changed the default offset fetching mechanism from consumer group assignment from Kafka to active fetch via AdminClient, which won't have such issue.
   https://github.com/apache/spark/blob/master/docs/ss-migration-guide.md#upgrading-from-structured-streaming-33-to-34
   
   That said, your test case should turn on `spark.sql.streaming.kafka.useDeprecatedOffsetFetching` to `true` to not test with default config.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] wecharyu commented on pull request #38898: [SPARK-41375][SS] Avoid empty latest KafkaSourceOffset

Posted by GitBox <gi...@apache.org>.
wecharyu commented on PR #38898:
URL: https://github.com/apache/spark/pull/38898#issuecomment-1342127570

   @HeartSaVioR yes you are right, the actual problem is that we may fetch empty partitions unexpectedly in one batch, and in the next batch we fetch the real partitions again. The new test is just used to mock the empty partitions, but it also make sense to not record the empty offset for the empty partitions.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org