This is an automated email from the ASF dual-hosted git repository.
jolshan pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git
The following commit(s) were added to refs/heads/trunk by this push:
new b3ba7bc929e KAFKA-18782: Extend ApplicationRecoverableException
related exceptions (#19354)
b3ba7bc929e is described below
commit b3ba7bc929e9307862d3ecdcb9ee3bf0c8eed1a0
Author: Kaushik Raina <[email protected]>
AuthorDate: Fri Apr 11 05:27:28 2025 +0530
KAFKA-18782: Extend ApplicationRecoverableException related exceptions
(#19354)
**Summary**
Extend ApplicationRecoverableException related exceptions
Reviewers: Artem Livshits <[email protected]>, Justine Olshan
<[email protected]>
---
.../errors/ApplicationRecoverableException.java | 15 +++++++++++++++
.../common/errors/FencedInstanceIdException.java | 2 +-
.../common/errors/IllegalGenerationException.java | 2 +-
.../common/errors/InvalidPidMappingException.java | 2 +-
.../common/errors/InvalidProducerEpochException.java | 2 +-
.../kafka/common/errors/ProducerFencedException.java | 2 +-
.../kafka/common/errors/UnknownMemberIdException.java | 2 +-
.../errors/TransactionExceptionHierarchyTest.java | 19 +++++++++++++++++++
8 files changed, 40 insertions(+), 6 deletions(-)
diff --git
a/clients/src/main/java/org/apache/kafka/common/errors/ApplicationRecoverableException.java
b/clients/src/main/java/org/apache/kafka/common/errors/ApplicationRecoverableException.java
index e4059e8e0c7..c07c20e86d9 100644
---
a/clients/src/main/java/org/apache/kafka/common/errors/ApplicationRecoverableException.java
+++
b/clients/src/main/java/org/apache/kafka/common/errors/ApplicationRecoverableException.java
@@ -22,7 +22,22 @@ package org.apache.kafka.common.errors;
* different recovery strategies (e.g., re-balancing task, restoring from
checkpoints) may be employed.
*/
public abstract class ApplicationRecoverableException extends ApiException {
+ private static final long serialVersionUID = 1L;
+
+ public ApplicationRecoverableException(String message, Throwable cause) {
+ super(message, cause);
+ }
+
public ApplicationRecoverableException(String message) {
super(message);
}
+
+ public ApplicationRecoverableException(Throwable cause) {
+ super(cause);
+ }
+
+ public ApplicationRecoverableException() {
+ super();
+ }
+
}
diff --git
a/clients/src/main/java/org/apache/kafka/common/errors/FencedInstanceIdException.java
b/clients/src/main/java/org/apache/kafka/common/errors/FencedInstanceIdException.java
index 78e4034a24a..ac4ae6bdcc6 100644
---
a/clients/src/main/java/org/apache/kafka/common/errors/FencedInstanceIdException.java
+++
b/clients/src/main/java/org/apache/kafka/common/errors/FencedInstanceIdException.java
@@ -16,7 +16,7 @@
*/
package org.apache.kafka.common.errors;
-public class FencedInstanceIdException extends ApiException {
+public class FencedInstanceIdException extends ApplicationRecoverableException
{
private static final long serialVersionUID = 1L;
public FencedInstanceIdException(String message) {
diff --git
a/clients/src/main/java/org/apache/kafka/common/errors/IllegalGenerationException.java
b/clients/src/main/java/org/apache/kafka/common/errors/IllegalGenerationException.java
index efd749f7f2c..2d739ce0290 100644
---
a/clients/src/main/java/org/apache/kafka/common/errors/IllegalGenerationException.java
+++
b/clients/src/main/java/org/apache/kafka/common/errors/IllegalGenerationException.java
@@ -16,7 +16,7 @@
*/
package org.apache.kafka.common.errors;
-public class IllegalGenerationException extends ApiException {
+public class IllegalGenerationException extends
ApplicationRecoverableException {
private static final long serialVersionUID = 1L;
public IllegalGenerationException() {
diff --git
a/clients/src/main/java/org/apache/kafka/common/errors/InvalidPidMappingException.java
b/clients/src/main/java/org/apache/kafka/common/errors/InvalidPidMappingException.java
index 69fb71ea42b..626a19507f4 100644
---
a/clients/src/main/java/org/apache/kafka/common/errors/InvalidPidMappingException.java
+++
b/clients/src/main/java/org/apache/kafka/common/errors/InvalidPidMappingException.java
@@ -16,7 +16,7 @@
*/
package org.apache.kafka.common.errors;
-public class InvalidPidMappingException extends ApiException {
+public class InvalidPidMappingException extends
ApplicationRecoverableException {
public InvalidPidMappingException(String message) {
super(message);
}
diff --git
a/clients/src/main/java/org/apache/kafka/common/errors/InvalidProducerEpochException.java
b/clients/src/main/java/org/apache/kafka/common/errors/InvalidProducerEpochException.java
index 79b82368feb..b65ea67aed5 100644
---
a/clients/src/main/java/org/apache/kafka/common/errors/InvalidProducerEpochException.java
+++
b/clients/src/main/java/org/apache/kafka/common/errors/InvalidProducerEpochException.java
@@ -22,7 +22,7 @@ package org.apache.kafka.common.errors;
* by calling KafkaProducer#abortTransaction which would try to send
initPidRequest and reinitialize the producer
* under the hood.
*/
-public class InvalidProducerEpochException extends ApiException {
+public class InvalidProducerEpochException extends
ApplicationRecoverableException {
private static final long serialVersionUID = 1L;
diff --git
a/clients/src/main/java/org/apache/kafka/common/errors/ProducerFencedException.java
b/clients/src/main/java/org/apache/kafka/common/errors/ProducerFencedException.java
index c47dbf53a8c..4a83bcbeb87 100644
---
a/clients/src/main/java/org/apache/kafka/common/errors/ProducerFencedException.java
+++
b/clients/src/main/java/org/apache/kafka/common/errors/ProducerFencedException.java
@@ -22,7 +22,7 @@ package org.apache.kafka.common.errors;
* given time, and the latest one to be started "fences" the previous
instances so that they can no longer
* make transactional requests. When you encounter this exception, you must
close the producer instance.
*/
-public class ProducerFencedException extends ApiException {
+public class ProducerFencedException extends ApplicationRecoverableException {
public ProducerFencedException(String msg) {
super(msg);
diff --git
a/clients/src/main/java/org/apache/kafka/common/errors/UnknownMemberIdException.java
b/clients/src/main/java/org/apache/kafka/common/errors/UnknownMemberIdException.java
index f6eea5bc8ea..88d717c120f 100644
---
a/clients/src/main/java/org/apache/kafka/common/errors/UnknownMemberIdException.java
+++
b/clients/src/main/java/org/apache/kafka/common/errors/UnknownMemberIdException.java
@@ -16,7 +16,7 @@
*/
package org.apache.kafka.common.errors;
-public class UnknownMemberIdException extends ApiException {
+public class UnknownMemberIdException extends ApplicationRecoverableException {
private static final long serialVersionUID = 1L;
public UnknownMemberIdException() {
diff --git
a/clients/src/test/java/org/apache/kafka/common/errors/TransactionExceptionHierarchyTest.java
b/clients/src/test/java/org/apache/kafka/common/errors/TransactionExceptionHierarchyTest.java
index d642351e9fa..1d6d47abc33 100644
---
a/clients/src/test/java/org/apache/kafka/common/errors/TransactionExceptionHierarchyTest.java
+++
b/clients/src/test/java/org/apache/kafka/common/errors/TransactionExceptionHierarchyTest.java
@@ -75,4 +75,23 @@ public class TransactionExceptionHierarchyTest {
assertTrue(RefreshRetriableException.class.isAssignableFrom(exceptionClass),
exceptionClass.getSimpleName() + " should extend
RefreshRetriableException");
}
+
+ /**
+ * Verifies that the given exception class extends
`ApplicationRecoverableException`
+ *
+ * @param exceptionClass the exception class to check
+ */
+ @ParameterizedTest
+ @ValueSource(classes = {
+ FencedInstanceIdException.class,
+ IllegalGenerationException.class,
+ InvalidPidMappingException.class,
+ InvalidProducerEpochException.class,
+ ProducerFencedException.class,
+ UnknownMemberIdException.class
+ })
+ void testApplicationRecoverableExceptionHierarchy(Class<? extends
Exception> exceptionClass) {
+
assertTrue(ApplicationRecoverableException.class.isAssignableFrom(exceptionClass),
+ exceptionClass.getSimpleName() + " should extend
ApplicationRecoverableException");
+ }
}