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 896f283e1e5 KAFKA-18783 : Extend InvalidConfigurationException related 
exceptions (#19731)
896f283e1e5 is described below

commit 896f283e1e5e9d6ce4d4a7591d2654d74c1ebf1a
Author: Kaushik Raina <[email protected]>
AuthorDate: Wed May 21 05:01:33 2025 +0530

    KAFKA-18783 : Extend InvalidConfigurationException related exceptions 
(#19731)
    
    ## Summary
    Extend InvalidConfigurationException related exceptions
      - `AuthenticationException`
      - `AuthorizationException`
      - `ClusterAuthorizationException`
      - `TransactionalIdAuthorizationException`
      - `UnsupportedVersionException`
      - `UnsupportedForMessageFormatException`
      - `InvalidRecordException`
      - `InvalidRequiredAcksException`
      - `RecordBatchTooLargeException`
      - `InvalidTopicException`
      - `TopicAuthorizationException`
      - `GroupAuthorizationException`
    
    ## Testing
    - Added unit tests in `TransactionExceptionHierarchyTest` to verify:
      - All configuration-related exceptions properly extend
    `InvalidConfigurationException`
    
    Reviewers: Justine Olshan <[email protected]>
---
 .../kafka/common/InvalidRecordException.java       |  4 ++--
 .../common/errors/AuthenticationException.java     |  2 +-
 .../common/errors/AuthorizationException.java      |  2 +-
 .../errors/InvalidConfigurationException.java      | 12 ++++++++--
 .../errors/InvalidReplicationFactorException.java  |  2 +-
 .../errors/InvalidRequiredAcksException.java       |  2 +-
 .../kafka/common/errors/InvalidTopicException.java |  2 +-
 .../errors/RecordBatchTooLargeException.java       |  2 +-
 .../UnsupportedForMessageFormatException.java      |  2 +-
 .../common/errors/UnsupportedVersionException.java |  2 +-
 .../errors/TransactionExceptionHierarchyTest.java  | 27 ++++++++++++++++++++++
 .../src/main/scala/kafka/admin/ConfigCommand.scala |  8 +++----
 12 files changed, 51 insertions(+), 16 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/InvalidRecordException.java 
b/clients/src/main/java/org/apache/kafka/common/InvalidRecordException.java
index 4c2815bb3bd..aa13ffd9936 100644
--- a/clients/src/main/java/org/apache/kafka/common/InvalidRecordException.java
+++ b/clients/src/main/java/org/apache/kafka/common/InvalidRecordException.java
@@ -16,9 +16,9 @@
  */
 package org.apache.kafka.common;
 
-import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.InvalidConfigurationException;
 
-public class InvalidRecordException extends ApiException {
+public class InvalidRecordException extends InvalidConfigurationException {
 
     private static final long serialVersionUID = 1;
 
diff --git 
a/clients/src/main/java/org/apache/kafka/common/errors/AuthenticationException.java
 
b/clients/src/main/java/org/apache/kafka/common/errors/AuthenticationException.java
index 7a05eba03f2..5d1d5070173 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/errors/AuthenticationException.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/errors/AuthenticationException.java
@@ -32,7 +32,7 @@ import javax.net.ssl.SSLException;
  *   <li>{@link SslAuthenticationException} if SSL handshake failed due to any 
{@link SSLException}.
  * </ul>
  */
-public class AuthenticationException extends ApiException {
+public class AuthenticationException extends InvalidConfigurationException {
 
     private static final long serialVersionUID = 1L;
 
diff --git 
a/clients/src/main/java/org/apache/kafka/common/errors/AuthorizationException.java
 
b/clients/src/main/java/org/apache/kafka/common/errors/AuthorizationException.java
index 0471fe643d9..8ff29af9597 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/errors/AuthorizationException.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/errors/AuthorizationException.java
@@ -16,7 +16,7 @@
  */
 package org.apache.kafka.common.errors;
 
-public class AuthorizationException extends ApiException {
+public class AuthorizationException extends InvalidConfigurationException {
 
     public AuthorizationException(String message) {
         super(message);
diff --git 
a/clients/src/main/java/org/apache/kafka/common/errors/InvalidConfigurationException.java
 
b/clients/src/main/java/org/apache/kafka/common/errors/InvalidConfigurationException.java
index 333566a64ba..85e63c42ec3 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/errors/InvalidConfigurationException.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/errors/InvalidConfigurationException.java
@@ -20,12 +20,20 @@ public class InvalidConfigurationException extends 
ApiException {
 
     private static final long serialVersionUID = 1L;
 
+    public InvalidConfigurationException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
     public InvalidConfigurationException(String message) {
         super(message);
     }
 
-    public InvalidConfigurationException(String message, Throwable cause) {
-        super(message, cause);
+    public InvalidConfigurationException(Throwable cause) {
+        super(cause);
+    }
+
+    public InvalidConfigurationException() {
+        super();
     }
 
 }
diff --git 
a/clients/src/main/java/org/apache/kafka/common/errors/InvalidReplicationFactorException.java
 
b/clients/src/main/java/org/apache/kafka/common/errors/InvalidReplicationFactorException.java
index 699d5a83a43..508d73a793f 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/errors/InvalidReplicationFactorException.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/errors/InvalidReplicationFactorException.java
@@ -16,7 +16,7 @@
  */
 package org.apache.kafka.common.errors;
 
-public class InvalidReplicationFactorException extends ApiException {
+public class InvalidReplicationFactorException extends 
InvalidConfigurationException {
 
     private static final long serialVersionUID = 1L;
 
diff --git 
a/clients/src/main/java/org/apache/kafka/common/errors/InvalidRequiredAcksException.java
 
b/clients/src/main/java/org/apache/kafka/common/errors/InvalidRequiredAcksException.java
index 423c09166b7..f861dbfee18 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/errors/InvalidRequiredAcksException.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/errors/InvalidRequiredAcksException.java
@@ -16,7 +16,7 @@
  */
 package org.apache.kafka.common.errors;
 
-public class InvalidRequiredAcksException extends ApiException {
+public class InvalidRequiredAcksException extends 
InvalidConfigurationException {
     private static final long serialVersionUID = 1L;
 
     public InvalidRequiredAcksException(String message) {
diff --git 
a/clients/src/main/java/org/apache/kafka/common/errors/InvalidTopicException.java
 
b/clients/src/main/java/org/apache/kafka/common/errors/InvalidTopicException.java
index 344d231ce9e..cf0ed5ed5cc 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/errors/InvalidTopicException.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/errors/InvalidTopicException.java
@@ -27,7 +27,7 @@ import java.util.Set;
  *
  * @see UnknownTopicOrPartitionException
  */
-public class InvalidTopicException extends ApiException {
+public class InvalidTopicException extends InvalidConfigurationException {
     private static final long serialVersionUID = 1L;
 
     private final Set<String> invalidTopics;
diff --git 
a/clients/src/main/java/org/apache/kafka/common/errors/RecordBatchTooLargeException.java
 
b/clients/src/main/java/org/apache/kafka/common/errors/RecordBatchTooLargeException.java
index d1ba8ff7160..24f563e1c27 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/errors/RecordBatchTooLargeException.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/errors/RecordBatchTooLargeException.java
@@ -19,7 +19,7 @@ package org.apache.kafka.common.errors;
 /**
  * This record batch is larger than the maximum allowable size
  */
-public class RecordBatchTooLargeException extends ApiException {
+public class RecordBatchTooLargeException extends 
InvalidConfigurationException {
 
     private static final long serialVersionUID = 1L;
 
diff --git 
a/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedForMessageFormatException.java
 
b/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedForMessageFormatException.java
index f66298e99c3..03add5c4ffc 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedForMessageFormatException.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedForMessageFormatException.java
@@ -20,7 +20,7 @@ package org.apache.kafka.common.errors;
  * The message format version does not support the requested function. For 
example, if idempotence is
  * requested and the topic is using a message format older than 0.11.0.0, then 
this error will be returned.
  */
-public class UnsupportedForMessageFormatException extends ApiException {
+public class UnsupportedForMessageFormatException extends 
InvalidConfigurationException {
     private static final long serialVersionUID = 1L;
 
     public UnsupportedForMessageFormatException(String message) {
diff --git 
a/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedVersionException.java
 
b/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedVersionException.java
index 484947b0ae2..00da3a1b2b8 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedVersionException.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedVersionException.java
@@ -28,7 +28,7 @@ import java.util.Map;
  * is raised from {@link 
org.apache.kafka.clients.consumer.KafkaConsumer#offsetsForTimes(Map)}, it would
  * be possible to revert to alternative logic to set the consumer's position.
  */
-public class UnsupportedVersionException extends ApiException {
+public class UnsupportedVersionException extends InvalidConfigurationException 
{
     private static final long serialVersionUID = 1L;
 
     public UnsupportedVersionException(String message, Throwable cause) {
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 1d6d47abc33..9738ccd6c81 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
@@ -16,6 +16,8 @@
  */
 package org.apache.kafka.common.errors;
 
+import org.apache.kafka.common.InvalidRecordException;
+
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.params.ParameterizedTest;
 import org.junit.jupiter.params.provider.ValueSource;
@@ -94,4 +96,29 @@ public class TransactionExceptionHierarchyTest {
         
assertTrue(ApplicationRecoverableException.class.isAssignableFrom(exceptionClass),
                 exceptionClass.getSimpleName() + " should extend 
ApplicationRecoverableException");
     }
+
+    /**
+     * Verifies that the given exception class extends 
`InvalidConfigurationException`
+     *
+     * @param exceptionClass the exception class to check
+     */
+    @ParameterizedTest
+    @ValueSource(classes = {
+        AuthenticationException.class,
+        AuthorizationException.class,
+        ClusterAuthorizationException.class,
+        TransactionalIdAuthorizationException.class,
+        UnsupportedVersionException.class,
+        UnsupportedForMessageFormatException.class,
+        InvalidRecordException.class,
+        InvalidRequiredAcksException.class,
+        RecordBatchTooLargeException.class,
+        InvalidTopicException.class,
+        TopicAuthorizationException.class,
+        GroupAuthorizationException.class
+    })
+    void testInvalidConfigurationExceptionHierarchy(Class<? extends Exception> 
exceptionClass) {
+        
assertTrue(InvalidConfigurationException.class.isAssignableFrom(exceptionClass),
+                exceptionClass.getSimpleName() + " should extend 
InvalidConfigurationException");
+    }
 }
diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala 
b/core/src/main/scala/kafka/admin/ConfigCommand.scala
index ee1fbbca2da..b23b770558f 100644
--- a/core/src/main/scala/kafka/admin/ConfigCommand.scala
+++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala
@@ -86,13 +86,13 @@ object ConfigCommand extends Logging {
       opts.checkArgs()
       processCommand(opts)
     } catch {
-      case e @ (_: IllegalArgumentException | _: InvalidConfigurationException 
| _: OptionException) =>
-        logger.debug(s"Failed config command with args '${args.mkString(" 
")}'", e)
+      case e: UnsupportedVersionException =>
+        logger.debug(s"Unsupported API encountered in server when executing 
config command with args '${args.mkString(" ")}'")
         System.err.println(e.getMessage)
         Exit.exit(1)
 
-      case e: UnsupportedVersionException =>
-        logger.debug(s"Unsupported API encountered in server when executing 
config command with args '${args.mkString(" ")}'")
+      case e @ (_: IllegalArgumentException | _: InvalidConfigurationException 
| _: OptionException) =>
+        logger.debug(s"Failed config command with args '${args.mkString(" 
")}'", e)
         System.err.println(e.getMessage)
         Exit.exit(1)
 

Reply via email to