lucasbru commented on code in PR #13876:
URL: https://github.com/apache/kafka/pull/13876#discussion_r1236863600


##########
clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java:
##########
@@ -1013,16 +1013,8 @@ private void maybeFailWithError() {
         if (!hasError()) {
             return;
         }
-        // for ProducerFencedException, do not wrap it as a KafkaException
-        // but create a new instance without the call trace since it was not 
thrown because of the current call
-        if (lastError instanceof ProducerFencedException) {
-            throw new ProducerFencedException("Producer with transactionalId 
'" + transactionalId
-                    + "' and " + producerIdAndEpoch + " has been fenced by 
another producer " +
-                    "with the same transactionalId");
-        }
-        if (lastError instanceof InvalidProducerEpochException) {

Review Comment:
   `INVALID_PRODUCER_EPOCH` is always converted to a `ProducerFencedException`, 
so we should not see `InvalidProducerEpochException` here anymore, right? There 
are tests mocking a `INVALID_PRODUCER_EPOCH` response though.



-- 
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: jira-unsubscr...@kafka.apache.org

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

Reply via email to