Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-10033

AdminClient should throw UnknownTopicOrPartitionException instead of UnknownServerException if altering configs of non-existing topic

    XMLWordPrintableJSON

Details

    • Improvement
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 2.4.0
    • 2.7.0
    • admin

    Description

      Currently, altering configs of non-existing topic leads to UnknownServerException:

      java.util.concurrent.ExecutionException: org.apache.kafka.common.errors.UnknownServerException: Topic "kgn_test" does not exist.
      	at org.apache.kafka.common.internals.KafkaFutureImpl.wrapAndThrow(KafkaFutureImpl.java:45)
      	at org.apache.kafka.common.internals.KafkaFutureImpl.access$000(KafkaFutureImpl.java:32)
      	at org.apache.kafka.common.internals.KafkaFutureImpl$SingleWaiter.await(KafkaFutureImpl.java:89)
      	at org.apache.kafka.common.internals.KafkaFutureImpl.get(KafkaFutureImpl.java:260)
      	at ru.kontur.vostok.hercules.stream.manager.kafka.KafkaManager.changeTtl(KafkaManager.java:130)
      	... 10 common frames omitted
      Caused by: org.apache.kafka.common.errors.UnknownServerException: Topic "kgn_test" does not exist.
      

      The output above is produced due to AdminZkClient.validateTopicConfig method:

        def validateTopicConfig(topic: String, configs: Properties): Unit = {
          Topic.validate(topic)
          if (!zkClient.topicExists(topic))
            throw new AdminOperationException("Topic \"%s\" does not exist.".format(topic))
          // remove the topic overrides
          LogConfig.validate(configs)
        }
      

      UnknownServerException is common exception but in this case cause is pretty clear. So this can be fixed easily by using UnknownTopicOrPartitionException.

      Attachments

        Activity

          People

            bbyrne Brian Byrne
            kgn Gregory Koshelev
            Votes:
            0 Vote for this issue
            Watchers:
            1 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: