chia7712 commented on code in PR #21176:
URL: https://github.com/apache/kafka/pull/21176#discussion_r2635801996


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -1798,13 +1798,17 @@ class KafkaApis(val requestChannel: RequestChannel,
               val error = if (exception == null) {
                 Errors.NONE
               } else {
-                Errors.forException(exception) match {
-                  case Errors.COORDINATOR_NOT_AVAILABLE | 
Errors.COORDINATOR_LOAD_IN_PROGRESS | Errors.NOT_COORDINATOR =>
-                    // The transaction coordinator does not expect those 
errors so we translate them
-                    // to NOT_LEADER_OR_FOLLOWER to signal to it that the 
coordinator is not ready yet.
-                    Errors.NOT_LEADER_OR_FOLLOWER
-                  case error =>
-                    error
+                if 
(Errors.maybeUnwrapException(exception).isInstanceOf[IdempotentTransactionMarkerException])

Review Comment:
   Could you add a helper method to `IdempotentTransactionMarkerException` to 
handle this condition?



##########
clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java:
##########
@@ -203,6 +218,10 @@ public PartitionResponse(
             this.currentLeader = currentLeader;
         }
 
+        public Throwable exception() {
+            return exception.orElse(error.exception(errorMessage));

Review Comment:
   ```java
   exception.orElseGet(() -> error.exception(errorMessage));
   ```



##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -1831,7 +1835,13 @@ class KafkaApis(val requestChannel: RequestChannel,
             requestLocal = requestLocal,
             responseCallback = errors => {
               errors.foreachEntry { (topicIdPartition, partitionResponse) =>
-                addResultAndMaybeComplete(topicIdPartition.topicPartition(), 
partitionResponse.error)
+                val error = if (partitionResponse.error == Errors.NONE)
+                  Errors.NONE
+                else if 
(partitionResponse.exception().isInstanceOf[IdempotentTransactionMarkerException])

Review Comment:
   please add comments



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/IdempotentTransactionMarkerException.java:
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.kafka.storage.internals.log;
+
+import org.apache.kafka.common.KafkaException;
+
+public class IdempotentTransactionMarkerException extends KafkaException {

Review Comment:
   Please add comments



##########
clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java:
##########
@@ -185,6 +187,19 @@ public PartitionResponse(Errors error, long baseOffset, 
long logAppendTime, long
             this(error, baseOffset, logAppendTime, logStartOffset, 
recordErrors, errorMessage, new ProduceResponseData.LeaderIdAndEpoch());
         }
 
+        public PartitionResponse(Optional<Throwable> exception, long 
baseOffset, long logAppendTime, long logStartOffset, List<RecordError> 
recordErrors, String errorMessage) {
+            this(
+                exception.isEmpty() ? Errors.NONE : 
Errors.forException(exception.get()),

Review Comment:
   ```
   exception.map(Errors::forException).orElse(Errors.NONE)
   ```



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to