You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2020/04/29 01:39:22 UTC

[GitHub] [kafka] efeg opened a new pull request #8579: KAFKA-9930: Prevent ReplicaFetcherThread from throwing UnknownTopicOrPartitionException upon topic creation and deletion.

efeg opened a new pull request #8579:
URL: https://github.com/apache/kafka/pull/8579


   When does UnknownTopicOrPartitionException typically occur?
    * Upon a topic creation, a follower broker of a new partition starts replica fetcher before the prospective leader broker of the new partition receives the leadership information from the controller (see [KAFKA-6221](https://issues.apache.org/jira/browse/KAFKA-6221)).
    * Upon a topic deletion, a follower broker of a to-be-deleted partition starts replica fetcher after the leader broker of the to-be-deleted partition processes the deletion information from the controller.
    * As expected, clusters with frequent topic creation and deletion report UnknownTopicOrPartitionException with relatively higher frequency.
   
   What is the impact?
    * Exception tracking systems identify the error logs with UnknownTopicOrPartitionException as an exception. This results in a lot of noise for a transient issue that is expected to recover by itself and a natural process in Kafka due to its asynchronous state propagation.
   
   Why not move it to a lower than warn-level log?
    * Despite typically being a transient issue, UnknownTopicOrPartitionException may also indicate real issues if it doesn't fix itself after a short period of time. To ensure detection of such scenarios, we set the log level to warn.
   
   *More detailed description of your change,
   if necessary. The PR title and PR message become
   the squashed commit message, so use a separate
   comment to ping reviewers.*
   
   *Summary of testing strategy (including rationale)
   for the feature or bug fix. Unit and/or integration
   tests are expected for any behaviour change and
   system tests should be considered for larger changes.*
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


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

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



[GitHub] [kafka] efeg commented on a change in pull request #8579: KAFKA-9930: Prevent ReplicaFetcherThread from throwing UnknownTopicOrPartitionException upon topic creation and deletion.

Posted by GitBox <gi...@apache.org>.
efeg commented on a change in pull request #8579:
URL: https://github.com/apache/kafka/pull/8579#discussion_r445833856



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -382,6 +382,11 @@ abstract class AbstractFetcherThread(name: String,
                     "that the partition is being moved")
                   partitionsWithError += topicPartition
 
+                case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
+                  warn(s"Remote broker does not host the partition $topicPartition, which could indicate " +
+                    "that the partition is being created or deleted.")

Review comment:
       @junrao @hachikuji @ijuma Thanks for the review!
   I updated the message as per Jun's recommendation -- please let me know if further changes needed.




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

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



[GitHub] [kafka] ijuma commented on pull request #8579: KAFKA-9930: Prevent ReplicaFetcherThread from throwing UnknownTopicOrPartitionException upon topic creation and deletion.

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #8579:
URL: https://github.com/apache/kafka/pull/8579#issuecomment-654436916


   retest this please


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

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



[GitHub] [kafka] hachikuji commented on a change in pull request #8579: KAFKA-9930: Prevent ReplicaFetcherThread from throwing UnknownTopicOrPartitionException upon topic creation and deletion.

Posted by GitBox <gi...@apache.org>.
hachikuji commented on a change in pull request #8579:
URL: https://github.com/apache/kafka/pull/8579#discussion_r436406760



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -382,6 +382,11 @@ abstract class AbstractFetcherThread(name: String,
                     "that the partition is being moved")
                   partitionsWithError += topicPartition
 
+                case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
+                  warn(s"Remote broker does not host the partition $topicPartition, which could indicate " +
+                    "that the partition is being created or deleted.")

Review comment:
       I don't have a strong opinion on this one. I guess we would most likely see this in startup cases where the remote broker has just started and may not have gotten the full metadata yet. If we fixed that common case, then I think there would be a stronger case to change this to warn. Otherwise, we'd just learn to ignore them in spite of the higher level.




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

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



[GitHub] [kafka] ijuma merged pull request #8579: KAFKA-9930: Prevent ReplicaFetcherThread from throwing UnknownTopicOrPartitionException upon topic creation and deletion.

Posted by GitBox <gi...@apache.org>.
ijuma merged pull request #8579:
URL: https://github.com/apache/kafka/pull/8579


   


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

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



[GitHub] [kafka] hachikuji commented on a change in pull request #8579: KAFKA-9930: Prevent ReplicaFetcherThread from throwing UnknownTopicOrPartitionException upon topic creation and deletion.

Posted by GitBox <gi...@apache.org>.
hachikuji commented on a change in pull request #8579:
URL: https://github.com/apache/kafka/pull/8579#discussion_r436406760



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -382,6 +382,11 @@ abstract class AbstractFetcherThread(name: String,
                     "that the partition is being moved")
                   partitionsWithError += topicPartition
 
+                case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
+                  warn(s"Remote broker does not host the partition $topicPartition, which could indicate " +
+                    "that the partition is being created or deleted.")

Review comment:
       I don't have a strong opinion on this one. I guess we would most likely see this in startup cases where the remote broker may not have gotten the full metadata yet. If we fixed that common case, then I think there would be a stronger case to change this to warn. Otherwise, we'd just learn to ignore them in spite of the higher level.




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

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



[GitHub] [kafka] ijuma commented on a change in pull request #8579: KAFKA-9930: Prevent ReplicaFetcherThread from throwing UnknownTopicOrPartitionException upon topic creation and deletion.

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8579:
URL: https://github.com/apache/kafka/pull/8579#discussion_r419038136



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -382,6 +382,11 @@ abstract class AbstractFetcherThread(name: String,
                     "that the partition is being moved")
                   partitionsWithError += topicPartition
 
+                case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
+                  warn(s"Remote broker does not host the partition $topicPartition, which could indicate " +
+                    "that the partition is being created or deleted.")

Review comment:
       I think that relying on warnings in logs to detect propagation issues is difficult. We have metrics to indicate issues with failed replica fetches and such. Are those enough for these cases?




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

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



[GitHub] [kafka] junrao commented on a change in pull request #8579: KAFKA-9930: Prevent ReplicaFetcherThread from throwing UnknownTopicOrPartitionException upon topic creation and deletion.

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #8579:
URL: https://github.com/apache/kafka/pull/8579#discussion_r445751897



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -382,6 +382,11 @@ abstract class AbstractFetcherThread(name: String,
                     "that the partition is being moved")
                   partitionsWithError += topicPartition
 
+                case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
+                  warn(s"Remote broker does not host the partition $topicPartition, which could indicate " +
+                    "that the partition is being created or deleted.")

Review comment:
       If a broker is restarted, it typically doesn't have any leader replicas and won't receive follower requests. So, this is mostly related to topic creation/deletion. When there is topic creation/deletion, receiving transient UNKNOWN_TOPIC_OR_PARTITION is expected. But sustained UNKNOWN_TOPIC_OR_PARTITION is unexpected. So, perhaps we could keep the logging at WARN, but tweak the message a bit to sth like "Receiving UNKNOWN_TOPIC_OR_PARTITION from the leader for partition XXXX. This can happen transiently if the partition is being created or deleted. However, this is unexpected if it sustains." 




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

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



[GitHub] [kafka] junrao commented on pull request #8579: KAFKA-9930: Prevent ReplicaFetcherThread from throwing UnknownTopicOrPartitionException upon topic creation and deletion.

Posted by GitBox <gi...@apache.org>.
junrao commented on pull request #8579:
URL: https://github.com/apache/kafka/pull/8579#issuecomment-649884287


   ok to test


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

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



[GitHub] [kafka] ijuma commented on pull request #8579: KAFKA-9930: Prevent ReplicaFetcherThread from throwing UnknownTopicOrPartitionException upon topic creation and deletion.

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #8579:
URL: https://github.com/apache/kafka/pull/8579#issuecomment-652800669


   retest this please


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

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



[GitHub] [kafka] efeg commented on a change in pull request #8579: KAFKA-9930: Prevent ReplicaFetcherThread from throwing UnknownTopicOrPartitionException upon topic creation and deletion.

Posted by GitBox <gi...@apache.org>.
efeg commented on a change in pull request #8579:
URL: https://github.com/apache/kafka/pull/8579#discussion_r419833344



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -382,6 +382,11 @@ abstract class AbstractFetcherThread(name: String,
                     "that the partition is being moved")
                   partitionsWithError += topicPartition
 
+                case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
+                  warn(s"Remote broker does not host the partition $topicPartition, which could indicate " +
+                    "that the partition is being created or deleted.")

Review comment:
       @ijuma Thanks for your reply!
   We both agree that relying on warnings in logs to _detect_ propagation issues is not practical -- metrics indeed should be used in "detection" of such issues.
   I believe the focus of this discussion is the _investigation_ of detected issues.
   
   Metrics typically fail to provide the level of details to examine the interaction of various events in a distributed environment. Hence, in case we need to look at the logs for figuring out such cases, I claim that being able to grep warns and see undesired events is useful -- as opposed to trying to browse info logs for bad/undesirable events.
   
   Note that in `trunk`, such `UNKNOWN_TOPIC_OR_PARTITION` events are already printed in a stronger level (i.e. `error`). 




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

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



[GitHub] [kafka] ijuma commented on a change in pull request #8579: KAFKA-9930: Prevent ReplicaFetcherThread from throwing UnknownTopicOrPartitionException upon topic creation and deletion.

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8579:
URL: https://github.com/apache/kafka/pull/8579#discussion_r445936549



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -382,6 +382,12 @@ abstract class AbstractFetcherThread(name: String,
                     "that the partition is being moved")
                   partitionsWithError += topicPartition
 
+                case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
+                  warn(s"Receiving ${Errors.UNKNOWN_TOPIC_OR_PARTITION} from the leader for partition $topicPartition. " +
+                       s"This can happen transiently if the partition is being created or deleted. " +
+                       s"However, this is unexpected if it sustains.")

Review comment:
       Thanks for the update. I think the following reads a bit better and is a bit more consistent with the other messages:
   
   ```scala
   s"Received ${Errors.UNKNOWN_TOPIC_OR_PARTITION} from the leader for partition $topicPartition. " +
   "This error may be returned transiently when the partition is being created or deleted, but it is not " +
   "expected to persist")
   ```




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

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



[GitHub] [kafka] efeg commented on pull request #8579: KAFKA-9930: Prevent ReplicaFetcherThread from throwing UnknownTopicOrPartitionException upon topic creation and deletion.

Posted by GitBox <gi...@apache.org>.
efeg commented on pull request #8579:
URL: https://github.com/apache/kafka/pull/8579#issuecomment-635547747


   @ijuma do you think we may be able to iterate on this PR?


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

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



[GitHub] [kafka] ijuma commented on pull request #8579: KAFKA-9930: Prevent ReplicaFetcherThread from throwing UnknownTopicOrPartitionException upon topic creation and deletion.

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #8579:
URL: https://github.com/apache/kafka/pull/8579#issuecomment-635553427


   ok to test


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

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



[GitHub] [kafka] ijuma commented on a change in pull request #8579: KAFKA-9930: Prevent ReplicaFetcherThread from throwing UnknownTopicOrPartitionException upon topic creation and deletion.

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8579:
URL: https://github.com/apache/kafka/pull/8579#discussion_r417044882



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -382,6 +382,11 @@ abstract class AbstractFetcherThread(name: String,
                     "that the partition is being moved")
                   partitionsWithError += topicPartition
 
+                case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
+                  warn(s"Remote broker does not host the partition $topicPartition, which could indicate " +
+                    "that the partition is being created or deleted.")

Review comment:
       Should it be `info` if we think this is expected? That would still show in the logs.




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

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



[GitHub] [kafka] efeg commented on a change in pull request #8579: KAFKA-9930: Prevent ReplicaFetcherThread from throwing UnknownTopicOrPartitionException upon topic creation and deletion.

Posted by GitBox <gi...@apache.org>.
efeg commented on a change in pull request #8579:
URL: https://github.com/apache/kafka/pull/8579#discussion_r446327952



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -382,6 +382,12 @@ abstract class AbstractFetcherThread(name: String,
                     "that the partition is being moved")
                   partitionsWithError += topicPartition
 
+                case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
+                  warn(s"Receiving ${Errors.UNKNOWN_TOPIC_OR_PARTITION} from the leader for partition $topicPartition. " +
+                       s"This can happen transiently if the partition is being created or deleted. " +
+                       s"However, this is unexpected if it sustains.")

Review comment:
       @ijuma thanks for the update -- let me know if further changes needed.




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

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



[GitHub] [kafka] ijuma commented on pull request #8579: KAFKA-9930: Prevent ReplicaFetcherThread from throwing UnknownTopicOrPartitionException upon topic creation and deletion.

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #8579:
URL: https://github.com/apache/kafka/pull/8579#issuecomment-650999166


   ok to test


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

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



[GitHub] [kafka] ijuma commented on a change in pull request #8579: KAFKA-9930: Prevent ReplicaFetcherThread from throwing UnknownTopicOrPartitionException upon topic creation and deletion.

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8579:
URL: https://github.com/apache/kafka/pull/8579#discussion_r432073518



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -382,6 +382,11 @@ abstract class AbstractFetcherThread(name: String,
                     "that the partition is being moved")
                   partitionsWithError += topicPartition
 
+                case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
+                  warn(s"Remote broker does not host the partition $topicPartition, which could indicate " +
+                    "that the partition is being created or deleted.")

Review comment:
       I still think that `info` would be a better debug level for this. But let's get a second opinion from @junrao or @hachikuji.




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

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



[GitHub] [kafka] ijuma commented on a change in pull request #8579: KAFKA-9930: Prevent ReplicaFetcherThread from throwing UnknownTopicOrPartitionException upon topic creation and deletion.

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8579:
URL: https://github.com/apache/kafka/pull/8579#discussion_r445936549



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -382,6 +382,12 @@ abstract class AbstractFetcherThread(name: String,
                     "that the partition is being moved")
                   partitionsWithError += topicPartition
 
+                case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
+                  warn(s"Receiving ${Errors.UNKNOWN_TOPIC_OR_PARTITION} from the leader for partition $topicPartition. " +
+                       s"This can happen transiently if the partition is being created or deleted. " +
+                       s"However, this is unexpected if it sustains.")

Review comment:
       Thanks for the update. I think the following reads a bit better and is a bit more consistent with the other messages:
   
   ```scala
   s"Received ${Errors.UNKNOWN_TOPIC_OR_PARTITION} from the leader for partition $topicPartition. " +
   "This error may be returned transiently when the partition is being created or deleted, but it is not " +
   "expected to persist.")
   ```




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

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



[GitHub] [kafka] efeg commented on a change in pull request #8579: KAFKA-9930: Prevent ReplicaFetcherThread from throwing UnknownTopicOrPartitionException upon topic creation and deletion.

Posted by GitBox <gi...@apache.org>.
efeg commented on a change in pull request #8579:
URL: https://github.com/apache/kafka/pull/8579#discussion_r419833344



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -382,6 +382,11 @@ abstract class AbstractFetcherThread(name: String,
                     "that the partition is being moved")
                   partitionsWithError += topicPartition
 
+                case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
+                  warn(s"Remote broker does not host the partition $topicPartition, which could indicate " +
+                    "that the partition is being created or deleted.")

Review comment:
       @ijuma Thanks for your reply!
   We both agree that relying on warnings in logs to _detect_ propagation issues is not practical -- metrics indeed should be used in "detection" of such issues.
   I believe the focus of this discussion isthe _investigation_ of detected issues.
   
   Metrics typically fail to provide the level of details to examine the interaction of various events in a distributed environment. Hence, in case we need to look at the logs for figuring out such cases, I claim that being able to grep warns and see undesired events is useful -- as opposed to trying to browse info logs for bad/undesirable events.
   
   Note that in `trunk`, such `UNKNOWN_TOPIC_OR_PARTITION` events are already printed in a stronger level (i.e. `error`). 




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

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



[GitHub] [kafka] efeg commented on a change in pull request #8579: KAFKA-9930: Prevent ReplicaFetcherThread from throwing UnknownTopicOrPartitionException upon topic creation and deletion.

Posted by GitBox <gi...@apache.org>.
efeg commented on a change in pull request #8579:
URL: https://github.com/apache/kafka/pull/8579#discussion_r417460683



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -382,6 +382,11 @@ abstract class AbstractFetcherThread(name: String,
                     "that the partition is being moved")
                   partitionsWithError += topicPartition
 
+                case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
+                  warn(s"Remote broker does not host the partition $topicPartition, which could indicate " +
+                    "that the partition is being created or deleted.")

Review comment:
       @ijuma I feel that `info` has an implicit implication that everything is "fine"; hence, I am concerned that it might be overlooked when investigating an issue.
   
   The current broadcast-based approach in Kafka is just one way -- which arguably, has room for improvement -- of implementing state propagation for topic creation/deletion, and this error is a side-effect of it. Extended occurrence of this error can hamper the read/write availability as well as state consistency. Thus, even if `UNKNOWN_TOPIC_OR_PARTITION` is expected under the current design/implementation, it is definitely not desired.
   
   What are your thoughts?




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

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



[GitHub] [kafka] ijuma commented on pull request #8579: KAFKA-9930: Prevent ReplicaFetcherThread from throwing UnknownTopicOrPartitionException upon topic creation and deletion.

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #8579:
URL: https://github.com/apache/kafka/pull/8579#issuecomment-654513658


   Unrelated flaky test failures:
   
   ```
   org.apache.kafka.streams.integration.EosBetaUpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosBeta[false]
   org.apache.kafka.streams.integration.EosBetaUpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosBeta[true]
   org.apache.kafka.streams.integration.EosBetaUpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosBeta[false]
   org.apache.kafka.streams.integration.EosBetaUpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosBeta[true]
   
   ```


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

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



[GitHub] [kafka] ijuma commented on a change in pull request #8579: KAFKA-9930: Prevent ReplicaFetcherThread from throwing UnknownTopicOrPartitionException upon topic creation and deletion.

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8579:
URL: https://github.com/apache/kafka/pull/8579#discussion_r432073518



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -382,6 +382,11 @@ abstract class AbstractFetcherThread(name: String,
                     "that the partition is being moved")
                   partitionsWithError += topicPartition
 
+                case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
+                  warn(s"Remote broker does not host the partition $topicPartition, which could indicate " +
+                    "that the partition is being created or deleted.")

Review comment:
       I still think that `info` would be a better log level for this. But let's get a second opinion from @junrao or @hachikuji.




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

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



[GitHub] [kafka] hachikuji commented on a change in pull request #8579: KAFKA-9930: Prevent ReplicaFetcherThread from throwing UnknownTopicOrPartitionException upon topic creation and deletion.

Posted by GitBox <gi...@apache.org>.
hachikuji commented on a change in pull request #8579:
URL: https://github.com/apache/kafka/pull/8579#discussion_r436406760



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -382,6 +382,11 @@ abstract class AbstractFetcherThread(name: String,
                     "that the partition is being moved")
                   partitionsWithError += topicPartition
 
+                case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
+                  warn(s"Remote broker does not host the partition $topicPartition, which could indicate " +
+                    "that the partition is being created or deleted.")

Review comment:
       I don't have a strong opinion on this one. I guess we would most likely see this in startup cases where the remote broker may not have gotten the full metadata yet. If we fixed that common case, then I think there would be a strong case to change this to warn. Otherwise, we'd just learn to ignore them in spite of the higher level.




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

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



[GitHub] [kafka] ijuma commented on pull request #8579: KAFKA-9930: Prevent ReplicaFetcherThread from throwing UnknownTopicOrPartitionException upon topic creation and deletion.

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #8579:
URL: https://github.com/apache/kafka/pull/8579#issuecomment-650319283


   ok to test


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

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