You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2020/10/11 09:49:18 UTC

[GitHub] [kafka] kowshik opened a new pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

kowshik opened a new pull request #9409:
URL: https://github.com/apache/kafka/pull/9409


   Implement basic CLI tool for feature versioning system, providing the following:
    - `--describe`:
       - Describe supported and finalized features.
       - Usage: `./bin/kafka-run-class.sh kafka.admin.FeatureCommand --describe [--from-controller]`
       - Optionally, use the `--from-controller` option to get features from the controller.
   
    - `--upgrade-all`:
        - Upgrades all features known to the tool to their highest max version levels.
        - Usage: `./bin/kafka-run-class.sh kafka.admin.FeatureCommand --upgrade-all [--dry-run]`
        - Optionally, use the `--dry-run` CLI option to preview the feature updates without actually applying them.
   
    - `--downgrade-all`:
       -  Downgrades existing finalized features to the highest max version levels known to this tool.
       - Usage: `./bin/kafka-run-class.sh kafka.admin.FeatureCommand --upgrade-all [--dry-run]`.
       - Optionally, use the `--dry-run` CLI option to preview the feature updates without actually applying them.
   
   **Tests:**
   Added a new `FeatureCommand` integration test suite to test the CLI tool.


----------------------------------------------------------------
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.

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



[GitHub] [kafka] kowshik commented on a change in pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#discussion_r504138505



##########
File path: core/src/test/scala/unit/kafka/admin/FeatureCommandTest.scala
##########
@@ -0,0 +1,245 @@
+/**
+ * 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 kafka.admin
+
+import kafka.api.KAFKA_2_7_IV0
+import kafka.server.{BaseRequestTest, KafkaConfig, KafkaServer}
+import kafka.utils.TestUtils
+import kafka.utils.TestUtils.waitUntilTrue
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+
+import org.junit.Assert.{assertEquals, assertTrue}
+import org.junit.Test
+import org.scalatest.Assertions.intercept
+
+class FeatureCommandTest extends BaseRequestTest {
+  override def brokerCount: Int = 3
+
+  override def brokerPropertyOverrides(props: Properties): Unit = {
+    props.put(KafkaConfig.InterBrokerProtocolVersionProp, KAFKA_2_7_IV0.toString)
+  }
+
+  private def defaultSupportedFeatures(): Features[SupportedVersionRange] = {
+    Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)),
+                                           Utils.mkEntry("feature_2", new SupportedVersionRange(1, 5))))
+  }
+
+  private def updateSupportedFeatures(features: Features[SupportedVersionRange],
+                                      targetServers: Set[KafkaServer]): Unit = {
+    targetServers.foreach(s => {
+      s.brokerFeatures.setSupportedFeatures(features)
+      s.zkClient.updateBrokerInfo(s.createBrokerInfo)
+    })
+
+    // Wait until updates to all BrokerZNode supported features propagate to the controller.
+    val brokerIds = targetServers.map(s => s.config.brokerId)
+    waitUntilTrue(
+      () => servers.exists(s => {
+        if (s.kafkaController.isActive) {
+          s.kafkaController.controllerContext.liveOrShuttingDownBrokers
+            .filter(b => brokerIds.contains(b.id))
+            .forall(b => {
+              b.features.equals(features)
+            })
+        } else {
+          false
+        }
+      }),
+      "Controller did not get broker updates")
+  }
+
+  private def updateSupportedFeaturesInAllBrokers(features: Features[SupportedVersionRange]): Unit = {
+    updateSupportedFeatures(features, Set[KafkaServer]() ++ servers)
+  }
+
+  /**
+   * Tests if the FeatureApis#describeFeatures API works as expected when describing features before and
+   * after upgrading features.
+   */
+  @Test
+  def testDescribeFeaturesSuccess(): Unit = {
+    updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+    val featureApis = new FeatureApis(new FeatureCommandOptions(Array("--bootstrap-server", brokerList, "--describe", "--from-controller")))
+    featureApis.setSupportedFeatures(defaultSupportedFeatures())
+    try {
+      val initialDescribeOutput = TestUtils.grabConsoleOutput(featureApis.describeFeatures())
+      val expectedInitialDescribeOutput =
+        "Feature: feature_1\tSupportedMinVersion: 1\tSupportedMaxVersion: 3\tFinalizedMinVersionLevel: -\tFinalizedMaxVersionLevel: -\tEpoch: 0\n" +
+        "Feature: feature_2\tSupportedMinVersion: 1\tSupportedMaxVersion: 5\tFinalizedMinVersionLevel: -\tFinalizedMaxVersionLevel: -\tEpoch: 0\n"
+      assertEquals(expectedInitialDescribeOutput, initialDescribeOutput)
+      featureApis.upgradeAllFeatures()
+      val finalDescribeOutput = TestUtils.grabConsoleOutput(featureApis.describeFeatures())
+      val expectedFinalDescribeOutput =
+        "Feature: feature_1\tSupportedMinVersion: 1\tSupportedMaxVersion: 3\tFinalizedMinVersionLevel: 1\tFinalizedMaxVersionLevel: 3\tEpoch: 1\n" +
+        "Feature: feature_2\tSupportedMinVersion: 1\tSupportedMaxVersion: 5\tFinalizedMinVersionLevel: 1\tFinalizedMaxVersionLevel: 5\tEpoch: 1\n"
+      assertEquals(expectedFinalDescribeOutput, finalDescribeOutput)
+    } finally {
+      featureApis.close()
+    }
+  }
+
+  /**
+   * Tests if the FeatureApis#upgradeAllFeatures API works as expected during a success case.
+   */
+  @Test
+  def testUpgradeAllFeaturesSuccess(): Unit = {
+    val upgradeOpts = new FeatureCommandOptions(Array("--bootstrap-server", brokerList, "--upgrade-all"))
+    val featureApis = new FeatureApis(upgradeOpts)
+    try {
+      // Step (1):
+      // - Update the supported features across all brokers.
+      // - Upgrade non-existing feature_1 to maxVersionLevel: 2.
+      // - Verify results.
+      val initialSupportedFeatures = Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new SupportedVersionRange(1, 2))))
+      updateSupportedFeaturesInAllBrokers(initialSupportedFeatures)
+      featureApis.setSupportedFeatures(initialSupportedFeatures)
+      var output = TestUtils.grabConsoleOutput(featureApis.upgradeAllFeatures())
+      var expected =
+        "      [Add]\tFeature: feature_1\tExistingFinalizedMaxVersion: -\tNewFinalizedMaxVersion: 2\tResult: OK\n"
+      assertEquals(expected, output)
+
+      // Step (2):
+      // - Update the supported features across all brokers.
+      // - Upgrade existing feature_1 to maxVersionLevel: 3.
+      // - Upgrade non-existing feature_2 to maxVersionLevel: 5.
+      // - Verify results.
+      updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+      featureApis.setSupportedFeatures(defaultSupportedFeatures())
+      output = TestUtils.grabConsoleOutput(featureApis.upgradeAllFeatures())
+      expected =
+        "  [Upgrade]\tFeature: feature_1\tExistingFinalizedMaxVersion: 2\tNewFinalizedMaxVersion: 3\tResult: OK\n" +
+        "      [Add]\tFeature: feature_2\tExistingFinalizedMaxVersion: -\tNewFinalizedMaxVersion: 5\tResult: OK\n"
+      assertEquals(expected, output)
+
+      // Step (3):
+      // - Perform an upgrade of all features again.
+      // - Since supported features have not changed, expect that the above action does not yield
+      //   any results.
+      output = TestUtils.grabConsoleOutput(featureApis.upgradeAllFeatures())
+      assertTrue(output.isEmpty)
+      featureApis.setOptions(upgradeOpts)
+      output = TestUtils.grabConsoleOutput(featureApis.upgradeAllFeatures())
+      assertTrue(output.isEmpty)
+    } finally {
+      featureApis.close()
+    }
+  }
+
+  /**
+   * Tests if the FeatureApis#downgradeAllFeatures API works as expected during a success case.
+   */
+  @Test
+  def testDowngradeFeaturesSuccess(): Unit = {
+    val downgradeOpts = new FeatureCommandOptions(Array("--bootstrap-server", brokerList, "--downgrade-all"))
+    val upgradeOpts = new FeatureCommandOptions(Array("--bootstrap-server", brokerList, "--upgrade-all"))
+    val featureApis = new FeatureApis(upgradeOpts)
+    try {
+      // Step (1):
+      // - Update the supported features across all brokers.
+      // - Upgrade non-existing feature_1 to maxVersionLevel: 3.
+      // - Upgrade non-existing feature_2 to maxVersionLevel: 5.
+      updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+      featureApis.setSupportedFeatures(defaultSupportedFeatures())
+      featureApis.upgradeAllFeatures()
+
+      // Step (2):
+      // - Downgrade existing feature_1 to maxVersionLevel: 2.
+      // - Delete feature_2 since it is no longer supported by the FeatureApis object.
+      // - Verify results.
+      val downgradedFeatures = Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new SupportedVersionRange(1, 2))))
+      featureApis.setSupportedFeatures(downgradedFeatures)
+      featureApis.setOptions(downgradeOpts)
+      var output = TestUtils.grabConsoleOutput(featureApis.downgradeAllFeatures())
+      var expected =
+        "[Downgrade]\tFeature: feature_1\tExistingFinalizedMaxVersion: 3\tNewFinalizedMaxVersion: 2\tResult: OK\n" +
+        "   [Delete]\tFeature: feature_2\tExistingFinalizedMaxVersion: 5\tNewFinalizedMaxVersion: -\tResult: OK\n"
+      assertEquals(expected, output)
+
+      // Step (3):
+      // - Perform a downgrade of all features again.
+      // - Since supported features have not changed, expect that the above action does not yield
+      //   any results.
+      updateSupportedFeaturesInAllBrokers(downgradedFeatures)
+      output = TestUtils.grabConsoleOutput(featureApis.downgradeAllFeatures())
+      assertTrue(output.isEmpty)
+
+      // Step (4):
+      // - Delete feature_1 since it is no longer supported by the FeatureApis object.
+      // - Verify results.
+      featureApis.setSupportedFeatures(Features.emptySupportedFeatures())
+      output = TestUtils.grabConsoleOutput(featureApis.downgradeAllFeatures())
+      expected =
+        "   [Delete]\tFeature: feature_1\tExistingFinalizedMaxVersion: 2\tNewFinalizedMaxVersion: -\tResult: OK\n"
+      assertEquals(expected, output)
+    } finally {
+      featureApis.close()
+    }
+  }
+
+  /**
+   * Tests if the FeatureApis#upgradeAllFeatures API works as expected during a partial failure case.
+   */
+  @Test
+  def testUpgradeFeaturesFailure(): Unit = {

Review comment:
       As such the downgrade can not fail currently on the server. The reason is the following: the `FeatureCommand` downgrades `maxVersionLevel` of existing finalized features to the max versions that it knows internally (since it links against the `BrokerFeatures` library). So the only situation when the command can fail is if it tries to downgrade below supported `minVersion`. But this can never happen since we don't (yet) have the facility on the broker to have a supported `minVersion` > 1 (this can only happen during feature deprecation, the support for this is future work).
   
   On a side note, we could maybe write a test using `AdminClientUnitTestEnv` to create a mock setup specifically for the downgrade test and inject a mock admin client into the `FeatureApis` class. But that looks overkill to me...




----------------------------------------------------------------
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.

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



[GitHub] [kafka] kowshik commented on a change in pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#discussion_r503494114



##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,350 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+    features.toList.sorted.foreach {
+      case feature =>
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        print(s"Feature: $feature")
+        print(s"\tSupportedMinVersion: $supportedMinVersion")
+        print(s"\tSupportedMaxVersion: $supportedMaxVersion")
+        print(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        print(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+        println(s"\tEpoch: $epoch")
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Downgrades existing finalized features to the highest max version levels known to this tool.
+   * The method may delete existing finalized features if they are no longer seen to be supported,
+   * but it does not add a feature that was not finalized previously. The results of the feature
+   * updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def downgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val supportedFeaturesMap = supportedFeatures.features
+    val updates = existingFinalizedFeatures.asScala.map {
+      case (feature, existingVersionRange) =>
+        val targetVersionRange = supportedFeaturesMap.get(feature)
+        if (targetVersionRange == null) {
+          val updateStr =
+            deleteOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+            s"\tNewFinalizedMaxVersion: -"
+          (feature, (updateStr, new FeatureUpdate(0, true)))
+        } else {
+          if (targetVersionRange.max < existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              downgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, true)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Applies the provided feature updates. If the --dry-run CLI option is provided, the method
+   * only prints the expected feature updates to STDOUT without applying them.
+   *
+   * @param updates the feature updates to be applied via the admin client
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = {
+    if (opts.hasDryRunOption) {
+      println("Expected feature updates:")
+      println(ListMap(updates.toSeq.sortBy(_._1):_*)
+                .map { case(_, (updateStr, _)) => updateStr}
+                .mkString("\n"))
+    } else {
+      val result = adminClient.updateFeatures(
+        updates.map { case(feature, (_, update)) => (feature, update)}.asJava,
+        new UpdateFeaturesOptions())
+      val failures = ListMap(result.values.asScala.toSeq.sortBy(_._1):_*).map {
+        case (feature, updateFuture) =>
+          val (updateStr, _) = updates(feature)
+          try {
+            updateFuture.get
+            println(updateStr + "\tResult: OK")
+            0
+          } catch {
+            case e: Exception =>
+              println(updateStr + "\tResult: FAILED due to " + e.getMessage)
+              1
+          }
+      }.sum
+      if (failures > 0) {
+        throw new UpdateFeaturesException(s"$failures feature updates failed!")
+      }
+    }
+  }
+
+  def execute(): Unit = {
+    if (opts.hasDescribeOption) {
+      describeFeatures()
+    } else if (opts.hasUpgradeAllOption) {
+      upgradeAllFeatures()
+    } else if (opts.hasDowngradeAllOption) {
+      downgradeAllFeatures()
+    } else {
+      throw new IllegalStateException("Unexpected state: no CLI command could be executed.")
+    }
+  }
+
+  def close(): Unit = {
+    adminClient.close()
+  }
+
+  private def createAdminClient(): Admin = {
+    val props = new Properties()
+    props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.bootstrapServers)
+    Admin.create(props)
+  }
+}
+
+class FeatureCommandOptions(args: Array[String]) extends CommandDefaultOptions(args) {
+  private val bootstrapServerOpt =
+    parser.accepts("bootstrap-server", "REQUIRED: The Kafka server(s) to connect to.")
+      .withRequiredArg
+      .describedAs("server to connect to")
+      .ofType(classOf[String])
+  private val describeOpt = parser.accepts(
+    "describe",
+    "Describe supported and finalized features. By default, the features are described from a" +
+    " random broker. The request can be optionally directed only to the controller using the" +
+    " --from-controller option.")
+  private val fromControllerOpt = parser.accepts(
+    "from-controller",
+    "Describe supported and finalized features from the controller.")
+  private val upgradeAllOpt = parser.accepts(
+    "upgrade-all",
+    "Upgrades all finalized features to the maximum version levels known to the tool." +
+    " This command finalizes new features known to the tool that were never finalized" +
+    " previously in the cluster, but it is guaranteed to not delete any existing feature.")
+  private val downgradeAllOpt = parser.accepts(
+    "downgrade-all",
+    "Downgrades all finalized features to the maximum version levels known to the tool." +
+    " This command deletes unknown features from the list of finalized features in the" +
+    " cluster, but it is guaranteed to not add a new feature.")
+  private val dryRunOpt = parser.accepts(
+    "dry-run",
+    "Performs a dry-run of upgrade/downgrade mutations to finalized feature without applying them.")
+
+  options = parser.parse(args : _*)
+
+  checkArgs()
+
+  def has(builder: OptionSpec[_]): Boolean = options.has(builder)
+
+  def hasDescribeOption: Boolean = has(describeOpt)
+
+  def hasFromControllerOption: Boolean = has(fromControllerOpt)
+
+  def hasDryRunOption: Boolean = has(dryRunOpt)
+
+  def hasUpgradeAllOption: Boolean = has(upgradeAllOpt)

Review comment:
       @abbccdda How do you envision the proposed `--action` flag for the advanced CLI scenarios where we would want to allow for `--upgrade`, `--downgrade` and `--delete` options as part of the same CLI command? Please [see this section](https://cwiki.apache.org/confluence/display/KAFKA/KIP-584%3A+Versioning+scheme+for+features#KIP584:Versioningschemeforfeatures-AdvancedCLItoolusage) of the KIP. As such, the existing technique allows us to support the advanced CLI facilities in the future.
   




----------------------------------------------------------------
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.

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



[GitHub] [kafka] abbccdda commented on a change in pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
abbccdda commented on a change in pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#discussion_r503443796



##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,350 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))

Review comment:
       Is the `\n` necessary?

##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,350 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+    features.toList.sorted.foreach {
+      case feature =>
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        print(s"Feature: $feature")
+        print(s"\tSupportedMinVersion: $supportedMinVersion")
+        print(s"\tSupportedMaxVersion: $supportedMaxVersion")
+        print(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")

Review comment:
       nit: we could do the comment immediately after the version get extracted, just like moving L132 to be after L113

##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,350 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+    features.toList.sorted.foreach {
+      case feature =>

Review comment:
       nit: case not necessary

##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,350 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+    features.toList.sorted.foreach {

Review comment:
       Why do we need to sort the features?

##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,350 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+    features.toList.sorted.foreach {
+      case feature =>
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        print(s"Feature: $feature")
+        print(s"\tSupportedMinVersion: $supportedMinVersion")
+        print(s"\tSupportedMaxVersion: $supportedMaxVersion")
+        print(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        print(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+        println(s"\tEpoch: $epoch")
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Downgrades existing finalized features to the highest max version levels known to this tool.
+   * The method may delete existing finalized features if they are no longer seen to be supported,
+   * but it does not add a feature that was not finalized previously. The results of the feature
+   * updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def downgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val supportedFeaturesMap = supportedFeatures.features
+    val updates = existingFinalizedFeatures.asScala.map {
+      case (feature, existingVersionRange) =>
+        val targetVersionRange = supportedFeaturesMap.get(feature)
+        if (targetVersionRange == null) {
+          val updateStr =
+            deleteOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+            s"\tNewFinalizedMaxVersion: -"
+          (feature, (updateStr, new FeatureUpdate(0, true)))
+        } else {
+          if (targetVersionRange.max < existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              downgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, true)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Applies the provided feature updates. If the --dry-run CLI option is provided, the method
+   * only prints the expected feature updates to STDOUT without applying them.
+   *
+   * @param updates the feature updates to be applied via the admin client
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = {
+    if (opts.hasDryRunOption) {
+      println("Expected feature updates:")
+      println(ListMap(updates.toSeq.sortBy(_._1):_*)
+                .map { case(_, (updateStr, _)) => updateStr}
+                .mkString("\n"))
+    } else {
+      val result = adminClient.updateFeatures(
+        updates.map { case(feature, (_, update)) => (feature, update)}.asJava,
+        new UpdateFeaturesOptions())
+      val failures = ListMap(result.values.asScala.toSeq.sortBy(_._1):_*).map {
+        case (feature, updateFuture) =>
+          val (updateStr, _) = updates(feature)
+          try {
+            updateFuture.get
+            println(updateStr + "\tResult: OK")
+            0
+          } catch {
+            case e: Exception =>
+              println(updateStr + "\tResult: FAILED due to " + e.getMessage)
+              1
+          }
+      }.sum
+      if (failures > 0) {
+        throw new UpdateFeaturesException(s"$failures feature updates failed!")
+      }
+    }
+  }
+
+  def execute(): Unit = {
+    if (opts.hasDescribeOption) {
+      describeFeatures()
+    } else if (opts.hasUpgradeAllOption) {
+      upgradeAllFeatures()
+    } else if (opts.hasDowngradeAllOption) {
+      downgradeAllFeatures()
+    } else {
+      throw new IllegalStateException("Unexpected state: no CLI command could be executed.")
+    }
+  }
+
+  def close(): Unit = {
+    adminClient.close()
+  }
+
+  private def createAdminClient(): Admin = {
+    val props = new Properties()
+    props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.bootstrapServers)
+    Admin.create(props)
+  }
+}
+
+class FeatureCommandOptions(args: Array[String]) extends CommandDefaultOptions(args) {
+  private val bootstrapServerOpt =
+    parser.accepts("bootstrap-server", "REQUIRED: The Kafka server(s) to connect to.")
+      .withRequiredArg
+      .describedAs("server to connect to")
+      .ofType(classOf[String])
+  private val describeOpt = parser.accepts(
+    "describe",
+    "Describe supported and finalized features. By default, the features are described from a" +
+    " random broker. The request can be optionally directed only to the controller using the" +
+    " --from-controller option.")
+  private val fromControllerOpt = parser.accepts(
+    "from-controller",
+    "Describe supported and finalized features from the controller.")
+  private val upgradeAllOpt = parser.accepts(
+    "upgrade-all",
+    "Upgrades all finalized features to the maximum version levels known to the tool." +
+    " This command finalizes new features known to the tool that were never finalized" +
+    " previously in the cluster, but it is guaranteed to not delete any existing feature.")
+  private val downgradeAllOpt = parser.accepts(
+    "downgrade-all",
+    "Downgrades all finalized features to the maximum version levels known to the tool." +
+    " This command deletes unknown features from the list of finalized features in the" +
+    " cluster, but it is guaranteed to not add a new feature.")
+  private val dryRunOpt = parser.accepts(
+    "dry-run",
+    "Performs a dry-run of upgrade/downgrade mutations to finalized feature without applying them.")
+
+  options = parser.parse(args : _*)
+
+  checkArgs()
+
+  def has(builder: OptionSpec[_]): Boolean = options.has(builder)
+
+  def hasDescribeOption: Boolean = has(describeOpt)
+
+  def hasFromControllerOption: Boolean = has(fromControllerOpt)
+
+  def hasDryRunOption: Boolean = has(dryRunOpt)
+
+  def hasUpgradeAllOption: Boolean = has(upgradeAllOpt)
+
+  def hasDowngradeAllOption: Boolean = has(downgradeAllOpt)
+
+  def bootstrapServers: String = options.valueOf(bootstrapServerOpt)
+
+  def checkArgs(): Unit = {
+    CommandLineUtils.printHelpAndExitIfNeeded(this, "This tool describes and updates finalized features.")
+    val numActions = Seq(describeOpt, upgradeAllOpt, downgradeAllOpt).count(has)
+    if (numActions != 1) {
+      CommandLineUtils.printUsageAndDie(
+        parser,
+        "Command must include exactly one action: --describe, --upgrade-all, --downgrade-all.")
+    }
+    CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServerOpt)
+    if (hasDryRunOption && !hasUpgradeAllOption && !hasDowngradeAllOption) {
+      CommandLineUtils.printUsageAndDie(
+        parser,
+        "Command can contain --dry-run option only when either --upgrade-all or --downgrade-all actions are provided.")
+    }
+    if (hasFromControllerOption && !hasDescribeOption) {

Review comment:
       Could we just log a warning in this case instead of failing?

##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,350 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+    features.toList.sorted.foreach {
+      case feature =>
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        print(s"Feature: $feature")
+        print(s"\tSupportedMinVersion: $supportedMinVersion")
+        print(s"\tSupportedMaxVersion: $supportedMaxVersion")
+        print(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        print(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+        println(s"\tEpoch: $epoch")
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Downgrades existing finalized features to the highest max version levels known to this tool.
+   * The method may delete existing finalized features if they are no longer seen to be supported,
+   * but it does not add a feature that was not finalized previously. The results of the feature
+   * updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def downgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val supportedFeaturesMap = supportedFeatures.features
+    val updates = existingFinalizedFeatures.asScala.map {
+      case (feature, existingVersionRange) =>
+        val targetVersionRange = supportedFeaturesMap.get(feature)
+        if (targetVersionRange == null) {
+          val updateStr =
+            deleteOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+            s"\tNewFinalizedMaxVersion: -"
+          (feature, (updateStr, new FeatureUpdate(0, true)))
+        } else {
+          if (targetVersionRange.max < existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              downgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, true)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Applies the provided feature updates. If the --dry-run CLI option is provided, the method
+   * only prints the expected feature updates to STDOUT without applying them.
+   *
+   * @param updates the feature updates to be applied via the admin client
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = {
+    if (opts.hasDryRunOption) {
+      println("Expected feature updates:")
+      println(ListMap(updates.toSeq.sortBy(_._1):_*)
+                .map { case(_, (updateStr, _)) => updateStr}
+                .mkString("\n"))
+    } else {
+      val result = adminClient.updateFeatures(
+        updates.map { case(feature, (_, update)) => (feature, update)}.asJava,
+        new UpdateFeaturesOptions())
+      val failures = ListMap(result.values.asScala.toSeq.sortBy(_._1):_*).map {
+        case (feature, updateFuture) =>
+          val (updateStr, _) = updates(feature)
+          try {
+            updateFuture.get
+            println(updateStr + "\tResult: OK")
+            0
+          } catch {
+            case e: Exception =>
+              println(updateStr + "\tResult: FAILED due to " + e.getMessage)
+              1
+          }
+      }.sum
+      if (failures > 0) {
+        throw new UpdateFeaturesException(s"$failures feature updates failed!")
+      }
+    }
+  }
+
+  def execute(): Unit = {
+    if (opts.hasDescribeOption) {
+      describeFeatures()
+    } else if (opts.hasUpgradeAllOption) {
+      upgradeAllFeatures()
+    } else if (opts.hasDowngradeAllOption) {
+      downgradeAllFeatures()
+    } else {
+      throw new IllegalStateException("Unexpected state: no CLI command could be executed.")
+    }
+  }
+
+  def close(): Unit = {
+    adminClient.close()
+  }
+
+  private def createAdminClient(): Admin = {
+    val props = new Properties()
+    props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.bootstrapServers)
+    Admin.create(props)
+  }
+}
+
+class FeatureCommandOptions(args: Array[String]) extends CommandDefaultOptions(args) {
+  private val bootstrapServerOpt =
+    parser.accepts("bootstrap-server", "REQUIRED: The Kafka server(s) to connect to.")
+      .withRequiredArg
+      .describedAs("server to connect to")
+      .ofType(classOf[String])
+  private val describeOpt = parser.accepts(
+    "describe",
+    "Describe supported and finalized features. By default, the features are described from a" +
+    " random broker. The request can be optionally directed only to the controller using the" +
+    " --from-controller option.")
+  private val fromControllerOpt = parser.accepts(
+    "from-controller",
+    "Describe supported and finalized features from the controller.")
+  private val upgradeAllOpt = parser.accepts(
+    "upgrade-all",
+    "Upgrades all finalized features to the maximum version levels known to the tool." +
+    " This command finalizes new features known to the tool that were never finalized" +
+    " previously in the cluster, but it is guaranteed to not delete any existing feature.")
+  private val downgradeAllOpt = parser.accepts(
+    "downgrade-all",
+    "Downgrades all finalized features to the maximum version levels known to the tool." +
+    " This command deletes unknown features from the list of finalized features in the" +
+    " cluster, but it is guaranteed to not add a new feature.")
+  private val dryRunOpt = parser.accepts(
+    "dry-run",
+    "Performs a dry-run of upgrade/downgrade mutations to finalized feature without applying them.")
+
+  options = parser.parse(args : _*)
+
+  checkArgs()
+
+  def has(builder: OptionSpec[_]): Boolean = options.has(builder)
+
+  def hasDescribeOption: Boolean = has(describeOpt)
+
+  def hasFromControllerOption: Boolean = has(fromControllerOpt)
+
+  def hasDryRunOption: Boolean = has(dryRunOpt)
+
+  def hasUpgradeAllOption: Boolean = has(upgradeAllOpt)

Review comment:
       A random thought I have is that since describe/upgrade/downgrade are mutually-exclusive, we could define one flag like `--action` and pass in the different options as strings to translate the an enum matching scenario.

##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,350 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+    features.toList.sorted.foreach {
+      case feature =>
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        print(s"Feature: $feature")
+        print(s"\tSupportedMinVersion: $supportedMinVersion")
+        print(s"\tSupportedMaxVersion: $supportedMaxVersion")
+        print(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        print(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+        println(s"\tEpoch: $epoch")
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Downgrades existing finalized features to the highest max version levels known to this tool.
+   * The method may delete existing finalized features if they are no longer seen to be supported,
+   * but it does not add a feature that was not finalized previously. The results of the feature
+   * updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def downgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val supportedFeaturesMap = supportedFeatures.features
+    val updates = existingFinalizedFeatures.asScala.map {
+      case (feature, existingVersionRange) =>
+        val targetVersionRange = supportedFeaturesMap.get(feature)
+        if (targetVersionRange == null) {
+          val updateStr =
+            deleteOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+            s"\tNewFinalizedMaxVersion: -"
+          (feature, (updateStr, new FeatureUpdate(0, true)))
+        } else {
+          if (targetVersionRange.max < existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              downgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, true)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Applies the provided feature updates. If the --dry-run CLI option is provided, the method
+   * only prints the expected feature updates to STDOUT without applying them.
+   *
+   * @param updates the feature updates to be applied via the admin client
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = {
+    if (opts.hasDryRunOption) {
+      println("Expected feature updates:")
+      println(ListMap(updates.toSeq.sortBy(_._1):_*)
+                .map { case(_, (updateStr, _)) => updateStr}
+                .mkString("\n"))
+    } else {
+      val result = adminClient.updateFeatures(
+        updates.map { case(feature, (_, update)) => (feature, update)}.asJava,
+        new UpdateFeaturesOptions())
+      val failures = ListMap(result.values.asScala.toSeq.sortBy(_._1):_*).map {
+        case (feature, updateFuture) =>
+          val (updateStr, _) = updates(feature)
+          try {
+            updateFuture.get
+            println(updateStr + "\tResult: OK")
+            0
+          } catch {
+            case e: Exception =>
+              println(updateStr + "\tResult: FAILED due to " + e.getMessage)
+              1
+          }
+      }.sum
+      if (failures > 0) {
+        throw new UpdateFeaturesException(s"$failures feature updates failed!")
+      }
+    }
+  }
+
+  def execute(): Unit = {
+    if (opts.hasDescribeOption) {
+      describeFeatures()
+    } else if (opts.hasUpgradeAllOption) {
+      upgradeAllFeatures()
+    } else if (opts.hasDowngradeAllOption) {
+      downgradeAllFeatures()
+    } else {
+      throw new IllegalStateException("Unexpected state: no CLI command could be executed.")
+    }
+  }
+
+  def close(): Unit = {
+    adminClient.close()
+  }
+
+  private def createAdminClient(): Admin = {
+    val props = new Properties()
+    props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.bootstrapServers)
+    Admin.create(props)
+  }
+}
+
+class FeatureCommandOptions(args: Array[String]) extends CommandDefaultOptions(args) {
+  private val bootstrapServerOpt =
+    parser.accepts("bootstrap-server", "REQUIRED: The Kafka server(s) to connect to.")
+      .withRequiredArg
+      .describedAs("server to connect to")
+      .ofType(classOf[String])
+  private val describeOpt = parser.accepts(
+    "describe",
+    "Describe supported and finalized features. By default, the features are described from a" +
+    " random broker. The request can be optionally directed only to the controller using the" +
+    " --from-controller option.")
+  private val fromControllerOpt = parser.accepts(
+    "from-controller",
+    "Describe supported and finalized features from the controller.")
+  private val upgradeAllOpt = parser.accepts(
+    "upgrade-all",
+    "Upgrades all finalized features to the maximum version levels known to the tool." +
+    " This command finalizes new features known to the tool that were never finalized" +
+    " previously in the cluster, but it is guaranteed to not delete any existing feature.")
+  private val downgradeAllOpt = parser.accepts(
+    "downgrade-all",
+    "Downgrades all finalized features to the maximum version levels known to the tool." +
+    " This command deletes unknown features from the list of finalized features in the" +
+    " cluster, but it is guaranteed to not add a new feature.")
+  private val dryRunOpt = parser.accepts(
+    "dry-run",
+    "Performs a dry-run of upgrade/downgrade mutations to finalized feature without applying them.")
+
+  options = parser.parse(args : _*)
+
+  checkArgs()
+
+  def has(builder: OptionSpec[_]): Boolean = options.has(builder)
+
+  def hasDescribeOption: Boolean = has(describeOpt)

Review comment:
       I feel we don't need the `has` prefix for all the options, which seem not matching with the conventions by looking at examples as `CofnigCommand.scala`

##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,350 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+    features.toList.sorted.foreach {
+      case feature =>
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        print(s"Feature: $feature")
+        print(s"\tSupportedMinVersion: $supportedMinVersion")
+        print(s"\tSupportedMaxVersion: $supportedMaxVersion")
+        print(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        print(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+        println(s"\tEpoch: $epoch")
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Downgrades existing finalized features to the highest max version levels known to this tool.
+   * The method may delete existing finalized features if they are no longer seen to be supported,
+   * but it does not add a feature that was not finalized previously. The results of the feature
+   * updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def downgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val supportedFeaturesMap = supportedFeatures.features
+    val updates = existingFinalizedFeatures.asScala.map {
+      case (feature, existingVersionRange) =>
+        val targetVersionRange = supportedFeaturesMap.get(feature)
+        if (targetVersionRange == null) {
+          val updateStr =
+            deleteOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+            s"\tNewFinalizedMaxVersion: -"
+          (feature, (updateStr, new FeatureUpdate(0, true)))
+        } else {
+          if (targetVersionRange.max < existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              downgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, true)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Applies the provided feature updates. If the --dry-run CLI option is provided, the method
+   * only prints the expected feature updates to STDOUT without applying them.
+   *
+   * @param updates the feature updates to be applied via the admin client
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = {
+    if (opts.hasDryRunOption) {
+      println("Expected feature updates:")
+      println(ListMap(updates.toSeq.sortBy(_._1):_*)
+                .map { case(_, (updateStr, _)) => updateStr}
+                .mkString("\n"))
+    } else {
+      val result = adminClient.updateFeatures(
+        updates.map { case(feature, (_, update)) => (feature, update)}.asJava,
+        new UpdateFeaturesOptions())
+      val failures = ListMap(result.values.asScala.toSeq.sortBy(_._1):_*).map {
+        case (feature, updateFuture) =>
+          val (updateStr, _) = updates(feature)
+          try {
+            updateFuture.get
+            println(updateStr + "\tResult: OK")
+            0
+          } catch {
+            case e: Exception =>
+              println(updateStr + "\tResult: FAILED due to " + e.getMessage)
+              1
+          }
+      }.sum
+      if (failures > 0) {

Review comment:
       I'm not sure the exception thrown here is necessary since we already printed out the result, is this just for testing purpose?

##########
File path: core/src/test/scala/unit/kafka/admin/FeatureCommandTest.scala
##########
@@ -0,0 +1,245 @@
+/**
+ * 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 kafka.admin
+
+import kafka.api.KAFKA_2_7_IV0
+import kafka.server.{BaseRequestTest, KafkaConfig, KafkaServer}
+import kafka.utils.TestUtils
+import kafka.utils.TestUtils.waitUntilTrue
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+
+import org.junit.Assert.{assertEquals, assertTrue}
+import org.junit.Test
+import org.scalatest.Assertions.intercept
+
+class FeatureCommandTest extends BaseRequestTest {
+  override def brokerCount: Int = 3
+
+  override def brokerPropertyOverrides(props: Properties): Unit = {
+    props.put(KafkaConfig.InterBrokerProtocolVersionProp, KAFKA_2_7_IV0.toString)
+  }
+
+  private def defaultSupportedFeatures(): Features[SupportedVersionRange] = {
+    Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)),
+                                           Utils.mkEntry("feature_2", new SupportedVersionRange(1, 5))))
+  }
+
+  private def updateSupportedFeatures(features: Features[SupportedVersionRange],
+                                      targetServers: Set[KafkaServer]): Unit = {
+    targetServers.foreach(s => {
+      s.brokerFeatures.setSupportedFeatures(features)
+      s.zkClient.updateBrokerInfo(s.createBrokerInfo)
+    })
+
+    // Wait until updates to all BrokerZNode supported features propagate to the controller.
+    val brokerIds = targetServers.map(s => s.config.brokerId)
+    waitUntilTrue(
+      () => servers.exists(s => {
+        if (s.kafkaController.isActive) {
+          s.kafkaController.controllerContext.liveOrShuttingDownBrokers
+            .filter(b => brokerIds.contains(b.id))
+            .forall(b => {
+              b.features.equals(features)
+            })
+        } else {
+          false
+        }
+      }),
+      "Controller did not get broker updates")
+  }
+
+  private def updateSupportedFeaturesInAllBrokers(features: Features[SupportedVersionRange]): Unit = {
+    updateSupportedFeatures(features, Set[KafkaServer]() ++ servers)
+  }
+
+  /**
+   * Tests if the FeatureApis#describeFeatures API works as expected when describing features before and
+   * after upgrading features.
+   */
+  @Test
+  def testDescribeFeaturesSuccess(): Unit = {
+    updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+    val featureApis = new FeatureApis(new FeatureCommandOptions(Array("--bootstrap-server", brokerList, "--describe", "--from-controller")))
+    featureApis.setSupportedFeatures(defaultSupportedFeatures())
+    try {
+      val initialDescribeOutput = TestUtils.grabConsoleOutput(featureApis.describeFeatures())
+      val expectedInitialDescribeOutput =
+        "Feature: feature_1\tSupportedMinVersion: 1\tSupportedMaxVersion: 3\tFinalizedMinVersionLevel: -\tFinalizedMaxVersionLevel: -\tEpoch: 0\n" +
+        "Feature: feature_2\tSupportedMinVersion: 1\tSupportedMaxVersion: 5\tFinalizedMinVersionLevel: -\tFinalizedMaxVersionLevel: -\tEpoch: 0\n"
+      assertEquals(expectedInitialDescribeOutput, initialDescribeOutput)
+      featureApis.upgradeAllFeatures()
+      val finalDescribeOutput = TestUtils.grabConsoleOutput(featureApis.describeFeatures())
+      val expectedFinalDescribeOutput =
+        "Feature: feature_1\tSupportedMinVersion: 1\tSupportedMaxVersion: 3\tFinalizedMinVersionLevel: 1\tFinalizedMaxVersionLevel: 3\tEpoch: 1\n" +
+        "Feature: feature_2\tSupportedMinVersion: 1\tSupportedMaxVersion: 5\tFinalizedMinVersionLevel: 1\tFinalizedMaxVersionLevel: 5\tEpoch: 1\n"
+      assertEquals(expectedFinalDescribeOutput, finalDescribeOutput)
+    } finally {
+      featureApis.close()
+    }
+  }
+
+  /**
+   * Tests if the FeatureApis#upgradeAllFeatures API works as expected during a success case.
+   */
+  @Test
+  def testUpgradeAllFeaturesSuccess(): Unit = {
+    val upgradeOpts = new FeatureCommandOptions(Array("--bootstrap-server", brokerList, "--upgrade-all"))
+    val featureApis = new FeatureApis(upgradeOpts)
+    try {
+      // Step (1):
+      // - Update the supported features across all brokers.
+      // - Upgrade non-existing feature_1 to maxVersionLevel: 2.
+      // - Verify results.
+      val initialSupportedFeatures = Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new SupportedVersionRange(1, 2))))
+      updateSupportedFeaturesInAllBrokers(initialSupportedFeatures)
+      featureApis.setSupportedFeatures(initialSupportedFeatures)
+      var output = TestUtils.grabConsoleOutput(featureApis.upgradeAllFeatures())
+      var expected =
+        "      [Add]\tFeature: feature_1\tExistingFinalizedMaxVersion: -\tNewFinalizedMaxVersion: 2\tResult: OK\n"
+      assertEquals(expected, output)
+
+      // Step (2):
+      // - Update the supported features across all brokers.
+      // - Upgrade existing feature_1 to maxVersionLevel: 3.
+      // - Upgrade non-existing feature_2 to maxVersionLevel: 5.
+      // - Verify results.
+      updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+      featureApis.setSupportedFeatures(defaultSupportedFeatures())
+      output = TestUtils.grabConsoleOutput(featureApis.upgradeAllFeatures())
+      expected =
+        "  [Upgrade]\tFeature: feature_1\tExistingFinalizedMaxVersion: 2\tNewFinalizedMaxVersion: 3\tResult: OK\n" +
+        "      [Add]\tFeature: feature_2\tExistingFinalizedMaxVersion: -\tNewFinalizedMaxVersion: 5\tResult: OK\n"
+      assertEquals(expected, output)
+
+      // Step (3):
+      // - Perform an upgrade of all features again.
+      // - Since supported features have not changed, expect that the above action does not yield
+      //   any results.
+      output = TestUtils.grabConsoleOutput(featureApis.upgradeAllFeatures())
+      assertTrue(output.isEmpty)
+      featureApis.setOptions(upgradeOpts)
+      output = TestUtils.grabConsoleOutput(featureApis.upgradeAllFeatures())
+      assertTrue(output.isEmpty)
+    } finally {
+      featureApis.close()
+    }
+  }
+
+  /**
+   * Tests if the FeatureApis#downgradeAllFeatures API works as expected during a success case.
+   */
+  @Test
+  def testDowngradeFeaturesSuccess(): Unit = {
+    val downgradeOpts = new FeatureCommandOptions(Array("--bootstrap-server", brokerList, "--downgrade-all"))
+    val upgradeOpts = new FeatureCommandOptions(Array("--bootstrap-server", brokerList, "--upgrade-all"))
+    val featureApis = new FeatureApis(upgradeOpts)
+    try {
+      // Step (1):
+      // - Update the supported features across all brokers.
+      // - Upgrade non-existing feature_1 to maxVersionLevel: 3.
+      // - Upgrade non-existing feature_2 to maxVersionLevel: 5.
+      updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+      featureApis.setSupportedFeatures(defaultSupportedFeatures())
+      featureApis.upgradeAllFeatures()
+
+      // Step (2):
+      // - Downgrade existing feature_1 to maxVersionLevel: 2.
+      // - Delete feature_2 since it is no longer supported by the FeatureApis object.
+      // - Verify results.
+      val downgradedFeatures = Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new SupportedVersionRange(1, 2))))
+      featureApis.setSupportedFeatures(downgradedFeatures)
+      featureApis.setOptions(downgradeOpts)
+      var output = TestUtils.grabConsoleOutput(featureApis.downgradeAllFeatures())
+      var expected =
+        "[Downgrade]\tFeature: feature_1\tExistingFinalizedMaxVersion: 3\tNewFinalizedMaxVersion: 2\tResult: OK\n" +
+        "   [Delete]\tFeature: feature_2\tExistingFinalizedMaxVersion: 5\tNewFinalizedMaxVersion: -\tResult: OK\n"
+      assertEquals(expected, output)
+
+      // Step (3):
+      // - Perform a downgrade of all features again.
+      // - Since supported features have not changed, expect that the above action does not yield
+      //   any results.
+      updateSupportedFeaturesInAllBrokers(downgradedFeatures)
+      output = TestUtils.grabConsoleOutput(featureApis.downgradeAllFeatures())
+      assertTrue(output.isEmpty)
+
+      // Step (4):
+      // - Delete feature_1 since it is no longer supported by the FeatureApis object.
+      // - Verify results.
+      featureApis.setSupportedFeatures(Features.emptySupportedFeatures())
+      output = TestUtils.grabConsoleOutput(featureApis.downgradeAllFeatures())
+      expected =
+        "   [Delete]\tFeature: feature_1\tExistingFinalizedMaxVersion: 2\tNewFinalizedMaxVersion: -\tResult: OK\n"
+      assertEquals(expected, output)
+    } finally {
+      featureApis.close()
+    }
+  }
+
+  /**
+   * Tests if the FeatureApis#upgradeAllFeatures API works as expected during a partial failure case.
+   */
+  @Test
+  def testUpgradeFeaturesFailure(): Unit = {

Review comment:
       Do we need a downgrade failure test as well?
   




----------------------------------------------------------------
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.

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



[GitHub] [kafka] kowshik commented on pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
kowshik commented on pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#issuecomment-708609368


   @abbccdda Thanks for the review! I've addressed the most recent comment.


----------------------------------------------------------------
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.

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



[GitHub] [kafka] kowshik commented on a change in pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#discussion_r504113561



##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,350 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))

Review comment:
       Yeah, it provides a separation between the regular o/p and the error.




----------------------------------------------------------------
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.

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



[GitHub] [kafka] kowshik commented on a change in pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#discussion_r504118379



##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,350 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+    features.toList.sorted.foreach {
+      case feature =>

Review comment:
       Done.




----------------------------------------------------------------
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.

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



[GitHub] [kafka] kowshik commented on a change in pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#discussion_r504915393



##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,359 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case _: UpdateFeaturesException =>
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {

Review comment:
       Done




----------------------------------------------------------------
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.

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



[GitHub] [kafka] junrao merged pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
junrao merged pull request #9409:
URL: https://github.com/apache/kafka/pull/9409


   


----------------------------------------------------------------
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.

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



[GitHub] [kafka] kowshik commented on pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
kowshik commented on pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#issuecomment-710503810


   @junrao Thank you for the review! I've address the comments and the PR is ready for review again.


----------------------------------------------------------------
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.

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



[GitHub] [kafka] kowshik commented on a change in pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#discussion_r506681887



##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,359 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case _: UpdateFeaturesException =>
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  private[admin] def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  private[admin] def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+
+    features.toList.sorted.foreach {
+      feature =>
+        val output = new StringBuilder()
+        output.append(s"Feature: $feature")
+
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        output.append(s"\tSupportedMinVersion: $supportedMinVersion")
+        output.append(s"\tSupportedMaxVersion: $supportedMaxVersion")
+
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        output.append(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        output.append(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        output.append(s"\tEpoch: $epoch")
+
+        println(output)
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Downgrades existing finalized features to the highest max version levels known to this tool.
+   * The method may delete existing finalized features if they are no longer seen to be supported,
+   * but it does not add a feature that was not finalized previously. The results of the feature
+   * updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def downgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val supportedFeaturesMap = supportedFeatures.features
+    val updates = existingFinalizedFeatures.asScala.map {
+      case (feature, existingVersionRange) =>
+        val targetVersionRange = supportedFeaturesMap.get(feature)
+        if (targetVersionRange == null) {
+          val updateStr =
+            deleteOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+            s"\tNewFinalizedMaxVersion: -"
+          (feature, (updateStr, new FeatureUpdate(0, true)))
+        } else {
+          if (targetVersionRange.max < existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              downgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, true)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Applies the provided feature updates. If the --dry-run CLI option is provided, the method
+   * only prints the expected feature updates to STDOUT without applying them.
+   *
+   * @param updates the feature updates to be applied via the admin client
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = {
+    if (opts.hasDryRunOption) {
+      println("Expected feature updates:")
+      println(ListMap(updates.toSeq.sortBy(_._1):_*)
+                .map { case(_, (updateStr, _)) => updateStr}
+                .mkString("\n"))
+    } else {
+      val result = adminClient.updateFeatures(
+        updates.map { case(feature, (_, update)) => (feature, update)}.asJava,
+        new UpdateFeaturesOptions())
+      val failures = ListMap(result.values.asScala.toSeq.sortBy(_._1):_*).map {
+        case (feature, updateFuture) =>
+          val (updateStr, _) = updates(feature)
+          try {
+            updateFuture.get
+            println(updateStr + "\tResult: OK")
+            0
+          } catch {
+            case e: Exception =>
+              println(updateStr + "\tResult: FAILED due to " + e.getMessage)

Review comment:
       Done. I'm printing the exception `toString` output now, instead of `getMessage`.




----------------------------------------------------------------
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.

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



[GitHub] [kafka] kowshik commented on a change in pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#discussion_r504296681



##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,350 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+    features.toList.sorted.foreach {
+      case feature =>
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        print(s"Feature: $feature")
+        print(s"\tSupportedMinVersion: $supportedMinVersion")
+        print(s"\tSupportedMaxVersion: $supportedMaxVersion")
+        print(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        print(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+        println(s"\tEpoch: $epoch")
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Downgrades existing finalized features to the highest max version levels known to this tool.
+   * The method may delete existing finalized features if they are no longer seen to be supported,
+   * but it does not add a feature that was not finalized previously. The results of the feature
+   * updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def downgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val supportedFeaturesMap = supportedFeatures.features
+    val updates = existingFinalizedFeatures.asScala.map {
+      case (feature, existingVersionRange) =>
+        val targetVersionRange = supportedFeaturesMap.get(feature)
+        if (targetVersionRange == null) {
+          val updateStr =
+            deleteOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+            s"\tNewFinalizedMaxVersion: -"
+          (feature, (updateStr, new FeatureUpdate(0, true)))
+        } else {
+          if (targetVersionRange.max < existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              downgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, true)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Applies the provided feature updates. If the --dry-run CLI option is provided, the method
+   * only prints the expected feature updates to STDOUT without applying them.
+   *
+   * @param updates the feature updates to be applied via the admin client
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = {
+    if (opts.hasDryRunOption) {
+      println("Expected feature updates:")
+      println(ListMap(updates.toSeq.sortBy(_._1):_*)
+                .map { case(_, (updateStr, _)) => updateStr}
+                .mkString("\n"))
+    } else {
+      val result = adminClient.updateFeatures(
+        updates.map { case(feature, (_, update)) => (feature, update)}.asJava,
+        new UpdateFeaturesOptions())
+      val failures = ListMap(result.values.asScala.toSeq.sortBy(_._1):_*).map {
+        case (feature, updateFuture) =>
+          val (updateStr, _) = updates(feature)
+          try {
+            updateFuture.get
+            println(updateStr + "\tResult: OK")
+            0
+          } catch {
+            case e: Exception =>
+              println(updateStr + "\tResult: FAILED due to " + e.getMessage)
+              1
+          }
+      }.sum
+      if (failures > 0) {
+        throw new UpdateFeaturesException(s"$failures feature updates failed!")
+      }
+    }
+  }
+
+  def execute(): Unit = {
+    if (opts.hasDescribeOption) {
+      describeFeatures()
+    } else if (opts.hasUpgradeAllOption) {
+      upgradeAllFeatures()
+    } else if (opts.hasDowngradeAllOption) {
+      downgradeAllFeatures()
+    } else {
+      throw new IllegalStateException("Unexpected state: no CLI command could be executed.")
+    }
+  }
+
+  def close(): Unit = {
+    adminClient.close()
+  }
+
+  private def createAdminClient(): Admin = {
+    val props = new Properties()
+    props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.bootstrapServers)
+    Admin.create(props)
+  }
+}
+
+class FeatureCommandOptions(args: Array[String]) extends CommandDefaultOptions(args) {
+  private val bootstrapServerOpt =
+    parser.accepts("bootstrap-server", "REQUIRED: The Kafka server(s) to connect to.")
+      .withRequiredArg
+      .describedAs("server to connect to")
+      .ofType(classOf[String])
+  private val describeOpt = parser.accepts(
+    "describe",
+    "Describe supported and finalized features. By default, the features are described from a" +
+    " random broker. The request can be optionally directed only to the controller using the" +
+    " --from-controller option.")
+  private val fromControllerOpt = parser.accepts(
+    "from-controller",
+    "Describe supported and finalized features from the controller.")
+  private val upgradeAllOpt = parser.accepts(
+    "upgrade-all",
+    "Upgrades all finalized features to the maximum version levels known to the tool." +
+    " This command finalizes new features known to the tool that were never finalized" +
+    " previously in the cluster, but it is guaranteed to not delete any existing feature.")
+  private val downgradeAllOpt = parser.accepts(
+    "downgrade-all",
+    "Downgrades all finalized features to the maximum version levels known to the tool." +
+    " This command deletes unknown features from the list of finalized features in the" +
+    " cluster, but it is guaranteed to not add a new feature.")
+  private val dryRunOpt = parser.accepts(
+    "dry-run",
+    "Performs a dry-run of upgrade/downgrade mutations to finalized feature without applying them.")
+
+  options = parser.parse(args : _*)
+
+  checkArgs()
+
+  def has(builder: OptionSpec[_]): Boolean = options.has(builder)
+
+  def hasDescribeOption: Boolean = has(describeOpt)
+
+  def hasFromControllerOption: Boolean = has(fromControllerOpt)
+
+  def hasDryRunOption: Boolean = has(dryRunOpt)
+
+  def hasUpgradeAllOption: Boolean = has(upgradeAllOpt)
+
+  def hasDowngradeAllOption: Boolean = has(downgradeAllOpt)
+
+  def bootstrapServers: String = options.valueOf(bootstrapServerOpt)
+
+  def checkArgs(): Unit = {
+    CommandLineUtils.printHelpAndExitIfNeeded(this, "This tool describes and updates finalized features.")
+    val numActions = Seq(describeOpt, upgradeAllOpt, downgradeAllOpt).count(has)
+    if (numActions != 1) {
+      CommandLineUtils.printUsageAndDie(
+        parser,
+        "Command must include exactly one action: --describe, --upgrade-all, --downgrade-all.")
+    }
+    CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServerOpt)
+    if (hasDryRunOption && !hasUpgradeAllOption && !hasDowngradeAllOption) {
+      CommandLineUtils.printUsageAndDie(
+        parser,
+        "Command can contain --dry-run option only when either --upgrade-all or --downgrade-all actions are provided.")
+    }
+    if (hasFromControllerOption && !hasDescribeOption) {

Review comment:
       Sounds good. I'll mark this resolved since you don't feel strongly about t.




----------------------------------------------------------------
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.

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



[GitHub] [kafka] kowshik commented on a change in pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#discussion_r504117818



##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,350 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+    features.toList.sorted.foreach {
+      case feature =>
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        print(s"Feature: $feature")
+        print(s"\tSupportedMinVersion: $supportedMinVersion")
+        print(s"\tSupportedMaxVersion: $supportedMaxVersion")
+        print(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        print(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+        println(s"\tEpoch: $epoch")
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Downgrades existing finalized features to the highest max version levels known to this tool.
+   * The method may delete existing finalized features if they are no longer seen to be supported,
+   * but it does not add a feature that was not finalized previously. The results of the feature
+   * updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def downgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val supportedFeaturesMap = supportedFeatures.features
+    val updates = existingFinalizedFeatures.asScala.map {
+      case (feature, existingVersionRange) =>
+        val targetVersionRange = supportedFeaturesMap.get(feature)
+        if (targetVersionRange == null) {
+          val updateStr =
+            deleteOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+            s"\tNewFinalizedMaxVersion: -"
+          (feature, (updateStr, new FeatureUpdate(0, true)))
+        } else {
+          if (targetVersionRange.max < existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              downgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, true)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Applies the provided feature updates. If the --dry-run CLI option is provided, the method
+   * only prints the expected feature updates to STDOUT without applying them.
+   *
+   * @param updates the feature updates to be applied via the admin client
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = {
+    if (opts.hasDryRunOption) {
+      println("Expected feature updates:")
+      println(ListMap(updates.toSeq.sortBy(_._1):_*)
+                .map { case(_, (updateStr, _)) => updateStr}
+                .mkString("\n"))
+    } else {
+      val result = adminClient.updateFeatures(
+        updates.map { case(feature, (_, update)) => (feature, update)}.asJava,
+        new UpdateFeaturesOptions())
+      val failures = ListMap(result.values.asScala.toSeq.sortBy(_._1):_*).map {
+        case (feature, updateFuture) =>
+          val (updateStr, _) = updates(feature)
+          try {
+            updateFuture.get
+            println(updateStr + "\tResult: OK")
+            0
+          } catch {
+            case e: Exception =>
+              println(updateStr + "\tResult: FAILED due to " + e.getMessage)
+              1
+          }
+      }.sum
+      if (failures > 0) {
+        throw new UpdateFeaturesException(s"$failures feature updates failed!")
+      }
+    }
+  }
+
+  def execute(): Unit = {
+    if (opts.hasDescribeOption) {
+      describeFeatures()
+    } else if (opts.hasUpgradeAllOption) {
+      upgradeAllFeatures()
+    } else if (opts.hasDowngradeAllOption) {
+      downgradeAllFeatures()
+    } else {
+      throw new IllegalStateException("Unexpected state: no CLI command could be executed.")
+    }
+  }
+
+  def close(): Unit = {
+    adminClient.close()
+  }
+
+  private def createAdminClient(): Admin = {
+    val props = new Properties()
+    props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.bootstrapServers)
+    Admin.create(props)
+  }
+}
+
+class FeatureCommandOptions(args: Array[String]) extends CommandDefaultOptions(args) {
+  private val bootstrapServerOpt =
+    parser.accepts("bootstrap-server", "REQUIRED: The Kafka server(s) to connect to.")
+      .withRequiredArg
+      .describedAs("server to connect to")
+      .ofType(classOf[String])
+  private val describeOpt = parser.accepts(
+    "describe",
+    "Describe supported and finalized features. By default, the features are described from a" +
+    " random broker. The request can be optionally directed only to the controller using the" +
+    " --from-controller option.")
+  private val fromControllerOpt = parser.accepts(
+    "from-controller",
+    "Describe supported and finalized features from the controller.")
+  private val upgradeAllOpt = parser.accepts(
+    "upgrade-all",
+    "Upgrades all finalized features to the maximum version levels known to the tool." +
+    " This command finalizes new features known to the tool that were never finalized" +
+    " previously in the cluster, but it is guaranteed to not delete any existing feature.")
+  private val downgradeAllOpt = parser.accepts(
+    "downgrade-all",
+    "Downgrades all finalized features to the maximum version levels known to the tool." +
+    " This command deletes unknown features from the list of finalized features in the" +
+    " cluster, but it is guaranteed to not add a new feature.")
+  private val dryRunOpt = parser.accepts(
+    "dry-run",
+    "Performs a dry-run of upgrade/downgrade mutations to finalized feature without applying them.")
+
+  options = parser.parse(args : _*)
+
+  checkArgs()
+
+  def has(builder: OptionSpec[_]): Boolean = options.has(builder)
+
+  def hasDescribeOption: Boolean = has(describeOpt)
+
+  def hasFromControllerOption: Boolean = has(fromControllerOpt)
+
+  def hasDryRunOption: Boolean = has(dryRunOpt)
+
+  def hasUpgradeAllOption: Boolean = has(upgradeAllOpt)
+
+  def hasDowngradeAllOption: Boolean = has(downgradeAllOpt)
+
+  def bootstrapServers: String = options.valueOf(bootstrapServerOpt)
+
+  def checkArgs(): Unit = {
+    CommandLineUtils.printHelpAndExitIfNeeded(this, "This tool describes and updates finalized features.")
+    val numActions = Seq(describeOpt, upgradeAllOpt, downgradeAllOpt).count(has)
+    if (numActions != 1) {
+      CommandLineUtils.printUsageAndDie(
+        parser,
+        "Command must include exactly one action: --describe, --upgrade-all, --downgrade-all.")
+    }
+    CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServerOpt)
+    if (hasDryRunOption && !hasUpgradeAllOption && !hasDowngradeAllOption) {
+      CommandLineUtils.printUsageAndDie(
+        parser,
+        "Command can contain --dry-run option only when either --upgrade-all or --downgrade-all actions are provided.")
+    }
+    if (hasFromControllerOption && !hasDescribeOption) {

Review comment:
       What do we lose by making a stricter check? This is not the expected usage of the command, and therefore it should be disallowed. 




----------------------------------------------------------------
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.

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



[GitHub] [kafka] kowshik commented on pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
kowshik commented on pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#issuecomment-708070565


   > one thing to point out is that we need to look at the KIP-584 definition for the basic tool usage and make sure the API names are aligned.
   
   This is done now. Please refer to the PR description.


----------------------------------------------------------------
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.

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



[GitHub] [kafka] kowshik commented on a change in pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#discussion_r506681706



##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,359 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case _: UpdateFeaturesException =>
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  private[admin] def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  private[admin] def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+
+    features.toList.sorted.foreach {
+      feature =>
+        val output = new StringBuilder()
+        output.append(s"Feature: $feature")
+
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        output.append(s"\tSupportedMinVersion: $supportedMinVersion")
+        output.append(s"\tSupportedMaxVersion: $supportedMaxVersion")
+
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        output.append(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        output.append(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        output.append(s"\tEpoch: $epoch")
+
+        println(output)
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Downgrades existing finalized features to the highest max version levels known to this tool.
+   * The method may delete existing finalized features if they are no longer seen to be supported,
+   * but it does not add a feature that was not finalized previously. The results of the feature
+   * updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def downgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val supportedFeaturesMap = supportedFeatures.features
+    val updates = existingFinalizedFeatures.asScala.map {
+      case (feature, existingVersionRange) =>
+        val targetVersionRange = supportedFeaturesMap.get(feature)
+        if (targetVersionRange == null) {
+          val updateStr =
+            deleteOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+            s"\tNewFinalizedMaxVersion: -"
+          (feature, (updateStr, new FeatureUpdate(0, true)))
+        } else {
+          if (targetVersionRange.max < existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              downgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, true)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Applies the provided feature updates. If the --dry-run CLI option is provided, the method
+   * only prints the expected feature updates to STDOUT without applying them.
+   *
+   * @param updates the feature updates to be applied via the admin client
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = {
+    if (opts.hasDryRunOption) {
+      println("Expected feature updates:")
+      println(ListMap(updates.toSeq.sortBy(_._1):_*)
+                .map { case(_, (updateStr, _)) => updateStr}
+                .mkString("\n"))
+    } else {
+      val result = adminClient.updateFeatures(
+        updates.map { case(feature, (_, update)) => (feature, update)}.asJava,
+        new UpdateFeaturesOptions())
+      val failures = ListMap(result.values.asScala.toSeq.sortBy(_._1):_*).map {
+        case (feature, updateFuture) =>
+          val (updateStr, _) = updates(feature)
+          try {
+            updateFuture.get
+            println(updateStr + "\tResult: OK")
+            0
+          } catch {
+            case e: Exception =>
+              println(updateStr + "\tResult: FAILED due to " + e.getMessage)
+              1
+          }
+      }.sum
+      if (failures > 0) {
+        throw new UpdateFeaturesException(s"$failures feature updates failed!")
+      }
+    }
+  }
+
+  def execute(): Unit = {
+    if (opts.hasDescribeOption) {
+      describeFeatures()
+    } else if (opts.hasUpgradeAllOption) {
+      upgradeAllFeatures()
+    } else if (opts.hasDowngradeAllOption) {
+      downgradeAllFeatures()
+    } else {
+      throw new IllegalStateException("Unexpected state: no CLI command could be executed.")
+    }
+  }
+
+  def close(): Unit = {
+    adminClient.close()
+  }
+
+  private def createAdminClient(): Admin = {
+    val props = new Properties()
+    props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.bootstrapServers)
+    Admin.create(props)
+  }
+}
+
+class FeatureCommandOptions(args: Array[String]) extends CommandDefaultOptions(args) {
+  private val bootstrapServerOpt =
+    parser.accepts("bootstrap-server", "REQUIRED: The Kafka server(s) to connect to.")

Review comment:
       Done




----------------------------------------------------------------
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.

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



[GitHub] [kafka] kowshik commented on a change in pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#discussion_r504138505



##########
File path: core/src/test/scala/unit/kafka/admin/FeatureCommandTest.scala
##########
@@ -0,0 +1,245 @@
+/**
+ * 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 kafka.admin
+
+import kafka.api.KAFKA_2_7_IV0
+import kafka.server.{BaseRequestTest, KafkaConfig, KafkaServer}
+import kafka.utils.TestUtils
+import kafka.utils.TestUtils.waitUntilTrue
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+
+import org.junit.Assert.{assertEquals, assertTrue}
+import org.junit.Test
+import org.scalatest.Assertions.intercept
+
+class FeatureCommandTest extends BaseRequestTest {
+  override def brokerCount: Int = 3
+
+  override def brokerPropertyOverrides(props: Properties): Unit = {
+    props.put(KafkaConfig.InterBrokerProtocolVersionProp, KAFKA_2_7_IV0.toString)
+  }
+
+  private def defaultSupportedFeatures(): Features[SupportedVersionRange] = {
+    Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)),
+                                           Utils.mkEntry("feature_2", new SupportedVersionRange(1, 5))))
+  }
+
+  private def updateSupportedFeatures(features: Features[SupportedVersionRange],
+                                      targetServers: Set[KafkaServer]): Unit = {
+    targetServers.foreach(s => {
+      s.brokerFeatures.setSupportedFeatures(features)
+      s.zkClient.updateBrokerInfo(s.createBrokerInfo)
+    })
+
+    // Wait until updates to all BrokerZNode supported features propagate to the controller.
+    val brokerIds = targetServers.map(s => s.config.brokerId)
+    waitUntilTrue(
+      () => servers.exists(s => {
+        if (s.kafkaController.isActive) {
+          s.kafkaController.controllerContext.liveOrShuttingDownBrokers
+            .filter(b => brokerIds.contains(b.id))
+            .forall(b => {
+              b.features.equals(features)
+            })
+        } else {
+          false
+        }
+      }),
+      "Controller did not get broker updates")
+  }
+
+  private def updateSupportedFeaturesInAllBrokers(features: Features[SupportedVersionRange]): Unit = {
+    updateSupportedFeatures(features, Set[KafkaServer]() ++ servers)
+  }
+
+  /**
+   * Tests if the FeatureApis#describeFeatures API works as expected when describing features before and
+   * after upgrading features.
+   */
+  @Test
+  def testDescribeFeaturesSuccess(): Unit = {
+    updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+    val featureApis = new FeatureApis(new FeatureCommandOptions(Array("--bootstrap-server", brokerList, "--describe", "--from-controller")))
+    featureApis.setSupportedFeatures(defaultSupportedFeatures())
+    try {
+      val initialDescribeOutput = TestUtils.grabConsoleOutput(featureApis.describeFeatures())
+      val expectedInitialDescribeOutput =
+        "Feature: feature_1\tSupportedMinVersion: 1\tSupportedMaxVersion: 3\tFinalizedMinVersionLevel: -\tFinalizedMaxVersionLevel: -\tEpoch: 0\n" +
+        "Feature: feature_2\tSupportedMinVersion: 1\tSupportedMaxVersion: 5\tFinalizedMinVersionLevel: -\tFinalizedMaxVersionLevel: -\tEpoch: 0\n"
+      assertEquals(expectedInitialDescribeOutput, initialDescribeOutput)
+      featureApis.upgradeAllFeatures()
+      val finalDescribeOutput = TestUtils.grabConsoleOutput(featureApis.describeFeatures())
+      val expectedFinalDescribeOutput =
+        "Feature: feature_1\tSupportedMinVersion: 1\tSupportedMaxVersion: 3\tFinalizedMinVersionLevel: 1\tFinalizedMaxVersionLevel: 3\tEpoch: 1\n" +
+        "Feature: feature_2\tSupportedMinVersion: 1\tSupportedMaxVersion: 5\tFinalizedMinVersionLevel: 1\tFinalizedMaxVersionLevel: 5\tEpoch: 1\n"
+      assertEquals(expectedFinalDescribeOutput, finalDescribeOutput)
+    } finally {
+      featureApis.close()
+    }
+  }
+
+  /**
+   * Tests if the FeatureApis#upgradeAllFeatures API works as expected during a success case.
+   */
+  @Test
+  def testUpgradeAllFeaturesSuccess(): Unit = {
+    val upgradeOpts = new FeatureCommandOptions(Array("--bootstrap-server", brokerList, "--upgrade-all"))
+    val featureApis = new FeatureApis(upgradeOpts)
+    try {
+      // Step (1):
+      // - Update the supported features across all brokers.
+      // - Upgrade non-existing feature_1 to maxVersionLevel: 2.
+      // - Verify results.
+      val initialSupportedFeatures = Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new SupportedVersionRange(1, 2))))
+      updateSupportedFeaturesInAllBrokers(initialSupportedFeatures)
+      featureApis.setSupportedFeatures(initialSupportedFeatures)
+      var output = TestUtils.grabConsoleOutput(featureApis.upgradeAllFeatures())
+      var expected =
+        "      [Add]\tFeature: feature_1\tExistingFinalizedMaxVersion: -\tNewFinalizedMaxVersion: 2\tResult: OK\n"
+      assertEquals(expected, output)
+
+      // Step (2):
+      // - Update the supported features across all brokers.
+      // - Upgrade existing feature_1 to maxVersionLevel: 3.
+      // - Upgrade non-existing feature_2 to maxVersionLevel: 5.
+      // - Verify results.
+      updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+      featureApis.setSupportedFeatures(defaultSupportedFeatures())
+      output = TestUtils.grabConsoleOutput(featureApis.upgradeAllFeatures())
+      expected =
+        "  [Upgrade]\tFeature: feature_1\tExistingFinalizedMaxVersion: 2\tNewFinalizedMaxVersion: 3\tResult: OK\n" +
+        "      [Add]\tFeature: feature_2\tExistingFinalizedMaxVersion: -\tNewFinalizedMaxVersion: 5\tResult: OK\n"
+      assertEquals(expected, output)
+
+      // Step (3):
+      // - Perform an upgrade of all features again.
+      // - Since supported features have not changed, expect that the above action does not yield
+      //   any results.
+      output = TestUtils.grabConsoleOutput(featureApis.upgradeAllFeatures())
+      assertTrue(output.isEmpty)
+      featureApis.setOptions(upgradeOpts)
+      output = TestUtils.grabConsoleOutput(featureApis.upgradeAllFeatures())
+      assertTrue(output.isEmpty)
+    } finally {
+      featureApis.close()
+    }
+  }
+
+  /**
+   * Tests if the FeatureApis#downgradeAllFeatures API works as expected during a success case.
+   */
+  @Test
+  def testDowngradeFeaturesSuccess(): Unit = {
+    val downgradeOpts = new FeatureCommandOptions(Array("--bootstrap-server", brokerList, "--downgrade-all"))
+    val upgradeOpts = new FeatureCommandOptions(Array("--bootstrap-server", brokerList, "--upgrade-all"))
+    val featureApis = new FeatureApis(upgradeOpts)
+    try {
+      // Step (1):
+      // - Update the supported features across all brokers.
+      // - Upgrade non-existing feature_1 to maxVersionLevel: 3.
+      // - Upgrade non-existing feature_2 to maxVersionLevel: 5.
+      updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+      featureApis.setSupportedFeatures(defaultSupportedFeatures())
+      featureApis.upgradeAllFeatures()
+
+      // Step (2):
+      // - Downgrade existing feature_1 to maxVersionLevel: 2.
+      // - Delete feature_2 since it is no longer supported by the FeatureApis object.
+      // - Verify results.
+      val downgradedFeatures = Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new SupportedVersionRange(1, 2))))
+      featureApis.setSupportedFeatures(downgradedFeatures)
+      featureApis.setOptions(downgradeOpts)
+      var output = TestUtils.grabConsoleOutput(featureApis.downgradeAllFeatures())
+      var expected =
+        "[Downgrade]\tFeature: feature_1\tExistingFinalizedMaxVersion: 3\tNewFinalizedMaxVersion: 2\tResult: OK\n" +
+        "   [Delete]\tFeature: feature_2\tExistingFinalizedMaxVersion: 5\tNewFinalizedMaxVersion: -\tResult: OK\n"
+      assertEquals(expected, output)
+
+      // Step (3):
+      // - Perform a downgrade of all features again.
+      // - Since supported features have not changed, expect that the above action does not yield
+      //   any results.
+      updateSupportedFeaturesInAllBrokers(downgradedFeatures)
+      output = TestUtils.grabConsoleOutput(featureApis.downgradeAllFeatures())
+      assertTrue(output.isEmpty)
+
+      // Step (4):
+      // - Delete feature_1 since it is no longer supported by the FeatureApis object.
+      // - Verify results.
+      featureApis.setSupportedFeatures(Features.emptySupportedFeatures())
+      output = TestUtils.grabConsoleOutput(featureApis.downgradeAllFeatures())
+      expected =
+        "   [Delete]\tFeature: feature_1\tExistingFinalizedMaxVersion: 2\tNewFinalizedMaxVersion: -\tResult: OK\n"
+      assertEquals(expected, output)
+    } finally {
+      featureApis.close()
+    }
+  }
+
+  /**
+   * Tests if the FeatureApis#upgradeAllFeatures API works as expected during a partial failure case.
+   */
+  @Test
+  def testUpgradeFeaturesFailure(): Unit = {

Review comment:
       As such the downgrade can not fail currently on the server. The reason is the following: the `FeatureCommand` downgrades `maxVersionLevel` of existing finalized features to the max versions that it knows internally (since it links against the `BrokerFeatures` library). As a result, the only situation when the `FeaturecCommand` can fail is if it tries to downgrade below the supported `minVersion`. But this can never happen since we don't (yet) have the facility on the broker to have a supported `minVersion` > 1 (this can only happen during feature deprecation, the support for this is future work).
   
   On a side note, we could maybe write a test using `AdminClientUnitTestEnv` to create a mock setup specifically for the downgrade test and inject a mock admin client into the `FeatureApis` class. But that looks overkill to me...




----------------------------------------------------------------
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.

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



[GitHub] [kafka] kowshik commented on a change in pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#discussion_r506681928



##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,359 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case _: UpdateFeaturesException =>
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  private[admin] def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  private[admin] def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+
+    features.toList.sorted.foreach {
+      feature =>
+        val output = new StringBuilder()
+        output.append(s"Feature: $feature")
+
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        output.append(s"\tSupportedMinVersion: $supportedMinVersion")
+        output.append(s"\tSupportedMaxVersion: $supportedMaxVersion")
+
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        output.append(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        output.append(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        output.append(s"\tEpoch: $epoch")
+
+        println(output)
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Downgrades existing finalized features to the highest max version levels known to this tool.
+   * The method may delete existing finalized features if they are no longer seen to be supported,
+   * but it does not add a feature that was not finalized previously. The results of the feature
+   * updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def downgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val supportedFeaturesMap = supportedFeatures.features
+    val updates = existingFinalizedFeatures.asScala.map {
+      case (feature, existingVersionRange) =>
+        val targetVersionRange = supportedFeaturesMap.get(feature)
+        if (targetVersionRange == null) {
+          val updateStr =
+            deleteOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+            s"\tNewFinalizedMaxVersion: -"
+          (feature, (updateStr, new FeatureUpdate(0, true)))
+        } else {
+          if (targetVersionRange.max < existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              downgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, true)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Applies the provided feature updates. If the --dry-run CLI option is provided, the method
+   * only prints the expected feature updates to STDOUT without applying them.
+   *
+   * @param updates the feature updates to be applied via the admin client
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = {
+    if (opts.hasDryRunOption) {
+      println("Expected feature updates:")
+      println(ListMap(updates.toSeq.sortBy(_._1):_*)
+                .map { case(_, (updateStr, _)) => updateStr}
+                .mkString("\n"))
+    } else {
+      val result = adminClient.updateFeatures(
+        updates.map { case(feature, (_, update)) => (feature, update)}.asJava,
+        new UpdateFeaturesOptions())
+      val failures = ListMap(result.values.asScala.toSeq.sortBy(_._1):_*).map {
+        case (feature, updateFuture) =>
+          val (updateStr, _) = updates(feature)
+          try {
+            updateFuture.get
+            println(updateStr + "\tResult: OK")
+            0
+          } catch {
+            case e: Exception =>
+              println(updateStr + "\tResult: FAILED due to " + e.getMessage)
+              1
+          }
+      }.sum
+      if (failures > 0) {
+        throw new UpdateFeaturesException(s"$failures feature updates failed!")
+      }
+    }
+  }
+
+  def execute(): Unit = {
+    if (opts.hasDescribeOption) {
+      describeFeatures()
+    } else if (opts.hasUpgradeAllOption) {
+      upgradeAllFeatures()
+    } else if (opts.hasDowngradeAllOption) {
+      downgradeAllFeatures()
+    } else {
+      throw new IllegalStateException("Unexpected state: no CLI command could be executed.")
+    }
+  }
+
+  def close(): Unit = {
+    adminClient.close()
+  }
+
+  private def createAdminClient(): Admin = {
+    val props = new Properties()
+    props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.bootstrapServers)
+    Admin.create(props)

Review comment:
       Done. Great point! I've provided an additional arg: `--command-config <path_to_properties>` file that can be used to inject the additional properties into admin client.




----------------------------------------------------------------
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.

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



[GitHub] [kafka] kowshik commented on a change in pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#discussion_r504121355



##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,350 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+    features.toList.sorted.foreach {
+      case feature =>
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        print(s"Feature: $feature")
+        print(s"\tSupportedMinVersion: $supportedMinVersion")
+        print(s"\tSupportedMaxVersion: $supportedMaxVersion")
+        print(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")

Review comment:
       Done.




----------------------------------------------------------------
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.

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



[GitHub] [kafka] junrao commented on a change in pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#discussion_r505019265



##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,359 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case _: UpdateFeaturesException =>
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  private[admin] def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  private[admin] def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+
+    features.toList.sorted.foreach {
+      feature =>
+        val output = new StringBuilder()
+        output.append(s"Feature: $feature")
+
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        output.append(s"\tSupportedMinVersion: $supportedMinVersion")
+        output.append(s"\tSupportedMaxVersion: $supportedMaxVersion")
+
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        output.append(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        output.append(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        output.append(s"\tEpoch: $epoch")
+
+        println(output)
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Downgrades existing finalized features to the highest max version levels known to this tool.
+   * The method may delete existing finalized features if they are no longer seen to be supported,
+   * but it does not add a feature that was not finalized previously. The results of the feature
+   * updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def downgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val supportedFeaturesMap = supportedFeatures.features
+    val updates = existingFinalizedFeatures.asScala.map {
+      case (feature, existingVersionRange) =>
+        val targetVersionRange = supportedFeaturesMap.get(feature)
+        if (targetVersionRange == null) {
+          val updateStr =
+            deleteOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+            s"\tNewFinalizedMaxVersion: -"
+          (feature, (updateStr, new FeatureUpdate(0, true)))
+        } else {
+          if (targetVersionRange.max < existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              downgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, true)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Applies the provided feature updates. If the --dry-run CLI option is provided, the method
+   * only prints the expected feature updates to STDOUT without applying them.
+   *
+   * @param updates the feature updates to be applied via the admin client
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = {
+    if (opts.hasDryRunOption) {
+      println("Expected feature updates:")
+      println(ListMap(updates.toSeq.sortBy(_._1):_*)
+                .map { case(_, (updateStr, _)) => updateStr}
+                .mkString("\n"))
+    } else {
+      val result = adminClient.updateFeatures(
+        updates.map { case(feature, (_, update)) => (feature, update)}.asJava,
+        new UpdateFeaturesOptions())
+      val failures = ListMap(result.values.asScala.toSeq.sortBy(_._1):_*).map {
+        case (feature, updateFuture) =>
+          val (updateStr, _) = updates(feature)
+          try {
+            updateFuture.get
+            println(updateStr + "\tResult: OK")
+            0
+          } catch {
+            case e: Exception =>
+              println(updateStr + "\tResult: FAILED due to " + e.getMessage)
+              1
+          }
+      }.sum
+      if (failures > 0) {
+        throw new UpdateFeaturesException(s"$failures feature updates failed!")
+      }
+    }
+  }
+
+  def execute(): Unit = {
+    if (opts.hasDescribeOption) {
+      describeFeatures()
+    } else if (opts.hasUpgradeAllOption) {
+      upgradeAllFeatures()
+    } else if (opts.hasDowngradeAllOption) {
+      downgradeAllFeatures()
+    } else {
+      throw new IllegalStateException("Unexpected state: no CLI command could be executed.")
+    }
+  }
+
+  def close(): Unit = {
+    adminClient.close()
+  }
+
+  private def createAdminClient(): Admin = {
+    val props = new Properties()
+    props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.bootstrapServers)
+    Admin.create(props)
+  }
+}
+
+class FeatureCommandOptions(args: Array[String]) extends CommandDefaultOptions(args) {
+  private val bootstrapServerOpt =
+    parser.accepts("bootstrap-server", "REQUIRED: The Kafka server(s) to connect to.")

Review comment:
       Perhaps we can describe the format is host:port list.

##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,359 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case _: UpdateFeaturesException =>
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  private[admin] def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  private[admin] def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+
+    features.toList.sorted.foreach {
+      feature =>
+        val output = new StringBuilder()
+        output.append(s"Feature: $feature")
+
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        output.append(s"\tSupportedMinVersion: $supportedMinVersion")
+        output.append(s"\tSupportedMaxVersion: $supportedMaxVersion")
+
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        output.append(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        output.append(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        output.append(s"\tEpoch: $epoch")
+
+        println(output)
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+          } else {
+            (feature, null)

Review comment:
       Perhaps we could use Option to avoid null?

##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,359 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case _: UpdateFeaturesException =>
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  private[admin] def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  private[admin] def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+
+    features.toList.sorted.foreach {
+      feature =>
+        val output = new StringBuilder()
+        output.append(s"Feature: $feature")
+
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        output.append(s"\tSupportedMinVersion: $supportedMinVersion")
+        output.append(s"\tSupportedMaxVersion: $supportedMaxVersion")
+
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        output.append(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        output.append(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        output.append(s"\tEpoch: $epoch")
+
+        println(output)
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Downgrades existing finalized features to the highest max version levels known to this tool.
+   * The method may delete existing finalized features if they are no longer seen to be supported,
+   * but it does not add a feature that was not finalized previously. The results of the feature
+   * updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def downgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val supportedFeaturesMap = supportedFeatures.features
+    val updates = existingFinalizedFeatures.asScala.map {
+      case (feature, existingVersionRange) =>
+        val targetVersionRange = supportedFeaturesMap.get(feature)
+        if (targetVersionRange == null) {
+          val updateStr =
+            deleteOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+            s"\tNewFinalizedMaxVersion: -"
+          (feature, (updateStr, new FeatureUpdate(0, true)))
+        } else {
+          if (targetVersionRange.max < existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              downgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, true)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Applies the provided feature updates. If the --dry-run CLI option is provided, the method
+   * only prints the expected feature updates to STDOUT without applying them.
+   *
+   * @param updates the feature updates to be applied via the admin client
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = {
+    if (opts.hasDryRunOption) {
+      println("Expected feature updates:")
+      println(ListMap(updates.toSeq.sortBy(_._1):_*)

Review comment:
       Could we use case to avoid unnamed reference _._1?

##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,359 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case _: UpdateFeaturesException =>
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  private[admin] def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  private[admin] def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+
+    features.toList.sorted.foreach {
+      feature =>
+        val output = new StringBuilder()
+        output.append(s"Feature: $feature")
+
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        output.append(s"\tSupportedMinVersion: $supportedMinVersion")
+        output.append(s"\tSupportedMaxVersion: $supportedMaxVersion")
+
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        output.append(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        output.append(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        output.append(s"\tEpoch: $epoch")
+
+        println(output)
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Downgrades existing finalized features to the highest max version levels known to this tool.
+   * The method may delete existing finalized features if they are no longer seen to be supported,
+   * but it does not add a feature that was not finalized previously. The results of the feature
+   * updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def downgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val supportedFeaturesMap = supportedFeatures.features
+    val updates = existingFinalizedFeatures.asScala.map {
+      case (feature, existingVersionRange) =>
+        val targetVersionRange = supportedFeaturesMap.get(feature)
+        if (targetVersionRange == null) {
+          val updateStr =
+            deleteOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+            s"\tNewFinalizedMaxVersion: -"
+          (feature, (updateStr, new FeatureUpdate(0, true)))
+        } else {
+          if (targetVersionRange.max < existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              downgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, true)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Applies the provided feature updates. If the --dry-run CLI option is provided, the method
+   * only prints the expected feature updates to STDOUT without applying them.
+   *
+   * @param updates the feature updates to be applied via the admin client
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = {
+    if (opts.hasDryRunOption) {
+      println("Expected feature updates:")
+      println(ListMap(updates.toSeq.sortBy(_._1):_*)
+                .map { case(_, (updateStr, _)) => updateStr}
+                .mkString("\n"))
+    } else {
+      val result = adminClient.updateFeatures(
+        updates.map { case(feature, (_, update)) => (feature, update)}.asJava,
+        new UpdateFeaturesOptions())
+      val failures = ListMap(result.values.asScala.toSeq.sortBy(_._1):_*).map {
+        case (feature, updateFuture) =>
+          val (updateStr, _) = updates(feature)
+          try {
+            updateFuture.get
+            println(updateStr + "\tResult: OK")
+            0
+          } catch {
+            case e: Exception =>
+              println(updateStr + "\tResult: FAILED due to " + e.getMessage)

Review comment:
       Do we need to get the cause from ExecutionException thrown from Future?

##########
File path: core/src/test/scala/unit/kafka/admin/FeatureCommandTest.scala
##########
@@ -0,0 +1,245 @@
+/**
+ * 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 kafka.admin
+
+import kafka.api.KAFKA_2_7_IV0
+import kafka.server.{BaseRequestTest, KafkaConfig, KafkaServer}
+import kafka.utils.TestUtils
+import kafka.utils.TestUtils.waitUntilTrue
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+
+import org.junit.Assert.{assertEquals, assertTrue}
+import org.junit.Test
+import org.scalatest.Assertions.intercept
+
+class FeatureCommandTest extends BaseRequestTest {
+  override def brokerCount: Int = 3
+
+  override def brokerPropertyOverrides(props: Properties): Unit = {
+    props.put(KafkaConfig.InterBrokerProtocolVersionProp, KAFKA_2_7_IV0.toString)
+  }
+
+  private def defaultSupportedFeatures(): Features[SupportedVersionRange] = {

Review comment:
       Could this be a val?

##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,359 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case _: UpdateFeaturesException =>
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  private[admin] def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  private[admin] def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+
+    features.toList.sorted.foreach {
+      feature =>
+        val output = new StringBuilder()
+        output.append(s"Feature: $feature")
+
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        output.append(s"\tSupportedMinVersion: $supportedMinVersion")
+        output.append(s"\tSupportedMaxVersion: $supportedMaxVersion")
+
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        output.append(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        output.append(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        output.append(s"\tEpoch: $epoch")
+
+        println(output)
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Downgrades existing finalized features to the highest max version levels known to this tool.
+   * The method may delete existing finalized features if they are no longer seen to be supported,
+   * but it does not add a feature that was not finalized previously. The results of the feature
+   * updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def downgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val supportedFeaturesMap = supportedFeatures.features
+    val updates = existingFinalizedFeatures.asScala.map {
+      case (feature, existingVersionRange) =>
+        val targetVersionRange = supportedFeaturesMap.get(feature)
+        if (targetVersionRange == null) {
+          val updateStr =
+            deleteOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+            s"\tNewFinalizedMaxVersion: -"
+          (feature, (updateStr, new FeatureUpdate(0, true)))
+        } else {
+          if (targetVersionRange.max < existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              downgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, true)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Applies the provided feature updates. If the --dry-run CLI option is provided, the method
+   * only prints the expected feature updates to STDOUT without applying them.
+   *
+   * @param updates the feature updates to be applied via the admin client
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = {
+    if (opts.hasDryRunOption) {
+      println("Expected feature updates:")
+      println(ListMap(updates.toSeq.sortBy(_._1):_*)
+                .map { case(_, (updateStr, _)) => updateStr}
+                .mkString("\n"))
+    } else {
+      val result = adminClient.updateFeatures(
+        updates.map { case(feature, (_, update)) => (feature, update)}.asJava,
+        new UpdateFeaturesOptions())
+      val failures = ListMap(result.values.asScala.toSeq.sortBy(_._1):_*).map {

Review comment:
       Could we use case to avoid unnamed reference _._1?

##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,359 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case _: UpdateFeaturesException =>
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  private[admin] def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  private[admin] def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+
+    features.toList.sorted.foreach {
+      feature =>
+        val output = new StringBuilder()
+        output.append(s"Feature: $feature")
+
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        output.append(s"\tSupportedMinVersion: $supportedMinVersion")
+        output.append(s"\tSupportedMaxVersion: $supportedMaxVersion")
+
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        output.append(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        output.append(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        output.append(s"\tEpoch: $epoch")
+
+        println(output)
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Downgrades existing finalized features to the highest max version levels known to this tool.
+   * The method may delete existing finalized features if they are no longer seen to be supported,
+   * but it does not add a feature that was not finalized previously. The results of the feature
+   * updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def downgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val supportedFeaturesMap = supportedFeatures.features
+    val updates = existingFinalizedFeatures.asScala.map {
+      case (feature, existingVersionRange) =>
+        val targetVersionRange = supportedFeaturesMap.get(feature)
+        if (targetVersionRange == null) {
+          val updateStr =
+            deleteOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+            s"\tNewFinalizedMaxVersion: -"
+          (feature, (updateStr, new FeatureUpdate(0, true)))
+        } else {
+          if (targetVersionRange.max < existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              downgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, true)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Applies the provided feature updates. If the --dry-run CLI option is provided, the method
+   * only prints the expected feature updates to STDOUT without applying them.
+   *
+   * @param updates the feature updates to be applied via the admin client
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = {
+    if (opts.hasDryRunOption) {
+      println("Expected feature updates:")
+      println(ListMap(updates.toSeq.sortBy(_._1):_*)
+                .map { case(_, (updateStr, _)) => updateStr}
+                .mkString("\n"))
+    } else {
+      val result = adminClient.updateFeatures(
+        updates.map { case(feature, (_, update)) => (feature, update)}.asJava,
+        new UpdateFeaturesOptions())
+      val failures = ListMap(result.values.asScala.toSeq.sortBy(_._1):_*).map {
+        case (feature, updateFuture) =>
+          val (updateStr, _) = updates(feature)
+          try {
+            updateFuture.get
+            println(updateStr + "\tResult: OK")
+            0
+          } catch {
+            case e: Exception =>
+              println(updateStr + "\tResult: FAILED due to " + e.getMessage)
+              1
+          }
+      }.sum
+      if (failures > 0) {
+        throw new UpdateFeaturesException(s"$failures feature updates failed!")
+      }
+    }
+  }
+
+  def execute(): Unit = {
+    if (opts.hasDescribeOption) {
+      describeFeatures()
+    } else if (opts.hasUpgradeAllOption) {
+      upgradeAllFeatures()
+    } else if (opts.hasDowngradeAllOption) {
+      downgradeAllFeatures()
+    } else {
+      throw new IllegalStateException("Unexpected state: no CLI command could be executed.")
+    }
+  }
+
+  def close(): Unit = {
+    adminClient.close()
+  }
+
+  private def createAdminClient(): Admin = {
+    val props = new Properties()
+    props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.bootstrapServers)
+    Admin.create(props)

Review comment:
       The bootstrap port may be secured (e.g., SSL, SASL). So we need to be able to pass in the security configs from the command line. See commandConfigOpt in TopicCommand.




----------------------------------------------------------------
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.

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



[GitHub] [kafka] kowshik commented on a change in pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#discussion_r506681748



##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,359 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case _: UpdateFeaturesException =>
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  private[admin] def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  private[admin] def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+
+    features.toList.sorted.foreach {
+      feature =>
+        val output = new StringBuilder()
+        output.append(s"Feature: $feature")
+
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        output.append(s"\tSupportedMinVersion: $supportedMinVersion")
+        output.append(s"\tSupportedMaxVersion: $supportedMaxVersion")
+
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        output.append(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        output.append(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        output.append(s"\tEpoch: $epoch")
+
+        println(output)
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+          } else {
+            (feature, null)

Review comment:
       Done

##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,359 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case _: UpdateFeaturesException =>
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  private[admin] def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  private[admin] def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+
+    features.toList.sorted.foreach {
+      feature =>
+        val output = new StringBuilder()
+        output.append(s"Feature: $feature")
+
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        output.append(s"\tSupportedMinVersion: $supportedMinVersion")
+        output.append(s"\tSupportedMaxVersion: $supportedMaxVersion")
+
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        output.append(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        output.append(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        output.append(s"\tEpoch: $epoch")
+
+        println(output)
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Downgrades existing finalized features to the highest max version levels known to this tool.
+   * The method may delete existing finalized features if they are no longer seen to be supported,
+   * but it does not add a feature that was not finalized previously. The results of the feature
+   * updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def downgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val supportedFeaturesMap = supportedFeatures.features
+    val updates = existingFinalizedFeatures.asScala.map {
+      case (feature, existingVersionRange) =>
+        val targetVersionRange = supportedFeaturesMap.get(feature)
+        if (targetVersionRange == null) {
+          val updateStr =
+            deleteOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+            s"\tNewFinalizedMaxVersion: -"
+          (feature, (updateStr, new FeatureUpdate(0, true)))
+        } else {
+          if (targetVersionRange.max < existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              downgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, true)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Applies the provided feature updates. If the --dry-run CLI option is provided, the method
+   * only prints the expected feature updates to STDOUT without applying them.
+   *
+   * @param updates the feature updates to be applied via the admin client
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = {
+    if (opts.hasDryRunOption) {
+      println("Expected feature updates:")
+      println(ListMap(updates.toSeq.sortBy(_._1):_*)

Review comment:
       Done

##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,359 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case _: UpdateFeaturesException =>
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  private[admin] def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  private[admin] def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+
+    features.toList.sorted.foreach {
+      feature =>
+        val output = new StringBuilder()
+        output.append(s"Feature: $feature")
+
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        output.append(s"\tSupportedMinVersion: $supportedMinVersion")
+        output.append(s"\tSupportedMaxVersion: $supportedMaxVersion")
+
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        output.append(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        output.append(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        output.append(s"\tEpoch: $epoch")
+
+        println(output)
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Downgrades existing finalized features to the highest max version levels known to this tool.
+   * The method may delete existing finalized features if they are no longer seen to be supported,
+   * but it does not add a feature that was not finalized previously. The results of the feature
+   * updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def downgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val supportedFeaturesMap = supportedFeatures.features
+    val updates = existingFinalizedFeatures.asScala.map {
+      case (feature, existingVersionRange) =>
+        val targetVersionRange = supportedFeaturesMap.get(feature)
+        if (targetVersionRange == null) {
+          val updateStr =
+            deleteOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+            s"\tNewFinalizedMaxVersion: -"
+          (feature, (updateStr, new FeatureUpdate(0, true)))
+        } else {
+          if (targetVersionRange.max < existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              downgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, true)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Applies the provided feature updates. If the --dry-run CLI option is provided, the method
+   * only prints the expected feature updates to STDOUT without applying them.
+   *
+   * @param updates the feature updates to be applied via the admin client
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = {
+    if (opts.hasDryRunOption) {
+      println("Expected feature updates:")
+      println(ListMap(updates.toSeq.sortBy(_._1):_*)
+                .map { case(_, (updateStr, _)) => updateStr}
+                .mkString("\n"))
+    } else {
+      val result = adminClient.updateFeatures(
+        updates.map { case(feature, (_, update)) => (feature, update)}.asJava,
+        new UpdateFeaturesOptions())
+      val failures = ListMap(result.values.asScala.toSeq.sortBy(_._1):_*).map {

Review comment:
       Done

##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,359 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case _: UpdateFeaturesException =>
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  private[admin] def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  private[admin] def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+
+    features.toList.sorted.foreach {
+      feature =>
+        val output = new StringBuilder()
+        output.append(s"Feature: $feature")
+
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        output.append(s"\tSupportedMinVersion: $supportedMinVersion")
+        output.append(s"\tSupportedMaxVersion: $supportedMaxVersion")
+
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        output.append(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        output.append(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        output.append(s"\tEpoch: $epoch")
+
+        println(output)
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Downgrades existing finalized features to the highest max version levels known to this tool.
+   * The method may delete existing finalized features if they are no longer seen to be supported,
+   * but it does not add a feature that was not finalized previously. The results of the feature
+   * updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def downgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val supportedFeaturesMap = supportedFeatures.features
+    val updates = existingFinalizedFeatures.asScala.map {
+      case (feature, existingVersionRange) =>
+        val targetVersionRange = supportedFeaturesMap.get(feature)
+        if (targetVersionRange == null) {
+          val updateStr =
+            deleteOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+            s"\tNewFinalizedMaxVersion: -"
+          (feature, (updateStr, new FeatureUpdate(0, true)))
+        } else {
+          if (targetVersionRange.max < existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              downgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, true)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Applies the provided feature updates. If the --dry-run CLI option is provided, the method
+   * only prints the expected feature updates to STDOUT without applying them.
+   *
+   * @param updates the feature updates to be applied via the admin client
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = {
+    if (opts.hasDryRunOption) {
+      println("Expected feature updates:")
+      println(ListMap(updates.toSeq.sortBy(_._1):_*)
+                .map { case(_, (updateStr, _)) => updateStr}
+                .mkString("\n"))
+    } else {
+      val result = adminClient.updateFeatures(
+        updates.map { case(feature, (_, update)) => (feature, update)}.asJava,
+        new UpdateFeaturesOptions())
+      val failures = ListMap(result.values.asScala.toSeq.sortBy(_._1):_*).map {
+        case (feature, updateFuture) =>
+          val (updateStr, _) = updates(feature)
+          try {
+            updateFuture.get
+            println(updateStr + "\tResult: OK")
+            0
+          } catch {
+            case e: Exception =>
+              println(updateStr + "\tResult: FAILED due to " + e.getMessage)

Review comment:
       Done

##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,359 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case _: UpdateFeaturesException =>
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  private[admin] def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  private[admin] def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+
+    features.toList.sorted.foreach {
+      feature =>
+        val output = new StringBuilder()
+        output.append(s"Feature: $feature")
+
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        output.append(s"\tSupportedMinVersion: $supportedMinVersion")
+        output.append(s"\tSupportedMaxVersion: $supportedMaxVersion")
+
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        output.append(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        output.append(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        output.append(s"\tEpoch: $epoch")
+
+        println(output)
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Downgrades existing finalized features to the highest max version levels known to this tool.
+   * The method may delete existing finalized features if they are no longer seen to be supported,
+   * but it does not add a feature that was not finalized previously. The results of the feature
+   * updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def downgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val supportedFeaturesMap = supportedFeatures.features
+    val updates = existingFinalizedFeatures.asScala.map {
+      case (feature, existingVersionRange) =>
+        val targetVersionRange = supportedFeaturesMap.get(feature)
+        if (targetVersionRange == null) {
+          val updateStr =
+            deleteOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+            s"\tNewFinalizedMaxVersion: -"
+          (feature, (updateStr, new FeatureUpdate(0, true)))
+        } else {
+          if (targetVersionRange.max < existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              downgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, true)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Applies the provided feature updates. If the --dry-run CLI option is provided, the method
+   * only prints the expected feature updates to STDOUT without applying them.
+   *
+   * @param updates the feature updates to be applied via the admin client
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = {
+    if (opts.hasDryRunOption) {
+      println("Expected feature updates:")
+      println(ListMap(updates.toSeq.sortBy(_._1):_*)
+                .map { case(_, (updateStr, _)) => updateStr}
+                .mkString("\n"))
+    } else {
+      val result = adminClient.updateFeatures(
+        updates.map { case(feature, (_, update)) => (feature, update)}.asJava,
+        new UpdateFeaturesOptions())
+      val failures = ListMap(result.values.asScala.toSeq.sortBy(_._1):_*).map {
+        case (feature, updateFuture) =>
+          val (updateStr, _) = updates(feature)
+          try {
+            updateFuture.get
+            println(updateStr + "\tResult: OK")
+            0
+          } catch {
+            case e: Exception =>
+              println(updateStr + "\tResult: FAILED due to " + e.getMessage)
+              1
+          }
+      }.sum
+      if (failures > 0) {
+        throw new UpdateFeaturesException(s"$failures feature updates failed!")
+      }
+    }
+  }
+
+  def execute(): Unit = {
+    if (opts.hasDescribeOption) {
+      describeFeatures()
+    } else if (opts.hasUpgradeAllOption) {
+      upgradeAllFeatures()
+    } else if (opts.hasDowngradeAllOption) {
+      downgradeAllFeatures()
+    } else {
+      throw new IllegalStateException("Unexpected state: no CLI command could be executed.")
+    }
+  }
+
+  def close(): Unit = {
+    adminClient.close()
+  }
+
+  private def createAdminClient(): Admin = {
+    val props = new Properties()
+    props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.bootstrapServers)
+    Admin.create(props)

Review comment:
       Done. Great point!

##########
File path: core/src/test/scala/unit/kafka/admin/FeatureCommandTest.scala
##########
@@ -0,0 +1,245 @@
+/**
+ * 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 kafka.admin
+
+import kafka.api.KAFKA_2_7_IV0
+import kafka.server.{BaseRequestTest, KafkaConfig, KafkaServer}
+import kafka.utils.TestUtils
+import kafka.utils.TestUtils.waitUntilTrue
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+
+import org.junit.Assert.{assertEquals, assertTrue}
+import org.junit.Test
+import org.scalatest.Assertions.intercept
+
+class FeatureCommandTest extends BaseRequestTest {
+  override def brokerCount: Int = 3
+
+  override def brokerPropertyOverrides(props: Properties): Unit = {
+    props.put(KafkaConfig.InterBrokerProtocolVersionProp, KAFKA_2_7_IV0.toString)
+  }
+
+  private def defaultSupportedFeatures(): Features[SupportedVersionRange] = {

Review comment:
       Done




----------------------------------------------------------------
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.

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



[GitHub] [kafka] abbccdda commented on a change in pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
abbccdda commented on a change in pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#discussion_r504831412



##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,359 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case _: UpdateFeaturesException =>
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {

Review comment:
       nit: we could make these functions package private




----------------------------------------------------------------
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.

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



[GitHub] [kafka] abbccdda commented on a change in pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
abbccdda commented on a change in pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#discussion_r504152360



##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,350 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+    features.toList.sorted.foreach {
+      case feature =>
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        print(s"Feature: $feature")
+        print(s"\tSupportedMinVersion: $supportedMinVersion")
+        print(s"\tSupportedMaxVersion: $supportedMaxVersion")
+        print(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        print(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+        println(s"\tEpoch: $epoch")
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Downgrades existing finalized features to the highest max version levels known to this tool.
+   * The method may delete existing finalized features if they are no longer seen to be supported,
+   * but it does not add a feature that was not finalized previously. The results of the feature
+   * updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def downgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val supportedFeaturesMap = supportedFeatures.features
+    val updates = existingFinalizedFeatures.asScala.map {
+      case (feature, existingVersionRange) =>
+        val targetVersionRange = supportedFeaturesMap.get(feature)
+        if (targetVersionRange == null) {
+          val updateStr =
+            deleteOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+            s"\tNewFinalizedMaxVersion: -"
+          (feature, (updateStr, new FeatureUpdate(0, true)))
+        } else {
+          if (targetVersionRange.max < existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              downgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, true)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Applies the provided feature updates. If the --dry-run CLI option is provided, the method
+   * only prints the expected feature updates to STDOUT without applying them.
+   *
+   * @param updates the feature updates to be applied via the admin client
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = {
+    if (opts.hasDryRunOption) {
+      println("Expected feature updates:")
+      println(ListMap(updates.toSeq.sortBy(_._1):_*)
+                .map { case(_, (updateStr, _)) => updateStr}
+                .mkString("\n"))
+    } else {
+      val result = adminClient.updateFeatures(
+        updates.map { case(feature, (_, update)) => (feature, update)}.asJava,
+        new UpdateFeaturesOptions())
+      val failures = ListMap(result.values.asScala.toSeq.sortBy(_._1):_*).map {
+        case (feature, updateFuture) =>
+          val (updateStr, _) = updates(feature)
+          try {
+            updateFuture.get
+            println(updateStr + "\tResult: OK")
+            0
+          } catch {
+            case e: Exception =>
+              println(updateStr + "\tResult: FAILED due to " + e.getMessage)
+              1
+          }
+      }.sum
+      if (failures > 0) {
+        throw new UpdateFeaturesException(s"$failures feature updates failed!")
+      }
+    }
+  }
+
+  def execute(): Unit = {
+    if (opts.hasDescribeOption) {
+      describeFeatures()
+    } else if (opts.hasUpgradeAllOption) {
+      upgradeAllFeatures()
+    } else if (opts.hasDowngradeAllOption) {
+      downgradeAllFeatures()
+    } else {
+      throw new IllegalStateException("Unexpected state: no CLI command could be executed.")
+    }
+  }
+
+  def close(): Unit = {
+    adminClient.close()
+  }
+
+  private def createAdminClient(): Admin = {
+    val props = new Properties()
+    props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.bootstrapServers)
+    Admin.create(props)
+  }
+}
+
+class FeatureCommandOptions(args: Array[String]) extends CommandDefaultOptions(args) {
+  private val bootstrapServerOpt =
+    parser.accepts("bootstrap-server", "REQUIRED: The Kafka server(s) to connect to.")
+      .withRequiredArg
+      .describedAs("server to connect to")
+      .ofType(classOf[String])
+  private val describeOpt = parser.accepts(
+    "describe",
+    "Describe supported and finalized features. By default, the features are described from a" +
+    " random broker. The request can be optionally directed only to the controller using the" +
+    " --from-controller option.")
+  private val fromControllerOpt = parser.accepts(
+    "from-controller",
+    "Describe supported and finalized features from the controller.")
+  private val upgradeAllOpt = parser.accepts(
+    "upgrade-all",
+    "Upgrades all finalized features to the maximum version levels known to the tool." +
+    " This command finalizes new features known to the tool that were never finalized" +
+    " previously in the cluster, but it is guaranteed to not delete any existing feature.")
+  private val downgradeAllOpt = parser.accepts(
+    "downgrade-all",
+    "Downgrades all finalized features to the maximum version levels known to the tool." +
+    " This command deletes unknown features from the list of finalized features in the" +
+    " cluster, but it is guaranteed to not add a new feature.")
+  private val dryRunOpt = parser.accepts(
+    "dry-run",
+    "Performs a dry-run of upgrade/downgrade mutations to finalized feature without applying them.")
+
+  options = parser.parse(args : _*)
+
+  checkArgs()
+
+  def has(builder: OptionSpec[_]): Boolean = options.has(builder)
+
+  def hasDescribeOption: Boolean = has(describeOpt)
+
+  def hasFromControllerOption: Boolean = has(fromControllerOpt)
+
+  def hasDryRunOption: Boolean = has(dryRunOpt)
+
+  def hasUpgradeAllOption: Boolean = has(upgradeAllOpt)
+
+  def hasDowngradeAllOption: Boolean = has(downgradeAllOpt)
+
+  def bootstrapServers: String = options.valueOf(bootstrapServerOpt)
+
+  def checkArgs(): Unit = {
+    CommandLineUtils.printHelpAndExitIfNeeded(this, "This tool describes and updates finalized features.")
+    val numActions = Seq(describeOpt, upgradeAllOpt, downgradeAllOpt).count(has)
+    if (numActions != 1) {
+      CommandLineUtils.printUsageAndDie(
+        parser,
+        "Command must include exactly one action: --describe, --upgrade-all, --downgrade-all.")
+    }
+    CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServerOpt)
+    if (hasDryRunOption && !hasUpgradeAllOption && !hasDowngradeAllOption) {
+      CommandLineUtils.printUsageAndDie(
+        parser,
+        "Command can contain --dry-run option only when either --upgrade-all or --downgrade-all actions are provided.")
+    }
+    if (hasFromControllerOption && !hasDescribeOption) {

Review comment:
       I just don't see it hinders the command to succeed, which means it's an ignorable property. Anyway, I'm not insisting either. 




----------------------------------------------------------------
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.

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



[GitHub] [kafka] kowshik commented on a change in pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#discussion_r504138505



##########
File path: core/src/test/scala/unit/kafka/admin/FeatureCommandTest.scala
##########
@@ -0,0 +1,245 @@
+/**
+ * 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 kafka.admin
+
+import kafka.api.KAFKA_2_7_IV0
+import kafka.server.{BaseRequestTest, KafkaConfig, KafkaServer}
+import kafka.utils.TestUtils
+import kafka.utils.TestUtils.waitUntilTrue
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+
+import org.junit.Assert.{assertEquals, assertTrue}
+import org.junit.Test
+import org.scalatest.Assertions.intercept
+
+class FeatureCommandTest extends BaseRequestTest {
+  override def brokerCount: Int = 3
+
+  override def brokerPropertyOverrides(props: Properties): Unit = {
+    props.put(KafkaConfig.InterBrokerProtocolVersionProp, KAFKA_2_7_IV0.toString)
+  }
+
+  private def defaultSupportedFeatures(): Features[SupportedVersionRange] = {
+    Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)),
+                                           Utils.mkEntry("feature_2", new SupportedVersionRange(1, 5))))
+  }
+
+  private def updateSupportedFeatures(features: Features[SupportedVersionRange],
+                                      targetServers: Set[KafkaServer]): Unit = {
+    targetServers.foreach(s => {
+      s.brokerFeatures.setSupportedFeatures(features)
+      s.zkClient.updateBrokerInfo(s.createBrokerInfo)
+    })
+
+    // Wait until updates to all BrokerZNode supported features propagate to the controller.
+    val brokerIds = targetServers.map(s => s.config.brokerId)
+    waitUntilTrue(
+      () => servers.exists(s => {
+        if (s.kafkaController.isActive) {
+          s.kafkaController.controllerContext.liveOrShuttingDownBrokers
+            .filter(b => brokerIds.contains(b.id))
+            .forall(b => {
+              b.features.equals(features)
+            })
+        } else {
+          false
+        }
+      }),
+      "Controller did not get broker updates")
+  }
+
+  private def updateSupportedFeaturesInAllBrokers(features: Features[SupportedVersionRange]): Unit = {
+    updateSupportedFeatures(features, Set[KafkaServer]() ++ servers)
+  }
+
+  /**
+   * Tests if the FeatureApis#describeFeatures API works as expected when describing features before and
+   * after upgrading features.
+   */
+  @Test
+  def testDescribeFeaturesSuccess(): Unit = {
+    updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+    val featureApis = new FeatureApis(new FeatureCommandOptions(Array("--bootstrap-server", brokerList, "--describe", "--from-controller")))
+    featureApis.setSupportedFeatures(defaultSupportedFeatures())
+    try {
+      val initialDescribeOutput = TestUtils.grabConsoleOutput(featureApis.describeFeatures())
+      val expectedInitialDescribeOutput =
+        "Feature: feature_1\tSupportedMinVersion: 1\tSupportedMaxVersion: 3\tFinalizedMinVersionLevel: -\tFinalizedMaxVersionLevel: -\tEpoch: 0\n" +
+        "Feature: feature_2\tSupportedMinVersion: 1\tSupportedMaxVersion: 5\tFinalizedMinVersionLevel: -\tFinalizedMaxVersionLevel: -\tEpoch: 0\n"
+      assertEquals(expectedInitialDescribeOutput, initialDescribeOutput)
+      featureApis.upgradeAllFeatures()
+      val finalDescribeOutput = TestUtils.grabConsoleOutput(featureApis.describeFeatures())
+      val expectedFinalDescribeOutput =
+        "Feature: feature_1\tSupportedMinVersion: 1\tSupportedMaxVersion: 3\tFinalizedMinVersionLevel: 1\tFinalizedMaxVersionLevel: 3\tEpoch: 1\n" +
+        "Feature: feature_2\tSupportedMinVersion: 1\tSupportedMaxVersion: 5\tFinalizedMinVersionLevel: 1\tFinalizedMaxVersionLevel: 5\tEpoch: 1\n"
+      assertEquals(expectedFinalDescribeOutput, finalDescribeOutput)
+    } finally {
+      featureApis.close()
+    }
+  }
+
+  /**
+   * Tests if the FeatureApis#upgradeAllFeatures API works as expected during a success case.
+   */
+  @Test
+  def testUpgradeAllFeaturesSuccess(): Unit = {
+    val upgradeOpts = new FeatureCommandOptions(Array("--bootstrap-server", brokerList, "--upgrade-all"))
+    val featureApis = new FeatureApis(upgradeOpts)
+    try {
+      // Step (1):
+      // - Update the supported features across all brokers.
+      // - Upgrade non-existing feature_1 to maxVersionLevel: 2.
+      // - Verify results.
+      val initialSupportedFeatures = Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new SupportedVersionRange(1, 2))))
+      updateSupportedFeaturesInAllBrokers(initialSupportedFeatures)
+      featureApis.setSupportedFeatures(initialSupportedFeatures)
+      var output = TestUtils.grabConsoleOutput(featureApis.upgradeAllFeatures())
+      var expected =
+        "      [Add]\tFeature: feature_1\tExistingFinalizedMaxVersion: -\tNewFinalizedMaxVersion: 2\tResult: OK\n"
+      assertEquals(expected, output)
+
+      // Step (2):
+      // - Update the supported features across all brokers.
+      // - Upgrade existing feature_1 to maxVersionLevel: 3.
+      // - Upgrade non-existing feature_2 to maxVersionLevel: 5.
+      // - Verify results.
+      updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+      featureApis.setSupportedFeatures(defaultSupportedFeatures())
+      output = TestUtils.grabConsoleOutput(featureApis.upgradeAllFeatures())
+      expected =
+        "  [Upgrade]\tFeature: feature_1\tExistingFinalizedMaxVersion: 2\tNewFinalizedMaxVersion: 3\tResult: OK\n" +
+        "      [Add]\tFeature: feature_2\tExistingFinalizedMaxVersion: -\tNewFinalizedMaxVersion: 5\tResult: OK\n"
+      assertEquals(expected, output)
+
+      // Step (3):
+      // - Perform an upgrade of all features again.
+      // - Since supported features have not changed, expect that the above action does not yield
+      //   any results.
+      output = TestUtils.grabConsoleOutput(featureApis.upgradeAllFeatures())
+      assertTrue(output.isEmpty)
+      featureApis.setOptions(upgradeOpts)
+      output = TestUtils.grabConsoleOutput(featureApis.upgradeAllFeatures())
+      assertTrue(output.isEmpty)
+    } finally {
+      featureApis.close()
+    }
+  }
+
+  /**
+   * Tests if the FeatureApis#downgradeAllFeatures API works as expected during a success case.
+   */
+  @Test
+  def testDowngradeFeaturesSuccess(): Unit = {
+    val downgradeOpts = new FeatureCommandOptions(Array("--bootstrap-server", brokerList, "--downgrade-all"))
+    val upgradeOpts = new FeatureCommandOptions(Array("--bootstrap-server", brokerList, "--upgrade-all"))
+    val featureApis = new FeatureApis(upgradeOpts)
+    try {
+      // Step (1):
+      // - Update the supported features across all brokers.
+      // - Upgrade non-existing feature_1 to maxVersionLevel: 3.
+      // - Upgrade non-existing feature_2 to maxVersionLevel: 5.
+      updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+      featureApis.setSupportedFeatures(defaultSupportedFeatures())
+      featureApis.upgradeAllFeatures()
+
+      // Step (2):
+      // - Downgrade existing feature_1 to maxVersionLevel: 2.
+      // - Delete feature_2 since it is no longer supported by the FeatureApis object.
+      // - Verify results.
+      val downgradedFeatures = Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new SupportedVersionRange(1, 2))))
+      featureApis.setSupportedFeatures(downgradedFeatures)
+      featureApis.setOptions(downgradeOpts)
+      var output = TestUtils.grabConsoleOutput(featureApis.downgradeAllFeatures())
+      var expected =
+        "[Downgrade]\tFeature: feature_1\tExistingFinalizedMaxVersion: 3\tNewFinalizedMaxVersion: 2\tResult: OK\n" +
+        "   [Delete]\tFeature: feature_2\tExistingFinalizedMaxVersion: 5\tNewFinalizedMaxVersion: -\tResult: OK\n"
+      assertEquals(expected, output)
+
+      // Step (3):
+      // - Perform a downgrade of all features again.
+      // - Since supported features have not changed, expect that the above action does not yield
+      //   any results.
+      updateSupportedFeaturesInAllBrokers(downgradedFeatures)
+      output = TestUtils.grabConsoleOutput(featureApis.downgradeAllFeatures())
+      assertTrue(output.isEmpty)
+
+      // Step (4):
+      // - Delete feature_1 since it is no longer supported by the FeatureApis object.
+      // - Verify results.
+      featureApis.setSupportedFeatures(Features.emptySupportedFeatures())
+      output = TestUtils.grabConsoleOutput(featureApis.downgradeAllFeatures())
+      expected =
+        "   [Delete]\tFeature: feature_1\tExistingFinalizedMaxVersion: 2\tNewFinalizedMaxVersion: -\tResult: OK\n"
+      assertEquals(expected, output)
+    } finally {
+      featureApis.close()
+    }
+  }
+
+  /**
+   * Tests if the FeatureApis#upgradeAllFeatures API works as expected during a partial failure case.
+   */
+  @Test
+  def testUpgradeFeaturesFailure(): Unit = {

Review comment:
       As such the downgrade can not fail currently on the server. The reason is the following: the `FeatureCommand` downgrades `maxVersionLevel` of existing finalized features to the max versions that it knows internally (since it links with the `BrokerFeatures` library). As a result, the only situation when the `FeatureCommand` can fail is if it tries to downgrade below the supported `minVersion`. But this can never happen since we don't (yet) have the facility on the broker to have a supported `minVersion` > 1 (this can only happen during feature deprecation, the support for this is future work).
   
   On a side note, we could maybe write a test using `AdminClientUnitTestEnv` to create a mock setup specifically for the downgrade test and inject a mock admin client into the `FeatureApis` class. But that looks overkill to me...




----------------------------------------------------------------
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.

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



[GitHub] [kafka] kowshik commented on a change in pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#discussion_r504118029



##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,350 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+    features.toList.sorted.foreach {
+      case feature =>
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        print(s"Feature: $feature")
+        print(s"\tSupportedMinVersion: $supportedMinVersion")
+        print(s"\tSupportedMaxVersion: $supportedMaxVersion")
+        print(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        print(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+        println(s"\tEpoch: $epoch")
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Downgrades existing finalized features to the highest max version levels known to this tool.
+   * The method may delete existing finalized features if they are no longer seen to be supported,
+   * but it does not add a feature that was not finalized previously. The results of the feature
+   * updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def downgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val supportedFeaturesMap = supportedFeatures.features
+    val updates = existingFinalizedFeatures.asScala.map {
+      case (feature, existingVersionRange) =>
+        val targetVersionRange = supportedFeaturesMap.get(feature)
+        if (targetVersionRange == null) {
+          val updateStr =
+            deleteOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+            s"\tNewFinalizedMaxVersion: -"
+          (feature, (updateStr, new FeatureUpdate(0, true)))
+        } else {
+          if (targetVersionRange.max < existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              downgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, true)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Applies the provided feature updates. If the --dry-run CLI option is provided, the method
+   * only prints the expected feature updates to STDOUT without applying them.
+   *
+   * @param updates the feature updates to be applied via the admin client
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = {
+    if (opts.hasDryRunOption) {
+      println("Expected feature updates:")
+      println(ListMap(updates.toSeq.sortBy(_._1):_*)
+                .map { case(_, (updateStr, _)) => updateStr}
+                .mkString("\n"))
+    } else {
+      val result = adminClient.updateFeatures(
+        updates.map { case(feature, (_, update)) => (feature, update)}.asJava,
+        new UpdateFeaturesOptions())
+      val failures = ListMap(result.values.asScala.toSeq.sortBy(_._1):_*).map {
+        case (feature, updateFuture) =>
+          val (updateStr, _) = updates(feature)
+          try {
+            updateFuture.get
+            println(updateStr + "\tResult: OK")
+            0
+          } catch {
+            case e: Exception =>
+              println(updateStr + "\tResult: FAILED due to " + e.getMessage)
+              1
+          }
+      }.sum
+      if (failures > 0) {

Review comment:
       Done. Good point. This is fixed now, as we are no longer printing exception from within `FeatureCommand.main`. Yes, the exception as such is useful for testing too.




----------------------------------------------------------------
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.

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



[GitHub] [kafka] kowshik commented on a change in pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#discussion_r506730195



##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,393 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case _: UpdateFeaturesException =>
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge feature APIs provided by the the Admin client with
+ * the requirements of the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = {
+    val props = new Properties()
+    props.putAll(opts.commandConfig)
+    props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.bootstrapServers)
+    Admin.create(props)
+  }
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  private[admin] def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  private[admin] def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+
+    features.toList.sorted.foreach {
+      feature =>
+        val output = new StringBuilder()
+        output.append(s"Feature: $feature")
+
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        output.append(s"\tSupportedMinVersion: $supportedMinVersion")
+        output.append(s"\tSupportedMaxVersion: $supportedMaxVersion")
+
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        output.append(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        output.append(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        output.append(s"\tEpoch: $epoch")
+
+        println(output)
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, Some((updateStr, new FeatureUpdate(targetVersionRange.max, false))))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, Some((updateStr, new FeatureUpdate(targetVersionRange.max, false))))
+          } else {
+            (feature, Option.empty)
+          }
+        }
+    }.filter {
+      case(_, updateInfo) => updateInfo.isDefined
+    }.map {
+      case(feature, updateInfo) => (feature, updateInfo.get)
+    }.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Downgrades existing finalized features to the highest max version levels known to this tool.
+   * The method may delete existing finalized features if they are no longer seen to be supported,
+   * but it does not add a feature that was not finalized previously. The results of the feature
+   * updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def downgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val supportedFeaturesMap = supportedFeatures.features
+    val updates = existingFinalizedFeatures.asScala.map {
+      case (feature, existingVersionRange) =>
+        val targetVersionRange = supportedFeaturesMap.get(feature)
+        if (targetVersionRange == null) {
+          val updateStr =
+            deleteOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+            s"\tNewFinalizedMaxVersion: -"
+          (feature, Some(updateStr, new FeatureUpdate(0, true)))
+        } else {
+          if (targetVersionRange.max < existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              downgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, Some(updateStr, new FeatureUpdate(targetVersionRange.max, true)))
+          } else {
+            (feature, Option.empty)
+          }
+        }
+    }.filter {
+      case(_, updateInfo) => updateInfo.isDefined
+    }.map {
+      case(feature, updateInfo) => (feature, updateInfo.get)
+    }.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Applies the provided feature updates. If the --dry-run CLI option is provided, the method
+   * only prints the expected feature updates to STDOUT without applying them.
+   *
+   * @param updates the feature updates to be applied via the admin client
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = {
+    if (opts.hasDryRunOption) {
+      println("Expected feature updates:" + ListMap(
+        updates
+          .toSeq
+          .sortBy { case(feature, _) => feature} :_*)
+          .map { case(_, (updateStr, _)) => updateStr}
+          .mkString("\n"))
+    } else {
+      val result = adminClient.updateFeatures(
+        updates
+          .map { case(feature, (_, update)) => (feature, update)}
+          .asJava,
+        new UpdateFeaturesOptions())
+      val resultSortedByFeature = ListMap(
+        result
+          .values
+          .asScala
+          .toSeq
+          .sortBy { case(feature, _) => feature} :_*)
+      val failures = resultSortedByFeature.map {
+        case (feature, updateFuture) =>
+          val (updateStr, _) = updates(feature)
+          try {
+            updateFuture.get
+            println(updateStr + "\tResult: OK")
+            0
+          } catch {
+            case e: Exception =>
+              println(updateStr + "\tResult: FAILED due to " + e)

Review comment:
       Done. Great point!

##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,393 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case _: UpdateFeaturesException =>
+        exitCode = 1
+      case e: Throwable =>

Review comment:
       Done.




----------------------------------------------------------------
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.

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



[GitHub] [kafka] junrao commented on a change in pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#discussion_r506696396



##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,393 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case _: UpdateFeaturesException =>
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge feature APIs provided by the the Admin client with
+ * the requirements of the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = {
+    val props = new Properties()
+    props.putAll(opts.commandConfig)
+    props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.bootstrapServers)
+    Admin.create(props)
+  }
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  private[admin] def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  private[admin] def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+
+    features.toList.sorted.foreach {
+      feature =>
+        val output = new StringBuilder()
+        output.append(s"Feature: $feature")
+
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        output.append(s"\tSupportedMinVersion: $supportedMinVersion")
+        output.append(s"\tSupportedMaxVersion: $supportedMaxVersion")
+
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        output.append(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        output.append(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        output.append(s"\tEpoch: $epoch")
+
+        println(output)
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, Some((updateStr, new FeatureUpdate(targetVersionRange.max, false))))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, Some((updateStr, new FeatureUpdate(targetVersionRange.max, false))))
+          } else {
+            (feature, Option.empty)
+          }
+        }
+    }.filter {
+      case(_, updateInfo) => updateInfo.isDefined
+    }.map {
+      case(feature, updateInfo) => (feature, updateInfo.get)
+    }.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Downgrades existing finalized features to the highest max version levels known to this tool.
+   * The method may delete existing finalized features if they are no longer seen to be supported,
+   * but it does not add a feature that was not finalized previously. The results of the feature
+   * updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def downgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val supportedFeaturesMap = supportedFeatures.features
+    val updates = existingFinalizedFeatures.asScala.map {
+      case (feature, existingVersionRange) =>
+        val targetVersionRange = supportedFeaturesMap.get(feature)
+        if (targetVersionRange == null) {
+          val updateStr =
+            deleteOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+            s"\tNewFinalizedMaxVersion: -"
+          (feature, Some(updateStr, new FeatureUpdate(0, true)))
+        } else {
+          if (targetVersionRange.max < existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              downgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, Some(updateStr, new FeatureUpdate(targetVersionRange.max, true)))
+          } else {
+            (feature, Option.empty)
+          }
+        }
+    }.filter {
+      case(_, updateInfo) => updateInfo.isDefined
+    }.map {
+      case(feature, updateInfo) => (feature, updateInfo.get)
+    }.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Applies the provided feature updates. If the --dry-run CLI option is provided, the method
+   * only prints the expected feature updates to STDOUT without applying them.
+   *
+   * @param updates the feature updates to be applied via the admin client
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = {
+    if (opts.hasDryRunOption) {
+      println("Expected feature updates:" + ListMap(
+        updates
+          .toSeq
+          .sortBy { case(feature, _) => feature} :_*)
+          .map { case(_, (updateStr, _)) => updateStr}
+          .mkString("\n"))
+    } else {
+      val result = adminClient.updateFeatures(
+        updates
+          .map { case(feature, (_, update)) => (feature, update)}
+          .asJava,
+        new UpdateFeaturesOptions())
+      val resultSortedByFeature = ListMap(
+        result
+          .values
+          .asScala
+          .toSeq
+          .sortBy { case(feature, _) => feature} :_*)
+      val failures = resultSortedByFeature.map {
+        case (feature, updateFuture) =>
+          val (updateStr, _) = updates(feature)
+          try {
+            updateFuture.get
+            println(updateStr + "\tResult: OK")
+            0
+          } catch {
+            case e: Exception =>
+              println(updateStr + "\tResult: FAILED due to " + e)

Review comment:
       If we hit ExecutionException and the exception has a cause, the wrapper ExecutionException is not really relevant and we should just print the cause. See TopicCommand as a reference. 

##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,393 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case _: UpdateFeaturesException =>
+        exitCode = 1
+      case e: Throwable =>

Review comment:
       It seems that we can hit ExecutionException through describeFeatures. We want to handle it properly too.




----------------------------------------------------------------
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.

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



[GitHub] [kafka] kowshik commented on a change in pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#discussion_r504118029



##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,350 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+    features.toList.sorted.foreach {
+      case feature =>
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        print(s"Feature: $feature")
+        print(s"\tSupportedMinVersion: $supportedMinVersion")
+        print(s"\tSupportedMaxVersion: $supportedMaxVersion")
+        print(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        print(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+        println(s"\tEpoch: $epoch")
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Downgrades existing finalized features to the highest max version levels known to this tool.
+   * The method may delete existing finalized features if they are no longer seen to be supported,
+   * but it does not add a feature that was not finalized previously. The results of the feature
+   * updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def downgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val supportedFeaturesMap = supportedFeatures.features
+    val updates = existingFinalizedFeatures.asScala.map {
+      case (feature, existingVersionRange) =>
+        val targetVersionRange = supportedFeaturesMap.get(feature)
+        if (targetVersionRange == null) {
+          val updateStr =
+            deleteOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+            s"\tNewFinalizedMaxVersion: -"
+          (feature, (updateStr, new FeatureUpdate(0, true)))
+        } else {
+          if (targetVersionRange.max < existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              downgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, true)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Applies the provided feature updates. If the --dry-run CLI option is provided, the method
+   * only prints the expected feature updates to STDOUT without applying them.
+   *
+   * @param updates the feature updates to be applied via the admin client
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = {
+    if (opts.hasDryRunOption) {
+      println("Expected feature updates:")
+      println(ListMap(updates.toSeq.sortBy(_._1):_*)
+                .map { case(_, (updateStr, _)) => updateStr}
+                .mkString("\n"))
+    } else {
+      val result = adminClient.updateFeatures(
+        updates.map { case(feature, (_, update)) => (feature, update)}.asJava,
+        new UpdateFeaturesOptions())
+      val failures = ListMap(result.values.asScala.toSeq.sortBy(_._1):_*).map {
+        case (feature, updateFuture) =>
+          val (updateStr, _) = updates(feature)
+          try {
+            updateFuture.get
+            println(updateStr + "\tResult: OK")
+            0
+          } catch {
+            case e: Exception =>
+              println(updateStr + "\tResult: FAILED due to " + e.getMessage)
+              1
+          }
+      }.sum
+      if (failures > 0) {

Review comment:
       Done. This is fixed now, as we are no longer printing exception.




----------------------------------------------------------------
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.

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



[GitHub] [kafka] kowshik commented on a change in pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#discussion_r504118829



##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,350 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+    features.toList.sorted.foreach {

Review comment:
       It is easy to compare command outputs when they are sorted based on feature names. Also, it is easier to write unit/integration tests when the output is uniform.




----------------------------------------------------------------
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.

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



[GitHub] [kafka] kowshik commented on pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
kowshik commented on pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#issuecomment-706762993


   @abbccdda @junrao : This PR is ready for review. 


----------------------------------------------------------------
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.

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



[GitHub] [kafka] kowshik commented on a change in pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#discussion_r504116128



##########
File path: core/src/main/scala/kafka/admin/FeatureCommand.scala
##########
@@ -0,0 +1,350 @@
+/**
+ * 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 kafka.admin
+
+import kafka.server.BrokerFeatures
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit}
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions}
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+import scala.collection.Seq
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters._
+
+import joptsimple.OptionSpec
+
+object FeatureCommand {
+
+  def main(args: Array[String]): Unit = {
+    val opts = new FeatureCommandOptions(args)
+    val featureApis = new FeatureApis(opts)
+    var exitCode = 0
+    try {
+      featureApis.execute()
+    } catch {
+      case e: IllegalArgumentException =>
+        printException(e)
+        opts.parser.printHelpOn(System.err)
+        exitCode = 1
+      case e: Throwable =>
+        printException(e)
+        exitCode = 1
+    } finally {
+      featureApis.close()
+      Exit.exit(exitCode)
+    }
+  }
+
+  private def printException(exception: Throwable): Unit = {
+    System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception))
+  }
+}
+
+class UpdateFeaturesException(message: String) extends RuntimeException(message)
+
+/**
+ * A class that provides necessary APIs to bridge the Admin client feature APIs with the CLI tool.
+ *
+ * @param opts the CLI options
+ */
+class FeatureApis(var opts: FeatureCommandOptions) {
+  private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures
+  private val adminClient = createAdminClient()
+
+  private def pad(op: String): String = {
+    f"$op%11s"
+  }
+
+  private val addOp = pad("[Add]")
+  private val upgradeOp = pad("[Upgrade]")
+  private val deleteOp = pad("[Delete]")
+  private val downgradeOp = pad("[Downgrade]")
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  def setOptions(newOpts: FeatureCommandOptions): Unit = {
+    opts = newOpts
+  }
+
+  private def describeFeatures(sendRequestToController: Boolean): FeatureMetadata = {
+    val options = new DescribeFeaturesOptions().sendRequestToController(sendRequestToController)
+    adminClient.describeFeatures(options).featureMetadata().get()
+  }
+
+  /**
+   * Describes the supported and finalized features. If the --from-controller CLI option
+   * is provided, then the request is issued only to the controller, otherwise the request is issued
+   * to any of the provided bootstrap servers.
+   */
+  def describeFeatures(): Unit = {
+    val result = describeFeatures(opts.hasFromControllerOption)
+    val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet
+    features.toList.sorted.foreach {
+      case feature =>
+        val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = {
+          val finalizedVersionRange = result.finalizedFeatures.get(feature)
+          if (finalizedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel)
+          }
+        }
+        val epoch = {
+          if (result.finalizedFeaturesEpoch.isPresent) {
+            result.finalizedFeaturesEpoch.get.toString
+          } else {
+            "-"
+          }
+        }
+        val (supportedMinVersion, supportedMaxVersion) = {
+          val supportedVersionRange = result.supportedFeatures.get(feature)
+          if (supportedVersionRange == null) {
+            ("-", "-")
+          } else {
+            (supportedVersionRange.minVersion, supportedVersionRange.maxVersion)
+          }
+        }
+        print(s"Feature: $feature")
+        print(s"\tSupportedMinVersion: $supportedMinVersion")
+        print(s"\tSupportedMaxVersion: $supportedMaxVersion")
+        print(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel")
+        print(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel")
+        println(s"\tEpoch: $epoch")
+    }
+  }
+
+  /**
+   * Upgrades all features known to this tool to their highest max version levels. The method may
+   * add new finalized features if they were not finalized previously, but it does not delete
+   * any existing finalized feature. The results of the feature updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def upgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val updates = supportedFeatures.features.asScala.map {
+      case (feature, targetVersionRange) =>
+        val existingVersionRange = existingFinalizedFeatures.get(feature)
+        if (existingVersionRange == null) {
+          val updateStr =
+            addOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: -" +
+            s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+          (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+        } else {
+          if (targetVersionRange.max > existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              upgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, false)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Downgrades existing finalized features to the highest max version levels known to this tool.
+   * The method may delete existing finalized features if they are no longer seen to be supported,
+   * but it does not add a feature that was not finalized previously. The results of the feature
+   * updates are written to STDOUT.
+   *
+   * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature
+   * updates to STDOUT, without applying them.
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  def downgradeAllFeatures(): Unit = {
+    val metadata = describeFeatures(true)
+    val existingFinalizedFeatures = metadata.finalizedFeatures
+    val supportedFeaturesMap = supportedFeatures.features
+    val updates = existingFinalizedFeatures.asScala.map {
+      case (feature, existingVersionRange) =>
+        val targetVersionRange = supportedFeaturesMap.get(feature)
+        if (targetVersionRange == null) {
+          val updateStr =
+            deleteOp +
+            s"\tFeature: $feature" +
+            s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+            s"\tNewFinalizedMaxVersion: -"
+          (feature, (updateStr, new FeatureUpdate(0, true)))
+        } else {
+          if (targetVersionRange.max < existingVersionRange.maxVersionLevel) {
+            val updateStr =
+              downgradeOp +
+              s"\tFeature: $feature" +
+              s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" +
+              s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}"
+            (feature, (updateStr, new FeatureUpdate(targetVersionRange.max, true)))
+          } else {
+            (feature, null)
+          }
+        }
+    }.filter{ case(_, updateInfo) => updateInfo != null}.toMap
+
+    if (updates.nonEmpty) {
+      maybeApplyFeatureUpdates(updates)
+    }
+  }
+
+  /**
+   * Applies the provided feature updates. If the --dry-run CLI option is provided, the method
+   * only prints the expected feature updates to STDOUT without applying them.
+   *
+   * @param updates the feature updates to be applied via the admin client
+   *
+   * @throws UpdateFeaturesException if at least one of the feature updates failed
+   */
+  private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = {
+    if (opts.hasDryRunOption) {
+      println("Expected feature updates:")
+      println(ListMap(updates.toSeq.sortBy(_._1):_*)
+                .map { case(_, (updateStr, _)) => updateStr}
+                .mkString("\n"))
+    } else {
+      val result = adminClient.updateFeatures(
+        updates.map { case(feature, (_, update)) => (feature, update)}.asJava,
+        new UpdateFeaturesOptions())
+      val failures = ListMap(result.values.asScala.toSeq.sortBy(_._1):_*).map {
+        case (feature, updateFuture) =>
+          val (updateStr, _) = updates(feature)
+          try {
+            updateFuture.get
+            println(updateStr + "\tResult: OK")
+            0
+          } catch {
+            case e: Exception =>
+              println(updateStr + "\tResult: FAILED due to " + e.getMessage)
+              1
+          }
+      }.sum
+      if (failures > 0) {
+        throw new UpdateFeaturesException(s"$failures feature updates failed!")
+      }
+    }
+  }
+
+  def execute(): Unit = {
+    if (opts.hasDescribeOption) {
+      describeFeatures()
+    } else if (opts.hasUpgradeAllOption) {
+      upgradeAllFeatures()
+    } else if (opts.hasDowngradeAllOption) {
+      downgradeAllFeatures()
+    } else {
+      throw new IllegalStateException("Unexpected state: no CLI command could be executed.")
+    }
+  }
+
+  def close(): Unit = {
+    adminClient.close()
+  }
+
+  private def createAdminClient(): Admin = {
+    val props = new Properties()
+    props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.bootstrapServers)
+    Admin.create(props)
+  }
+}
+
+class FeatureCommandOptions(args: Array[String]) extends CommandDefaultOptions(args) {
+  private val bootstrapServerOpt =
+    parser.accepts("bootstrap-server", "REQUIRED: The Kafka server(s) to connect to.")
+      .withRequiredArg
+      .describedAs("server to connect to")
+      .ofType(classOf[String])
+  private val describeOpt = parser.accepts(
+    "describe",
+    "Describe supported and finalized features. By default, the features are described from a" +
+    " random broker. The request can be optionally directed only to the controller using the" +
+    " --from-controller option.")
+  private val fromControllerOpt = parser.accepts(
+    "from-controller",
+    "Describe supported and finalized features from the controller.")
+  private val upgradeAllOpt = parser.accepts(
+    "upgrade-all",
+    "Upgrades all finalized features to the maximum version levels known to the tool." +
+    " This command finalizes new features known to the tool that were never finalized" +
+    " previously in the cluster, but it is guaranteed to not delete any existing feature.")
+  private val downgradeAllOpt = parser.accepts(
+    "downgrade-all",
+    "Downgrades all finalized features to the maximum version levels known to the tool." +
+    " This command deletes unknown features from the list of finalized features in the" +
+    " cluster, but it is guaranteed to not add a new feature.")
+  private val dryRunOpt = parser.accepts(
+    "dry-run",
+    "Performs a dry-run of upgrade/downgrade mutations to finalized feature without applying them.")
+
+  options = parser.parse(args : _*)
+
+  checkArgs()
+
+  def has(builder: OptionSpec[_]): Boolean = options.has(builder)
+
+  def hasDescribeOption: Boolean = has(describeOpt)

Review comment:
       This convention is used in `TopicCommand.scala`. See: https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/admin/TopicCommand.scala#L698-L702




----------------------------------------------------------------
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.

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



[GitHub] [kafka] kowshik commented on a change in pull request #9409: KAFKA-10599: Implement basic CLI tool for feature versioning system

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #9409:
URL: https://github.com/apache/kafka/pull/9409#discussion_r504138505



##########
File path: core/src/test/scala/unit/kafka/admin/FeatureCommandTest.scala
##########
@@ -0,0 +1,245 @@
+/**
+ * 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 kafka.admin
+
+import kafka.api.KAFKA_2_7_IV0
+import kafka.server.{BaseRequestTest, KafkaConfig, KafkaServer}
+import kafka.utils.TestUtils
+import kafka.utils.TestUtils.waitUntilTrue
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.utils.Utils
+
+import java.util.Properties
+
+import org.junit.Assert.{assertEquals, assertTrue}
+import org.junit.Test
+import org.scalatest.Assertions.intercept
+
+class FeatureCommandTest extends BaseRequestTest {
+  override def brokerCount: Int = 3
+
+  override def brokerPropertyOverrides(props: Properties): Unit = {
+    props.put(KafkaConfig.InterBrokerProtocolVersionProp, KAFKA_2_7_IV0.toString)
+  }
+
+  private def defaultSupportedFeatures(): Features[SupportedVersionRange] = {
+    Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)),
+                                           Utils.mkEntry("feature_2", new SupportedVersionRange(1, 5))))
+  }
+
+  private def updateSupportedFeatures(features: Features[SupportedVersionRange],
+                                      targetServers: Set[KafkaServer]): Unit = {
+    targetServers.foreach(s => {
+      s.brokerFeatures.setSupportedFeatures(features)
+      s.zkClient.updateBrokerInfo(s.createBrokerInfo)
+    })
+
+    // Wait until updates to all BrokerZNode supported features propagate to the controller.
+    val brokerIds = targetServers.map(s => s.config.brokerId)
+    waitUntilTrue(
+      () => servers.exists(s => {
+        if (s.kafkaController.isActive) {
+          s.kafkaController.controllerContext.liveOrShuttingDownBrokers
+            .filter(b => brokerIds.contains(b.id))
+            .forall(b => {
+              b.features.equals(features)
+            })
+        } else {
+          false
+        }
+      }),
+      "Controller did not get broker updates")
+  }
+
+  private def updateSupportedFeaturesInAllBrokers(features: Features[SupportedVersionRange]): Unit = {
+    updateSupportedFeatures(features, Set[KafkaServer]() ++ servers)
+  }
+
+  /**
+   * Tests if the FeatureApis#describeFeatures API works as expected when describing features before and
+   * after upgrading features.
+   */
+  @Test
+  def testDescribeFeaturesSuccess(): Unit = {
+    updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+    val featureApis = new FeatureApis(new FeatureCommandOptions(Array("--bootstrap-server", brokerList, "--describe", "--from-controller")))
+    featureApis.setSupportedFeatures(defaultSupportedFeatures())
+    try {
+      val initialDescribeOutput = TestUtils.grabConsoleOutput(featureApis.describeFeatures())
+      val expectedInitialDescribeOutput =
+        "Feature: feature_1\tSupportedMinVersion: 1\tSupportedMaxVersion: 3\tFinalizedMinVersionLevel: -\tFinalizedMaxVersionLevel: -\tEpoch: 0\n" +
+        "Feature: feature_2\tSupportedMinVersion: 1\tSupportedMaxVersion: 5\tFinalizedMinVersionLevel: -\tFinalizedMaxVersionLevel: -\tEpoch: 0\n"
+      assertEquals(expectedInitialDescribeOutput, initialDescribeOutput)
+      featureApis.upgradeAllFeatures()
+      val finalDescribeOutput = TestUtils.grabConsoleOutput(featureApis.describeFeatures())
+      val expectedFinalDescribeOutput =
+        "Feature: feature_1\tSupportedMinVersion: 1\tSupportedMaxVersion: 3\tFinalizedMinVersionLevel: 1\tFinalizedMaxVersionLevel: 3\tEpoch: 1\n" +
+        "Feature: feature_2\tSupportedMinVersion: 1\tSupportedMaxVersion: 5\tFinalizedMinVersionLevel: 1\tFinalizedMaxVersionLevel: 5\tEpoch: 1\n"
+      assertEquals(expectedFinalDescribeOutput, finalDescribeOutput)
+    } finally {
+      featureApis.close()
+    }
+  }
+
+  /**
+   * Tests if the FeatureApis#upgradeAllFeatures API works as expected during a success case.
+   */
+  @Test
+  def testUpgradeAllFeaturesSuccess(): Unit = {
+    val upgradeOpts = new FeatureCommandOptions(Array("--bootstrap-server", brokerList, "--upgrade-all"))
+    val featureApis = new FeatureApis(upgradeOpts)
+    try {
+      // Step (1):
+      // - Update the supported features across all brokers.
+      // - Upgrade non-existing feature_1 to maxVersionLevel: 2.
+      // - Verify results.
+      val initialSupportedFeatures = Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new SupportedVersionRange(1, 2))))
+      updateSupportedFeaturesInAllBrokers(initialSupportedFeatures)
+      featureApis.setSupportedFeatures(initialSupportedFeatures)
+      var output = TestUtils.grabConsoleOutput(featureApis.upgradeAllFeatures())
+      var expected =
+        "      [Add]\tFeature: feature_1\tExistingFinalizedMaxVersion: -\tNewFinalizedMaxVersion: 2\tResult: OK\n"
+      assertEquals(expected, output)
+
+      // Step (2):
+      // - Update the supported features across all brokers.
+      // - Upgrade existing feature_1 to maxVersionLevel: 3.
+      // - Upgrade non-existing feature_2 to maxVersionLevel: 5.
+      // - Verify results.
+      updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+      featureApis.setSupportedFeatures(defaultSupportedFeatures())
+      output = TestUtils.grabConsoleOutput(featureApis.upgradeAllFeatures())
+      expected =
+        "  [Upgrade]\tFeature: feature_1\tExistingFinalizedMaxVersion: 2\tNewFinalizedMaxVersion: 3\tResult: OK\n" +
+        "      [Add]\tFeature: feature_2\tExistingFinalizedMaxVersion: -\tNewFinalizedMaxVersion: 5\tResult: OK\n"
+      assertEquals(expected, output)
+
+      // Step (3):
+      // - Perform an upgrade of all features again.
+      // - Since supported features have not changed, expect that the above action does not yield
+      //   any results.
+      output = TestUtils.grabConsoleOutput(featureApis.upgradeAllFeatures())
+      assertTrue(output.isEmpty)
+      featureApis.setOptions(upgradeOpts)
+      output = TestUtils.grabConsoleOutput(featureApis.upgradeAllFeatures())
+      assertTrue(output.isEmpty)
+    } finally {
+      featureApis.close()
+    }
+  }
+
+  /**
+   * Tests if the FeatureApis#downgradeAllFeatures API works as expected during a success case.
+   */
+  @Test
+  def testDowngradeFeaturesSuccess(): Unit = {
+    val downgradeOpts = new FeatureCommandOptions(Array("--bootstrap-server", brokerList, "--downgrade-all"))
+    val upgradeOpts = new FeatureCommandOptions(Array("--bootstrap-server", brokerList, "--upgrade-all"))
+    val featureApis = new FeatureApis(upgradeOpts)
+    try {
+      // Step (1):
+      // - Update the supported features across all brokers.
+      // - Upgrade non-existing feature_1 to maxVersionLevel: 3.
+      // - Upgrade non-existing feature_2 to maxVersionLevel: 5.
+      updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+      featureApis.setSupportedFeatures(defaultSupportedFeatures())
+      featureApis.upgradeAllFeatures()
+
+      // Step (2):
+      // - Downgrade existing feature_1 to maxVersionLevel: 2.
+      // - Delete feature_2 since it is no longer supported by the FeatureApis object.
+      // - Verify results.
+      val downgradedFeatures = Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new SupportedVersionRange(1, 2))))
+      featureApis.setSupportedFeatures(downgradedFeatures)
+      featureApis.setOptions(downgradeOpts)
+      var output = TestUtils.grabConsoleOutput(featureApis.downgradeAllFeatures())
+      var expected =
+        "[Downgrade]\tFeature: feature_1\tExistingFinalizedMaxVersion: 3\tNewFinalizedMaxVersion: 2\tResult: OK\n" +
+        "   [Delete]\tFeature: feature_2\tExistingFinalizedMaxVersion: 5\tNewFinalizedMaxVersion: -\tResult: OK\n"
+      assertEquals(expected, output)
+
+      // Step (3):
+      // - Perform a downgrade of all features again.
+      // - Since supported features have not changed, expect that the above action does not yield
+      //   any results.
+      updateSupportedFeaturesInAllBrokers(downgradedFeatures)
+      output = TestUtils.grabConsoleOutput(featureApis.downgradeAllFeatures())
+      assertTrue(output.isEmpty)
+
+      // Step (4):
+      // - Delete feature_1 since it is no longer supported by the FeatureApis object.
+      // - Verify results.
+      featureApis.setSupportedFeatures(Features.emptySupportedFeatures())
+      output = TestUtils.grabConsoleOutput(featureApis.downgradeAllFeatures())
+      expected =
+        "   [Delete]\tFeature: feature_1\tExistingFinalizedMaxVersion: 2\tNewFinalizedMaxVersion: -\tResult: OK\n"
+      assertEquals(expected, output)
+    } finally {
+      featureApis.close()
+    }
+  }
+
+  /**
+   * Tests if the FeatureApis#upgradeAllFeatures API works as expected during a partial failure case.
+   */
+  @Test
+  def testUpgradeFeaturesFailure(): Unit = {

Review comment:
       As such the downgrade can not fail currently on the server. The reason is the following: the `FeatureCommand` downgrades `maxVersionLevel` of existing finalized features to the max versions that it knows internally (since it links against the `BrokerFeatures` library). So the only situation when the command can fail is if it tries to downgrade below supported `minVersion`. But this can never happen since we don't (yet) have the facility on the broker to have a supported `minVersion` > 1 (this can only happen during feature deprecation, the support for this is future work).




----------------------------------------------------------------
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.

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