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/07/09 10:51:01 UTC

[GitHub] [kafka] kowshik opened a new pull request #9001: KAFKA-10028: Implement KIP-584 write path

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


   


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/resources/common/message/UpdateFeaturesRequest.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "request",
+  "name": "UpdateFeaturesRequest",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "FeatureUpdates", "type": "[]FeatureUpdateKey", "versions": "0+",
+      "about": "The list of updates to finalized features.", "fields": [
+      {"name": "Feature", "type": "string", "versions": "0+", "mapKey": true,
+        "about": "The name of the finalized feature to be updated."},
+      {"name": "MaxVersionLevel", "type": "int16", "versions": "0+",
+        "about": "The new maximum version level for the finalized feature. A value >= 1 is valid. A value < 1, is special, and can be used to request the deletion of the finalized feature."},
+      {"name": "AllowDowngrade", "type": "bool", "versions": "0+",

Review comment:
       Done. Fixed now.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1844,185 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns a suitable error.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+    // NOTE: Below we set the finalized min version level to be the default minimum version
+    // level. If the finalized feature already exists, then, this can cause deprecation of all
+    // version levels in the closed range:
+    // [existingVersionRange.min(), defaultMinVersionLevel - 1].
+    val defaultMinVersionLevel = brokerFeatures.defaultMinVersionLevel(update.feature)
+    val newVersionRange = new FinalizedVersionRange(defaultMinVersionLevel, update.maxVersionLevel)
+    val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+      val singleFinalizedFeature =
+        Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange)))
+      BrokerFeatures.hasIncompatibleFeatures(broker.features, singleFinalizedFeature)
+    })
+    if (numIncompatibleBrokers == 0) {
+      Left(newVersionRange)
+    } else {
+      Right(
+        new ApiError(Errors.INVALID_REQUEST,
+                     s"Could not apply finalized feature update because $numIncompatibleBrokers" +
+                     " brokers were found to have incompatible features."))
+    }
+  }
+
+  /**
+   * Validate and process a finalized feature update on an existing FinalizedVersionRange for the
+   * feature.
+   *
+   * If the processing is successful, then, the return value contains:
+   * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature.
+   * 2. Option.empty, if the feature update was meant to delete the feature.
+   *
+   * If the processing failed, then returned value contains a suitable ApiError.
+   *
+   * @param update                 the feature update to be processed.
+   * @param existingVersionRange   the existing FinalizedVersionRange which can be empty when no
+   *                               FinalizedVersionRange exists for the associated feature
+   *
+   * @return                       the new FinalizedVersionRange or error, as described above.
+   */
+  private def processFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey,
+                                   existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = {
+    def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = {
+      newFinalizedVersionRangeOrIncompatibilityError(update)
+        .fold(versionRange => Left(Some(versionRange)), error => Right(error))
+    }
+
+    if (update.feature.isEmpty) {
+      // Check that the feature name is not empty.
+      Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty."))
+    } else {
+      // We handle deletion requests separately from non-deletion requests.
+      if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+        if (existingVersionRange.isEmpty) {
+          // Disallow deletion of a non-existing finalized feature.
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             s"Can not delete non-existing finalized feature: '${update.feature}'"))
+        } else {
+          Left(Option.empty)
+        }
+      } else if (update.maxVersionLevel() < 1) {
+        // Disallow deletion of a finalized feature without allowDowngrade flag set.
+        Right(new ApiError(Errors.INVALID_REQUEST,
+                           s"Can not provide maxVersionLevel: ${update.maxVersionLevel} less" +
+                           s" than 1 for feature: '${update.feature}' without setting the" +
+                           " allowDowngrade flag to true in the request."))
+      } else {
+        existingVersionRange.map(existing =>
+          if (update.maxVersionLevel == existing.max) {
+            // Disallow a case where target maxVersionLevel matches existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+              s"Can not ${if (update.allowDowngrade) "downgrade" else "upgrade"}" +
+                s" a finalized feature: '${update.feature}' from existing" +
+                s" maxVersionLevel:${existing.max} to the same value."))
+          } else if (update.maxVersionLevel < existing.max && !update.allowDowngrade) {
+            // Disallow downgrade of a finalized feature without the allowDowngrade flag set.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+              s"Can not downgrade finalized feature: '${update.feature}' from" +
+                s" existing maxVersionLevel:${existing.max} to provided" +
+                s" maxVersionLevel:${update.maxVersionLevel} without setting the" +
+                " allowDowngrade flag in the request."))
+          } else if (update.allowDowngrade && update.maxVersionLevel > existing.max) {
+            // Disallow a request that sets allowDowngrade flag without specifying a
+            // maxVersionLevel that's lower than the existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+              s"When finalized feature: '${update.feature}' has the allowDowngrade" +
+                " flag set in the request, the provided" +
+                s" maxVersionLevel:${update.maxVersionLevel} can not be greater than" +
+                s" existing maxVersionLevel:${existing.max}."))
+          } else if (update.maxVersionLevel() < existing.min) {
+            // Disallow downgrade of a finalized feature below the existing finalized
+            // minVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+              s"Can not downgrade finalized feature: '${update.feature}' to" +
+                s" maxVersionLevel:${update.maxVersionLevel} because it's lower than" +
+                s" the existing minVersionLevel:${existing.min}."))
+          } else {
+            newVersionRangeOrError(update)
+          }
+        ).getOrElse(newVersionRangeOrError(update))
+      }
+    }
+  }
+
+  private def processFeatureUpdates(request: UpdateFeaturesRequest,
+                                    callback: UpdateFeaturesCallback): Unit = {
+    if (isActive) {
+      processFeatureUpdatesWithActiveController(request, callback)
+    } else {
+      val results = request.data().featureUpdates().asScala.map {
+        update => update.feature() -> new ApiError(Errors.NOT_CONTROLLER)
+      }.toMap
+      callback(results)
+    }
+  }
+
+  private def processFeatureUpdatesWithActiveController(request: UpdateFeaturesRequest,
+                                                        callback: UpdateFeaturesCallback): Unit = {
+    val updates = request.data.featureUpdates
+    val existingFeatures = featureCache.get
+      .map(featuresAndEpoch => featuresAndEpoch.features.features().asScala)
+      .getOrElse(Map[String, FinalizedVersionRange]())
+    // Map of feature to FinalizedVersionRange. This contains the target features to be eventually
+    // written to FeatureZNode.
+    val targetFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]() ++ existingFeatures
+    // Map of feature to error.
+    var errors = scala.collection.mutable.Map[String, ApiError]()
+
+    // Process each FeatureUpdate.
+    // If a FeatureUpdate is found to be valid, then the corresponding entry in errors would contain
+    // Errors.NONE. Otherwise the entry would contain the appropriate error.
+    updates.asScala.iterator.foreach { update =>
+      processFeatureUpdate(update, existingFeatures.get(update.feature())) match {
+        case Left(newVersionRangeOrNone) =>
+          newVersionRangeOrNone
+            .map(newVersionRange => targetFeatures += (update.feature() -> newVersionRange))
+            .getOrElse(targetFeatures -= update.feature())
+          errors += (update.feature() -> new ApiError(Errors.NONE))
+        case Right(featureUpdateFailureReason) =>
+          errors += (update.feature() -> featureUpdateFailureReason)
+      }
+    }
+
+    if (existingFeatures.equals(targetFeatures)) {

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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesResult.java
##########
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.common.KafkaFuture;
+
+/**
+ * The result of the {@link Admin#describeFeatures(DescribeFeaturesOptions)} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+public class DescribeFeaturesResult {
+
+    private final KafkaFuture<FeatureMetadata> future;
+
+    public DescribeFeaturesResult(KafkaFuture<FeatureMetadata> future) {

Review comment:
       Could we make the constructor non-public?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesResult.java
##########
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Map;
+import org.apache.kafka.common.KafkaFuture;
+
+/**
+ * The result of the {@link Admin#updateFeatures(Map, UpdateFeaturesOptions)} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+public class UpdateFeaturesResult {
+    private final Map<String, KafkaFuture<Void>> futures;
+
+    /**
+     * @param futures   a map from feature name to future, which can be used to check the status of
+     *                  individual feature updates.
+     */
+    public UpdateFeaturesResult(final Map<String, KafkaFuture<Void>> futures) {

Review comment:
       Could we make the constructor non-public?




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -109,7 +109,9 @@ class KafkaApis(val requestChannel: RequestChannel,
                 brokerTopicStats: BrokerTopicStats,
                 val clusterId: String,
                 time: Time,
-                val tokenManager: DelegationTokenManager) extends Logging {
+                val tokenManager: DelegationTokenManager,
+                val brokerFeatures: BrokerFeatures,

Review comment:
       The `FinalizedFeatureCache.getSupportedFeatures` API is not the right fit for the cache's public interface (it is quite unrelated to the other public APIs of the cache). I'd rather not pollute the public API there, just for the sake of convenience.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/errors/FeatureUpdateFailedException.java
##########
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.errors;
+
+public class FeatureUpdateFailedException extends ApiException {

Review comment:
       This exception corresponds to `Errors.FEATURE_UPDATE_FAILED`. The caller of `AdminClient#updateFeatures` can receive this exception whenever a feature update can not be written to ZK (due to a ZK issue). So this has to be a public 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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,192 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables 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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala
##########
@@ -0,0 +1,580 @@
+/**
+ * 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.server
+
+import java.util.{Optional, Properties}
+import java.util.concurrent.ExecutionException
+
+import kafka.api.KAFKA_2_7_IV0
+import kafka.utils.TestUtils
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion}
+import kafka.utils.TestUtils.waitUntilTrue
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureUpdate, UpdateFeaturesOptions, UpdateFeaturesResult}
+import org.apache.kafka.common.errors.InvalidRequestException
+import org.apache.kafka.common.feature.FinalizedVersionRange
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.message.UpdateFeaturesRequestData
+import org.apache.kafka.common.message.UpdateFeaturesRequestData.FeatureUpdateKeyCollection
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{UpdateFeaturesRequest, UpdateFeaturesResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.Test
+import org.junit.Assert.{assertEquals, assertFalse, assertNotEquals, assertNotNull, assertTrue}
+import org.scalatest.Assertions.intercept
+
+import scala.jdk.CollectionConverters._
+import scala.reflect.ClassTag
+import scala.util.matching.Regex
+
+class UpdateFeaturesTest extends BaseRequestTest {
+
+  override def brokerCount = 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))))
+  }
+
+  private def defaultFinalizedFeatures(): Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2))))
+  }
+
+  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)
+  }
+
+  private def updateFeatureZNode(features: Features[FinalizedVersionRange]): Int = {
+    val server = serverForId(0).get
+    val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, features)
+    val newVersion = server.zkClient.updateFeatureZNode(newNode)
+    servers.foreach(s => {
+      s.featureCache.waitUntilEpochOrThrow(newVersion, s.config.zkConnectionTimeoutMs)
+    })
+    newVersion
+  }
+
+  private def getFeatureZNode(): FeatureZNode = {
+    val (mayBeFeatureZNodeBytes, version) = serverForId(0).get.zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(version, ZkVersion.UnknownVersion)
+    FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+  }
+
+  private def finalizedFeatures(features: java.util.Map[String, org.apache.kafka.clients.admin.FinalizedVersionRange]): Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(features.asScala.map {
+      case(name, versionRange) =>
+        (name, new FinalizedVersionRange(versionRange.minVersionLevel(), versionRange.maxVersionLevel()))
+    }.asJava)
+  }
+
+  private def supportedFeatures(features: java.util.Map[String, org.apache.kafka.clients.admin.SupportedVersionRange]): Features[SupportedVersionRange] = {
+    Features.supportedFeatures(features.asScala.map {
+      case(name, versionRange) =>
+        (name, new SupportedVersionRange(versionRange.minVersion(), versionRange.maxVersion()))
+    }.asJava)
+  }
+
+  private def checkFeatures(client: Admin,
+                            expectedNode: FeatureZNode,
+                            expectedFinalizedFeatures: Features[FinalizedVersionRange],
+                            expectedFinalizedFeaturesEpoch: Long,
+                            expectedSupportedFeatures: Features[SupportedVersionRange]): Unit = {
+    assertEquals(expectedNode, getFeatureZNode())
+    val featureMetadata = client.describeFeatures(
+      new DescribeFeaturesOptions().sendRequestToController(true)).featureMetadata.get
+    assertEquals(expectedFinalizedFeatures, finalizedFeatures(featureMetadata.finalizedFeatures))
+    assertEquals(expectedSupportedFeatures, supportedFeatures(featureMetadata.supportedFeatures))
+    assertEquals(Optional.of(expectedFinalizedFeaturesEpoch), featureMetadata.finalizedFeaturesEpoch)
+  }
+
+  private def checkException[ExceptionType <: Throwable](result: UpdateFeaturesResult,
+                                                         featureExceptionMsgPatterns: Map[String, Regex])
+                                                        (implicit tag: ClassTag[ExceptionType]): Unit = {
+    featureExceptionMsgPatterns.foreach {
+      case (feature, exceptionMsgPattern) =>
+        val exception = intercept[ExecutionException] {
+          result.values().get(feature).get()
+        }
+        val cause = exception.getCause
+        assertNotNull(cause)
+        assertEquals(cause.getClass, tag.runtimeClass)
+        assertTrue(s"Received unexpected error message: ${cause.getMessage}",
+                   exceptionMsgPattern.findFirstIn(cause.getMessage).isDefined)
+    }
+  }
+
+  /**
+   * Tests whether an invalid feature update does not get processed on the server as expected,
+   * and raises the ExceptionType on the client side as expected.
+   *
+   * @param invalidUpdate         the invalid feature update to be sent in the
+   *                              updateFeatures request to the server
+   * @param exceptionMsgPattern   a pattern for the expected exception message
+   */
+  private def testWithInvalidFeatureUpdate[ExceptionType <: Throwable](feature: String,
+                                                                       invalidUpdate: FeatureUpdate,
+                                                                       exceptionMsgPattern: Regex)
+                                                                      (implicit tag: ClassTag[ExceptionType]): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+    val adminClient = createAdminClient()
+    val nodeBefore = getFeatureZNode()
+
+    val result = adminClient.updateFeatures(Utils.mkMap(Utils.mkEntry(feature, invalidUpdate)), new UpdateFeaturesOptions())
+
+    checkException[ExceptionType](result, Map(feature -> exceptionMsgPattern))
+    checkFeatures(
+      adminClient,
+      nodeBefore,
+      defaultFinalizedFeatures(),
+      versionBefore,
+      defaultSupportedFeatures())
+  }
+
+  /**
+   * Tests that an UpdateFeatures request sent to a non-Controller node fails as expected.
+   */
+  @Test
+  def testShouldFailRequestIfNotController(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val nodeBefore = getFeatureZNode()
+    val validUpdates = new FeatureUpdateKeyCollection()
+    val validUpdate = new UpdateFeaturesRequestData.FeatureUpdateKey();
+    validUpdate.setFeature("feature_1");
+    validUpdate.setMaxVersionLevel(defaultSupportedFeatures().get("feature_1").max())
+    validUpdate.setAllowDowngrade(false)
+    validUpdates.add(validUpdate)
+
+    val response = connectAndReceive[UpdateFeaturesResponse](
+      new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData().setFeatureUpdates(validUpdates)).build(),
+      notControllerSocketServer)
+
+    assertEquals(Errors.NOT_CONTROLLER, Errors.forCode(response.data.errorCode()))
+    assertNotNull(response.data.errorMessage())
+    assertEquals(0, response.data.results.size)
+    checkFeatures(
+      createAdminClient(),
+      nodeBefore,
+      defaultFinalizedFeatures(),
+      versionBefore,
+      defaultSupportedFeatures())
+  }
+
+  /**
+   * Tests that an UpdateFeatures request fails in the Controller, when, for a feature the
+   * allowDowngrade flag is not set during a downgrade request.
+   */
+  @Test
+  def testShouldFailRequestWhenDowngradeFlagIsNotSetDuringDowngrade(): Unit = {
+    val targetMaxVersionLevel = (defaultFinalizedFeatures().get("feature_1").max() - 1).asInstanceOf[Short]
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      "feature_1",
+      new FeatureUpdate(targetMaxVersionLevel,false),
+      ".*Can not downgrade finalized feature.*allowDowngrade.*".r)
+  }
+
+  /**
+   * Tests that an UpdateFeatures request fails in the Controller, when, for a feature the downgrade
+   * is attempted to a max version level thats higher than the existing max version level.
+   */
+  @Test
+  def testShouldFailRequestWhenDowngradeToHigherVersionLevelIsAttempted(): Unit = {
+    val targetMaxVersionLevel = (defaultFinalizedFeatures().get("feature_1").max() + 1).asInstanceOf[Short]
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      "feature_1",
+      new FeatureUpdate(targetMaxVersionLevel, true),
+      ".*When the allowDowngrade flag set in the request, the provided maxVersionLevel:3.*existing maxVersionLevel:2.*".r)
+  }
+
+  /**
+   * Tests that an UpdateFeatures request fails in the Controller, when, a feature deletion is
+   * attempted without setting the allowDowngrade flag.
+   */
+  @Test
+  def testShouldFailRequestInServerWhenDowngradeFlagIsNotSetDuringDeletion(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val adminClient = createAdminClient()
+    val nodeBefore = getFeatureZNode()
+
+    val invalidUpdates
+      = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection();
+    val invalidUpdate = new UpdateFeaturesRequestData.FeatureUpdateKey();
+    invalidUpdate.setFeature("feature_1")
+    invalidUpdate.setMaxVersionLevel(0)
+    invalidUpdate.setAllowDowngrade(false)
+    invalidUpdates.add(invalidUpdate);
+    val requestData = new UpdateFeaturesRequestData()
+    requestData.setFeatureUpdates(invalidUpdates);
+
+    val response = connectAndReceive[UpdateFeaturesResponse](
+      new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData().setFeatureUpdates(invalidUpdates)).build(),
+      controllerSocketServer)
+
+    assertEquals(1, response.data().results().size())
+    val result = response.data.results.asScala.head
+    assertEquals("feature_1", result.feature)
+    assertEquals(Errors.INVALID_REQUEST, Errors.forCode(result.errorCode))
+    assertNotNull(result.errorMessage)
+    assertFalse(result.errorMessage.isEmpty)
+    val exceptionMsgPattern = ".*Can not provide maxVersionLevel: 0 less than 1.*allowDowngrade.*".r
+    assertTrue(result.errorMessage, exceptionMsgPattern.findFirstIn(result.errorMessage).isDefined)
+    checkFeatures(
+      adminClient,
+      nodeBefore,
+      defaultFinalizedFeatures(),
+      versionBefore,
+      defaultSupportedFeatures())
+  }
+
+  /**
+   * Tests that an UpdateFeatures request fails in the Controller, when, a feature version level
+   * upgrade is attempted for a non-existing feature.
+   */
+  @Test
+  def testShouldFailRequestDuringDeletionOfNonExistingFeature(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      "feature_non_existing",
+      new FeatureUpdate(0, true),
+      ".*Can not delete non-existing finalized feature.*".r)
+  }
+
+  /**
+   * Tests that an UpdateFeatures request fails in the Controller, when, a feature version level
+   * upgrade is attempted to a version level thats the same as the existing max version level.

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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1215,71 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker. It can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter. This is particularly useful if the user requires strongly consistent reads of
+     * finalized features.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the {@link DescribeFeaturesResult} containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);

Review comment:
       Note in the post-KIP-500 world, this feature could still work, but the request must be redirected to the controller inherently on the broker side, instead of sending it directly. So in the comment, we may try to phrase it to convey the principal is that `the request must be handled by the controller` instead of `the admin client must send this request to the controller`. 

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1215,70 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker. It can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter. This is particularly useful if the user requires strongly consistent reads of
+     * finalized features.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the {@link DescribeFeaturesResult} containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);
+
+    /**
+     * Applies specified updates to finalized features. This operation is not transactional so it
+     * may succeed for some features while fail for others.
+     * <p>
+     * The API takes in a map of finalized feature name to {@link FeatureUpdate} that need to be

Review comment:
       nit: s/name/names

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -983,8 +1144,25 @@ class KafkaController(val config: KafkaConfig,
    */
   private[controller] def sendUpdateMetadataRequest(brokers: Seq[Int], partitions: Set[TopicPartition]): Unit = {
     try {
+      val filteredBrokers = scala.collection.mutable.Set[Int]() ++ brokers
+      if (config.isFeatureVersioningEnabled) {
+        def hasIncompatibleFeatures(broker: Broker): Boolean = {
+          val latestFinalizedFeatures = featureCache.get
+          if (latestFinalizedFeatures.isDefined) {
+            BrokerFeatures.hasIncompatibleFeatures(broker.features, latestFinalizedFeatures.get.features)
+          } else {
+            false
+          }
+        }
+        controllerContext.liveOrShuttingDownBrokers.foreach(broker => {
+          if (filteredBrokers.contains(broker.id) && hasIncompatibleFeatures(broker)) {

Review comment:
       I see, what would happen to a currently live broker if it couldn't get any metadata update for a while, will it shut down itself?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java
##########
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+/**
+ * Options for {@link AdminClient#describeFeatures(DescribeFeaturesOptions)}
+ *
+ * The API of this class is evolving. See {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeFeaturesOptions extends AbstractOptions<DescribeFeaturesOptions> {
+
+    /**
+     * - True means the {@link Admin#describeFeatures(DescribeFeaturesOptions)} request can be

Review comment:
       `can be issued only to the controller.`/ `must be processed by the controller`

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)

Review comment:
       Yea, I mean you could use `val newVersion = zkClient.getDataAndVersion(FeatureZNode.path)._2`, but it's up to you.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4052,6 +4058,128 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        final NodeProvider provider =
+            options.sendRequestToController() ? new ControllerNodeProvider() : new LeastLoadedNodeProvider();
+
+        Call call = new Call(
+            "describeFeatures", calcDeadlineMs(now, options.timeoutMs()), provider) {
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(
+                        new FeatureMetadata(
+                            apiVersionsResponse.finalizedFeatures(),
+                            apiVersionsResponse.finalizedFeaturesEpoch(),
+                            apiVersionsResponse.supportedFeatures()));
+                } else if (options.sendRequestToController() && apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) {
+                    handleNotControllerError(Errors.NOT_CONTROLLER);
+                } else {
+                    future.completeExceptionally(
+                        Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        runnable.call(call, now);
+        return new DescribeFeaturesResult(future);
+    }
+
+    @Override
+    public UpdateFeaturesResult updateFeatures(
+        final Map<String, FeatureUpdate> featureUpdates, final UpdateFeaturesOptions options) {
+        if (featureUpdates == null || featureUpdates.isEmpty()) {
+            throw new IllegalArgumentException("Feature updates can not be null or empty.");
+        }
+        Objects.requireNonNull(options, "UpdateFeaturesOptions can not be null");
+
+        final Map<String, KafkaFutureImpl<Void>> updateFutures = new HashMap<>();
+        final UpdateFeaturesRequestData.FeatureUpdateKeyCollection featureUpdatesRequestData

Review comment:
       I suggest we build a static method in the `UpdateFeaturesRequest` class to avoid exposing the sub modules of feature data, such like:
   ```
   public static UpdateFeaturesRequestData getFeatureRequest(final Map<String, FeatureUpdate> featureUpdate);
   ```

##########
File path: clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
##########
@@ -466,6 +477,42 @@ private static DescribeGroupsResponseData prepareDescribeGroupsResponseData(Stri
                 Collections.emptySet()));
         return data;
     }
+
+    private static UpdateFeaturesResponse prepareUpdateFeaturesResponse(Map<String, Errors> featureUpdateErrors) {

Review comment:
       Could be moved to the `UpdateFeaturesResponse`

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -82,18 +108,54 @@ object FinalizedFeatureCache extends Logging {
         " The existing cache contents are %s").format(latest, oldFeatureAndEpoch)
       throw new FeatureCacheUpdateException(errorMsg)
     } else {
-      val incompatibleFeatures = SupportedFeatures.incompatibleFeatures(latest.features)
+      val incompatibleFeatures = brokerFeatures.incompatibleFeatures(latest.features)
       if (!incompatibleFeatures.empty) {
         val errorMsg = ("FinalizedFeatureCache update failed since feature compatibility" +
           " checks failed! Supported %s has incompatibilities with the latest %s."
-          ).format(SupportedFeatures.get, latest)
+          ).format(brokerFeatures.supportedFeatures, latest)
         throw new FeatureCacheUpdateException(errorMsg)
       } else {
-        val logMsg = "Updated cache from existing finalized %s to latest finalized %s".format(
+        val logMsg = "Updated cache from existing %s to latest %s".format(
           oldFeatureAndEpoch, latest)
-        featuresAndEpoch = Some(latest)
+        synchronized {
+          featuresAndEpoch = Some(latest)
+          notifyAll()
+        }
         info(logMsg)
       }
     }
   }
+
+  /**
+   * Causes the current thread to wait no more than timeoutMs for the specified condition to be met.
+   * It is guaranteed that the provided condition will always be invoked only from within a
+   * synchronized block.
+   *
+   * @param waitCondition   the condition to be waited upon:
+   *                         - if the condition returns true, then, the wait will stop.
+   *                         - if the condition returns false, it means the wait must continue until
+   *                           timeout.
+   *
+   * @param timeoutMs       the timeout (in milli seconds)
+   *
+   * @throws                TimeoutException if the condition is not met within timeoutMs.
+   */
+  private def waitUntilConditionOrThrow(waitCondition: () => Boolean, timeoutMs: Long): Unit = {
+    if(timeoutMs < 0L) {
+      throw new IllegalArgumentException(s"Expected timeoutMs >= 0, but $timeoutMs was provided.")
+    }
+    val waitEndTimeNanos = System.nanoTime() + (timeoutMs * 1_000_000)

Review comment:
       Why don't we just use `System.currentTimeMillis()` to avoid conversion between nano time?

##########
File path: core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala
##########
@@ -0,0 +1,550 @@
+/**
+ * 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.server
+
+import java.util
+import java.util.Properties
+import java.util.concurrent.ExecutionException
+
+import kafka.api.KAFKA_2_7_IV0
+import kafka.utils.TestUtils
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion}
+import kafka.utils.TestUtils.waitUntilTrue
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions, UpdateFeaturesResult}
+import org.apache.kafka.common.errors.InvalidRequestException
+import org.apache.kafka.common.feature.FinalizedVersionRange
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.message.UpdateFeaturesRequestData
+import org.apache.kafka.common.message.UpdateFeaturesRequestData.FeatureUpdateKeyCollection
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{UpdateFeaturesRequest, UpdateFeaturesResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.Test
+import org.junit.Assert.{assertEquals, assertFalse, assertNotEquals, assertNotNull, assertTrue}
+import org.scalatest.Assertions.{assertThrows, intercept}
+
+import scala.jdk.CollectionConverters._
+import scala.reflect.ClassTag
+import scala.util.matching.Regex
+
+class UpdateFeaturesTest extends BaseRequestTest {
+
+  override def brokerCount = 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))))
+  }
+
+  private def defaultFinalizedFeatures(): Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2))))
+  }
+
+  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)
+  }
+
+  private def updateDefaultMinVersionLevelsInAllBrokers(newMinVersionLevels: Map[String, Short]): Unit = {
+    servers.foreach(s => {
+      s.brokerFeatures.setDefaultMinVersionLevels(newMinVersionLevels)
+    })
+  }
+
+  private def updateFeatureZNode(features: Features[FinalizedVersionRange]): Int = {
+    val server = serverForId(0).get
+    val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, features)
+    val newVersion = server.zkClient.updateFeatureZNode(newNode)
+    servers.foreach(s => {
+      s.featureCache.waitUntilEpochOrThrow(newVersion, s.config.zkConnectionTimeoutMs)
+    })
+    newVersion
+  }
+
+  private def getFeatureZNode(): FeatureZNode = {
+    val (mayBeFeatureZNodeBytes, version) = serverForId(0).get.zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(version, ZkVersion.UnknownVersion)
+    FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+  }
+
+  private def checkFeatures(client: Admin, expectedNode: FeatureZNode, expectedMetadata: FeatureMetadata): Unit = {
+    assertEquals(expectedNode, getFeatureZNode())
+    val featureMetadata = client.describeFeatures(
+      new DescribeFeaturesOptions().sendRequestToController(true)).featureMetadata().get()
+    assertEquals(expectedMetadata, featureMetadata)
+  }
+
+  private def checkException[ExceptionType <: Throwable](result: UpdateFeaturesResult,
+                                                         featureExceptionMsgPatterns: Map[String, Regex])
+                                                        (implicit tag: ClassTag[ExceptionType]): Unit = {
+    featureExceptionMsgPatterns.foreach {
+      case (feature, exceptionMsgPattern) =>
+        val exception = intercept[ExecutionException] {
+          result.values().get(feature).get()
+        }
+        val cause = exception.getCause
+        assertNotNull(cause)
+        assertEquals(cause.getClass, tag.runtimeClass)
+        assertTrue(cause.getMessage, exceptionMsgPattern.findFirstIn(cause.getMessage).isDefined)
+    }
+
+  }
+
+  /**
+   * Tests whether an invalid feature update does not get processed on the server as expected,
+   * and raises the ExceptionType on the client side as expected.
+   *
+   * @param invalidUpdate         the invalid feature update to be sent in the
+   *                              updateFeatures request to the server
+   * @param exceptionMsgPattern   a pattern for the expected exception message
+   */
+  private def testWithInvalidFeatureUpdate[ExceptionType <: Throwable](feature: String,
+                                                                       invalidUpdate: FeatureUpdate,
+                                                                       exceptionMsgPattern: Regex)
+                                                                      (implicit tag: ClassTag[ExceptionType]): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+    val adminClient = createAdminClient()
+    val nodeBefore = getFeatureZNode()
+
+    val result = adminClient.updateFeatures(Utils.mkMap(Utils.mkEntry(feature, invalidUpdate)), new UpdateFeaturesOptions())
+
+    checkException[ExceptionType](result, Map(feature -> exceptionMsgPattern))
+    checkFeatures(
+      adminClient,
+      nodeBefore,
+      new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures()))
+  }
+
+  @Test
+  def testShouldFailRequestIfNotController(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val nodeBefore = getFeatureZNode()
+    val updates = new FeatureUpdateKeyCollection()
+    val update = new UpdateFeaturesRequestData.FeatureUpdateKey();
+    update.setFeature("feature_1");
+    update.setMaxVersionLevel(defaultSupportedFeatures().get("feature_1").max())
+    update.setAllowDowngrade(false)
+    updates.add(update)
+
+    val response = connectAndReceive[UpdateFeaturesResponse](
+      new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData().setFeatureUpdates(updates)).build(),
+      notControllerSocketServer)
+
+    assertEquals(1, response.data.results.size)
+    val result = response.data.results.asScala.head
+    assertEquals("feature_1", result.feature)
+    assertEquals(Errors.NOT_CONTROLLER, Errors.forCode(result.errorCode))
+    assertNotNull(result.errorMessage)
+    assertFalse(result.errorMessage.isEmpty)
+    checkFeatures(
+      createAdminClient(),
+      nodeBefore,
+      new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures()))
+  }
+
+  @Test
+  def testShouldFailRequestForEmptyUpdates(): Unit = {
+    val nullMap: util.Map[String, FeatureUpdate] = null
+    val emptyMap: util.Map[String, FeatureUpdate] = Utils.mkMap()
+    Set(nullMap, emptyMap).foreach { updates =>
+      val client = createAdminClient()
+      val exception = intercept[IllegalArgumentException] {
+        client.updateFeatures(updates, new UpdateFeaturesOptions())
+      }
+      assertNotNull(exception)
+      assertEquals("Feature updates can not be null or empty.", exception.getMessage)
+    }
+  }
+
+  @Test
+  def testShouldFailRequestForNullUpdateFeaturesOptions(): Unit = {
+    val client = createAdminClient()
+    val update = new FeatureUpdate(defaultSupportedFeatures().get("feature_1").max(), false)
+    val exception = intercept[NullPointerException] {
+      client.updateFeatures(Utils.mkMap(Utils.mkEntry("feature_1", update)), null)
+    }
+    assertNotNull(exception)
+    assertEquals("UpdateFeaturesOptions can not be null", exception.getMessage)
+  }
+
+  @Test
+  def testShouldFailRequestForInvalidFeatureName(): Unit = {
+    val client = createAdminClient()
+    val update = new FeatureUpdate(defaultSupportedFeatures().get("feature_1").max(), false)
+    val exception = intercept[IllegalArgumentException] {
+      client.updateFeatures(Utils.mkMap(Utils.mkEntry("", update)), new UpdateFeaturesOptions())
+    }
+    assertNotNull(exception)
+    assertTrue((".*Provided feature can not be null or empty.*"r).findFirstIn(exception.getMessage).isDefined)
+  }
+
+  @Test
+  def testShouldFailRequestWhenDowngradeFlagIsNotSetDuringDowngrade(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      "feature_1",
+      new FeatureUpdate((defaultFinalizedFeatures().get("feature_1").max() - 1).asInstanceOf[Short],false),
+      ".*Can not downgrade finalized feature: 'feature_1'.*allowDowngrade.*".r)
+  }
+
+  @Test
+  def testShouldFailRequestWhenDowngradeToHigherVersionLevelIsAttempted(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      "feature_1",
+      new FeatureUpdate(defaultSupportedFeatures().get("feature_1").max(), true),
+      ".*finalized feature: 'feature_1'.*allowDowngrade.* provided maxVersionLevel:3.*existing maxVersionLevel:2.*".r)
+  }
+
+  @Test
+  def testShouldFailRequestInClientWhenDowngradeFlagIsNotSetDuringDeletion(): Unit = {
+    assertThrows[IllegalArgumentException] {
+      new FeatureUpdate(0, false)
+    }
+  }
+
+  @Test
+  def testShouldFailRequestInServerWhenDowngradeFlagIsNotSetDuringDeletion(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val adminClient = createAdminClient()
+    val nodeBefore = getFeatureZNode()
+
+    val updates
+      = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection();
+    val update = new UpdateFeaturesRequestData.FeatureUpdateKey();
+    update.setFeature("feature_1")
+    update.setMaxVersionLevel(0)
+    update.setAllowDowngrade(false)
+    updates.add(update);
+    val requestData = new UpdateFeaturesRequestData()
+    requestData.setFeatureUpdates(updates);
+
+    val response = connectAndReceive[UpdateFeaturesResponse](
+      new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData().setFeatureUpdates(updates)).build(),
+      controllerSocketServer)
+
+    assertEquals(1, response.data().results().size())
+    val result = response.data.results.asScala.head
+    assertEquals("feature_1", result.feature)
+    assertEquals(Errors.INVALID_REQUEST, Errors.forCode(result.errorCode))
+    assertNotNull(result.errorMessage)
+    assertFalse(result.errorMessage.isEmpty)
+    val exceptionMsgPattern = ".*Can not provide maxVersionLevel: 0 less than 1 for feature: 'feature_1'.*allowDowngrade.*".r
+    assertTrue(exceptionMsgPattern.findFirstIn(result.errorMessage).isDefined)
+    checkFeatures(
+      adminClient,
+      nodeBefore,
+      new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures()))
+  }
+
+  @Test
+  def testShouldFailRequestDuringDeletionOfNonExistingFeature(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      "feature_non_existing",
+      new FeatureUpdate(0, true),
+      ".*Can not delete non-existing finalized feature: 'feature_non_existing'.*".r)
+  }
+
+  @Test
+  def testShouldFailRequestWhenUpgradingToSameVersionLevel(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      "feature_1",
+      new FeatureUpdate(defaultFinalizedFeatures().get("feature_1").max(), false),
+      ".*Can not upgrade a finalized feature: 'feature_1'.*to the same value.*".r)
+  }
+
+  @Test
+  def testShouldFailRequestWhenDowngradingBelowMinVersionLevel(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+    val minVersionLevel = 2.asInstanceOf[Short]
+    updateDefaultMinVersionLevelsInAllBrokers(Map[String, Short]("feature_1" -> minVersionLevel))
+    val initialFinalizedFeatures = Features.finalizedFeatures(
+      Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(minVersionLevel, 2))))
+    val versionBefore = updateFeatureZNode(initialFinalizedFeatures)
+
+    val update = new FeatureUpdate((minVersionLevel - 1).asInstanceOf[Short], true)
+    val adminClient = createAdminClient()
+    val nodeBefore = getFeatureZNode()
+
+    val result = adminClient.updateFeatures(
+      Utils.mkMap(Utils.mkEntry("feature_1", update)), new UpdateFeaturesOptions())
+
+    checkException[InvalidRequestException](
+      result,
+      Map("feature_1" -> ".*Can not downgrade finalized feature: 'feature_1' to maxVersionLevel:1.*existing minVersionLevel:2.*".r))
+    checkFeatures(
+      adminClient,
+      nodeBefore,
+      new FeatureMetadata(initialFinalizedFeatures, versionBefore, defaultSupportedFeatures()))
+  }
+
+  @Test
+  def testShouldFailRequestDuringBrokerMaxVersionLevelIncompatibility(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    val controller = servers.filter { server => server.kafkaController.isActive}.head
+    val nonControllerServers = servers.filter { server => !server.kafkaController.isActive}
+    val unsupportedBrokers = Set[KafkaServer](nonControllerServers.head)
+    val supportedBrokers = Set[KafkaServer](nonControllerServers(1), controller)
+
+    updateSupportedFeatures(defaultSupportedFeatures(), supportedBrokers)
+
+    val validMinVersion = defaultSupportedFeatures().get("feature_1").min()
+    val unsupportedMaxVersion =
+      (defaultSupportedFeatures().get("feature_1").max() - 1).asInstanceOf[Short]
+    val badSupportedFeatures = Features.supportedFeatures(
+      Utils.mkMap(
+        Utils.mkEntry("feature_1",
+          new SupportedVersionRange(
+            validMinVersion,
+            unsupportedMaxVersion))))
+    updateSupportedFeatures(badSupportedFeatures, unsupportedBrokers)
+
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val invalidUpdate = new FeatureUpdate(defaultSupportedFeatures().get("feature_1").max(), false)
+    val nodeBefore = getFeatureZNode()
+    val adminClient = createAdminClient()
+    val result = adminClient.updateFeatures(
+      Utils.mkMap(Utils.mkEntry("feature_1", invalidUpdate)),
+      new UpdateFeaturesOptions())
+
+    checkException[InvalidRequestException](result, Map("feature_1" -> ".*1 broker.*incompatible.*".r))
+    checkFeatures(
+      adminClient,
+      nodeBefore,
+      new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures()))
+  }
+
+  @Test
+  def testSuccessfulFeatureUpgradeAndWithNoExistingFinalizedFeatures(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeaturesInAllBrokers(
+      Features.supportedFeatures(
+        Utils.mkMap(
+          Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)),
+          Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5)))))
+    updateDefaultMinVersionLevelsInAllBrokers(Map[String, Short]("feature_1" -> 1, "feature_2" -> 2))
+    val versionBefore = updateFeatureZNode(Features.emptyFinalizedFeatures())
+
+    val targetFinalizedFeatures = Features.finalizedFeatures(
+      Utils.mkMap(
+        Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 3)),
+        Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 3))))
+    val update1 = new FeatureUpdate(targetFinalizedFeatures.get("feature_1").max(), false)
+    val update2 = new FeatureUpdate(targetFinalizedFeatures.get("feature_2").max(), false)
+
+    val expected = new FeatureMetadata(
+      targetFinalizedFeatures,
+      versionBefore + 1,
+      Features.supportedFeatures(
+        Utils.mkMap(
+          Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)),
+          Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5)))))
+
+    val adminClient = createAdminClient()
+    adminClient.updateFeatures(
+      Utils.mkMap(Utils.mkEntry("feature_1", update1), Utils.mkEntry("feature_2", update2)),
+      new UpdateFeaturesOptions()
+    ).all().get()
+
+    checkFeatures(
+      adminClient,
+      new FeatureZNode(FeatureZNodeStatus.Enabled, targetFinalizedFeatures),
+      expected)
+  }
+
+  @Test
+  def testSuccessfulFeatureUpgradeAndDowngrade(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeaturesInAllBrokers(
+      Features.supportedFeatures(
+        Utils.mkMap(
+          Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)),
+          Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5)))))
+    updateDefaultMinVersionLevelsInAllBrokers(Map[String, Short]("feature_1" -> 1, "feature_2" -> 2))
+    val versionBefore = updateFeatureZNode(
+      Features.finalizedFeatures(
+        Utils.mkMap(
+          Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2)),
+          Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 4)))))
+
+    val targetFinalizedFeatures = Features.finalizedFeatures(
+      Utils.mkMap(
+        Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 3)),
+        Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 3))))
+    val update1 = new FeatureUpdate(targetFinalizedFeatures.get("feature_1").max(), false)
+    val update2 = new FeatureUpdate(targetFinalizedFeatures.get("feature_2").max(), true)
+
+    val expected = new FeatureMetadata(
+      targetFinalizedFeatures,
+      versionBefore + 1,
+      Features.supportedFeatures(
+        Utils.mkMap(
+          Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)),
+          Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5)))))
+
+    val adminClient = createAdminClient()
+    adminClient.updateFeatures(
+      Utils.mkMap(Utils.mkEntry("feature_1", update1), Utils.mkEntry("feature_2", update2)),
+      new UpdateFeaturesOptions()
+    ).all().get()
+
+    checkFeatures(
+      adminClient,
+      new FeatureZNode(FeatureZNodeStatus.Enabled, targetFinalizedFeatures),
+      expected)
+  }
+
+  @Test
+  def testPartialSuccessDuringValidFeatureUpgradeAndInvalidDowngrade(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    val initialSupportedFeatures = Features.supportedFeatures(

Review comment:
       nit: this could be extracted as a common struct.

##########
File path: clients/src/main/resources/common/message/UpdateFeaturesRequest.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "request",
+  "name": "UpdateFeaturesRequest",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "FeatureUpdates", "type": "[]FeatureUpdateKey", "versions": "0+",
+      "about": "The list of updates to finalized features.", "fields": [
+      {"name": "Feature", "type": "string", "versions": "0+", "mapKey": true,
+        "about": "The name of the finalized feature to be updated."},
+      {"name": "MaxVersionLevel", "type": "int16", "versions": "0+",

Review comment:
       Same here

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java
##########
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+import java.util.Optional;
+import org.apache.kafka.common.feature.Features;
+import org.apache.kafka.common.feature.FinalizedVersionRange;
+import org.apache.kafka.common.feature.SupportedVersionRange;
+
+/**
+ * Encapsulates details about finalized as well as supported features. This is particularly useful
+ * to hold the result returned by the {@link Admin#describeFeatures(DescribeFeaturesOptions)} API.
+ */
+public class FeatureMetadata {
+
+    private final Features<FinalizedVersionRange> finalizedFeatures;
+
+    private final Optional<Integer> finalizedFeaturesEpoch;
+
+    private final Features<SupportedVersionRange> supportedFeatures;
+
+    public FeatureMetadata(

Review comment:
       Try to put first parameter on the same line as the constructor, and align the rest parameters.

##########
File path: clients/src/main/resources/common/message/UpdateFeaturesRequest.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "request",
+  "name": "UpdateFeaturesRequest",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "FeatureUpdates", "type": "[]FeatureUpdateKey", "versions": "0+",
+      "about": "The list of updates to finalized features.", "fields": [
+      {"name": "Feature", "type": "string", "versions": "0+", "mapKey": true,

Review comment:
       Space

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1215,71 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker. It can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter. This is particularly useful if the user requires strongly consistent reads of
+     * finalized features.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the {@link DescribeFeaturesResult} containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);
+
+    /**
+     * Applies specified updates to finalized features. This operation is not transactional so it
+     * may succeed for some features while fail for others.
+     * <p>
+     * The API takes in a map of finalized feature name to {@link FeatureUpdate} that needs to be
+     * applied. Each entry in the map specifies the finalized feature to be added or updated or
+     * deleted, along with the new max feature version level value. This request is issued only to
+     * the controller since the API is only served by the controller. The return value contains an
+     * error code for each supplied {@link FeatureUpdate}, and the code indicates if the update
+     * succeeded or failed in the controller.
+     * <ul>
+     * <li>Downgrade of feature version level is not a regular operation/intent. It is only allowed
+     * in the controller if the {@link FeatureUpdate} has the allowDowngrade flag set - setting this
+     * flag conveys user intent to attempt downgrade of a feature max version level. Note that
+     * despite the allowDowngrade flag being set, certain downgrades may be rejected by the
+     * controller if it is deemed impossible.</li>
+     * <li>Deletion of a finalized feature version is not a regular operation/intent. It could be
+     * done by setting the allowDowngrade flag to true in the {@link FeatureUpdate}, and, setting
+     * the max version level to be less than 1.</li>
+     * </ul>
+     *<p>
+     * The following exceptions can be anticipated when calling {@code get()} on the futures
+     * obtained from the returned {@link UpdateFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.ClusterAuthorizationException}
+     *   If the authenticated user didn't have alter access to the cluster.</li>
+     *   <li>{@link org.apache.kafka.common.errors.InvalidRequestException}

Review comment:
       should this a per feature error or a top level error?

##########
File path: clients/src/main/resources/common/message/UpdateFeaturesResponse.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "response",
+  "name": "UpdateFeaturesResponse",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [

Review comment:
       For top level exception such as cluster authorization exception, we could just define a top level error code instead of check-marking every feature with the redundant error code. I know we have been a bit inconsistent in such a case, but personally feel having layered error codes could make the response handling clear of whether it is per feature issue, or a high level issue.

##########
File path: clients/src/main/resources/common/message/UpdateFeaturesRequest.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "request",
+  "name": "UpdateFeaturesRequest",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "FeatureUpdates", "type": "[]FeatureUpdateKey", "versions": "0+",
+      "about": "The list of updates to finalized features.", "fields": [
+      {"name": "Feature", "type": "string", "versions": "0+", "mapKey": true,
+        "about": "The name of the finalized feature to be updated."},
+      {"name": "MaxVersionLevel", "type": "int16", "versions": "0+",
+        "about": "The new maximum version level for the finalized feature. A value >= 1 is valid. A value < 1, is special, and can be used to request the deletion of the finalized feature."},
+      {"name": "AllowDowngrade", "type": "bool", "versions": "0+",

Review comment:
       Same here

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java
##########
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+/**
+ * Options for {@link AdminClient#describeFeatures(DescribeFeaturesOptions)}
+ *
+ * The API of this class is evolving. See {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeFeaturesOptions extends AbstractOptions<DescribeFeaturesOptions> {
+
+    /**
+     * - True means the {@link Admin#describeFeatures(DescribeFeaturesOptions)} request can be
+     *   issued only to the controller.
+     * - False means the {@link Admin#describeFeatures(DescribeFeaturesOptions)} request can be

Review comment:
       `could be processed by any random broker`

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4052,6 +4058,128 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        final NodeProvider provider =
+            options.sendRequestToController() ? new ControllerNodeProvider() : new LeastLoadedNodeProvider();
+
+        Call call = new Call(
+            "describeFeatures", calcDeadlineMs(now, options.timeoutMs()), provider) {
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(
+                        new FeatureMetadata(
+                            apiVersionsResponse.finalizedFeatures(),
+                            apiVersionsResponse.finalizedFeaturesEpoch(),
+                            apiVersionsResponse.supportedFeatures()));
+                } else if (options.sendRequestToController() && apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) {
+                    handleNotControllerError(Errors.NOT_CONTROLLER);
+                } else {
+                    future.completeExceptionally(
+                        Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        runnable.call(call, now);
+        return new DescribeFeaturesResult(future);
+    }
+
+    @Override
+    public UpdateFeaturesResult updateFeatures(
+        final Map<String, FeatureUpdate> featureUpdates, final UpdateFeaturesOptions options) {
+        if (featureUpdates == null || featureUpdates.isEmpty()) {
+            throw new IllegalArgumentException("Feature updates can not be null or empty.");
+        }
+        Objects.requireNonNull(options, "UpdateFeaturesOptions can not be null");
+
+        final Map<String, KafkaFutureImpl<Void>> updateFutures = new HashMap<>();
+        final UpdateFeaturesRequestData.FeatureUpdateKeyCollection featureUpdatesRequestData
+            = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection();
+        for (Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+            final String feature = entry.getKey();
+            final FeatureUpdate update = entry.getValue();
+            if (feature.trim().isEmpty()) {
+                throw new IllegalArgumentException("Provided feature can not be null or empty.");
+            }
+
+            updateFutures.put(feature, new KafkaFutureImpl<>());
+            final UpdateFeaturesRequestData.FeatureUpdateKey requestItem =
+                new UpdateFeaturesRequestData.FeatureUpdateKey();
+            requestItem.setFeature(feature);
+            requestItem.setMaxVersionLevel(update.maxVersionLevel());
+            requestItem.setAllowDowngrade(update.allowDowngrade());
+            featureUpdatesRequestData.add(requestItem);
+        }
+        final UpdateFeaturesRequestData request = new UpdateFeaturesRequestData().setFeatureUpdates(featureUpdatesRequestData);
+
+        final long now = time.milliseconds();
+        final Call call = new Call("updateFeatures", calcDeadlineMs(now, options.timeoutMs()),
+            new ControllerNodeProvider()) {
+
+            @Override
+            UpdateFeaturesRequest.Builder createRequest(int timeoutMs) {
+                return new UpdateFeaturesRequest.Builder(request);
+            }
+
+            @Override
+            void handleResponse(AbstractResponse abstractResponse) {
+                final UpdateFeaturesResponse response =
+                    (UpdateFeaturesResponse) abstractResponse;
+
+                // Check for controller change.
+                for (UpdatableFeatureResult result : response.data().results()) {
+                    final Errors error = Errors.forCode(result.errorCode());
+                    if (error == Errors.NOT_CONTROLLER) {
+                        handleNotControllerError(error);
+                        throw error.exception();
+                    }
+                }
+
+                for (UpdatableFeatureResult result : response.data().results()) {
+                    final KafkaFutureImpl<Void> future = updateFutures.get(result.feature());
+                    if (future == null) {

Review comment:
       Does this overlap with `completeUnrealizedFutures` check? We could just keep one to reduce the checking complexity. 

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java
##########
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+import java.util.Optional;
+import org.apache.kafka.common.feature.Features;
+import org.apache.kafka.common.feature.FinalizedVersionRange;
+import org.apache.kafka.common.feature.SupportedVersionRange;
+
+/**
+ * Encapsulates details about finalized as well as supported features. This is particularly useful
+ * to hold the result returned by the {@link Admin#describeFeatures(DescribeFeaturesOptions)} API.
+ */
+public class FeatureMetadata {
+
+    private final Features<FinalizedVersionRange> finalizedFeatures;
+
+    private final Optional<Integer> finalizedFeaturesEpoch;
+
+    private final Features<SupportedVersionRange> supportedFeatures;
+
+    public FeatureMetadata(
+        final Features<FinalizedVersionRange> finalizedFeatures,
+        final int finalizedFeaturesEpoch,
+        final Features<SupportedVersionRange> supportedFeatures) {
+        Objects.requireNonNull(finalizedFeatures, "Provided finalizedFeatures can not be null.");
+        Objects.requireNonNull(supportedFeatures, "Provided supportedFeatures can not be null.");
+        this.finalizedFeatures = finalizedFeatures;
+        if (finalizedFeaturesEpoch >= 0) {
+            this.finalizedFeaturesEpoch = Optional.of(finalizedFeaturesEpoch);
+        } else {
+            this.finalizedFeaturesEpoch = Optional.empty();
+        }
+        this.supportedFeatures = supportedFeatures;
+    }
+
+    /**
+     * A map of finalized feature versions, with key being finalized feature name and value
+     * containing the min/max version levels for the finalized feature.
+     */
+    public Features<FinalizedVersionRange> finalizedFeatures() {
+        return finalizedFeatures;
+    }
+
+    /**
+     * The epoch for the finalized features.
+     * If the returned value is empty, it means the finalized features are absent/unavailable.
+     */
+    public Optional<Integer> finalizedFeaturesEpoch() {
+        return finalizedFeaturesEpoch;
+    }
+
+    /**
+     * A map of supported feature versions, with key being supported feature name and value
+     * containing the min/max version for the supported feature.
+     */
+    public Features<SupportedVersionRange> supportedFeatures() {
+        return supportedFeatures;
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (this == other) {
+            return true;
+        }
+        if (!(other instanceof FeatureMetadata)) {
+            return false;
+        }
+
+        final FeatureMetadata that = (FeatureMetadata) other;
+        return Objects.equals(this.finalizedFeatures, that.finalizedFeatures) &&
+            Objects.equals(this.finalizedFeaturesEpoch, that.finalizedFeaturesEpoch) &&
+            Objects.equals(this.supportedFeatures, that.supportedFeatures);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(finalizedFeatures, finalizedFeaturesEpoch, supportedFeatures);
+    }
+
+    @Override
+    public String toString() {
+        return String.format(
+            "FeatureMetadata{finalized:%s, finalizedFeaturesEpoch:%d, supported:%s}",
+            finalizedFeatures,
+            finalizedFeaturesEpoch,

Review comment:
       This won't work well with string format, consider doing `orElse`

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java
##########
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+/**
+ * Options for {@link AdminClient#describeFeatures(DescribeFeaturesOptions)}
+ *
+ * The API of this class is evolving. See {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeFeaturesOptions extends AbstractOptions<DescribeFeaturesOptions> {
+
+    /**
+     * - True means the {@link Admin#describeFeatures(DescribeFeaturesOptions)} request can be
+     *   issued only to the controller.
+     * - False means the {@link Admin#describeFeatures(DescribeFeaturesOptions)} request can be
+     *   issued to any random broker.
+     */
+    private boolean sendRequestToController = false;
+
+    /**
+     * Sets a flag indicating that the describe features request should be issued to the controller.

Review comment:
       Same here

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Encapsulates details about an update to a finalized feature. This is particularly useful to
+ * define each feature update in the {@link Admin#updateFeatures(Map, UpdateFeaturesOptions)} API.
+ */
+public class FeatureUpdate {
+    private final short maxVersionLevel;
+    private final boolean allowDowngrade;
+
+    /**
+     * @param maxVersionLevel   the new maximum version level for the finalized feature.
+     *                          a value < 1 is special and indicates that the update is intended to
+     *                          delete the finalized feature, and should be accompanied by setting
+     *                          the allowDowngrade flag to true.
+     * @param allowDowngrade    - true, if this feature update was meant to downgrade the existing
+     *                            maximum version level of the finalized feature.
+     *                          - false, otherwise.
+     */
+    public FeatureUpdate(final short maxVersionLevel, final boolean allowDowngrade) {
+        if (maxVersionLevel < 1 && !allowDowngrade) {
+            throw new IllegalArgumentException(String.format(
+                "The allowDowngrade flag should be set when the provided maxVersionLevel:%d is < 1.",
+                maxVersionLevel));
+        }
+        this.maxVersionLevel = maxVersionLevel;
+        this.allowDowngrade = allowDowngrade;
+    }
+
+    public short maxVersionLevel() {
+        return maxVersionLevel;
+    }
+
+    public boolean allowDowngrade() {
+        return allowDowngrade;
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (this == other) {
+            return true;
+        }
+
+        if (!(other instanceof FeatureUpdate)) {
+            return false;
+        }
+
+        final FeatureUpdate that = (FeatureUpdate) other;
+        return this.maxVersionLevel == that.maxVersionLevel && this.allowDowngrade == that.allowDowngrade;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(maxVersionLevel, allowDowngrade);
+    }
+
+    @Override
+    public String toString() {
+        return String.format("FeatureUpdate{maxVersionLevel:%d, allowDowngrade:%s}", maxVersionLevel, allowDowngrade);
+    }
+}

Review comment:
       new line

##########
File path: clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
##########
@@ -3193,6 +3238,104 @@ public void testListOffsetsNonRetriableErrors() throws Exception {
         }
     }
 
+    @Test
+    public void testUpdateFeaturesDuringSuccess() throws Exception {
+        testUpdateFeaturesDuringError(Errors.NONE);
+    }
+
+    @Test
+    public void testUpdateFeaturesInvalidRequestError() throws Exception {
+        testUpdateFeaturesDuringError(Errors.INVALID_REQUEST);
+    }
+
+    @Test
+    public void testUpdateFeaturesUpdateFailedError() throws Exception {
+        testUpdateFeaturesDuringError(Errors.FEATURE_UPDATE_FAILED);
+    }
+
+    private void testUpdateFeaturesDuringError(Errors error) throws Exception {
+        try (final AdminClientUnitTestEnv env = mockClientEnv()) {
+            env.kafkaClient().prepareResponse(
+                body -> body instanceof UpdateFeaturesRequest,
+                prepareUpdateFeaturesResponse(error));
+            final KafkaFuture<Void> future = env.adminClient().updateFeatures(
+                new HashSet<>(
+                    Arrays.asList(
+                        new FeatureUpdate(
+                            "test_feature_1", (short) 2, false),
+                        new FeatureUpdate(
+                            "test_feature_2", (short) 3, true))),
+                new UpdateFeaturesOptions().timeoutMs(10000)).result();
+            if (error.exception() == null) {
+                future.get();
+            } else {
+                final ExecutionException e = assertThrows(ExecutionException.class,
+                    () -> future.get());
+                assertEquals(e.getCause().getClass(), error.exception().getClass());
+            }
+        }
+    }
+
+    @Test
+    public void testUpdateFeaturesHandleNotControllerException() throws Exception {
+        try (final AdminClientUnitTestEnv env = mockClientEnv()) {
+            env.kafkaClient().prepareResponseFrom(
+                prepareUpdateFeaturesResponse(Errors.NOT_CONTROLLER),
+                env.cluster().nodeById(0));
+            env.kafkaClient().prepareResponse(MetadataResponse.prepareResponse(env.cluster().nodes(),
+                env.cluster().clusterResource().clusterId(),
+                1,
+                Collections.<MetadataResponse.TopicMetadata>emptyList()));
+            env.kafkaClient().prepareResponseFrom(

Review comment:
       You are right, it seems not necessary.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.kafka.common.message.UpdateFeaturesResponseData;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.Struct;
+
+
+/**
+ * Possible error codes:
+ *
+ *   - {@link Errors#CLUSTER_AUTHORIZATION_FAILED}
+ *   - {@link Errors#NOT_CONTROLLER}
+ *   - {@link Errors#INVALID_REQUEST}
+ *   - {@link Errors#FEATURE_UPDATE_FAILED}
+ */
+public class UpdateFeaturesResponse extends AbstractResponse {
+
+    private final UpdateFeaturesResponseData data;
+
+    public UpdateFeaturesResponse(UpdateFeaturesResponseData data) {
+        this.data = data;
+    }
+
+    public UpdateFeaturesResponse(Struct struct) {
+        final short latestVersion = (short) (UpdateFeaturesResponseData.SCHEMAS.length - 1);
+        this.data = new UpdateFeaturesResponseData(struct, latestVersion);
+    }
+
+    public UpdateFeaturesResponse(Struct struct, short version) {
+        this.data = new UpdateFeaturesResponseData(struct, version);
+    }
+
+    public Map<String, ApiError> errors() {
+        return data.results().valuesSet().stream().collect(
+            Collectors.toMap(
+                result -> result.feature(),

Review comment:
       nit: could be replaced with lambda

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -143,7 +172,13 @@ public static ApiVersionsResponse apiVersionsResponse(
         Features<FinalizedVersionRange> finalizedFeatures,
         int finalizedFeaturesEpoch) {
         if (maxMagic == RecordBatch.CURRENT_MAGIC_VALUE && throttleTimeMs == DEFAULT_THROTTLE_TIME) {
-            return DEFAULT_API_VERSIONS_RESPONSE;
+            return new ApiVersionsResponse(createApiVersionsResponseData(
+                DEFAULT_API_VERSIONS_RESPONSE.throttleTimeMs(),
+                Errors.forCode(DEFAULT_API_VERSIONS_RESPONSE.data().errorCode()),
+                DEFAULT_API_VERSIONS_RESPONSE.data().apiKeys(),
+                latestSupportedFeatures,
+                finalizedFeatures,
+                finalizedFeaturesEpoch));

Review comment:
       Comment here since no better place: createApiVersionsResponse on L198 could be made private

##########
File path: clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
##########
@@ -3615,6 +3662,137 @@ public void testListOffsetsNonRetriableErrors() throws Exception {
         }
     }
 
+    @Test
+    public void testUpdateFeaturesDuringSuccess() throws Exception {
+        testUpdateFeatures(
+            makeTestFeatureUpdates(),
+            makeTestFeatureUpdateErrors(Errors.NONE));
+    }
+
+    @Test
+    public void testUpdateFeaturesInvalidRequestError() throws Exception {
+        testUpdateFeatures(
+            makeTestFeatureUpdates(),
+            makeTestFeatureUpdateErrors(Errors.INVALID_REQUEST));
+    }
+
+    @Test
+    public void testUpdateFeaturesUpdateFailedError() throws Exception {
+        testUpdateFeatures(
+            makeTestFeatureUpdates(),
+            makeTestFeatureUpdateErrors(Errors.FEATURE_UPDATE_FAILED));
+    }
+
+    @Test
+    public void testUpdateFeaturesPartialSuccess() throws Exception {
+        final Map<String, Errors> errors = makeTestFeatureUpdateErrors(Errors.NONE);
+        errors.put("test_feature_2", Errors.INVALID_REQUEST);
+        testUpdateFeatures(makeTestFeatureUpdates(), errors);
+    }
+
+    private Map<String, FeatureUpdate> makeTestFeatureUpdates() {
+        return Utils.mkMap(
+            Utils.mkEntry("test_feature_1", new FeatureUpdate((short) 2, false)),
+            Utils.mkEntry("test_feature_2", new FeatureUpdate((short) 3, true)));
+    }
+
+    private Map<String, Errors> makeTestFeatureUpdateErrors(final Errors error) {
+        final Map<String, FeatureUpdate> updates = makeTestFeatureUpdates();
+        final Map<String, Errors> errors = new HashMap<>();
+        for (Map.Entry<String, FeatureUpdate> entry : updates.entrySet()) {
+            errors.put(entry.getKey(), error);
+        }
+        return errors;
+    }
+
+    private void testUpdateFeatures(Map<String, FeatureUpdate> featureUpdates,
+                                    Map<String, Errors> featureUpdateErrors) throws Exception {
+        try (final AdminClientUnitTestEnv env = mockClientEnv()) {
+            env.kafkaClient().prepareResponse(
+                body -> body instanceof UpdateFeaturesRequest,
+                prepareUpdateFeaturesResponse(featureUpdateErrors));
+            final Map<String, KafkaFuture<Void>> futures = env.adminClient().updateFeatures(
+                featureUpdates,
+                new UpdateFeaturesOptions().timeoutMs(10000)).values();
+            for (Map.Entry<String, KafkaFuture<Void>> entry : futures.entrySet()) {
+                final KafkaFuture<Void> future = entry.getValue();
+                final Errors error = featureUpdateErrors.get(entry.getKey());
+                if (error == Errors.NONE) {
+                    future.get();
+                } else {
+                    final ExecutionException e = assertThrows(ExecutionException.class,
+                        () -> future.get());

Review comment:
       nit: could use lambda

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,178 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of its own supported features in its
+   * own BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. This process ensures we do not enable all the possible features immediately after
+   *    an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, it will
+   *        react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, its existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) =>
+            val brokerDefaultVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (brokerDefaultVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (brokerDefaultVersionRange.max() >= existingVersionRange.max() &&
+                       brokerDefaultVersionRange.min() <= existingVersionRange.max()) {
+              // Through this change, we deprecate all version levels in the closed range:
+              // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1]
+              (featureName, new FinalizedVersionRange(brokerDefaultVersionRange.min(), existingVersionRange.max()))
+            } else {
+              // If the existing version levels fall completely outside the

Review comment:
       Are we good to proceed in this case? When there is no overlapping between broker default features and remote finalized features, is the current controller still eligible?

##########
File path: clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
##########
@@ -3615,6 +3662,137 @@ public void testListOffsetsNonRetriableErrors() throws Exception {
         }
     }
 
+    @Test
+    public void testUpdateFeaturesDuringSuccess() throws Exception {
+        testUpdateFeatures(
+            makeTestFeatureUpdates(),
+            makeTestFeatureUpdateErrors(Errors.NONE));
+    }
+
+    @Test
+    public void testUpdateFeaturesInvalidRequestError() throws Exception {
+        testUpdateFeatures(
+            makeTestFeatureUpdates(),
+            makeTestFeatureUpdateErrors(Errors.INVALID_REQUEST));
+    }
+
+    @Test
+    public void testUpdateFeaturesUpdateFailedError() throws Exception {
+        testUpdateFeatures(
+            makeTestFeatureUpdates(),
+            makeTestFeatureUpdateErrors(Errors.FEATURE_UPDATE_FAILED));
+    }
+
+    @Test
+    public void testUpdateFeaturesPartialSuccess() throws Exception {
+        final Map<String, Errors> errors = makeTestFeatureUpdateErrors(Errors.NONE);
+        errors.put("test_feature_2", Errors.INVALID_REQUEST);
+        testUpdateFeatures(makeTestFeatureUpdates(), errors);
+    }
+
+    private Map<String, FeatureUpdate> makeTestFeatureUpdates() {
+        return Utils.mkMap(
+            Utils.mkEntry("test_feature_1", new FeatureUpdate((short) 2, false)),
+            Utils.mkEntry("test_feature_2", new FeatureUpdate((short) 3, true)));
+    }
+
+    private Map<String, Errors> makeTestFeatureUpdateErrors(final Errors error) {
+        final Map<String, FeatureUpdate> updates = makeTestFeatureUpdates();

Review comment:
       Could we make `updates` as a pass-in parameter to avoid calling `makeTestFeatureUpdates` twice?

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -2956,6 +2959,37 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFeatures(request: RequestChannel.Request): Unit = {
+    val updateFeaturesRequest = request.body[UpdateFeaturesRequest]
+    def featureUpdateErrors(error: Errors, msgOverride: Option[String]): Map[String, ApiError] = {
+      updateFeaturesRequest.data().featureUpdates().asScala.map(
+        update => update.feature() -> new ApiError(error, msgOverride.getOrElse(error.message()))
+      ).toMap
+    }
+
+    def sendResponseCallback(updateErrors: Map[String, ApiError]): Unit = {
+      val results = new UpdatableFeatureResultCollection()

Review comment:
       Could be moved to `UpdateFeaturesResponse` as a utility.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.
+   *
+   * This method sets up the FeatureZNode with enabled status. This status means the feature
+   * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode
+   * are active. This status should be written by the controller to the FeatureZNode only when the
+   * broker IBP config is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. The reason to do this is that enabling all the possible features immediately after
+   *    an upgrade could be harmful to the cluster.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, then it
+   *        will react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, it's existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) => {
+            val brokerDefaultVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (brokerDefaultVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (existingVersionRange.max() >= brokerDefaultVersionRange.min() &&
+                       brokerDefaultVersionRange.max() >= existingVersionRange.max()) {
+              // Through this change, we deprecate all version levels in the closed range:
+              // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1]
+              (featureName, new FinalizedVersionRange(brokerDefaultVersionRange.min(), existingVersionRange.max()))
+            } else {
+              // If the existing version levels fall completely outside the
+              // range of the default finalized version levels (i.e. no intersection), or, if the
+              // existing version levels are ineligible for a modification since they are
+              // incompatible with default finalized version levels, then we skip the update.
+              warn(s"Can not update minimum version level in finalized feature: $featureName,"
+                + s" since the existing $existingVersionRange is not eligible for a change"
+                + s" based on the default $brokerDefaultVersionRange.")
+              (featureName, existingVersionRange)
+            }
+          }
+        }.asJava)
+      }
+      val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures)
+      if (!newFeatureZNode.equals(existingFeatureZNode)) {

Review comment:
       I see, still wondering if we could just check whether `newFeatures` is equal to `existingFeatureZNode.features`

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1844,185 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns a suitable error.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }

Review comment:
       nit: new line

##########
File path: clients/src/main/resources/common/message/UpdateFeaturesRequest.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "request",
+  "name": "UpdateFeaturesRequest",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "FeatureUpdates", "type": "[]FeatureUpdateKey", "versions": "0+",
+      "about": "The list of updates to finalized features.", "fields": [
+      {"name": "Feature", "type": "string", "versions": "0+", "mapKey": true,
+        "about": "The name of the finalized feature to be updated."},
+      {"name": "MaxVersionLevel", "type": "int16", "versions": "0+",
+        "about": "The new maximum version level for the finalized feature. A value >= 1 is valid. A value < 1, is special, and can be used to request the deletion of the finalized feature."},
+      {"name": "AllowDowngrade", "type": "bool", "versions": "0+",
+        "about": "When set to true, the finalized feature version level is allowed to be downgraded/deleted."}

Review comment:
       Should we also mention that this flag would fail the request when we are not actually doing a downgrade? 

##########
File path: clients/src/main/java/org/apache/kafka/common/errors/FeatureUpdateFailedException.java
##########
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.errors;
+
+public class FeatureUpdateFailedException extends ApiException {

Review comment:
       Do we need to make this a public error? It seems only be used internally, so could be made private if we don't have intention to let user catch.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,178 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of its own supported features in its
+   * own BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. This process ensures we do not enable all the possible features immediately after
+   *    an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, it will
+   *        react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, its existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) =>
+            val brokerDefaultVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (brokerDefaultVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (brokerDefaultVersionRange.max() >= existingVersionRange.max() &&
+                       brokerDefaultVersionRange.min() <= existingVersionRange.max()) {
+              // Through this change, we deprecate all version levels in the closed range:
+              // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1]
+              (featureName, new FinalizedVersionRange(brokerDefaultVersionRange.min(), existingVersionRange.max()))
+            } else {
+              // If the existing version levels fall completely outside the
+              // range of the default finalized version levels (i.e. no intersection), or, if the
+              // existing version levels are ineligible for a modification since they are
+              // incompatible with default finalized version levels, then we skip the update.
+              warn(s"Can not update minimum version level in finalized feature: $featureName,"
+                + s" since the existing $existingVersionRange is not eligible for a change"
+                + s" based on the default $brokerDefaultVersionRange.")
+              (featureName, existingVersionRange)
+            }
+        }.asJava)
+      }
+      val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures)
+      if (!newFeatureZNode.equals(existingFeatureZNode)) {
+        val newVersion = updateFeatureZNode(newFeatureZNode)
+        featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+      }
+    }
+  }
+
+  /**
+   * Disables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with disabled status. This status means the feature versioning system
+   * (KIP-584) is disabled, and, the finalized features stored in the FeatureZNode are not relevant.
+   * This status should be written by the controller to the FeatureZNode only when the broker
+   * IBP config is less than KAFKA_2_7_IV0.
+   *
+   * NOTE:
+   * 1. When this method returns, existing finalized features (if any) will be cleared from the
+   *    FeatureZNode.
+   * 2. This method, unlike enableFeatureVersioning() need not wait for the FinalizedFeatureCache
+   *    to be updated, because, such updates to the cache (via FinalizedFeatureChangeListener)
+   *    are disabled when IBP config is < than KAFKA_2_7_IV0.
+   */
+  private def disableFeatureVersioning(): Unit = {
+    val newNode = FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures())
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      createFeatureZNode(newNode)

Review comment:
       Do we need to call `featureCache.waitUntilEpochOrThrow(newNode, config.zkConnectionTimeoutMs)` here to ensure the update is successful?

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,178 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature
+ *    version level deprecation. This is how it works: in order to deprecate feature version levels,
+ *    in this map the default minimum version level of a feature can be set to a new value that's
+ *    higher than 1 (let's call this latest_min_version_level). In doing so, the feature version levels
+ *    in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic
+ *    that applies this map to persistent finalized feature state in ZK (this mutation happens
+ *    during controller election and during finalized feature updates via the
+ *    ApiKeys.UPDATE_FINALIZED_FEATURES api). This will automatically mean external clients of Kafka
+ *    would need to stop using the finalized min version levels that have been deprecated.
+ *
+ * This class also provides APIs to check for incompatibilities between the features supported by
+ * the Broker and finalized features. This class is immutable in production. It provides few APIs to
+ * mutate state only for the purpose of testing.
+ */
+class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange],
+                              @volatile var defaultFeatureMinVersionLevels: Map[String, Short]) {
+  require(BrokerFeatures.areFeatureMinVersionLevelsCompatible(
+    supportedFeatures, defaultFeatureMinVersionLevels))
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(newFeatures, defaultFeatureMinVersionLevels))
+    supportedFeatures = newFeatures
+  }
+
+  /**
+   * Returns the default minimum version level for a specific feature.
+   *
+   * @param feature   the name of the feature
+   *
+   * @return          the default minimum version level for the feature if its defined.
+   *                  otherwise, returns 1.
+   */
+  def defaultMinVersionLevel(feature: String): Short = {
+    defaultFeatureMinVersionLevels.getOrElse(feature, 1)
+  }
+
+  // For testing only.
+  def setDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(supportedFeatures, newMinVersionLevels))
+    defaultFeatureMinVersionLevels = newMinVersionLevels
+  }
+
+  /**
+   * Returns the default finalized features that a new Kafka cluster with IBP config >= KAFKA_2_7_IV0
+   * needs to be bootstrapped with.
+   */
+  def getDefaultFinalizedFeatures: Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(
+      supportedFeatures.features.asScala.map {
+        case(name, versionRange) => (
+          name, new FinalizedVersionRange(defaultMinVersionLevel(name), versionRange.max))
+      }.asJava)
+  }
+
+  /**
+   * Returns the set of feature names found to be incompatible.
+   * A feature incompatibility is a version mismatch between the latest feature supported by the
+   * Broker, and the provided finalized feature. This can happen because a provided finalized
+   * feature:
+   *  1) Does not exist in the Broker (i.e. it is unknown to the Broker).
+   *           [OR]
+   *  2) Exists but the FinalizedVersionRange does not match with the
+   *     supported feature's SupportedVersionRange.
+   *
+   * @param finalized   The finalized features against which incompatibilities need to be checked for.
+   *
+   * @return            The subset of input features which are incompatible. If the returned object
+   *                    is empty, it means there were no feature incompatibilities found.
+   */
+  def incompatibleFeatures(finalized: Features[FinalizedVersionRange]): Features[FinalizedVersionRange] = {
+    BrokerFeatures.incompatibleFeatures(supportedFeatures, finalized, logIncompatibilities = true)
+  }
+}
+
+object BrokerFeatures extends Logging {
+
+  def createDefault(): BrokerFeatures = {
+    // The arguments are currently empty, but, in the future as we define features we should
+    // populate the required values here.
+    new BrokerFeatures(emptySupportedFeatures, Map[String, Short]())
+  }
+
+  /**
+   * Returns true if any of the provided finalized features are incompatible with the provided
+   * supported features.
+   *
+   * @param supportedFeatures   The supported features to be compared
+   * @param finalizedFeatures   The finalized features to be compared
+   *
+   * @return                    - True if there are any feature incompatibilities found.
+   *                            - False otherwise.
+   */
+  def hasIncompatibleFeatures(supportedFeatures: Features[SupportedVersionRange],
+                              finalizedFeatures: Features[FinalizedVersionRange]): Boolean = {
+    !incompatibleFeatures(supportedFeatures, finalizedFeatures, false).empty
+  }
+
+  private def incompatibleFeatures(supportedFeatures: Features[SupportedVersionRange],
+                                   finalizedFeatures: Features[FinalizedVersionRange],
+                                   logIncompatibilities: Boolean): Features[FinalizedVersionRange] = {
+    val incompatibleFeaturesInfo = finalizedFeatures.features.asScala.map {
+      case (feature, versionLevels) =>
+        val supportedVersions = supportedFeatures.get(feature)
+        if (supportedVersions == null) {
+          (feature, versionLevels, "{feature=%s, reason='Unsupported feature'}".format(feature))
+        } else if (versionLevels.isIncompatibleWith(supportedVersions)) {
+          (feature, versionLevels, "{feature=%s, reason='%s is incompatible with %s'}".format(
+            feature, versionLevels, supportedVersions))
+        } else {
+          (feature, versionLevels, null)
+        }
+    }.filter{ case(_, _, errorReason) => errorReason != null}.toList
+
+    if (logIncompatibilities && incompatibleFeaturesInfo.nonEmpty) {
+      warn(
+        "Feature incompatibilities seen: " + incompatibleFeaturesInfo.map {
+          case(_, _, errorReason) => errorReason })
+    }
+    Features.finalizedFeatures(incompatibleFeaturesInfo.map {
+      case(feature, versionLevels, _) => (feature, versionLevels) }.toMap.asJava)
+  }
+
+  /**
+   * A check that ensures each feature defined with min version level is a supported feature, and
+   * the min version level value is valid (i.e. it is compatible with the supported version range).
+   *
+   * @param supportedFeatures         the supported features
+   * @param featureMinVersionLevels   the feature minimum version levels
+   *
+   * @return                          - true, if the above described check passes.
+   *                                  - false, otherwise.
+   */
+  private def areFeatureMinVersionLevelsCompatible(
+    supportedFeatures: Features[SupportedVersionRange],
+    featureMinVersionLevels: Map[String, Short]
+  ): Boolean = {
+    featureMinVersionLevels.forall {
+      case(featureName, minVersionLevel) =>
+        val supportedFeature = supportedFeatures.get(featureName)
+        (supportedFeature != null) &&
+          !new FinalizedVersionRange(minVersionLevel, supportedFeature.max())

Review comment:
       Could we get a static method instead of initiating a new `FinalizedVersionRange` for a comparison every time?

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1844,188 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns a suitable error.

Review comment:
       State the error explicitly here.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1844,188 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns a suitable error.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+    // NOTE: Below we set the finalized min version level to be the default minimum version
+    // level. If the finalized feature already exists, then, this can cause deprecation of all
+    // version levels in the closed range:
+    // [existingVersionRange.min(), defaultMinVersionLevel - 1].
+    val defaultMinVersionLevel = brokerFeatures.defaultMinVersionLevel(update.feature)
+    val newVersionRange = new FinalizedVersionRange(defaultMinVersionLevel, update.maxVersionLevel)
+    val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+      val singleFinalizedFeature =
+        Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange)))
+      BrokerFeatures.hasIncompatibleFeatures(broker.features, singleFinalizedFeature)
+    })
+    if (numIncompatibleBrokers == 0) {
+      Left(newVersionRange)
+    } else {
+      Right(
+        new ApiError(Errors.INVALID_REQUEST,
+                     s"Could not apply finalized feature update because $numIncompatibleBrokers" +
+                     " brokers were found to have incompatible features."))
+    }
+  }
+
+  /**
+   * Validate and process a finalized feature update.
+   *
+   * If the processing is successful, then, the return value contains:
+   * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature.
+   * 2. Option.empty, if the feature update was meant to delete the feature.
+   *
+   * If the processing failed, then returned value contains a suitable ApiError.
+   *
+   * @param update   the feature update to be processed.
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def processFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = {
+    val existingFeatures = featureCache.get
+      .map(featuresAndEpoch => featuresAndEpoch.features.features().asScala)
+      .getOrElse(Map[String, FinalizedVersionRange]())
+
+    def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = {
+      newFinalizedVersionRangeOrIncompatibilityError(update)
+        .fold(versionRange => Left(Some(versionRange)), error => Right(error))
+    }
+
+    if (update.feature.isEmpty) {
+      // Check that the feature name is not empty.
+      Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty."))
+    } else {
+      val cacheEntry = existingFeatures.get(update.feature).orNull
+
+      // We handle deletion requests separately from non-deletion requests.
+      if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+        if (cacheEntry == null) {
+          // Disallow deletion of a non-existing finalized feature.
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             s"Can not delete non-existing finalized feature: '${update.feature}'"))
+        } else {
+          Left(Option.empty)
+        }
+      } else if (update.maxVersionLevel() < 1) {
+        // Disallow deletion of a finalized feature without allowDowngrade flag set.
+        Right(new ApiError(Errors.INVALID_REQUEST,
+                           s"Can not provide maxVersionLevel: ${update.maxVersionLevel} less" +
+                           s" than 1 for feature: '${update.feature}' without setting the" +
+                           " allowDowngrade flag to true in the request."))
+      } else {
+        if (cacheEntry == null) {
+          newVersionRangeOrError(update)
+        } else {
+          if (update.maxVersionLevel == cacheEntry.max()) {
+            // Disallow a case where target maxVersionLevel matches existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not ${if (update.allowDowngrade) "downgrade" else "upgrade"}" +
+                               s" a finalized feature: '${update.feature}' from existing" +
+                               s" maxVersionLevel:${cacheEntry.max} to the same value."))
+          } else if (update.maxVersionLevel < cacheEntry.max && !update.allowDowngrade) {
+            // Disallow downgrade of a finalized feature without the allowDowngrade flag set.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature: '${update.feature}' from" +
+                               s" existing maxVersionLevel:${cacheEntry.max} to provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} without setting the" +
+                               " allowDowngrade flag in the request."))
+          } else if (update.allowDowngrade && update.maxVersionLevel > cacheEntry.max) {

Review comment:
       I'm actually wondering whether this is too strict in the perspective of a user. If they accidentally set a feature version larger than the cache, what they only care about is to be able to change the version to it. So it's a matter of whether we think this is a user error, or this could happen when user gets stale feature information from a broker while the downgrade already succeed eventually. 
   
   If we want to keep this check, it makes sense to update the meta comments around `allowDowngrade` to inform user that the request could fail when the target version is actually higher than the current finalized feature.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1844,185 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns a suitable error.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+    // NOTE: Below we set the finalized min version level to be the default minimum version
+    // level. If the finalized feature already exists, then, this can cause deprecation of all
+    // version levels in the closed range:
+    // [existingVersionRange.min(), defaultMinVersionLevel - 1].
+    val defaultMinVersionLevel = brokerFeatures.defaultMinVersionLevel(update.feature)
+    val newVersionRange = new FinalizedVersionRange(defaultMinVersionLevel, update.maxVersionLevel)
+    val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+      val singleFinalizedFeature =
+        Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange)))
+      BrokerFeatures.hasIncompatibleFeatures(broker.features, singleFinalizedFeature)
+    })
+    if (numIncompatibleBrokers == 0) {
+      Left(newVersionRange)
+    } else {
+      Right(
+        new ApiError(Errors.INVALID_REQUEST,
+                     s"Could not apply finalized feature update because $numIncompatibleBrokers" +
+                     " brokers were found to have incompatible features."))
+    }
+  }
+
+  /**
+   * Validate and process a finalized feature update on an existing FinalizedVersionRange for the
+   * feature.
+   *
+   * If the processing is successful, then, the return value contains:
+   * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature.
+   * 2. Option.empty, if the feature update was meant to delete the feature.
+   *
+   * If the processing failed, then returned value contains a suitable ApiError.
+   *
+   * @param update                 the feature update to be processed.
+   * @param existingVersionRange   the existing FinalizedVersionRange which can be empty when no
+   *                               FinalizedVersionRange exists for the associated feature
+   *
+   * @return                       the new FinalizedVersionRange or error, as described above.
+   */
+  private def processFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey,
+                                   existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = {
+    def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = {
+      newFinalizedVersionRangeOrIncompatibilityError(update)
+        .fold(versionRange => Left(Some(versionRange)), error => Right(error))
+    }
+
+    if (update.feature.isEmpty) {
+      // Check that the feature name is not empty.
+      Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty."))
+    } else {
+      // We handle deletion requests separately from non-deletion requests.
+      if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+        if (existingVersionRange.isEmpty) {
+          // Disallow deletion of a non-existing finalized feature.
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             s"Can not delete non-existing finalized feature: '${update.feature}'"))
+        } else {
+          Left(Option.empty)
+        }
+      } else if (update.maxVersionLevel() < 1) {

Review comment:
       Is this case covered by the case on L1931? Could we merge both?

##########
File path: core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala
##########
@@ -0,0 +1,84 @@
+/**
+ * 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.server
+
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.junit.Assert.{assertEquals, assertThrows, assertTrue}
+import org.junit.Test
+
+import scala.jdk.CollectionConverters._
+
+class BrokerFeaturesTest {

Review comment:
       Some methods in the `BrokerFeatures` are not covered by this suite, such as `defaultMinVersionLevel`, `getDefaultFinalizedFeatures` and `hasIncompatibleFeatures`, you could use code coverage tool to figure out any missing part.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1395,7 +1596,7 @@ class KafkaController(val config: KafkaConfig,
     if (newMetadataOpt.nonEmpty && oldMetadataOpt.nonEmpty) {
       val oldMetadata = oldMetadataOpt.get
       val newMetadata = newMetadataOpt.get
-      if (newMetadata.endPoints != oldMetadata.endPoints) {
+      if (newMetadata.endPoints != oldMetadata.endPoints || !oldMetadata.features.equals(newMetadata.features)) {

Review comment:
       I see, still I'm a bit worried future changes could break this assumption. Not a bad idea to check `features != null`?

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1844,185 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns a suitable error.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+    // NOTE: Below we set the finalized min version level to be the default minimum version
+    // level. If the finalized feature already exists, then, this can cause deprecation of all
+    // version levels in the closed range:
+    // [existingVersionRange.min(), defaultMinVersionLevel - 1].
+    val defaultMinVersionLevel = brokerFeatures.defaultMinVersionLevel(update.feature)
+    val newVersionRange = new FinalizedVersionRange(defaultMinVersionLevel, update.maxVersionLevel)
+    val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+      val singleFinalizedFeature =
+        Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange)))
+      BrokerFeatures.hasIncompatibleFeatures(broker.features, singleFinalizedFeature)
+    })
+    if (numIncompatibleBrokers == 0) {
+      Left(newVersionRange)
+    } else {
+      Right(
+        new ApiError(Errors.INVALID_REQUEST,
+                     s"Could not apply finalized feature update because $numIncompatibleBrokers" +
+                     " brokers were found to have incompatible features."))
+    }
+  }
+
+  /**
+   * Validate and process a finalized feature update on an existing FinalizedVersionRange for the
+   * feature.
+   *
+   * If the processing is successful, then, the return value contains:
+   * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature.
+   * 2. Option.empty, if the feature update was meant to delete the feature.
+   *
+   * If the processing failed, then returned value contains a suitable ApiError.
+   *
+   * @param update                 the feature update to be processed.
+   * @param existingVersionRange   the existing FinalizedVersionRange which can be empty when no
+   *                               FinalizedVersionRange exists for the associated feature
+   *
+   * @return                       the new FinalizedVersionRange or error, as described above.
+   */
+  private def processFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey,
+                                   existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = {
+    def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = {
+      newFinalizedVersionRangeOrIncompatibilityError(update)
+        .fold(versionRange => Left(Some(versionRange)), error => Right(error))
+    }
+
+    if (update.feature.isEmpty) {
+      // Check that the feature name is not empty.
+      Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty."))
+    } else {
+      // We handle deletion requests separately from non-deletion requests.
+      if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+        if (existingVersionRange.isEmpty) {
+          // Disallow deletion of a non-existing finalized feature.
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             s"Can not delete non-existing finalized feature: '${update.feature}'"))
+        } else {
+          Left(Option.empty)
+        }
+      } else if (update.maxVersionLevel() < 1) {
+        // Disallow deletion of a finalized feature without allowDowngrade flag set.
+        Right(new ApiError(Errors.INVALID_REQUEST,
+                           s"Can not provide maxVersionLevel: ${update.maxVersionLevel} less" +
+                           s" than 1 for feature: '${update.feature}' without setting the" +
+                           " allowDowngrade flag to true in the request."))
+      } else {
+        existingVersionRange.map(existing =>
+          if (update.maxVersionLevel == existing.max) {
+            // Disallow a case where target maxVersionLevel matches existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+              s"Can not ${if (update.allowDowngrade) "downgrade" else "upgrade"}" +
+                s" a finalized feature: '${update.feature}' from existing" +
+                s" maxVersionLevel:${existing.max} to the same value."))
+          } else if (update.maxVersionLevel < existing.max && !update.allowDowngrade) {
+            // Disallow downgrade of a finalized feature without the allowDowngrade flag set.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+              s"Can not downgrade finalized feature: '${update.feature}' from" +
+                s" existing maxVersionLevel:${existing.max} to provided" +
+                s" maxVersionLevel:${update.maxVersionLevel} without setting the" +
+                " allowDowngrade flag in the request."))
+          } else if (update.allowDowngrade && update.maxVersionLevel > existing.max) {
+            // Disallow a request that sets allowDowngrade flag without specifying a
+            // maxVersionLevel that's lower than the existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+              s"When finalized feature: '${update.feature}' has the allowDowngrade" +
+                " flag set in the request, the provided" +
+                s" maxVersionLevel:${update.maxVersionLevel} can not be greater than" +
+                s" existing maxVersionLevel:${existing.max}."))
+          } else if (update.maxVersionLevel() < existing.min) {
+            // Disallow downgrade of a finalized feature below the existing finalized
+            // minVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+              s"Can not downgrade finalized feature: '${update.feature}' to" +
+                s" maxVersionLevel:${update.maxVersionLevel} because it's lower than" +
+                s" the existing minVersionLevel:${existing.min}."))
+          } else {
+            newVersionRangeOrError(update)
+          }
+        ).getOrElse(newVersionRangeOrError(update))
+      }
+    }
+  }
+
+  private def processFeatureUpdates(request: UpdateFeaturesRequest,
+                                    callback: UpdateFeaturesCallback): Unit = {
+    if (isActive) {
+      processFeatureUpdatesWithActiveController(request, callback)
+    } else {
+      val results = request.data().featureUpdates().asScala.map {
+        update => update.feature() -> new ApiError(Errors.NOT_CONTROLLER)
+      }.toMap
+      callback(results)
+    }
+  }
+
+  private def processFeatureUpdatesWithActiveController(request: UpdateFeaturesRequest,
+                                                        callback: UpdateFeaturesCallback): Unit = {
+    val updates = request.data.featureUpdates
+    val existingFeatures = featureCache.get
+      .map(featuresAndEpoch => featuresAndEpoch.features.features().asScala)
+      .getOrElse(Map[String, FinalizedVersionRange]())
+    // Map of feature to FinalizedVersionRange. This contains the target features to be eventually
+    // written to FeatureZNode.
+    val targetFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]() ++ existingFeatures
+    // Map of feature to error.
+    var errors = scala.collection.mutable.Map[String, ApiError]()
+
+    // Process each FeatureUpdate.
+    // If a FeatureUpdate is found to be valid, then the corresponding entry in errors would contain
+    // Errors.NONE. Otherwise the entry would contain the appropriate error.
+    updates.asScala.iterator.foreach { update =>
+      processFeatureUpdate(update, existingFeatures.get(update.feature())) match {
+        case Left(newVersionRangeOrNone) =>
+          newVersionRangeOrNone
+            .map(newVersionRange => targetFeatures += (update.feature() -> newVersionRange))
+            .getOrElse(targetFeatures -= update.feature())
+          errors += (update.feature() -> new ApiError(Errors.NONE))
+        case Right(featureUpdateFailureReason) =>
+          errors += (update.feature() -> featureUpdateFailureReason)
+      }
+    }
+
+    if (existingFeatures.equals(targetFeatures)) {

Review comment:
       Could you clarify the reasoning here? If structs are not the same, are we going to do a partial update?

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1844,185 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns a suitable error.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+    // NOTE: Below we set the finalized min version level to be the default minimum version
+    // level. If the finalized feature already exists, then, this can cause deprecation of all
+    // version levels in the closed range:
+    // [existingVersionRange.min(), defaultMinVersionLevel - 1].
+    val defaultMinVersionLevel = brokerFeatures.defaultMinVersionLevel(update.feature)
+    val newVersionRange = new FinalizedVersionRange(defaultMinVersionLevel, update.maxVersionLevel)
+    val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+      val singleFinalizedFeature =
+        Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange)))
+      BrokerFeatures.hasIncompatibleFeatures(broker.features, singleFinalizedFeature)
+    })
+    if (numIncompatibleBrokers == 0) {
+      Left(newVersionRange)
+    } else {
+      Right(
+        new ApiError(Errors.INVALID_REQUEST,
+                     s"Could not apply finalized feature update because $numIncompatibleBrokers" +
+                     " brokers were found to have incompatible features."))
+    }
+  }
+
+  /**
+   * Validate and process a finalized feature update on an existing FinalizedVersionRange for the
+   * feature.
+   *
+   * If the processing is successful, then, the return value contains:
+   * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature.
+   * 2. Option.empty, if the feature update was meant to delete the feature.
+   *
+   * If the processing failed, then returned value contains a suitable ApiError.
+   *
+   * @param update                 the feature update to be processed.
+   * @param existingVersionRange   the existing FinalizedVersionRange which can be empty when no
+   *                               FinalizedVersionRange exists for the associated feature
+   *
+   * @return                       the new FinalizedVersionRange or error, as described above.
+   */
+  private def processFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey,
+                                   existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = {
+    def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = {
+      newFinalizedVersionRangeOrIncompatibilityError(update)
+        .fold(versionRange => Left(Some(versionRange)), error => Right(error))
+    }
+
+    if (update.feature.isEmpty) {
+      // Check that the feature name is not empty.
+      Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty."))
+    } else {
+      // We handle deletion requests separately from non-deletion requests.
+      if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+        if (existingVersionRange.isEmpty) {
+          // Disallow deletion of a non-existing finalized feature.
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             s"Can not delete non-existing finalized feature: '${update.feature}'"))
+        } else {
+          Left(Option.empty)
+        }
+      } else if (update.maxVersionLevel() < 1) {
+        // Disallow deletion of a finalized feature without allowDowngrade flag set.
+        Right(new ApiError(Errors.INVALID_REQUEST,
+                           s"Can not provide maxVersionLevel: ${update.maxVersionLevel} less" +
+                           s" than 1 for feature: '${update.feature}' without setting the" +
+                           " allowDowngrade flag to true in the request."))
+      } else {
+        existingVersionRange.map(existing =>
+          if (update.maxVersionLevel == existing.max) {
+            // Disallow a case where target maxVersionLevel matches existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+              s"Can not ${if (update.allowDowngrade) "downgrade" else "upgrade"}" +
+                s" a finalized feature: '${update.feature}' from existing" +
+                s" maxVersionLevel:${existing.max} to the same value."))
+          } else if (update.maxVersionLevel < existing.max && !update.allowDowngrade) {
+            // Disallow downgrade of a finalized feature without the allowDowngrade flag set.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+              s"Can not downgrade finalized feature: '${update.feature}' from" +
+                s" existing maxVersionLevel:${existing.max} to provided" +
+                s" maxVersionLevel:${update.maxVersionLevel} without setting the" +
+                " allowDowngrade flag in the request."))
+          } else if (update.allowDowngrade && update.maxVersionLevel > existing.max) {
+            // Disallow a request that sets allowDowngrade flag without specifying a
+            // maxVersionLevel that's lower than the existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+              s"When finalized feature: '${update.feature}' has the allowDowngrade" +
+                " flag set in the request, the provided" +
+                s" maxVersionLevel:${update.maxVersionLevel} can not be greater than" +
+                s" existing maxVersionLevel:${existing.max}."))
+          } else if (update.maxVersionLevel() < existing.min) {

Review comment:
       We should be consistent and remove `()` from `maxVersionLevel`

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -39,7 +42,7 @@ case class FinalizedFeaturesAndEpoch(features: Features[FinalizedVersionRange],
  *
  * @see FinalizedFeatureChangeListener
  */
-object FinalizedFeatureCache extends Logging {
+class FinalizedFeatureCache(private val brokerFeatures: BrokerFeatures) extends Logging {

Review comment:
       The meta comment for `FinalizedFeatureCache` should be updated as it is now being accessed for both read and write

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -2945,6 +2948,130 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFeatures(request: RequestChannel.Request): Unit = {

Review comment:
       Seems not covered yet

##########
File path: core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala
##########
@@ -78,25 +76,42 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness {
   /**
    * Tests that the listener can be initialized, and that it can listen to ZK notifications
    * successfully from an "Enabled" FeatureZNode (the ZK data has no feature incompatibilities).
+   * Particularly the test checks if multiple notifications can be processed in ZK
+   * (i.e. whether the FeatureZNode watch can be re-established).
    */
   @Test
   def testInitSuccessAndNotificationSuccess(): Unit = {
-    createSupportedFeatures()
     val initialFinalizedFeatures = createFinalizedFeatures()
-    val listener = createListener(Some(initialFinalizedFeatures))
+    val brokerFeatures = createBrokerFeatures()
+    val cache = new FinalizedFeatureCache(brokerFeatures)
+    val listener = createListener(cache, Some(initialFinalizedFeatures))
 
-    val updatedFinalizedFeaturesMap = Map[String, FinalizedVersionRange](
-      "feature_1" -> new FinalizedVersionRange(2, 4))
-    val updatedFinalizedFeatures = Features.finalizedFeatures(updatedFinalizedFeaturesMap.asJava)
-    zkClient.updateFeatureZNode(FeatureZNode(FeatureZNodeStatus.Enabled, updatedFinalizedFeatures))
-    val (mayBeFeatureZNodeNewBytes, updatedVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
-    assertNotEquals(updatedVersion, ZkVersion.UnknownVersion)
-    assertFalse(mayBeFeatureZNodeNewBytes.isEmpty)
-    assertTrue(updatedVersion > initialFinalizedFeatures.epoch)
-    TestUtils.waitUntilTrue(() => {
-      FinalizedFeatureCache.get.get.equals(FinalizedFeaturesAndEpoch(updatedFinalizedFeatures, updatedVersion))
-    }, "Timed out waiting for FinalizedFeatureCache to be updated with new features")
-    assertTrue(listener.isListenerInitiated)
+    def updateAndCheckCache(finalizedFeatures: Features[FinalizedVersionRange]): Unit = {
+      zkClient.updateFeatureZNode(FeatureZNode(FeatureZNodeStatus.Enabled, finalizedFeatures))
+      val (mayBeFeatureZNodeNewBytes, updatedVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+      assertNotEquals(updatedVersion, ZkVersion.UnknownVersion)
+      assertFalse(mayBeFeatureZNodeNewBytes.isEmpty)
+      assertTrue(updatedVersion > initialFinalizedFeatures.epoch)
+
+      cache.waitUntilEpochOrThrow(updatedVersion, JTestUtils.DEFAULT_MAX_WAIT_MS)
+      assertEquals(FinalizedFeaturesAndEpoch(finalizedFeatures, updatedVersion), cache.get.get)
+      assertTrue(listener.isListenerInitiated)
+    }
+
+    // Check if the write succeeds and a ZK notification is received that causes the feature cache
+    // to be populated.
+    updateAndCheckCache(
+      Features.finalizedFeatures(
+        Map[String, FinalizedVersionRange](
+      "feature_1" -> new FinalizedVersionRange(2, 4)).asJava))

Review comment:
       Indentation is not right.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -109,7 +109,9 @@ class KafkaApis(val requestChannel: RequestChannel,
                 brokerTopicStats: BrokerTopicStats,
                 val clusterId: String,
                 time: Time,
-                val tokenManager: DelegationTokenManager) extends Logging {
+                val tokenManager: DelegationTokenManager,
+                val brokerFeatures: BrokerFeatures,

Review comment:
       Could we only pass in `featureCache` to reduce the class coupling here? As we already have `brokerFeatures` as a private parameter, it shouldn't be too hard to set a helper to get supported features.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java
##########
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+@InterfaceStability.Evolving
+public class DescribeFeaturesOptions extends AbstractOptions<DescribeFeaturesOptions> {
+    /**
+     * - True means the {@link Admin#describeFeatures(DescribeFeaturesOptions)} request can be
+     *   issued only to the controller.
+     * - False means the {@link Admin#describeFeatures(DescribeFeaturesOptions)} request can be
+     *   issued to any random broker.
+     */
+    private boolean shouldSendRequestToController = false;

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1215,70 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker. It can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter. This is particularly useful if the user requires strongly consistent reads of
+     * finalized features.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the DescribeFeaturesResult containing the result

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -68,6 +69,36 @@ public ApiVersionsResponse(Struct struct, short version) {
         this(new ApiVersionsResponseData(struct, version));
     }
 
+    public ApiVersionsResponseData data() {
+        return data;
+    }
+
+    public Features<SupportedVersionRange> supportedFeatures() {
+        final Map<String, SupportedVersionRange> features = new HashMap<>();
+
+        for (SupportedFeatureKey key : data.supportedFeatures().valuesSet()) {
+            features.put(key.name(), new SupportedVersionRange(key.minVersion(), key.maxVersion()));
+        }
+
+        return Features.supportedFeatures(features);
+    }
+
+    public Features<FinalizedVersionRange> finalizedFeatures() {
+        final Map<String, FinalizedVersionRange> features = new HashMap<>();
+
+        for (FinalizedFeatureKey key : data.finalizedFeatures().valuesSet()) {
+            features.put(

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java
##########
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+@InterfaceStability.Evolving
+public class DescribeFeaturesOptions extends AbstractOptions<DescribeFeaturesOptions> {

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -3984,6 +3988,120 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+
+        Call callViaLeastLoadedNode = new Call("describeFeatures", calcDeadlineMs(now, options.timeoutMs()),
+            new LeastLoadedNodeProvider()) {
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(
+                        new FeatureMetadata(
+                            apiVersionsResponse.finalizedFeatures(),
+                            apiVersionsResponse.finalizedFeaturesEpoch(),
+                            apiVersionsResponse.supportedFeatures()));
+                } else {
+                    future.completeExceptionally(
+                        Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        Call callViaControllerNode = new Call("describeFeatures", calcDeadlineMs(now, options.timeoutMs()),

Review comment:
       Done. Good point.

##########
File path: clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
##########
@@ -413,6 +422,42 @@ private static DescribeGroupsResponseData prepareDescribeGroupsResponseData(Stri
                 Collections.emptySet()));
         return data;
     }
+
+    private static UpdateFeaturesResponse prepareUpdateFeaturesResponse(Errors error) {
+        final UpdateFeaturesResponseData data = new UpdateFeaturesResponseData();
+        data.setErrorCode(error.code());
+        return new UpdateFeaturesResponse(data);
+    }
+
+    private static FeatureMetadata getDefaultFeatureMetadata() {
+        return new FeatureMetadata(
+            Features.finalizedFeatures(new HashMap<String, FinalizedVersionRange>() {
+                {

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesOptions.java
##########
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+@InterfaceStability.Evolving
+public class UpdateFeaturesOptions extends AbstractOptions<UpdateFeaturesOptions> {
+    /**
+     * Sets the timeout in milliseconds for this operation or {@code null} if the default API
+     * timeout for the AdminClient should be used.
+     */
+    public UpdateFeaturesOptions timeoutMs(final Integer timeoutMs) {

Review comment:
       Done. Removed now.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/FinalizedVersionRange.java
##########
@@ -50,4 +50,8 @@ public static FinalizedVersionRange fromMap(Map<String, Short> versionRangeMap)
     public boolean isIncompatibleWith(SupportedVersionRange supportedVersionRange) {
         return min() < supportedVersionRange.min() || max() > supportedVersionRange.max();
     }
+
+    public boolean isCompatibleWith(SupportedVersionRange supportedVersionRange) {

Review comment:
       Done. Removed this method now.

##########
File path: clients/src/main/resources/common/message/UpdateFeaturesRequest.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "request",
+  "name": "UpdateFeaturesRequest",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "FeatureUpdates", "type": "[]FeatureUpdateKey", "versions": "0+",
+      "about": "The list of updates to finalized features.", "fields": [
+      {"name": "Name", "type":  "string", "versions":  "0+", "mapKey": true,
+        "about": "The name of the finalized feature to be updated."},
+      {"name":  "MaxVersionLevel", "type": "int16", "versions":  "0+",

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1848,36 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def processUpdateFeatures(newFeatures: Features[FinalizedVersionRange],
+                                             callback: UpdateFeaturesCallback): Unit = {

Review comment:
       Done.

##########
File path: clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
##########
@@ -413,6 +422,42 @@ private static DescribeGroupsResponseData prepareDescribeGroupsResponseData(Stri
                 Collections.emptySet()));
         return data;
     }
+
+    private static UpdateFeaturesResponse prepareUpdateFeaturesResponse(Errors error) {
+        final UpdateFeaturesResponseData data = new UpdateFeaturesResponseData();
+        data.setErrorCode(error.code());
+        return new UpdateFeaturesResponse(data);
+    }
+
+    private static FeatureMetadata getDefaultFeatureMetadata() {

Review comment:
       Done.

##########
File path: clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
##########
@@ -3193,6 +3238,104 @@ public void testListOffsetsNonRetriableErrors() throws Exception {
         }
     }
 
+    @Test
+    public void testUpdateFeaturesDuringSuccess() throws Exception {
+        testUpdateFeaturesDuringError(Errors.NONE);
+    }
+
+    @Test
+    public void testUpdateFeaturesInvalidRequestError() throws Exception {
+        testUpdateFeaturesDuringError(Errors.INVALID_REQUEST);
+    }
+
+    @Test
+    public void testUpdateFeaturesUpdateFailedError() throws Exception {
+        testUpdateFeaturesDuringError(Errors.FEATURE_UPDATE_FAILED);
+    }
+
+    private void testUpdateFeaturesDuringError(Errors error) throws Exception {
+        try (final AdminClientUnitTestEnv env = mockClientEnv()) {
+            env.kafkaClient().prepareResponse(
+                body -> body instanceof UpdateFeaturesRequest,
+                prepareUpdateFeaturesResponse(error));
+            final KafkaFuture<Void> future = env.adminClient().updateFeatures(
+                new HashSet<>(
+                    Arrays.asList(
+                        new FeatureUpdate(
+                            "test_feature_1", (short) 2, false),
+                        new FeatureUpdate(
+                            "test_feature_2", (short) 3, true))),
+                new UpdateFeaturesOptions().timeoutMs(10000)).result();
+            if (error.exception() == null) {
+                future.get();
+            } else {
+                final ExecutionException e = assertThrows(ExecutionException.class,
+                    () -> future.get());
+                assertEquals(e.getCause().getClass(), error.exception().getClass());
+            }
+        }
+    }
+
+    @Test
+    public void testUpdateFeaturesHandleNotControllerException() throws Exception {
+        try (final AdminClientUnitTestEnv env = mockClientEnv()) {
+            env.kafkaClient().prepareResponseFrom(
+                prepareUpdateFeaturesResponse(Errors.NOT_CONTROLLER),
+                env.cluster().nodeById(0));
+            env.kafkaClient().prepareResponse(MetadataResponse.prepareResponse(env.cluster().nodes(),
+                env.cluster().clusterResource().clusterId(),
+                1,

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.
+   *
+   * This method sets up the FeatureZNode with enabled status. This status means the feature
+   * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode
+   * are active. This status should be written by the controller to the FeatureZNode only when the
+   * broker IBP config is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined

Review comment:
       Done.

##########
File path: core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala
##########
@@ -0,0 +1,83 @@
+/**
+ * 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.server
+
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.junit.Assert.{assertEquals, assertThrows, assertTrue}
+import org.junit.Test
+
+import scala.jdk.CollectionConverters._
+
+class BrokerFeaturesTest {
+  @Test
+  def testEmpty(): Unit = {
+    assertTrue(BrokerFeatures.createDefault().supportedFeatures.empty)
+  }
+
+  @Test
+  def testIncompatibleFeatures(): Unit = {
+    val brokerFeatures = BrokerFeatures.createDefault()
+    val supportedFeatures = Map[String, SupportedVersionRange](
+      "test_feature_1" -> new SupportedVersionRange(1, 4),
+      "test_feature_2" -> new SupportedVersionRange(1, 3))
+    brokerFeatures.setSupportedFeatures(Features.supportedFeatures(supportedFeatures.asJava))
+
+    val compatibleFeatures = Map[String, FinalizedVersionRange](
+      "test_feature_1" -> new FinalizedVersionRange(2, 3))
+    val inCompatibleFeatures = Map[String, FinalizedVersionRange](
+      "test_feature_2" -> new FinalizedVersionRange(1, 4),
+      "test_feature_3" -> new FinalizedVersionRange(3, 4))
+    val features = compatibleFeatures++inCompatibleFeatures
+    val finalizedFeatures = Features.finalizedFeatures(features.asJava)
+
+    assertEquals(
+      Features.finalizedFeatures(inCompatibleFeatures.asJava),
+      brokerFeatures.incompatibleFeatures(finalizedFeatures))
+  }
+
+  @Test
+  def testFeatureVersionAssertions(): Unit = {
+    val brokerFeatures = BrokerFeatures.createDefault()
+    val supportedFeatures = Features.supportedFeatures(Map[String, SupportedVersionRange](
+      "test_feature_1" -> new SupportedVersionRange(1, 4),
+      "test_feature_2" -> new SupportedVersionRange(1, 3)).asJava)
+    brokerFeatures.setSupportedFeatures(supportedFeatures)
+
+    val defaultMinVersionLevelsWithNonExistingFeature = Map[String, Short](
+      "test_feature_1" -> 2,
+      "test_feature_2" -> 2,
+      "test_feature_non_existing" -> 5)
+    assertThrows(
+      classOf[IllegalArgumentException],
+      () => brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevelsWithNonExistingFeature))
+
+    val defaultMinVersionLevelsWithInvalidValue1 = Map[String, Short](
+      "test_feature_1" -> 2,
+      "test_feature_2" -> (supportedFeatures.get("test_feature_2").min() - 1).asInstanceOf[Short])
+    assertThrows(
+      classOf[IllegalArgumentException],
+      () => brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevelsWithInvalidValue1))
+
+    val defaultMinVersionLevelsWithInvalidValue2 = Map[String, Short](

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,158 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * Enables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with enabled status. This status means the feature versioning system
+   * (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode are active. This
+   * status should be written by the controller to the FeatureZNode only when the broker IBP config
+   * is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    For a new Kafka cluster (i.e. it is deployed first time), we would like to start the cluster
+   *    with all the possible supported features finalized immediately. The new cluster will almost
+   *    never be started with an old IBP config that’s less than KAFKA_2_7_IV0. Assuming this is the
+   *    case, then here is how we it: the controller will start up and notice that the FeatureZNode
+   *    is absent in the new cluster, it will then create a FeatureZNode (with enabled status)
+   *    containing the entire list of default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. The reason to do this is that enabling all the possible features immediately after
+   *    an upgrade could be harmful to the cluster.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, then it
+   *        will react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, it's existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) => {
+            val updatedVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (updatedVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (existingVersionRange.max() >= updatedVersionRange.min()) {
+              // Through this change, we deprecate all version levels in the closed range:
+              // [existingVersionRange.min(), updatedVersionRange.min() - 1]
+              (featureName, new FinalizedVersionRange(updatedVersionRange.min(), existingVersionRange.max()))
+            } else {
+              // This is a special case: If the existing version levels fall completely outside the
+              // range of the default finalized version levels (i.e. no intersection), then, this
+              // case is not eligible for deprecation. This requires that the max version level be
+              // upgraded first to a value that's equal to the the default minimum version level.
+              info(s"Can not update minimum version level in finalized feature: $featureName,"
+              + s" since the existing $existingVersionRange does not intersect with the default"
+              + s" $updatedVersionRange.")
+              (featureName, existingVersionRange)
+            }
+          }
+        }.asJava)
+      }
+      val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures)
+      if (!newFeatureZNode.equals(existingFeatureZNode)) {
+        val newVersion = updateFeatureZNode(newFeatureZNode)
+        featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+      }
+    }
+  }
+
+  /**
+   * Disables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with disabled status. This status means the feature versioning system
+   * (KIP-584) is disabled, and, the finalized features stored in the FeatureZNode are not relevant.
+   * This status should be written by the controller to the FeatureZNode only when the broker
+   * IBP config is less than KAFKA_2_7_IV0.
+   *
+   * NOTE:
+   * 1. When this method returns, existing finalized features (if any) will be cleared from the
+   *    FeatureZNode.
+   * 2. This method, unlike enableFeatureVersioning() need not wait for the FinalizedFeatureCache
+   *    to be updated, because, such updates to the caceh (via FinalizedFeatureChangeListener)
+   *    are disabled when IBP config is < than KAFKA_2_7_IV0.
+   */
+  private def disableFeatureVersioning(): Unit = {

Review comment:
       We can not just push the update, because, we have to decide if the node needs to be created or existing node should be updated. That is why we read the node first to understand if it exists or not, then we update the existing node only if the status does not match (this avoids a ZK write in the most common cases).

##########
File path: core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala
##########
@@ -0,0 +1,83 @@
+/**
+ * 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.server
+
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.junit.Assert.{assertEquals, assertThrows, assertTrue}
+import org.junit.Test
+
+import scala.jdk.CollectionConverters._
+
+class BrokerFeaturesTest {

Review comment:
       Done.

##########
File path: core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala
##########
@@ -0,0 +1,467 @@
+/**
+ * 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.server
+
+import java.util
+import java.util.Arrays
+import java.util.Collections
+import java.util.HashSet
+import java.util.Properties
+import java.util.concurrent.ExecutionException
+
+import kafka.api.KAFKA_2_7_IV0
+import kafka.utils.TestUtils
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion}
+import kafka.utils.TestUtils.waitUntilTrue
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions, UpdateFeaturesResult}
+import org.apache.kafka.common.errors.InvalidRequestException
+import org.apache.kafka.common.feature.FinalizedVersionRange
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.message.UpdateFeaturesRequestData
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{UpdateFeaturesRequest, UpdateFeaturesResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.Test
+import org.junit.Assert.{assertEquals, assertNotEquals, assertNotNull, assertTrue}
+import org.scalatest.Assertions.{assertThrows, intercept}
+
+import scala.reflect.ClassTag
+import scala.util.matching.Regex
+
+class UpdateFeaturesTest extends BaseRequestTest {
+
+  override def brokerCount = 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))))
+  }
+
+  private def defaultFinalizedFeatures(): Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2))))
+  }
+
+  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 updateSupportedFeatures(features: Features[SupportedVersionRange]): Unit = {
+    updateSupportedFeatures(features, Set[KafkaServer]() ++ servers)
+  }
+
+  private def updateDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = {
+    servers.foreach(s => {
+      s.brokerFeatures.setDefaultMinVersionLevels(newMinVersionLevels)
+    })
+  }
+
+  private def updateFeatureZNode(features: Features[FinalizedVersionRange]): Int = {
+    val server = serverForId(0).get
+    val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, features)
+    val newVersion = server.zkClient.updateFeatureZNode(newNode)
+    servers.foreach(s => {
+      s.featureCache.waitUntilEpochOrThrow(newVersion, s.config.zkConnectionTimeoutMs)
+    })
+    newVersion
+  }
+
+  private def getFeatureZNode(): FeatureZNode = {
+    val (mayBeFeatureZNodeBytes, version) = serverForId(0).get.zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(version, ZkVersion.UnknownVersion)
+    FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+  }
+
+  private def checkFeatures(client: Admin, expectedNode: FeatureZNode, expectedMetadata: FeatureMetadata): Unit = {
+    assertEquals(expectedNode, getFeatureZNode())
+    val featureMetadata = client.describeFeatures(
+      new DescribeFeaturesOptions().sendRequestToController(true)).featureMetadata().get()
+    assertEquals(expectedMetadata, featureMetadata)
+  }
+
+  private def checkException[ExceptionType <: Throwable](

Review comment:
       Done.

##########
File path: core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala
##########
@@ -0,0 +1,467 @@
+/**
+ * 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.server
+
+import java.util
+import java.util.Arrays
+import java.util.Collections
+import java.util.HashSet
+import java.util.Properties
+import java.util.concurrent.ExecutionException
+
+import kafka.api.KAFKA_2_7_IV0
+import kafka.utils.TestUtils
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion}
+import kafka.utils.TestUtils.waitUntilTrue
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions, UpdateFeaturesResult}
+import org.apache.kafka.common.errors.InvalidRequestException
+import org.apache.kafka.common.feature.FinalizedVersionRange
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.message.UpdateFeaturesRequestData
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{UpdateFeaturesRequest, UpdateFeaturesResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.Test
+import org.junit.Assert.{assertEquals, assertNotEquals, assertNotNull, assertTrue}
+import org.scalatest.Assertions.{assertThrows, intercept}
+
+import scala.reflect.ClassTag
+import scala.util.matching.Regex
+
+class UpdateFeaturesTest extends BaseRequestTest {
+
+  override def brokerCount = 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))))
+  }
+
+  private def defaultFinalizedFeatures(): Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2))))
+  }
+
+  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 updateSupportedFeatures(features: Features[SupportedVersionRange]): Unit = {
+    updateSupportedFeatures(features, Set[KafkaServer]() ++ servers)
+  }
+
+  private def updateDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = {
+    servers.foreach(s => {
+      s.brokerFeatures.setDefaultMinVersionLevels(newMinVersionLevels)
+    })
+  }
+
+  private def updateFeatureZNode(features: Features[FinalizedVersionRange]): Int = {
+    val server = serverForId(0).get
+    val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, features)
+    val newVersion = server.zkClient.updateFeatureZNode(newNode)
+    servers.foreach(s => {
+      s.featureCache.waitUntilEpochOrThrow(newVersion, s.config.zkConnectionTimeoutMs)
+    })
+    newVersion
+  }
+
+  private def getFeatureZNode(): FeatureZNode = {
+    val (mayBeFeatureZNodeBytes, version) = serverForId(0).get.zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(version, ZkVersion.UnknownVersion)
+    FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+  }
+
+  private def checkFeatures(client: Admin, expectedNode: FeatureZNode, expectedMetadata: FeatureMetadata): Unit = {
+    assertEquals(expectedNode, getFeatureZNode())
+    val featureMetadata = client.describeFeatures(
+      new DescribeFeaturesOptions().sendRequestToController(true)).featureMetadata().get()
+    assertEquals(expectedMetadata, featureMetadata)
+  }
+
+  private def checkException[ExceptionType <: Throwable](
+                                                          result: UpdateFeaturesResult,
+                                                          exceptionMsgPattern: Regex
+  )(implicit tag: ClassTag[ExceptionType]): Unit = {
+    val exception = intercept[ExecutionException] {
+      result.result().get()
+    }
+    assertNotNull(exception.getCause)
+    assertEquals(exception.getCause.getClass, tag.runtimeClass)
+    assertTrue(exceptionMsgPattern.findFirstIn(exception.getCause.getMessage).isDefined)
+  }
+
+  /**
+   * Tests whether an invalid feature update does not get processed on the server as expected,
+   * and raises the ExceptionType on the client side as expected.
+   *
+   * @param invalidUpdate         the invalid feature update to be sent in the
+   *                              updateFeatures request to the server
+   * @param exceptionMsgPattern   a pattern for the expected exception message
+   */
+  private def testWithInvalidFeatureUpdate[ExceptionType <: Throwable](
+                                                                        invalidUpdate: FeatureUpdate,
+                                                                        exceptionMsgPattern: Regex
+  )(implicit tag: ClassTag[ExceptionType]): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeatures(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val adminClient = createAdminClient()
+    val nodeBefore = getFeatureZNode()
+
+    val result = adminClient.updateFeatures(
+      new HashSet[FeatureUpdate](Collections.singletonList(invalidUpdate)), new UpdateFeaturesOptions())
+
+    checkException[ExceptionType](result, exceptionMsgPattern)
+    checkFeatures(
+      adminClient,
+      nodeBefore,
+      new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures()))
+  }
+
+  @Test
+  def testShouldFailRequestIfNotController(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeatures(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val nodeBefore = getFeatureZNode()
+    val requestData = FeatureUpdate.createRequest(
+      new util.HashSet[FeatureUpdate](
+        Collections.singletonList(new FeatureUpdate("feature_1",
+          defaultSupportedFeatures().get("feature_1").max(),
+        false))))
+
+    val response = connectAndReceive[UpdateFeaturesResponse](
+      new UpdateFeaturesRequest.Builder(requestData).build(), notControllerSocketServer)
+
+    assertEquals(Errors.NOT_CONTROLLER, response.error())
+    checkFeatures(
+      createAdminClient(),
+      nodeBefore,
+      new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures()))
+  }
+
+  @Test
+  def testShouldFailRequestForInvalidFeatureName(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](new FeatureUpdate(
+      "",
+      defaultSupportedFeatures().get("feature_1").max(),
+      false),
+      ".*empty feature name.*".r)
+  }
+
+  @Test
+  def testShouldFailRequestWhenDowngradeFlagIsNotSetDuringDowngrade(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](new FeatureUpdate(
+      "feature_1",
+      (defaultFinalizedFeatures().get("feature_1").max() - 1).asInstanceOf[Short],
+      false),
+      ".*Can not downgrade finalized feature: 'feature_1'.*allowDowngrade.*".r)
+  }
+
+  @Test
+  def testShouldFailRequestWhenDowngradeToHigherVersionLevelIsAttempted(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      new FeatureUpdate(
+        "feature_1",
+        defaultSupportedFeatures().get("feature_1").max(),
+        true),
+      ".*finalized feature: 'feature_1'.*allowDowngrade.* provided maxVersionLevel:3.*existing maxVersionLevel:2.*".r)
+  }
+
+  @Test
+  def testShouldFailRequestInClientWhenDowngradeFlagIsNotSetDuringDeletion(): Unit = {
+    assertThrows[IllegalArgumentException] {
+      new FeatureUpdate("feature_1", 0, false)
+    }
+  }
+
+  @Test
+  def testShouldFailRequestInServerWhenDowngradeFlagIsNotSetDuringDeletion(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeatures(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val adminClient = createAdminClient()
+    val nodeBefore = getFeatureZNode()
+
+    val featureUpdates
+      = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection();
+    val featureUpdate = new UpdateFeaturesRequestData.FeatureUpdateKey();
+    featureUpdate.setName("feature_1")
+    featureUpdate.setMaxVersionLevel(0)
+    featureUpdate.setAllowDowngrade(false)
+    featureUpdates.add(featureUpdate);
+    val requestData = new UpdateFeaturesRequestData()
+    requestData.setFeatureUpdates(featureUpdates);
+
+    val response = connectAndReceive[UpdateFeaturesResponse](
+      new UpdateFeaturesRequest.Builder(requestData).build(), controllerSocketServer)
+
+    assertEquals(Errors.INVALID_REQUEST, response.error)
+    val exceptionMsgPattern = ".*Can not delete feature: 'feature_1'.*allowDowngrade.*".r
+    assertTrue(exceptionMsgPattern.findFirstIn(response.data.errorMessage).isDefined)
+    checkFeatures(
+      adminClient,
+      nodeBefore,
+      new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures()))
+  }
+
+  @Test
+  def testShouldFailRequestDuringDeletionOfNonExistingFeature(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      new FeatureUpdate("feature_non_existing", 0, true),
+      ".*Can not delete non-existing finalized feature: 'feature_non_existing'.*".r)
+  }
+
+  @Test
+  def testShouldFailRequestWhenUpgradingToSameVersionLevel(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      new FeatureUpdate(
+        "feature_1", defaultFinalizedFeatures().get("feature_1").max(), false),
+      ".*Can not upgrade a finalized feature: 'feature_1'.*to the same value.*".r)
+  }
+
+  @Test
+  def testShouldFailRequestWhenDowngradingBelowMinVersionLevel(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeatures(defaultSupportedFeatures())
+    val minVersionLevel = 2.asInstanceOf[Short]
+    updateDefaultMinVersionLevels(Map[String, Short]("feature_1" -> minVersionLevel))
+    val initialFinalizedFeatures = Features.finalizedFeatures(
+      Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(minVersionLevel, 2))))
+    val versionBefore = updateFeatureZNode(initialFinalizedFeatures)
+
+    val update = new FeatureUpdate(
+      "feature_1", (minVersionLevel - 1).asInstanceOf[Short], true)
+    val adminClient = createAdminClient()
+    val nodeBefore = getFeatureZNode()
+
+    val result = adminClient.updateFeatures(
+      new HashSet[FeatureUpdate](Collections.singletonList(update)), new UpdateFeaturesOptions())
+
+    checkException[InvalidRequestException](
+      result, ".*Can not downgrade finalized feature: 'feature_1' to maxVersionLevel:1.*existing minVersionLevel:2.*".r)
+    checkFeatures(
+      adminClient,
+      nodeBefore,
+      new FeatureMetadata(initialFinalizedFeatures, versionBefore, defaultSupportedFeatures()))
+  }
+
+  @Test
+  def testShouldFailRequestDuringBrokerMaxVersionLevelIncompatibility(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    val controller = servers.filter { server => server.kafkaController.isActive}.head
+    val nonControllerServers = servers.filter { server => !server.kafkaController.isActive}
+    val unsupportedBrokers = Set[KafkaServer](nonControllerServers(0))

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java
##########
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+@InterfaceStability.Evolving
+public class DescribeFeaturesOptions extends AbstractOptions<DescribeFeaturesOptions> {
+    /**
+     * - True means the {@link Admin#describeFeatures(DescribeFeaturesOptions)} request can be
+     *   issued only to the controller.
+     * - False means the {@link Admin#describeFeatures(DescribeFeaturesOptions)} request can be
+     *   issued to any random broker.
+     */
+    private boolean shouldSendRequestToController = false;
+
+    /**
+     * Sets a flag indicating that the describe features request should be issued to the controller.
+     */
+    public DescribeFeaturesOptions sendRequestToController(boolean shouldSendRequestToController) {
+        this.shouldSendRequestToController = shouldSendRequestToController;
+        return this;
+    }
+
+    public boolean sendRequestToController() {
+        return shouldSendRequestToController;
+    }
+
+    /**
+     * Sets the timeout in milliseconds for this operation or {@code null} if the default API
+     * timeout for the AdminClient should be used.
+     */
+    public DescribeFeaturesOptions timeoutMs(Integer timeoutMs) {

Review comment:
       Done. Removed now. Didn't realize it was present in super class too.

##########
File path: clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
##########
@@ -3193,6 +3238,104 @@ public void testListOffsetsNonRetriableErrors() throws Exception {
         }
     }
 
+    @Test
+    public void testUpdateFeaturesDuringSuccess() throws Exception {
+        testUpdateFeaturesDuringError(Errors.NONE);
+    }
+
+    @Test
+    public void testUpdateFeaturesInvalidRequestError() throws Exception {
+        testUpdateFeaturesDuringError(Errors.INVALID_REQUEST);
+    }
+
+    @Test
+    public void testUpdateFeaturesUpdateFailedError() throws Exception {
+        testUpdateFeaturesDuringError(Errors.FEATURE_UPDATE_FAILED);
+    }
+
+    private void testUpdateFeaturesDuringError(Errors error) throws Exception {

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java
##########
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+import java.util.Set;
+import org.apache.kafka.common.message.UpdateFeaturesRequestData;
+
+/**
+ * Encapsulates details about an update to a finalized feature. This is particularly useful to
+ * define each feature update in the {@link Admin#updateFeatures(Set, UpdateFeaturesOptions)} API.
+ */
+public class FeatureUpdate {
+    private final String featureName;
+    private final short maxVersionLevel;
+    private final boolean allowDowngrade;
+
+    /**
+     * @param featureName       the name of the finalized feature to be updated.
+     * @param maxVersionLevel   the new maximum version level for the finalized feature.
+     *                          a value < 1 is special and indicates that the update is intended to
+     *                          delete the finalized feature, and should be accompanied by setting
+     *                          the allowDowngrade flag to true.
+     * @param allowDowngrade    - true, if this feature update was meant to downgrade the existing
+     *                            maximum version level of the finalized feature.
+     *                          - false, otherwise.
+     */
+    public FeatureUpdate(final String featureName, final short maxVersionLevel, final boolean allowDowngrade) {
+        Objects.requireNonNull(featureName, "Provided feature name can not be null.");
+        if (maxVersionLevel < 1 && !allowDowngrade) {
+            throw new IllegalArgumentException(
+                String.format(
+                    "For featureName: %s, the allowDowngrade flag is not set when the" +

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
##########
@@ -319,7 +320,8 @@
     GROUP_SUBSCRIBED_TO_TOPIC(86, "Deleting offsets of a topic is forbidden while the consumer group is actively subscribed to it.",
         GroupSubscribedToTopicException::new),
     INVALID_RECORD(87, "This record has failed the validation on broker and hence will be rejected.", InvalidRecordException::new),
-    UNSTABLE_OFFSET_COMMIT(88, "There are unstable offsets that need to be cleared.", UnstableOffsetCommitException::new);
+    UNSTABLE_OFFSET_COMMIT(88, "There are unstable offsets that need to be cleared.", UnstableOffsetCommitException::new),
+    FEATURE_UPDATE_FAILED(91, "Unable to update finalized features.", FeatureUpdateFailedException::new);

Review comment:
       Done. Not intentional. Changed to 89 now.

##########
File path: clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
##########
@@ -3193,6 +3238,104 @@ public void testListOffsetsNonRetriableErrors() throws Exception {
         }
     }
 
+    @Test
+    public void testUpdateFeaturesDuringSuccess() throws Exception {
+        testUpdateFeaturesDuringError(Errors.NONE);
+    }
+
+    @Test
+    public void testUpdateFeaturesInvalidRequestError() throws Exception {
+        testUpdateFeaturesDuringError(Errors.INVALID_REQUEST);
+    }
+
+    @Test
+    public void testUpdateFeaturesUpdateFailedError() throws Exception {
+        testUpdateFeaturesDuringError(Errors.FEATURE_UPDATE_FAILED);
+    }
+
+    private void testUpdateFeaturesDuringError(Errors error) throws Exception {
+        try (final AdminClientUnitTestEnv env = mockClientEnv()) {
+            env.kafkaClient().prepareResponse(
+                body -> body instanceof UpdateFeaturesRequest,
+                prepareUpdateFeaturesResponse(error));
+            final KafkaFuture<Void> future = env.adminClient().updateFeatures(
+                new HashSet<>(
+                    Arrays.asList(
+                        new FeatureUpdate(
+                            "test_feature_1", (short) 2, false),
+                        new FeatureUpdate(
+                            "test_feature_2", (short) 3, true))),
+                new UpdateFeaturesOptions().timeoutMs(10000)).result();
+            if (error.exception() == null) {
+                future.get();
+            } else {
+                final ExecutionException e = assertThrows(ExecutionException.class,
+                    () -> future.get());
+                assertEquals(e.getCause().getClass(), error.exception().getClass());
+            }
+        }
+    }
+
+    @Test
+    public void testUpdateFeaturesHandleNotControllerException() throws Exception {
+        try (final AdminClientUnitTestEnv env = mockClientEnv()) {
+            env.kafkaClient().prepareResponseFrom(
+                prepareUpdateFeaturesResponse(Errors.NOT_CONTROLLER),
+                env.cluster().nodeById(0));
+            env.kafkaClient().prepareResponse(MetadataResponse.prepareResponse(env.cluster().nodes(),
+                env.cluster().clusterResource().clusterId(),
+                1,
+                Collections.<MetadataResponse.TopicMetadata>emptyList()));

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -2945,6 +2948,130 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFeatures(request: RequestChannel.Request): Unit = {
+    val updateFeaturesRequest = request.body[UpdateFeaturesRequest]
+    def sendResponseCallback(error: Errors, msgOverride: Option[String]): Unit = {
+      val data = new UpdateFeaturesResponseData().setErrorCode(error.code())
+      msgOverride.map(msg => data.setErrorMessage(msg))
+      sendResponseExemptThrottle(request, new UpdateFeaturesResponse(data))
+    }
+
+    if (!authorize(request.context, ALTER, CLUSTER, CLUSTER_NAME)) {
+      sendResponseCallback(Errors.CLUSTER_AUTHORIZATION_FAILED, Option.empty)
+    } else if (!controller.isActive) {
+      sendResponseCallback(Errors.NOT_CONTROLLER, Option.empty)
+    } else if (!config.isFeatureVersioningEnabled) {
+      sendResponseCallback(Errors.INVALID_REQUEST, Some("Feature versioning system is disabled."))
+    } else if (updateFeaturesRequest.data.featureUpdates.isEmpty) {
+      sendResponseCallback(Errors.INVALID_REQUEST, Some("Can not provide empty FinalizedFeatureUpdates in the request."))
+    } else {
+      val targetFeaturesOrError = getTargetFinalizedFeaturesOrError(updateFeaturesRequest)
+      targetFeaturesOrError match {
+        case Left(targetFeatures) =>
+          controller.updateFeatures(targetFeatures, sendResponseCallback)
+        case Right(error) =>
+          sendResponseCallback(error.error, Some(error.message))
+      }
+    }
+  }
+
+  /**
+   * Validates the provided UpdateFinalizedFeaturesRequest, checking for various error cases.
+   * If the validation is successful, returns the target finalized features constructed from the
+   * request.
+   *
+   * @param request   the request to be validated
+   *
+   * @return          - the target finalized features, if request validation is successful
+   *                  - an ApiError if request validation fails
+   */
+  private def getTargetFinalizedFeaturesOrError(request: UpdateFeaturesRequest): Either[Features[FinalizedVersionRange], ApiError] = {
+    val updates = request.data.featureUpdates
+    val newFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]()
+
+    def addFeature(update: UpdateFeaturesRequestData.FeatureUpdateKey): Unit = {
+      // NOTE: Below we set the finalized min version level to be the default minimum version
+      // level. If the finalized feature already exists, then, this can cause deprecation of all
+      // version levels in the closed range:
+      // [existingVersionRange.min(), defaultMinVersionLevel - 1].
+      val defaultMinVersionLevel = brokerFeatures.defaultMinVersionLevel(update.name)
+      newFeatures += (
+        update.name -> new FinalizedVersionRange(
+          defaultMinVersionLevel,
+          update.maxVersionLevel))
+    }
+
+    val latestFeatures = featureCache.get
+    updates.asScala.iterator.map(
+      update => {
+        if (update.name.isEmpty) {
+          // Rule #1) Check that the feature name is not empty.
+          Some(new ApiError(Errors.INVALID_REQUEST,
+                   "Can not contain empty feature name in the request."))
+        } else {
+          val cacheEntry = latestFeatures.map(lf => lf.features.get(update.name)).orNull
+
+          // We handle deletion requests separately from non-deletion requests.
+          if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+            if (!update.allowDowngrade) {
+              // Rule #2) Disallow deletion of a finalized feature without allowDowngrade flag set.
+              Some(new ApiError(Errors.INVALID_REQUEST,
+                                s"Can not delete feature: '${update.name}' without setting the" +
+                                " allowDowngrade flag to true in the request."))
+            } else if (cacheEntry == null) {
+              // Rule #3) Disallow deletion of a non-existing finalized feature.
+              Some(new ApiError(Errors.INVALID_REQUEST,
+                       s"Can not delete non-existing finalized feature: '${update.name}'"))
+            }
+          } else {
+            if (cacheEntry == null) {

Review comment:
       Added a test now in `UpdateFeaturesTest.scala`. Look for `testSuccessfulFeatureUpgradeAndWithNoExistingFinalizedFeatures`.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java
##########
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+import org.apache.kafka.common.message.UpdateFeaturesResponseData;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.Struct;
+
+
+/**
+ * Possible error codes:
+ *
+ *   - {@link Errors#CLUSTER_AUTHORIZATION_FAILED}
+ *   - {@link Errors#NOT_CONTROLLER}
+ *   - {@link Errors#INVALID_REQUEST}
+ *   - {@link Errors#FEATURE_UPDATE_FAILED}
+ */
+public class UpdateFeaturesResponse extends AbstractResponse {
+
+    public final UpdateFeaturesResponseData data;
+
+    public UpdateFeaturesResponse(UpdateFeaturesResponseData data) {
+        this.data = data;
+    }
+
+    public UpdateFeaturesResponse(Struct struct) {
+        final short latestVersion = (short) (UpdateFeaturesResponseData.SCHEMAS.length - 1);
+        this.data = new UpdateFeaturesResponseData(struct, latestVersion);
+    }
+
+    public UpdateFeaturesResponse(Struct struct, short version) {
+        this.data = new UpdateFeaturesResponseData(struct, version);
+    }
+
+    public Errors error() {
+        return Errors.forCode(data.errorCode());
+    }
+
+    @Override
+    public Map<Errors, Integer> errorCounts() {
+        return errorCounts(Errors.forCode(data.errorCode()));
+    }
+
+    @Override
+    protected Struct toStruct(short version) {
+        return data.toStruct(version);
+    }
+
+    @Override
+    public String toString() {
+        return data.toString();
+    }
+
+    public UpdateFeaturesResponseData data() {

Review comment:
       Done. Made the attribute private.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import java.nio.ByteBuffer;
+import org.apache.kafka.common.message.UpdateFeaturesResponseData;
+import org.apache.kafka.common.message.UpdateFeaturesRequestData;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.types.Struct;
+
+public class UpdateFeaturesRequest extends AbstractRequest {
+
+    public static class Builder extends AbstractRequest.Builder<UpdateFeaturesRequest> {
+
+        private final UpdateFeaturesRequestData data;
+
+        public Builder(UpdateFeaturesRequestData data) {
+            super(ApiKeys.UPDATE_FEATURES);
+            this.data = data;
+        }
+
+        @Override
+        public UpdateFeaturesRequest build(short version) {
+            return new UpdateFeaturesRequest(data, version);
+        }
+
+        @Override
+        public String toString() {
+            return data.toString();
+        }
+    }
+
+    public final UpdateFeaturesRequestData data;
+
+    public UpdateFeaturesRequest(UpdateFeaturesRequestData data, short version) {
+        super(ApiKeys.UPDATE_FEATURES, version);
+        this.data = data;
+    }
+
+    public UpdateFeaturesRequest(Struct struct, short version) {
+        super(ApiKeys.UPDATE_FEATURES, version);
+        this.data = new UpdateFeaturesRequestData(struct, version);
+    }
+
+    @Override
+    public AbstractResponse getErrorResponse(int throttleTimeMsIgnored, Throwable e) {
+        final ApiError apiError = ApiError.fromThrowable(e);
+        return new UpdateFeaturesResponse(
+            new UpdateFeaturesResponseData()
+                .setErrorCode(apiError.error().code())
+                .setErrorMessage(apiError.message()));
+    }
+
+    @Override
+    protected Struct toStruct() {
+        return data.toStruct(version());
+    }
+
+    public UpdateFeaturesRequestData data() {

Review comment:
       Done. Made it private.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.
+   *
+   * This method sets up the FeatureZNode with enabled status. This status means the feature
+   * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode
+   * are active. This status should be written by the controller to the FeatureZNode only when the
+   * broker IBP config is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. The reason to do this is that enabling all the possible features immediately after
+   *    an upgrade could be harmful to the cluster.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, then it
+   *        will react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, it's existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) => {

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.
+   *
+   * This method sets up the FeatureZNode with enabled status. This status means the feature
+   * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode
+   * are active. This status should be written by the controller to the FeatureZNode only when the
+   * broker IBP config is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. The reason to do this is that enabling all the possible features immediately after
+   *    an upgrade could be harmful to the cluster.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, then it
+   *        will react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, it's existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) => {
+            val brokerDefaultVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (brokerDefaultVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (existingVersionRange.max() >= brokerDefaultVersionRange.min() &&
+                       brokerDefaultVersionRange.max() >= existingVersionRange.max()) {
+              // Through this change, we deprecate all version levels in the closed range:
+              // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1]
+              (featureName, new FinalizedVersionRange(brokerDefaultVersionRange.min(), existingVersionRange.max()))
+            } else {
+              // If the existing version levels fall completely outside the
+              // range of the default finalized version levels (i.e. no intersection), or, if the
+              // existing version levels are ineligible for a modification since they are
+              // incompatible with default finalized version levels, then we skip the update.
+              warn(s"Can not update minimum version level in finalized feature: $featureName,"
+                + s" since the existing $existingVersionRange is not eligible for a change"
+                + s" based on the default $brokerDefaultVersionRange.")
+              (featureName, existingVersionRange)
+            }
+          }
+        }.asJava)
+      }
+      val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures)
+      if (!newFeatureZNode.equals(existingFeatureZNode)) {
+        val newVersion = updateFeatureZNode(newFeatureZNode)
+        featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+      }
+    }
+  }
+
+  /**
+   * Disables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with disabled status. This status means the feature versioning system
+   * (KIP-584) is disabled, and, the finalized features stored in the FeatureZNode are not relevant.
+   * This status should be written by the controller to the FeatureZNode only when the broker
+   * IBP config is less than KAFKA_2_7_IV0.
+   *
+   * NOTE:
+   * 1. When this method returns, existing finalized features (if any) will be cleared from the
+   *    FeatureZNode.
+   * 2. This method, unlike enableFeatureVersioning() need not wait for the FinalizedFeatureCache
+   *    to be updated, because, such updates to the caceh (via FinalizedFeatureChangeListener)

Review comment:
       Done.

##########
File path: clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
##########
@@ -3193,6 +3238,104 @@ public void testListOffsetsNonRetriableErrors() throws Exception {
         }
     }
 
+    @Test
+    public void testUpdateFeaturesDuringSuccess() throws Exception {
+        testUpdateFeaturesDuringError(Errors.NONE);
+    }
+
+    @Test
+    public void testUpdateFeaturesInvalidRequestError() throws Exception {
+        testUpdateFeaturesDuringError(Errors.INVALID_REQUEST);
+    }
+
+    @Test
+    public void testUpdateFeaturesUpdateFailedError() throws Exception {
+        testUpdateFeaturesDuringError(Errors.FEATURE_UPDATE_FAILED);
+    }
+
+    private void testUpdateFeaturesDuringError(Errors error) throws Exception {
+        try (final AdminClientUnitTestEnv env = mockClientEnv()) {
+            env.kafkaClient().prepareResponse(
+                body -> body instanceof UpdateFeaturesRequest,
+                prepareUpdateFeaturesResponse(error));
+            final KafkaFuture<Void> future = env.adminClient().updateFeatures(
+                new HashSet<>(
+                    Arrays.asList(
+                        new FeatureUpdate(
+                            "test_feature_1", (short) 2, false),
+                        new FeatureUpdate(
+                            "test_feature_2", (short) 3, true))),
+                new UpdateFeaturesOptions().timeoutMs(10000)).result();
+            if (error.exception() == null) {

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.
+   *
+   * This method sets up the FeatureZNode with enabled status. This status means the feature
+   * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode
+   * are active. This status should be written by the controller to the FeatureZNode only when the
+   * broker IBP config is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. The reason to do this is that enabling all the possible features immediately after
+   *    an upgrade could be harmful to the cluster.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, then it
+   *        will react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, it's existing minimum version level is updated to the default

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.
+   *
+   * This method sets up the FeatureZNode with enabled status. This status means the feature
+   * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode
+   * are active. This status should be written by the controller to the FeatureZNode only when the
+   * broker IBP config is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. The reason to do this is that enabling all the possible features immediately after
+   *    an upgrade could be harmful to the cluster.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, then it
+   *        will react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, it's existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) => {
+            val brokerDefaultVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (brokerDefaultVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (existingVersionRange.max() >= brokerDefaultVersionRange.min() &&
+                       brokerDefaultVersionRange.max() >= existingVersionRange.max()) {
+              // Through this change, we deprecate all version levels in the closed range:
+              // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1]
+              (featureName, new FinalizedVersionRange(brokerDefaultVersionRange.min(), existingVersionRange.max()))
+            } else {
+              // If the existing version levels fall completely outside the
+              // range of the default finalized version levels (i.e. no intersection), or, if the
+              // existing version levels are ineligible for a modification since they are
+              // incompatible with default finalized version levels, then we skip the update.
+              warn(s"Can not update minimum version level in finalized feature: $featureName,"
+                + s" since the existing $existingVersionRange is not eligible for a change"
+                + s" based on the default $brokerDefaultVersionRange.")
+              (featureName, existingVersionRange)
+            }
+          }
+        }.asJava)
+      }
+      val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures)
+      if (!newFeatureZNode.equals(existingFeatureZNode)) {

Review comment:
       `FeatureZNode` is a `case class`, and therefore the `equals` method is auto generated. Let me know if I'm missing something. Here is the doc: https://docs.scala-lang.org/overviews/scala-book/case-classes.html.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)

Review comment:
       `newVersion` is more readable than `_2`.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying

Review comment:
       Done.

##########
File path: clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
##########
@@ -3193,6 +3238,104 @@ public void testListOffsetsNonRetriableErrors() throws Exception {
         }
     }
 
+    @Test
+    public void testUpdateFeaturesDuringSuccess() throws Exception {
+        testUpdateFeaturesDuringError(Errors.NONE);
+    }
+
+    @Test
+    public void testUpdateFeaturesInvalidRequestError() throws Exception {
+        testUpdateFeaturesDuringError(Errors.INVALID_REQUEST);
+    }
+
+    @Test
+    public void testUpdateFeaturesUpdateFailedError() throws Exception {
+        testUpdateFeaturesDuringError(Errors.FEATURE_UPDATE_FAILED);
+    }
+
+    private void testUpdateFeaturesDuringError(Errors error) throws Exception {
+        try (final AdminClientUnitTestEnv env = mockClientEnv()) {
+            env.kafkaClient().prepareResponse(
+                body -> body instanceof UpdateFeaturesRequest,
+                prepareUpdateFeaturesResponse(error));
+            final KafkaFuture<Void> future = env.adminClient().updateFeatures(
+                new HashSet<>(
+                    Arrays.asList(
+                        new FeatureUpdate(
+                            "test_feature_1", (short) 2, false),
+                        new FeatureUpdate(
+                            "test_feature_2", (short) 3, true))),
+                new UpdateFeaturesOptions().timeoutMs(10000)).result();
+            if (error.exception() == null) {
+                future.get();
+            } else {
+                final ExecutionException e = assertThrows(ExecutionException.class,
+                    () -> future.get());
+                assertEquals(e.getCause().getClass(), error.exception().getClass());
+            }
+        }
+    }
+
+    @Test
+    public void testUpdateFeaturesHandleNotControllerException() throws Exception {
+        try (final AdminClientUnitTestEnv env = mockClientEnv()) {
+            env.kafkaClient().prepareResponseFrom(
+                prepareUpdateFeaturesResponse(Errors.NOT_CONTROLLER),
+                env.cluster().nodeById(0));
+            env.kafkaClient().prepareResponse(MetadataResponse.prepareResponse(env.cluster().nodes(),
+                env.cluster().clusterResource().clusterId(),
+                1,
+                Collections.<MetadataResponse.TopicMetadata>emptyList()));
+            env.kafkaClient().prepareResponseFrom(

Review comment:
       I have improved the matcher now, but how do I check the correct controller id?

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -85,25 +85,25 @@ class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
       //                                           a case.
       if (version == ZkVersion.UnknownVersion) {
         info(s"Feature ZK node at path: $featureZkNodePath does not exist")
-        FinalizedFeatureCache.clear()
+        featureCache.clear()
       } else {
         var maybeFeatureZNode: Option[FeatureZNode] = Option.empty
         try {
           maybeFeatureZNode = Some(FeatureZNode.decode(mayBeFeatureZNodeBytes.get))
         } catch {
           case e: IllegalArgumentException => {
             error(s"Unable to deserialize feature ZK node at path: $featureZkNodePath", e)
-            FinalizedFeatureCache.clear()
+            featureCache.clear()
           }
         }
-        maybeFeatureZNode.map(featureZNode => {
+        maybeFeatureZNode.foreach(featureZNode => {
           featureZNode.status match {
             case FeatureZNodeStatus.Disabled => {
               info(s"Feature ZK node at path: $featureZkNodePath is in disabled status.")
-              FinalizedFeatureCache.clear()
+              featureCache.clear()

Review comment:
       It became painful to write tests using singletons. Particularly in `kafka.server.UpdateFeaturesTest` we would like to simulate presence of multiple brokers and a controller within the same test process. Then we would like to set incompatible features for some brokers, and compatible features for some others. Using a singleton for feature cache made it impossible to set up such an environment for testing. That is why we no longer use a singleton, instead we instantiate the class once in `KafkaServer` and we use the object wherever needed.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.
+   *
+   * This method sets up the FeatureZNode with enabled status. This status means the feature
+   * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.
+   *
+   * This method sets up the FeatureZNode with enabled status. This status means the feature
+   * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode
+   * are active. This status should be written by the controller to the FeatureZNode only when the
+   * broker IBP config is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.
+   *
+   * This method sets up the FeatureZNode with enabled status. This status means the feature
+   * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode
+   * are active. This status should be written by the controller to the FeatureZNode only when the
+   * broker IBP config is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. The reason to do this is that enabling all the possible features immediately after

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.
+   *
+   * This method sets up the FeatureZNode with enabled status. This status means the feature
+   * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode
+   * are active. This status should be written by the controller to the FeatureZNode only when the

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -977,14 +1159,33 @@ class KafkaController(val config: KafkaConfig,
 
   /**
    * Send the leader information for selected partitions to selected brokers so that they can correctly respond to
-   * metadata requests
+   * metadata requests. Particularly, when feature versioning is enabled, we filter out brokers with incompatible
+   * features from receiving the metadata requests. This is because we do not want to activate incompatible brokers,
+   * as these may have harmful consequences to the cluster.
    *
    * @param brokers The brokers that the update metadata request should be sent to
    */
   private[controller] def sendUpdateMetadataRequest(brokers: Seq[Int], partitions: Set[TopicPartition]): Unit = {
     try {
+      val filteredBrokers = scala.collection.mutable.Set[Int]() ++ brokers
+      if (config.isFeatureVersioningEnabled) {
+        def hasIncompatibleFeatures(broker: Broker): Boolean = {
+          val latestFinalizedFeatures = featureCache.get
+          if (latestFinalizedFeatures.isDefined) {
+            BrokerFeatures.hasIncompatibleFeatures(broker.features, latestFinalizedFeatures.get.features)
+          } else {
+            false
+          }
+        }
+        controllerContext.liveOrShuttingDownBrokers.foreach(broker => {
+          if (filteredBrokers.contains(broker.id) && hasIncompatibleFeatures(broker)) {
+            warn(s"Ignoring UpdateMetadataRequest to broker: ${broker.id} due to incompatible features")

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.
+   *
+   * This method sets up the FeatureZNode with enabled status. This status means the feature
+   * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode
+   * are active. This status should be written by the controller to the FeatureZNode only when the
+   * broker IBP config is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. The reason to do this is that enabling all the possible features immediately after
+   *    an upgrade could be harmful to the cluster.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, then it

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import java.nio.ByteBuffer;
+import org.apache.kafka.common.message.UpdateFeaturesResponseData;
+import org.apache.kafka.common.message.UpdateFeaturesRequestData;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.types.Struct;
+
+public class UpdateFeaturesRequest extends AbstractRequest {
+
+    public static class Builder extends AbstractRequest.Builder<UpdateFeaturesRequest> {
+
+        private final UpdateFeaturesRequestData data;
+
+        public Builder(UpdateFeaturesRequestData data) {
+            super(ApiKeys.UPDATE_FEATURES);
+            this.data = data;
+        }
+
+        @Override
+        public UpdateFeaturesRequest build(short version) {
+            return new UpdateFeaturesRequest(data, version);
+        }
+
+        @Override
+        public String toString() {
+            return data.toString();
+        }
+    }
+
+    public final UpdateFeaturesRequestData data;
+
+    public UpdateFeaturesRequest(UpdateFeaturesRequestData data, short version) {
+        super(ApiKeys.UPDATE_FEATURES, version);
+        this.data = data;
+    }
+
+    public UpdateFeaturesRequest(Struct struct, short version) {
+        super(ApiKeys.UPDATE_FEATURES, version);
+        this.data = new UpdateFeaturesRequestData(struct, version);
+    }
+
+    @Override
+    public AbstractResponse getErrorResponse(int throttleTimeMsIgnored, Throwable e) {
+        final ApiError apiError = ApiError.fromThrowable(e);
+        return new UpdateFeaturesResponse(
+            new UpdateFeaturesResponseData()
+                .setErrorCode(apiError.error().code())
+                .setErrorMessage(apiError.message()));
+    }
+
+    @Override
+    protected Struct toStruct() {
+        return data.toStruct(version());
+    }
+
+    public UpdateFeaturesRequestData data() {
+        return data;
+    }
+
+    public static UpdateFeaturesRequest parse(ByteBuffer buffer, short version) {
+        return new UpdateFeaturesRequest(
+            ApiKeys.UPDATE_FEATURES.parseRequest(version, buffer), version);
+    }
+
+    public static boolean isDeleteRequest(UpdateFeaturesRequestData.FeatureUpdateKey update) {
+        return update.maxVersionLevel() < 1;

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,178 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature
+ *    version level deprecation. This is how it works: in order to deprecate feature version levels,
+ *    in this map the default minimum version level of a feature can be set to a new value that's
+ *    higher than 1 (let's call this new_min_version_level). In doing so, the feature version levels
+ *    in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic
+ *    that applies this map to persistent finalized feature state in ZK (this mutation happens
+ *    during controller election and during finalized feature updates via the
+ *    ApiKeys.UPDATE_FINALIZED_FEATURES api). This will automatically mean clients have to stop
+ *    using the finalized min version levels that have been deprecated.
+ *
+ * This class also provides APIs to check for incompatibilities between the features supported by
+ * the Broker and finalized features. The class is generally immutable. It provides few APIs to

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,178 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature
+ *    version level deprecation. This is how it works: in order to deprecate feature version levels,
+ *    in this map the default minimum version level of a feature can be set to a new value that's
+ *    higher than 1 (let's call this new_min_version_level). In doing so, the feature version levels
+ *    in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic
+ *    that applies this map to persistent finalized feature state in ZK (this mutation happens
+ *    during controller election and during finalized feature updates via the
+ *    ApiKeys.UPDATE_FINALIZED_FEATURES api). This will automatically mean clients have to stop

Review comment:
       Done. I was referring to external clients of Kafka. Have updated the doc now.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1395,7 +1596,7 @@ class KafkaController(val config: KafkaConfig,
     if (newMetadataOpt.nonEmpty && oldMetadataOpt.nonEmpty) {
       val oldMetadata = oldMetadataOpt.get
       val newMetadata = newMetadataOpt.get
-      if (newMetadata.endPoints != oldMetadata.endPoints) {
+      if (newMetadata.endPoints != oldMetadata.endPoints || !oldMetadata.features.equals(newMetadata.features)) {

Review comment:
       Yes, `Broker.features` is just empty when there are no features set or none decoded from the `BrokerIdZNode`.

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,178 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature
+ *    version level deprecation. This is how it works: in order to deprecate feature version levels,
+ *    in this map the default minimum version level of a feature can be set to a new value that's
+ *    higher than 1 (let's call this new_min_version_level). In doing so, the feature version levels
+ *    in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic
+ *    that applies this map to persistent finalized feature state in ZK (this mutation happens
+ *    during controller election and during finalized feature updates via the
+ *    ApiKeys.UPDATE_FINALIZED_FEATURES api). This will automatically mean clients have to stop
+ *    using the finalized min version levels that have been deprecated.
+ *
+ * This class also provides APIs to check for incompatibilities between the features supported by
+ * the Broker and finalized features. The class is generally immutable. It provides few APIs to
+ * mutate state only for the purpose of testing.
+ */
+class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange],
+                              @volatile var defaultFeatureMinVersionLevels: Map[String, Short]) {
+  require(BrokerFeatures.areFeatureMinVersionLevelsCompatible(
+    supportedFeatures, defaultFeatureMinVersionLevels))
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(newFeatures, defaultFeatureMinVersionLevels))
+    supportedFeatures = newFeatures
+  }
+
+  /**
+   * Returns the default minimum version level for a specific feature.
+   *
+   * @param feature   the name of the feature
+   *
+   * @return          the default minimum version level for the feature if its defined.
+   *                  otherwise, returns 1.
+   */
+  def defaultMinVersionLevel(feature: String): Short = {
+    defaultFeatureMinVersionLevels.getOrElse(feature, 1)
+  }
+
+  // For testing only.
+  def setDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(supportedFeatures, newMinVersionLevels))
+    defaultFeatureMinVersionLevels = newMinVersionLevels
+  }
+
+  /**
+   * Returns the default finalized features that a new Kafka cluster with IBP config >= KAFKA_2_7_IV0
+   * needs to be bootstrapped with.
+   */
+  def getDefaultFinalizedFeatures: Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(
+      supportedFeatures.features.asScala.map {
+        case(name, versionRange) => (
+          name, new FinalizedVersionRange(defaultMinVersionLevel(name), versionRange.max))
+      }.asJava)
+  }
+
+  /**
+   * Returns the set of feature names found to be 'incompatible'.

Review comment:
       Done. Removed quotes.

##########
File path: core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala
##########
@@ -598,6 +608,21 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness {
     }, "Broker fail to initialize after restart")
   }
 
+  private def testControllerFeatureZNodeSetup(interBrokerProtocolVersion: ApiVersion): Unit = {
+    servers = makeServers(1, interBrokerProtocolVersion = Some(interBrokerProtocolVersion))
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(version, ZkVersion.UnknownVersion)

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -2945,6 +2948,130 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFeatures(request: RequestChannel.Request): Unit = {
+    val updateFeaturesRequest = request.body[UpdateFeaturesRequest]
+    def sendResponseCallback(error: Errors, msgOverride: Option[String]): Unit = {
+      val data = new UpdateFeaturesResponseData().setErrorCode(error.code())
+      msgOverride.map(msg => data.setErrorMessage(msg))
+      sendResponseExemptThrottle(request, new UpdateFeaturesResponse(data))
+    }
+
+    if (!authorize(request.context, ALTER, CLUSTER, CLUSTER_NAME)) {
+      sendResponseCallback(Errors.CLUSTER_AUTHORIZATION_FAILED, Option.empty)
+    } else if (!controller.isActive) {
+      sendResponseCallback(Errors.NOT_CONTROLLER, Option.empty)
+    } else if (!config.isFeatureVersioningEnabled) {
+      sendResponseCallback(Errors.INVALID_REQUEST, Some("Feature versioning system is disabled."))
+    } else if (updateFeaturesRequest.data.featureUpdates.isEmpty) {
+      sendResponseCallback(Errors.INVALID_REQUEST, Some("Can not provide empty FinalizedFeatureUpdates in the request."))
+    } else {
+      val targetFeaturesOrError = getTargetFinalizedFeaturesOrError(updateFeaturesRequest)
+      targetFeaturesOrError match {
+        case Left(targetFeatures) =>
+          controller.updateFeatures(targetFeatures, sendResponseCallback)
+        case Right(error) =>
+          sendResponseCallback(error.error, Some(error.message))
+      }
+    }
+  }
+
+  /**
+   * Validates the provided UpdateFinalizedFeaturesRequest, checking for various error cases.
+   * If the validation is successful, returns the target finalized features constructed from the
+   * request.
+   *
+   * @param request   the request to be validated
+   *
+   * @return          - the target finalized features, if request validation is successful
+   *                  - an ApiError if request validation fails
+   */
+  private def getTargetFinalizedFeaturesOrError(request: UpdateFeaturesRequest): Either[Features[FinalizedVersionRange], ApiError] = {
+    val updates = request.data.featureUpdates

Review comment:
       This method has changed greatly and it has been moved to `KafkaController.scala`.

##########
File path: core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala
##########
@@ -0,0 +1,83 @@
+/**
+ * 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.server
+
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.junit.Assert.{assertEquals, assertThrows, assertTrue}
+import org.junit.Test
+
+import scala.jdk.CollectionConverters._
+
+class BrokerFeaturesTest {
+  @Test
+  def testEmpty(): Unit = {
+    assertTrue(BrokerFeatures.createDefault().supportedFeatures.empty)
+  }
+
+  @Test
+  def testIncompatibleFeatures(): Unit = {
+    val brokerFeatures = BrokerFeatures.createDefault()
+    val supportedFeatures = Map[String, SupportedVersionRange](
+      "test_feature_1" -> new SupportedVersionRange(1, 4),
+      "test_feature_2" -> new SupportedVersionRange(1, 3))
+    brokerFeatures.setSupportedFeatures(Features.supportedFeatures(supportedFeatures.asJava))
+
+    val compatibleFeatures = Map[String, FinalizedVersionRange](
+      "test_feature_1" -> new FinalizedVersionRange(2, 3))
+    val inCompatibleFeatures = Map[String, FinalizedVersionRange](
+      "test_feature_2" -> new FinalizedVersionRange(1, 4),
+      "test_feature_3" -> new FinalizedVersionRange(3, 4))
+    val features = compatibleFeatures++inCompatibleFeatures
+    val finalizedFeatures = Features.finalizedFeatures(features.asJava)
+
+    assertEquals(
+      Features.finalizedFeatures(inCompatibleFeatures.asJava),
+      brokerFeatures.incompatibleFeatures(finalizedFeatures))
+  }
+
+  @Test
+  def testFeatureVersionAssertions(): Unit = {
+    val brokerFeatures = BrokerFeatures.createDefault()
+    val supportedFeatures = Features.supportedFeatures(Map[String, SupportedVersionRange](
+      "test_feature_1" -> new SupportedVersionRange(1, 4),
+      "test_feature_2" -> new SupportedVersionRange(1, 3)).asJava)
+    brokerFeatures.setSupportedFeatures(supportedFeatures)
+
+    val defaultMinVersionLevelsWithNonExistingFeature = Map[String, Short](
+      "test_feature_1" -> 2,
+      "test_feature_2" -> 2,
+      "test_feature_non_existing" -> 5)
+    assertThrows(
+      classOf[IllegalArgumentException],
+      () => brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevelsWithNonExistingFeature))
+
+    val defaultMinVersionLevelsWithInvalidValue1 = Map[String, Short](

Review comment:
       Done.

##########
File path: core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala
##########
@@ -78,25 +76,37 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness {
   /**
    * Tests that the listener can be initialized, and that it can listen to ZK notifications
    * successfully from an "Enabled" FeatureZNode (the ZK data has no feature incompatibilities).
+   * Particularly the test checks if multiple notifications can be processed in ZK
+   * (i.e. whether the FeatureZNode watch can be re-established).
    */
   @Test
   def testInitSuccessAndNotificationSuccess(): Unit = {
-    createSupportedFeatures()
     val initialFinalizedFeatures = createFinalizedFeatures()
-    val listener = createListener(Some(initialFinalizedFeatures))
+    val brokerFeatures = createBrokerFeatures()
+    val cache = new FinalizedFeatureCache(brokerFeatures)
+    val listener = createListener(cache, Some(initialFinalizedFeatures))
 
-    val updatedFinalizedFeaturesMap = Map[String, FinalizedVersionRange](
-      "feature_1" -> new FinalizedVersionRange(2, 4))
-    val updatedFinalizedFeatures = Features.finalizedFeatures(updatedFinalizedFeaturesMap.asJava)
-    zkClient.updateFeatureZNode(FeatureZNode(FeatureZNodeStatus.Enabled, updatedFinalizedFeatures))
-    val (mayBeFeatureZNodeNewBytes, updatedVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
-    assertNotEquals(updatedVersion, ZkVersion.UnknownVersion)
-    assertFalse(mayBeFeatureZNodeNewBytes.isEmpty)
-    assertTrue(updatedVersion > initialFinalizedFeatures.epoch)
-    TestUtils.waitUntilTrue(() => {
-      FinalizedFeatureCache.get.get.equals(FinalizedFeaturesAndEpoch(updatedFinalizedFeatures, updatedVersion))
-    }, "Timed out waiting for FinalizedFeatureCache to be updated with new features")
-    assertTrue(listener.isListenerInitiated)
+    def updateAndCheckCache(finalizedFeatures: Features[FinalizedVersionRange]): Unit = {
+      zkClient.updateFeatureZNode(FeatureZNode(FeatureZNodeStatus.Enabled, finalizedFeatures))
+      val (mayBeFeatureZNodeNewBytes, updatedVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+      assertNotEquals(updatedVersion, ZkVersion.UnknownVersion)
+      assertFalse(mayBeFeatureZNodeNewBytes.isEmpty)
+      assertTrue(updatedVersion > initialFinalizedFeatures.epoch)
+
+      cache.waitUntilEpochOrThrow(updatedVersion, JTestUtils.DEFAULT_MAX_WAIT_MS)
+      assertEquals(FinalizedFeaturesAndEpoch(finalizedFeatures, updatedVersion), cache.get.get)
+      assertTrue(listener.isListenerInitiated)
+    }
+
+    updateAndCheckCache(
+      Features.finalizedFeatures(
+        Map[String, FinalizedVersionRange](
+      "feature_1" -> new FinalizedVersionRange(2, 4)).asJava))
+    updateAndCheckCache(

Review comment:
       It is explained in the test doc above, and, I have also added comments now. The purpose is to check that the ZK watch on the FeatureZNode was re-established by the broker, after the first update triggers a ZK notification that populates the cache. The best way to check it is to update the node again and see if the notification is received by the broker again.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesResult.java
##########
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.common.KafkaFuture;
+
+public class UpdateFeaturesResult {

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -2945,6 +2948,130 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFeatures(request: RequestChannel.Request): Unit = {

Review comment:
       Will take a look.

##########
File path: core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala
##########
@@ -0,0 +1,467 @@
+/**
+ * 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.server
+
+import java.util
+import java.util.Arrays
+import java.util.Collections
+import java.util.HashSet
+import java.util.Properties
+import java.util.concurrent.ExecutionException
+
+import kafka.api.KAFKA_2_7_IV0
+import kafka.utils.TestUtils
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion}
+import kafka.utils.TestUtils.waitUntilTrue
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions, UpdateFeaturesResult}
+import org.apache.kafka.common.errors.InvalidRequestException
+import org.apache.kafka.common.feature.FinalizedVersionRange
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.message.UpdateFeaturesRequestData
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{UpdateFeaturesRequest, UpdateFeaturesResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.Test
+import org.junit.Assert.{assertEquals, assertNotEquals, assertNotNull, assertTrue}
+import org.scalatest.Assertions.{assertThrows, intercept}
+
+import scala.reflect.ClassTag
+import scala.util.matching.Regex
+
+class UpdateFeaturesTest extends BaseRequestTest {
+
+  override def brokerCount = 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))))
+  }
+
+  private def defaultFinalizedFeatures(): Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2))))
+  }
+
+  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 updateSupportedFeatures(features: Features[SupportedVersionRange]): Unit = {
+    updateSupportedFeatures(features, Set[KafkaServer]() ++ servers)
+  }
+
+  private def updateDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = {
+    servers.foreach(s => {
+      s.brokerFeatures.setDefaultMinVersionLevels(newMinVersionLevels)
+    })
+  }
+
+  private def updateFeatureZNode(features: Features[FinalizedVersionRange]): Int = {
+    val server = serverForId(0).get
+    val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, features)
+    val newVersion = server.zkClient.updateFeatureZNode(newNode)
+    servers.foreach(s => {
+      s.featureCache.waitUntilEpochOrThrow(newVersion, s.config.zkConnectionTimeoutMs)
+    })
+    newVersion
+  }
+
+  private def getFeatureZNode(): FeatureZNode = {
+    val (mayBeFeatureZNodeBytes, version) = serverForId(0).get.zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(version, ZkVersion.UnknownVersion)
+    FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+  }
+
+  private def checkFeatures(client: Admin, expectedNode: FeatureZNode, expectedMetadata: FeatureMetadata): Unit = {
+    assertEquals(expectedNode, getFeatureZNode())
+    val featureMetadata = client.describeFeatures(
+      new DescribeFeaturesOptions().sendRequestToController(true)).featureMetadata().get()
+    assertEquals(expectedMetadata, featureMetadata)
+  }
+
+  private def checkException[ExceptionType <: Throwable](
+                                                          result: UpdateFeaturesResult,
+                                                          exceptionMsgPattern: Regex
+  )(implicit tag: ClassTag[ExceptionType]): Unit = {
+    val exception = intercept[ExecutionException] {
+      result.result().get()
+    }
+    assertNotNull(exception.getCause)
+    assertEquals(exception.getCause.getClass, tag.runtimeClass)
+    assertTrue(exceptionMsgPattern.findFirstIn(exception.getCause.getMessage).isDefined)
+  }
+
+  /**
+   * Tests whether an invalid feature update does not get processed on the server as expected,
+   * and raises the ExceptionType on the client side as expected.
+   *
+   * @param invalidUpdate         the invalid feature update to be sent in the
+   *                              updateFeatures request to the server
+   * @param exceptionMsgPattern   a pattern for the expected exception message
+   */
+  private def testWithInvalidFeatureUpdate[ExceptionType <: Throwable](
+                                                                        invalidUpdate: FeatureUpdate,
+                                                                        exceptionMsgPattern: Regex
+  )(implicit tag: ClassTag[ExceptionType]): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeatures(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val adminClient = createAdminClient()
+    val nodeBefore = getFeatureZNode()
+
+    val result = adminClient.updateFeatures(
+      new HashSet[FeatureUpdate](Collections.singletonList(invalidUpdate)), new UpdateFeaturesOptions())
+
+    checkException[ExceptionType](result, exceptionMsgPattern)
+    checkFeatures(
+      adminClient,
+      nodeBefore,
+      new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures()))
+  }
+
+  @Test
+  def testShouldFailRequestIfNotController(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeatures(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val nodeBefore = getFeatureZNode()
+    val requestData = FeatureUpdate.createRequest(
+      new util.HashSet[FeatureUpdate](
+        Collections.singletonList(new FeatureUpdate("feature_1",
+          defaultSupportedFeatures().get("feature_1").max(),
+        false))))
+
+    val response = connectAndReceive[UpdateFeaturesResponse](
+      new UpdateFeaturesRequest.Builder(requestData).build(), notControllerSocketServer)
+
+    assertEquals(Errors.NOT_CONTROLLER, response.error())
+    checkFeatures(
+      createAdminClient(),
+      nodeBefore,
+      new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures()))
+  }
+
+  @Test
+  def testShouldFailRequestForInvalidFeatureName(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](new FeatureUpdate(
+      "",
+      defaultSupportedFeatures().get("feature_1").max(),
+      false),
+      ".*empty feature name.*".r)
+  }
+
+  @Test
+  def testShouldFailRequestWhenDowngradeFlagIsNotSetDuringDowngrade(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](new FeatureUpdate(
+      "feature_1",
+      (defaultFinalizedFeatures().get("feature_1").max() - 1).asInstanceOf[Short],
+      false),
+      ".*Can not downgrade finalized feature: 'feature_1'.*allowDowngrade.*".r)
+  }
+
+  @Test
+  def testShouldFailRequestWhenDowngradeToHigherVersionLevelIsAttempted(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      new FeatureUpdate(
+        "feature_1",
+        defaultSupportedFeatures().get("feature_1").max(),
+        true),
+      ".*finalized feature: 'feature_1'.*allowDowngrade.* provided maxVersionLevel:3.*existing maxVersionLevel:2.*".r)
+  }
+
+  @Test
+  def testShouldFailRequestInClientWhenDowngradeFlagIsNotSetDuringDeletion(): Unit = {
+    assertThrows[IllegalArgumentException] {
+      new FeatureUpdate("feature_1", 0, false)
+    }
+  }
+
+  @Test
+  def testShouldFailRequestInServerWhenDowngradeFlagIsNotSetDuringDeletion(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeatures(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val adminClient = createAdminClient()
+    val nodeBefore = getFeatureZNode()
+
+    val featureUpdates
+      = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection();
+    val featureUpdate = new UpdateFeaturesRequestData.FeatureUpdateKey();

Review comment:
       Hmm, there seem to be very few call sites and therefore seems ok to inline it. Let me know!

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1214,70 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker, but it can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the DescribeFeaturesResult containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);
+
+    /**
+     * Applies specified updates to finalized features. The API is atomic, meaning that if a single
+     * feature update in the request can't succeed on the controller, then none of the feature
+     * updates are carried out. This request is issued only to the controller since the API is
+     * only served by the controller.
+     * <p>
+     * The API takes as input a set of FinalizedFeatureUpdate that need to be applied. Each such
+     * update specifies the finalized feature to be added or updated or deleted, along with the new
+     * max feature version level value.
+     * <ul>
+     * <li>Downgrade of feature version level is not a regular operation/intent. It is only allowed
+     * in the controller if the feature update has the allowDowngrade flag set - setting this flag
+     * conveys user intent to attempt downgrade of a feature max version level. Note that despite
+     * the allowDowngrade flag being set, certain downgrades may be rejected by the controller if it
+     * is deemed impossible.</li>
+     * <li>Deletion of a finalized feature version is not a regular operation/intent. It is allowed
+     * only if the allowDowngrade flag is set in the feature update, and, if the max version level
+     * is set to a value less than 1.</li>
+     * </ul>
+     *<p>
+     * The following exceptions can be anticipated when calling {@code get()} on the futures
+     * obtained from the returned {@link UpdateFinalizedFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.ClusterAuthorizationException}
+     *   If the authenticated user didn't have alter access to the cluster.</li>
+     *   <li>{@link org.apache.kafka.common.errors.InvalidRequestException}
+     *   If the request details are invalid. e.g., a non-existing finalized feature is attempted

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.
+   *
+   * This method sets up the FeatureZNode with enabled status. This status means the feature
+   * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode
+   * are active. This status should be written by the controller to the FeatureZNode only when the
+   * broker IBP config is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. The reason to do this is that enabling all the possible features immediately after
+   *    an upgrade could be harmful to the cluster.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, then it
+   *        will react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, it's existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) => {
+            val brokerDefaultVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (brokerDefaultVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (existingVersionRange.max() >= brokerDefaultVersionRange.min() &&

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1825,36 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def processUpdateFinalizedFeatures(newFeatures: Features[FinalizedVersionRange],
+                                             callback: UpdateFinalizedFeaturesCallback): Unit = {
+    if (isActive) {
+      val incompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.filter(broker => {
+        BrokerFeatures.hasIncompatibleFeatures(broker.features, newFeatures)
+      })
+      if (incompatibleBrokers.size > 0) {
+        callback(
+          Errors.INVALID_REQUEST,
+          Some(
+            s"Could not apply finalized feature updates because ${incompatibleBrokers.size} brokers"
+            + s" were found to have incompatible features. newFeatures: $newFeatures"
+           + s", incompatibleBrokers: $incompatibleBrokers."))
+      } else {
+        try {
+          val newVersion = zkClient.updateFeatureZNode(
+            new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures))
+          featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)

Review comment:
       Sure, we can hear what others say.

##########
File path: core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala
##########
@@ -0,0 +1,467 @@
+/**
+ * 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.server
+
+import java.util
+import java.util.Arrays
+import java.util.Collections
+import java.util.HashSet
+import java.util.Properties
+import java.util.concurrent.ExecutionException
+
+import kafka.api.KAFKA_2_7_IV0
+import kafka.utils.TestUtils
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion}
+import kafka.utils.TestUtils.waitUntilTrue
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions, UpdateFeaturesResult}
+import org.apache.kafka.common.errors.InvalidRequestException
+import org.apache.kafka.common.feature.FinalizedVersionRange
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.message.UpdateFeaturesRequestData
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{UpdateFeaturesRequest, UpdateFeaturesResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.Test
+import org.junit.Assert.{assertEquals, assertNotEquals, assertNotNull, assertTrue}
+import org.scalatest.Assertions.{assertThrows, intercept}
+
+import scala.reflect.ClassTag
+import scala.util.matching.Regex
+
+class UpdateFeaturesTest extends BaseRequestTest {
+
+  override def brokerCount = 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))))
+  }
+
+  private def defaultFinalizedFeatures(): Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2))))
+  }
+
+  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 updateSupportedFeatures(features: Features[SupportedVersionRange]): Unit = {
+    updateSupportedFeatures(features, Set[KafkaServer]() ++ servers)
+  }
+
+  private def updateDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = {
+    servers.foreach(s => {
+      s.brokerFeatures.setDefaultMinVersionLevels(newMinVersionLevels)
+    })
+  }
+
+  private def updateFeatureZNode(features: Features[FinalizedVersionRange]): Int = {
+    val server = serverForId(0).get
+    val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, features)
+    val newVersion = server.zkClient.updateFeatureZNode(newNode)
+    servers.foreach(s => {
+      s.featureCache.waitUntilEpochOrThrow(newVersion, s.config.zkConnectionTimeoutMs)
+    })
+    newVersion
+  }
+
+  private def getFeatureZNode(): FeatureZNode = {
+    val (mayBeFeatureZNodeBytes, version) = serverForId(0).get.zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(version, ZkVersion.UnknownVersion)
+    FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+  }
+
+  private def checkFeatures(client: Admin, expectedNode: FeatureZNode, expectedMetadata: FeatureMetadata): Unit = {
+    assertEquals(expectedNode, getFeatureZNode())
+    val featureMetadata = client.describeFeatures(
+      new DescribeFeaturesOptions().sendRequestToController(true)).featureMetadata().get()
+    assertEquals(expectedMetadata, featureMetadata)
+  }
+
+  private def checkException[ExceptionType <: Throwable](
+                                                          result: UpdateFeaturesResult,
+                                                          exceptionMsgPattern: Regex
+  )(implicit tag: ClassTag[ExceptionType]): Unit = {
+    val exception = intercept[ExecutionException] {
+      result.result().get()
+    }
+    assertNotNull(exception.getCause)
+    assertEquals(exception.getCause.getClass, tag.runtimeClass)
+    assertTrue(exceptionMsgPattern.findFirstIn(exception.getCause.getMessage).isDefined)
+  }
+
+  /**
+   * Tests whether an invalid feature update does not get processed on the server as expected,
+   * and raises the ExceptionType on the client side as expected.
+   *
+   * @param invalidUpdate         the invalid feature update to be sent in the
+   *                              updateFeatures request to the server
+   * @param exceptionMsgPattern   a pattern for the expected exception message
+   */
+  private def testWithInvalidFeatureUpdate[ExceptionType <: Throwable](

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -2945,6 +2948,130 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFeatures(request: RequestChannel.Request): Unit = {
+    val updateFeaturesRequest = request.body[UpdateFeaturesRequest]
+    def sendResponseCallback(error: Errors, msgOverride: Option[String]): Unit = {
+      val data = new UpdateFeaturesResponseData().setErrorCode(error.code())
+      msgOverride.map(msg => data.setErrorMessage(msg))
+      sendResponseExemptThrottle(request, new UpdateFeaturesResponse(data))
+    }
+
+    if (!authorize(request.context, ALTER, CLUSTER, CLUSTER_NAME)) {
+      sendResponseCallback(Errors.CLUSTER_AUTHORIZATION_FAILED, Option.empty)
+    } else if (!controller.isActive) {
+      sendResponseCallback(Errors.NOT_CONTROLLER, Option.empty)
+    } else if (!config.isFeatureVersioningEnabled) {
+      sendResponseCallback(Errors.INVALID_REQUEST, Some("Feature versioning system is disabled."))
+    } else if (updateFeaturesRequest.data.featureUpdates.isEmpty) {
+      sendResponseCallback(Errors.INVALID_REQUEST, Some("Can not provide empty FinalizedFeatureUpdates in the request."))
+    } else {
+      val targetFeaturesOrError = getTargetFinalizedFeaturesOrError(updateFeaturesRequest)
+      targetFeaturesOrError match {
+        case Left(targetFeatures) =>
+          controller.updateFeatures(targetFeatures, sendResponseCallback)
+        case Right(error) =>
+          sendResponseCallback(error.error, Some(error.message))
+      }
+    }
+  }
+
+  /**
+   * Validates the provided UpdateFinalizedFeaturesRequest, checking for various error cases.
+   * If the validation is successful, returns the target finalized features constructed from the
+   * request.
+   *
+   * @param request   the request to be validated
+   *
+   * @return          - the target finalized features, if request validation is successful

Review comment:
       This method has changed greatly and it has been moved to `KafkaController.scala`.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java
##########
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+import java.util.Set;
+import org.apache.kafka.common.message.UpdateFeaturesRequestData;
+
+/**
+ * Encapsulates details about an update to a finalized feature. This is particularly useful to
+ * define each feature update in the {@link Admin#updateFeatures(Set, UpdateFeaturesOptions)} API.
+ */
+public class FeatureUpdate {
+    private final String featureName;
+    private final short maxVersionLevel;
+    private final boolean allowDowngrade;
+
+    /**
+     * @param featureName       the name of the finalized feature to be updated.
+     * @param maxVersionLevel   the new maximum version level for the finalized feature.
+     *                          a value < 1 is special and indicates that the update is intended to
+     *                          delete the finalized feature, and should be accompanied by setting
+     *                          the allowDowngrade flag to true.
+     * @param allowDowngrade    - true, if this feature update was meant to downgrade the existing
+     *                            maximum version level of the finalized feature.
+     *                          - false, otherwise.
+     */
+    public FeatureUpdate(final String featureName, final short maxVersionLevel, final boolean allowDowngrade) {
+        Objects.requireNonNull(featureName, "Provided feature name can not be null.");
+        if (maxVersionLevel < 1 && !allowDowngrade) {
+            throw new IllegalArgumentException(
+                String.format(
+                    "For featureName: %s, the allowDowngrade flag is not set when the" +

Review comment:
       Done. Actually `feature` is removed from this class now.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import java.nio.ByteBuffer;
+import org.apache.kafka.common.message.UpdateFeaturesResponseData;
+import org.apache.kafka.common.message.UpdateFeaturesRequestData;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.types.Struct;
+
+public class UpdateFeaturesRequest extends AbstractRequest {
+
+    public static class Builder extends AbstractRequest.Builder<UpdateFeaturesRequest> {
+
+        private final UpdateFeaturesRequestData data;
+
+        public Builder(UpdateFeaturesRequestData data) {
+            super(ApiKeys.UPDATE_FEATURES);
+            this.data = data;
+        }
+
+        @Override
+        public UpdateFeaturesRequest build(short version) {
+            return new UpdateFeaturesRequest(data, version);
+        }
+
+        @Override
+        public String toString() {
+            return data.toString();
+        }
+    }
+
+    public final UpdateFeaturesRequestData data;
+
+    public UpdateFeaturesRequest(UpdateFeaturesRequestData data, short version) {
+        super(ApiKeys.UPDATE_FEATURES, version);
+        this.data = data;
+    }
+
+    public UpdateFeaturesRequest(Struct struct, short version) {
+        super(ApiKeys.UPDATE_FEATURES, version);
+        this.data = new UpdateFeaturesRequestData(struct, version);
+    }
+
+    @Override
+    public AbstractResponse getErrorResponse(int throttleTimeMsIgnored, Throwable e) {
+        final ApiError apiError = ApiError.fromThrowable(e);
+        return new UpdateFeaturesResponse(
+            new UpdateFeaturesResponseData()
+                .setErrorCode(apiError.error().code())
+                .setErrorMessage(apiError.message()));
+    }
+
+    @Override
+    protected Struct toStruct() {
+        return data.toStruct(version());
+    }
+
+    public UpdateFeaturesRequestData data() {

Review comment:
       Done. Made the attribute 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] kowshik commented on a change in pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1395,7 +1596,7 @@ class KafkaController(val config: KafkaConfig,
     if (newMetadataOpt.nonEmpty && oldMetadataOpt.nonEmpty) {
       val oldMetadata = oldMetadataOpt.get
       val newMetadata = newMetadataOpt.get
-      if (newMetadata.endPoints != oldMetadata.endPoints) {
+      if (newMetadata.endPoints != oldMetadata.endPoints || !oldMetadata.features.equals(newMetadata.features)) {

Review comment:
       I do not understand the concern.
   Which code path can possibly introduce `null` features attribute in `Broker` object? It is impossible....




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,178 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature
+ *    version level deprecation. This is how it works: in order to deprecate feature version levels,
+ *    in this map the default minimum version level of a feature can be set to a new value that's
+ *    higher than 1 (let's call this latest_min_version_level). In doing so, the feature version levels
+ *    in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic
+ *    that applies this map to persistent finalized feature state in ZK (this mutation happens
+ *    during controller election and during finalized feature updates via the
+ *    ApiKeys.UPDATE_FINALIZED_FEATURES api). This will automatically mean external clients of Kafka
+ *    would need to stop using the finalized min version levels that have been deprecated.
+ *
+ * This class also provides APIs to check for incompatibilities between the features supported by
+ * the Broker and finalized features. This class is immutable in production. It provides few APIs to
+ * mutate state only for the purpose of testing.
+ */
+class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange],
+                              @volatile var defaultFeatureMinVersionLevels: Map[String, Short]) {
+  require(BrokerFeatures.areFeatureMinVersionLevelsCompatible(
+    supportedFeatures, defaultFeatureMinVersionLevels))
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(newFeatures, defaultFeatureMinVersionLevels))
+    supportedFeatures = newFeatures
+  }
+
+  /**
+   * Returns the default minimum version level for a specific feature.
+   *
+   * @param feature   the name of the feature
+   *
+   * @return          the default minimum version level for the feature if its defined.
+   *                  otherwise, returns 1.
+   */
+  def defaultMinVersionLevel(feature: String): Short = {
+    defaultFeatureMinVersionLevels.getOrElse(feature, 1)
+  }
+
+  // For testing only.
+  def setDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(supportedFeatures, newMinVersionLevels))
+    defaultFeatureMinVersionLevels = newMinVersionLevels
+  }
+
+  /**
+   * Returns the default finalized features that a new Kafka cluster with IBP config >= KAFKA_2_7_IV0
+   * needs to be bootstrapped with.
+   */
+  def getDefaultFinalizedFeatures: Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(
+      supportedFeatures.features.asScala.map {
+        case(name, versionRange) => (
+          name, new FinalizedVersionRange(defaultMinVersionLevel(name), versionRange.max))
+      }.asJava)
+  }
+
+  /**
+   * Returns the set of feature names found to be incompatible.
+   * A feature incompatibility is a version mismatch between the latest feature supported by the
+   * Broker, and the provided finalized feature. This can happen because a provided finalized
+   * feature:
+   *  1) Does not exist in the Broker (i.e. it is unknown to the Broker).
+   *           [OR]
+   *  2) Exists but the FinalizedVersionRange does not match with the
+   *     supported feature's SupportedVersionRange.
+   *
+   * @param finalized   The finalized features against which incompatibilities need to be checked for.
+   *
+   * @return            The subset of input features which are incompatible. If the returned object
+   *                    is empty, it means there were no feature incompatibilities found.
+   */
+  def incompatibleFeatures(finalized: Features[FinalizedVersionRange]): Features[FinalizedVersionRange] = {
+    BrokerFeatures.incompatibleFeatures(supportedFeatures, finalized, logIncompatibilities = true)
+  }
+}
+
+object BrokerFeatures extends Logging {
+
+  def createDefault(): BrokerFeatures = {
+    // The arguments are currently empty, but, in the future as we define features we should
+    // populate the required values here.
+    new BrokerFeatures(emptySupportedFeatures, Map[String, Short]())
+  }
+
+  /**
+   * Returns true if any of the provided finalized features are incompatible with the provided
+   * supported features.
+   *
+   * @param supportedFeatures   The supported features to be compared
+   * @param finalizedFeatures   The finalized features to be compared
+   *
+   * @return                    - True if there are any feature incompatibilities found.
+   *                            - False otherwise.
+   */
+  def hasIncompatibleFeatures(supportedFeatures: Features[SupportedVersionRange],
+                              finalizedFeatures: Features[FinalizedVersionRange]): Boolean = {
+    !incompatibleFeatures(supportedFeatures, finalizedFeatures, false).empty
+  }
+
+  private def incompatibleFeatures(supportedFeatures: Features[SupportedVersionRange],
+                                   finalizedFeatures: Features[FinalizedVersionRange],
+                                   logIncompatibilities: Boolean): Features[FinalizedVersionRange] = {
+    val incompatibleFeaturesInfo = finalizedFeatures.features.asScala.map {
+      case (feature, versionLevels) =>
+        val supportedVersions = supportedFeatures.get(feature)
+        if (supportedVersions == null) {
+          (feature, versionLevels, "{feature=%s, reason='Unsupported feature'}".format(feature))
+        } else if (versionLevels.isIncompatibleWith(supportedVersions)) {
+          (feature, versionLevels, "{feature=%s, reason='%s is incompatible with %s'}".format(
+            feature, versionLevels, supportedVersions))
+        } else {
+          (feature, versionLevels, null)
+        }
+    }.filter{ case(_, _, errorReason) => errorReason != null}.toList
+
+    if (logIncompatibilities && incompatibleFeaturesInfo.nonEmpty) {
+      warn(
+        "Feature incompatibilities seen: " + incompatibleFeaturesInfo.map {
+          case(_, _, errorReason) => errorReason })
+    }
+    Features.finalizedFeatures(incompatibleFeaturesInfo.map {
+      case(feature, versionLevels, _) => (feature, versionLevels) }.toMap.asJava)
+  }
+
+  /**
+   * A check that ensures each feature defined with min version level is a supported feature, and
+   * the min version level value is valid (i.e. it is compatible with the supported version range).
+   *
+   * @param supportedFeatures         the supported features
+   * @param featureMinVersionLevels   the feature minimum version levels
+   *
+   * @return                          - true, if the above described check passes.
+   *                                  - false, otherwise.
+   */
+  private def areFeatureMinVersionLevelsCompatible(
+    supportedFeatures: Features[SupportedVersionRange],
+    featureMinVersionLevels: Map[String, Short]
+  ): Boolean = {
+    featureMinVersionLevels.forall {
+      case(featureName, minVersionLevel) =>
+        val supportedFeature = supportedFeatures.get(featureName)
+        (supportedFeature != null) &&
+          !new FinalizedVersionRange(minVersionLevel, supportedFeature.max())

Review comment:
       It's cheap. I'd rather leave it this way.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,178 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of its own supported features in its
+   * own BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. This process ensures we do not enable all the possible features immediately after
+   *    an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, it will
+   *        react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, its existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) =>
+            val brokerDefaultVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (brokerDefaultVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (brokerDefaultVersionRange.max() >= existingVersionRange.max() &&
+                       brokerDefaultVersionRange.min() <= existingVersionRange.max()) {
+              // Through this change, we deprecate all version levels in the closed range:
+              // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1]
+              (featureName, new FinalizedVersionRange(brokerDefaultVersionRange.min(), existingVersionRange.max()))
+            } else {
+              // If the existing version levels fall completely outside the

Review comment:
       Actually this is an error case now. Have updated the code with the fix, and with good documentation.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java
##########
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import static java.util.stream.Collectors.joining;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * Encapsulates details about finalized as well as supported features. This is particularly useful
+ * to hold the result returned by the {@link Admin#describeFeatures(DescribeFeaturesOptions)} API.
+ */
+public class FeatureMetadata {
+
+    private final Map<String, FinalizedVersionRange> finalizedFeatures;
+
+    private final Optional<Long> finalizedFeaturesEpoch;
+
+    private final Map<String, SupportedVersionRange> supportedFeatures;
+
+    public FeatureMetadata(final Map<String, FinalizedVersionRange> finalizedFeatures,

Review comment:
       Done. Good catch. Also I've modified `org.apache.kafka.clients.admin.{Supported|Finalized}VersionRange` classes to make constructors non-public.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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


   @junrao The test failure in `MirrorConnectorsIntegrationTest.testReplication` does not seem related. I have rebased the PR now against latest AK trunk, I'd like to see if the failure happens 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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.
+   *
+   * This method sets up the FeatureZNode with enabled status. This status means the feature
+   * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode
+   * are active. This status should be written by the controller to the FeatureZNode only when the
+   * broker IBP config is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. The reason to do this is that enabling all the possible features immediately after
+   *    an upgrade could be harmful to the cluster.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, then it
+   *        will react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, it's existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) => {
+            val brokerDefaultVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (brokerDefaultVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (existingVersionRange.max() >= brokerDefaultVersionRange.min() &&
+                       brokerDefaultVersionRange.max() >= existingVersionRange.max()) {
+              // Through this change, we deprecate all version levels in the closed range:
+              // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1]
+              (featureName, new FinalizedVersionRange(brokerDefaultVersionRange.min(), existingVersionRange.max()))
+            } else {
+              // If the existing version levels fall completely outside the
+              // range of the default finalized version levels (i.e. no intersection), or, if the
+              // existing version levels are ineligible for a modification since they are
+              // incompatible with default finalized version levels, then we skip the update.
+              warn(s"Can not update minimum version level in finalized feature: $featureName,"
+                + s" since the existing $existingVersionRange is not eligible for a change"
+                + s" based on the default $brokerDefaultVersionRange.")
+              (featureName, existingVersionRange)
+            }
+          }
+        }.asJava)
+      }
+      val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures)
+      if (!newFeatureZNode.equals(existingFeatureZNode)) {

Review comment:
       We should keep the existing check as it is. The reason is that if the existing node is `(disabled, {})` then here we would like to change it to `(enabled, features)`. Therefore, we have to check the features as well as the `FeatureZNodeStatus`.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java
##########
@@ -17,9 +17,16 @@
 package org.apache.kafka.common.feature;
 
 import java.util.Map;
+import java.util.Objects;
+import org.apache.kafka.common.utils.Utils;
 
 /**
- * An extended {@link BaseVersionRange} representing the min/max versions for supported features.
+ * An extended {@link BaseVersionRange} representing the min, max and first active versions for a
+ * supported feature:
+ *  - minVersion: This is the minimum supported version for the feature.
+ *  - maxVersion: This the maximum supported version for the feature.
+ *  - firstActiveVersion: This is the first active version for the feature. Versions in the range

Review comment:
       @junrao My response below.
   
   > I was thinking what if we relax the current check by just making sure that maxVersion of finalized is within the supported range. Basically in your example, if supported minVersion goes to 2, it's still allowed since it's less than maxVersion of finalized. However, if supported minVersion goes to 7, this fails the broker since it's more than maxVersion of finalized.
   
   If we relaxed the current check like you suggested, then, as of today the following will happen:
   The controller can not effectively finalize `minVersionLevel` for the feature, because, with a relaxed check we do not know whether all brokers in the cluster support a particular `minVersion` when the controller finalizes it at a particular value.
   
   Supposing we want to keep the concept of `minVersionLevel` like the way it is now (i.e. it tells you the lowest feature version level guaranteed to be supported by any broker in the cluster), then, `firstActiveVersion` provides the door to mutate it safely whenever the need arises.
   
   So when thinking about your suggestion, it is not clear to me if you are intending to alter the meaning of `minVersionLevel`? If so, please could you tell me what is the new meaning of `minVersionLevel` that you are proposing?
   
   > Your concern for the relaxed check seems to be around deploying a wrong version of the broker by mistake.
   
   Not exactly, my concern is a bit different. It is to be noted that `minVersionLevel` can not be changed through the `UPDATE_FEATURES` api. Assuming we want to maintain the meaning of `minVersionLevel` the way it is today, my concern is about how would one alter `minVersionLevel` for a feature when the need arises? This needs to be done without letting the versioning system misconstrue the intent with a feature version incompatibility. This is where usage of `firstActiveVersion` acts as a solution.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,179 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the latest features supported by the Broker and also provides APIs to
+ * check for incompatibilities between the features supported by the Broker and finalized features.
+ * The class also enables feature version level deprecation, as explained below. This class is
+ * immutable in production. It provides few APIs to mutate state only for the purpose of testing.
+ *
+ * Feature version level deprecation:
+ * ==================================
+ *
+ * Deprecation of certain version levels of a feature is a process to stop supporting the
+ * functionality offered by the feature at a those version levels, across the entire Kafka cluster.
+ * Feature version deprecation is a simple 2-step process explained below. In each step below, an
+ * example is provided to help understand the process better:
+ *
+ * STEP 1:
+ * In the first step, a major Kafka release is made with a Broker code change (explained later
+ * below) that establishes the intent to deprecate certain versions of one or more features
+ * cluster-wide. When this new Kafka release is deployed to the cluster, the feature versioning
+ * system (via the controller) will automatically persist the new minVersionLevel for the feature in
+ * Zk to propagate the deprecation of certain versions. After this happens, any external client that
+ * queries the Broker to learn the feature versions will at some point start to see the new value
+ * for the finalized minVersionLevel for the feature. This makes the version deprecation permanent.
+ *
+ * Here is how the above code change needs to be done:
+ * In order to deprecate feature version levels, in the supportedFeatures map you need to supply a
+ * specific firstActiveVersion value that's higher than the minVersion for the feature. The
+ * value for firstActiveVersion should be 1 beyond the highest version that you intend to deprecate
+ * for that feature. When features are finalized via the ApiKeys.UPDATE_FEATURES api, the feature
+ * version levels in the closed range: [minVersion, firstActiveVersion - 1] are automatically
+ * deprecated in ZK by the controller logic.
+ * Example:
+ * - Let us assume the existing finalized feature in ZK:
+ *   {
+ *      "feature_1" -> FinalizedVersionRange(minVersionLevel=1, maxVersionLevel=5)
+ *   }
+ *   Now, supposing you would like to deprecate feature version levels: [1, 2].
+ *   Then, in the supportedFeatures map you should supply the following:
+ *   supportedFeatures = {
+ *     "feature1" -> SupportedVersionRange(minVersion=1, firstActiveVersion=3, maxVersion=5)
+ *   }
+ * - If you do NOT want to deprecate a version level for a feature, then in the supportedFeatures
+ *   map you should supply the firstActiveVersion to be the same as the minVersion supplied for that
+ *   feature.
+ *   Example:
+ *   supportedFeatures = {
+ *     "feature1" -> SupportedVersionRange(minVersion=1, firstActiveVersion=1, maxVersion=5)
+ *   }
+ *   This indicates no intent to deprecate any version levels for the feature.
+ *
+ * STEP 2:
+ * After the first step is over, you may (at some point) want to permanently remove the code/logic
+ * for the functionality offered by the deprecated feature versions. This is the second step. Here a
+ * subsequent major Kafka release is made with another Broker code change that removes the code for
+ * the functionality offered by the deprecated feature versions. This would completely drop support
+ * for the deprecated versions. Such a code change needs to be supplemented by supplying a
+ * suitable higher minVersion value for the feature in the supportedFeatures map.
+ * Example:
+ * - In the example above in step 1, we showed how to deprecate version levels [1, 2] for
+ *   "feature_1". Now let us assume the following finalized feature in ZK (after the deprecation
+ *   has been carried out):
+ *   {
+ *     "feature_1" -> FinalizedVersionRange(minVersionLevel=3, maxVersionLevel=5)
+ *   }
+ *   Now, supposing you would like to permanently remove support for feature versions: [1, 2].
+ *   Then, in the supportedFeatures map you should now supply the following:
+ *   supportedFeatures = {
+ *     "feature1" -> SupportedVersionRange(minVersion=3, firstActiveVersion=3, maxVersion=5)

Review comment:
       This is a really good point. Yes, I feel it is useful to expose it to the client via `ApiVersionsResponse`. I can change the KIP suitably.

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,179 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the latest features supported by the Broker and also provides APIs to
+ * check for incompatibilities between the features supported by the Broker and finalized features.
+ * The class also enables feature version level deprecation, as explained below. This class is
+ * immutable in production. It provides few APIs to mutate state only for the purpose of testing.
+ *
+ * Feature version level deprecation:
+ * ==================================
+ *
+ * Deprecation of certain version levels of a feature is a process to stop supporting the
+ * functionality offered by the feature at a those version levels, across the entire Kafka cluster.
+ * Feature version deprecation is a simple 2-step process explained below. In each step below, an
+ * example is provided to help understand the process better:
+ *
+ * STEP 1:
+ * In the first step, a major Kafka release is made with a Broker code change (explained later
+ * below) that establishes the intent to deprecate certain versions of one or more features
+ * cluster-wide. When this new Kafka release is deployed to the cluster, the feature versioning
+ * system (via the controller) will automatically persist the new minVersionLevel for the feature in
+ * Zk to propagate the deprecation of certain versions. After this happens, any external client that
+ * queries the Broker to learn the feature versions will at some point start to see the new value
+ * for the finalized minVersionLevel for the feature. This makes the version deprecation permanent.
+ *
+ * Here is how the above code change needs to be done:
+ * In order to deprecate feature version levels, in the supportedFeatures map you need to supply a
+ * specific firstActiveVersion value that's higher than the minVersion for the feature. The
+ * value for firstActiveVersion should be 1 beyond the highest version that you intend to deprecate
+ * for that feature. When features are finalized via the ApiKeys.UPDATE_FEATURES api, the feature
+ * version levels in the closed range: [minVersion, firstActiveVersion - 1] are automatically
+ * deprecated in ZK by the controller logic.
+ * Example:
+ * - Let us assume the existing finalized feature in ZK:
+ *   {
+ *      "feature_1" -> FinalizedVersionRange(minVersionLevel=1, maxVersionLevel=5)
+ *   }
+ *   Now, supposing you would like to deprecate feature version levels: [1, 2].
+ *   Then, in the supportedFeatures map you should supply the following:
+ *   supportedFeatures = {
+ *     "feature1" -> SupportedVersionRange(minVersion=1, firstActiveVersion=3, maxVersion=5)
+ *   }
+ * - If you do NOT want to deprecate a version level for a feature, then in the supportedFeatures
+ *   map you should supply the firstActiveVersion to be the same as the minVersion supplied for that
+ *   feature.
+ *   Example:
+ *   supportedFeatures = {
+ *     "feature1" -> SupportedVersionRange(minVersion=1, firstActiveVersion=1, maxVersion=5)
+ *   }
+ *   This indicates no intent to deprecate any version levels for the feature.
+ *
+ * STEP 2:
+ * After the first step is over, you may (at some point) want to permanently remove the code/logic
+ * for the functionality offered by the deprecated feature versions. This is the second step. Here a
+ * subsequent major Kafka release is made with another Broker code change that removes the code for
+ * the functionality offered by the deprecated feature versions. This would completely drop support
+ * for the deprecated versions. Such a code change needs to be supplemented by supplying a
+ * suitable higher minVersion value for the feature in the supportedFeatures map.
+ * Example:
+ * - In the example above in step 1, we showed how to deprecate version levels [1, 2] for
+ *   "feature_1". Now let us assume the following finalized feature in ZK (after the deprecation
+ *   has been carried out):
+ *   {
+ *     "feature_1" -> FinalizedVersionRange(minVersionLevel=3, maxVersionLevel=5)
+ *   }
+ *   Now, supposing you would like to permanently remove support for feature versions: [1, 2].
+ *   Then, in the supportedFeatures map you should now supply the following:
+ *   supportedFeatures = {
+ *     "feature1" -> SupportedVersionRange(minVersion=3, firstActiveVersion=3, maxVersion=5)

Review comment:
       This is a really good point. Yes, I feel it is useful to expose it to the client via `ApiVersionsResponse`. I can change the KIP suitably and then update the PR.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -272,6 +281,199 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (string) and a range of versions (defined by a
+   * SupportedVersionRange). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of its own supported features in its
+   * own BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (string) and a range of version levels (defined
+   * by a FinalizedVersionRange). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. This process ensures we do not enable all the possible features immediately after
+   *    an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, it will
+   *        react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, its existing minimum version level is updated to the default

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 edited a comment on pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #9001:
URL: https://github.com/apache/kafka/pull/9001#issuecomment-704786154


   The test failures in the latest CI runs do not seem related to this PR:
    * JDK 8: `org.apache.kafka.connect.integration.ExampleConnectIntegrationTest.testSourceConnector`
    * JDK 11: `org.apache.kafka.streams.integration.EosBetaUpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosBeta`
   
   The test that failed previously under JDK 15 has passed in the latest CI run: `MirrorConnectorsIntegrationTest.testReplication`.


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,187 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the latest features supported by the Broker and also provides APIs to
+ * check for incompatibilities between the features supported by the Broker and finalized features.
+ * The class also enables feature version level deprecation, as explained below. This class is
+ * immutable in production. It provides few APIs to mutate state only for the purpose of testing.
+ *
+ * Feature version level deprecation:
+ * ==================================
+ *
+ * Deprecation of certain version levels of a feature is a process to stop supporting the
+ * functionality offered by the feature at those version levels, across the entire Kafka cluster.
+ * Feature version deprecation is a simple 2-step process explained below. In each step below, an
+ * example is provided to help understand the process better:
+ *
+ * STEP 1:
+ * =======
+ *
+ * In the first step, a major Kafka release is made with a Broker code change (explained later
+ * below) that establishes the intent to deprecate certain versions of one or more features
+ * cluster-wide. When this new Kafka release is deployed to the cluster, deprecated finalized
+ * feature versions are no longer advertised to the client, but they can still be used by existing
+ * connections. The way it works is that the feature versioning system (via the controller) will
+ * automatically persist the new minVersionLevel for the feature in ZK to propagate the deprecation
+ * of certain versions. After this happens, any external client that queries the Broker to learn the
+ * feature versions will at some point start to see the new value for the finalized minVersionLevel
+ * for the feature. The external clients are expected to stop using the deprecated versions at least
+ * by the time that they learn about it.
+ *
+ * Here is how the above code change needs to be done:
+ * In order to deprecate feature version levels, in the supportedFeatures map you need to supply a
+ * specific firstActiveVersion value that's higher than the minVersion for the feature. The
+ * value for firstActiveVersion should be 1 beyond the highest version that you intend to deprecate
+ * for that feature. Whenever the controller is elected or the features are finalized via the
+ * ApiKeys.UPDATE_FEATURES api, the feature version levels in the closed range:
+ * [minVersion, firstActiveVersion - 1] are automatically deprecated in ZK by the controller logic.
+ *
+ * Example:
+ * - Let us assume the existing finalized feature in ZK:
+ *   {
+ *      "feature" -> FinalizedVersionRange(minVersionLevel=1, maxVersionLevel=5)
+ *   }
+ *   Now, supposing you would like to deprecate feature version levels: [1, 2].
+ *   Then, in the supportedFeatures map you should supply the following:
+ *   supportedFeatures = {
+ *     "feature" -> SupportedVersionRange(minVersion=1, firstActiveVersion=3, maxVersion=5)
+ *   }
+ * - If you do NOT want to deprecate a version level for a feature, then, in the supportedFeatures
+ *   map you should supply the firstActiveVersion to be the same as the minVersion supplied for that
+ *   feature.
+ *   Example:
+ *   supportedFeatures = {
+ *     "feature" -> SupportedVersionRange(minVersion=1, firstActiveVersion=1, maxVersion=5)
+ *   }
+ *   The above indicates no intent to deprecate any version levels for the feature.
+ *
+ * STEP 2:
+ * =======
+ *
+ * After the first step is over, you may (at some point) want to permanently remove the code/logic
+ * for the functionality offered by the deprecated feature versions. This is the second step. Here a
+ * subsequent major Kafka release is made with another Broker code change that removes the code for
+ * the functionality offered by the deprecated feature versions. This would completely drop support
+ * for the deprecated versions. Such a code change needs to be supplemented by supplying a
+ * suitable higher minVersion value for the feature in the supportedFeatures map.
+ * Example:
+ * - In the example above in step 1, we showed how to deprecate version levels [1, 2] for
+ *   "feature". Now let us assume the following finalized feature in ZK (after the deprecation
+ *   has been carried out):
+ *   {
+ *     "feature" -> FinalizedVersionRange(minVersionLevel=3, maxVersionLevel=5)
+ *   }
+ *   Now, supposing you would like to permanently remove support for feature versions: [1, 2].
+ *   Then, in the supportedFeatures map you should now supply the following:

Review comment:
       As mentioned in above response to a different comment, I've provided an overloaded c'tor now in `org.apache.kafka.common.feature.SupportedVersionRange` that only takes `minVersion` and `maxVersion` as parameters.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala
##########
@@ -0,0 +1,580 @@
+/**
+ * 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.server
+
+import java.util.{Optional, Properties}
+import java.util.concurrent.ExecutionException
+
+import kafka.api.KAFKA_2_7_IV0
+import kafka.utils.TestUtils
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion}
+import kafka.utils.TestUtils.waitUntilTrue
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureUpdate, UpdateFeaturesOptions, UpdateFeaturesResult}
+import org.apache.kafka.common.errors.InvalidRequestException
+import org.apache.kafka.common.feature.FinalizedVersionRange
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.message.UpdateFeaturesRequestData
+import org.apache.kafka.common.message.UpdateFeaturesRequestData.FeatureUpdateKeyCollection
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{UpdateFeaturesRequest, UpdateFeaturesResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.Test
+import org.junit.Assert.{assertEquals, assertFalse, assertNotEquals, assertNotNull, assertTrue}
+import org.scalatest.Assertions.intercept
+
+import scala.jdk.CollectionConverters._
+import scala.reflect.ClassTag
+import scala.util.matching.Regex
+
+class UpdateFeaturesTest extends BaseRequestTest {
+
+  override def brokerCount = 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))))
+  }
+
+  private def defaultFinalizedFeatures(): Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2))))
+  }
+
+  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)
+  }
+
+  private def updateFeatureZNode(features: Features[FinalizedVersionRange]): Int = {
+    val server = serverForId(0).get
+    val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, features)
+    val newVersion = server.zkClient.updateFeatureZNode(newNode)
+    servers.foreach(s => {
+      s.featureCache.waitUntilEpochOrThrow(newVersion, s.config.zkConnectionTimeoutMs)
+    })
+    newVersion
+  }
+
+  private def getFeatureZNode(): FeatureZNode = {
+    val (mayBeFeatureZNodeBytes, version) = serverForId(0).get.zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(version, ZkVersion.UnknownVersion)
+    FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+  }
+
+  private def finalizedFeatures(features: java.util.Map[String, org.apache.kafka.clients.admin.FinalizedVersionRange]): Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(features.asScala.map {
+      case(name, versionRange) =>
+        (name, new FinalizedVersionRange(versionRange.minVersionLevel(), versionRange.maxVersionLevel()))
+    }.asJava)
+  }
+
+  private def supportedFeatures(features: java.util.Map[String, org.apache.kafka.clients.admin.SupportedVersionRange]): Features[SupportedVersionRange] = {
+    Features.supportedFeatures(features.asScala.map {
+      case(name, versionRange) =>
+        (name, new SupportedVersionRange(versionRange.minVersion(), versionRange.maxVersion()))
+    }.asJava)
+  }
+
+  private def checkFeatures(client: Admin,
+                            expectedNode: FeatureZNode,
+                            expectedFinalizedFeatures: Features[FinalizedVersionRange],
+                            expectedFinalizedFeaturesEpoch: Long,
+                            expectedSupportedFeatures: Features[SupportedVersionRange]): Unit = {
+    assertEquals(expectedNode, getFeatureZNode())
+    val featureMetadata = client.describeFeatures(
+      new DescribeFeaturesOptions().sendRequestToController(true)).featureMetadata.get
+    assertEquals(expectedFinalizedFeatures, finalizedFeatures(featureMetadata.finalizedFeatures))
+    assertEquals(expectedSupportedFeatures, supportedFeatures(featureMetadata.supportedFeatures))
+    assertEquals(Optional.of(expectedFinalizedFeaturesEpoch), featureMetadata.finalizedFeaturesEpoch)
+  }
+
+  private def checkException[ExceptionType <: Throwable](result: UpdateFeaturesResult,
+                                                         featureExceptionMsgPatterns: Map[String, Regex])
+                                                        (implicit tag: ClassTag[ExceptionType]): Unit = {
+    featureExceptionMsgPatterns.foreach {
+      case (feature, exceptionMsgPattern) =>
+        val exception = intercept[ExecutionException] {
+          result.values().get(feature).get()
+        }
+        val cause = exception.getCause
+        assertNotNull(cause)
+        assertEquals(cause.getClass, tag.runtimeClass)
+        assertTrue(s"Received unexpected error message: ${cause.getMessage}",
+                   exceptionMsgPattern.findFirstIn(cause.getMessage).isDefined)
+    }
+  }
+
+  /**
+   * Tests whether an invalid feature update does not get processed on the server as expected,
+   * and raises the ExceptionType on the client side as expected.
+   *
+   * @param invalidUpdate         the invalid feature update to be sent in the
+   *                              updateFeatures request to the server
+   * @param exceptionMsgPattern   a pattern for the expected exception message
+   */
+  private def testWithInvalidFeatureUpdate[ExceptionType <: Throwable](feature: String,

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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/resources/common/message/UpdateFeaturesResponse.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "response",
+  "name": "UpdateFeaturesResponse",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "Results", "type": "[]UpdatableFeatureResult", "versions": "0+",
+      "about": "Results for each feature update.", "fields": [
+      {"name": "Feature", "type": "string", "versions": "0+", "mapKey": true,

Review comment:
       Done. I've updated the KIP-584 write up, please refer to [this section](https://cwiki.apache.org/confluence/display/KAFKA/KIP-584%3A+Versioning+scheme+for+features#KIP584:Versioningschemeforfeatures-UpdateFeaturesResponseschema) in the KIP.




----------------------------------------------------------------
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] chia7712 commented on a change in pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4335,6 +4343,150 @@ void handleFailure(Throwable throwable) {
                 .hi(password, salt, iterations);
     }
 
+    public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        final NodeProvider provider =
+            options.sendRequestToController() ? new ControllerNodeProvider() : new LeastLoadedNodeProvider();
+
+        final Call call = new Call(
+            "describeFeatures", calcDeadlineMs(now, options.timeoutMs()), provider) {
+
+            private FeatureMetadata createFeatureMetadata(final ApiVersionsResponse response) {
+                final Map<String, FinalizedVersionRange> finalizedFeatures = new HashMap<>();
+                for (final FinalizedFeatureKey key : response.data().finalizedFeatures().valuesSet()) {
+                    finalizedFeatures.put(key.name(), new FinalizedVersionRange(key.minVersionLevel(), key.maxVersionLevel()));
+                }
+
+                Optional<Long> finalizedFeaturesEpoch;
+                if (response.data().finalizedFeaturesEpoch() >= 0L) {
+                    finalizedFeaturesEpoch = Optional.of(response.data().finalizedFeaturesEpoch());
+                } else {
+                    finalizedFeaturesEpoch = Optional.empty();
+                }
+
+                final Map<String, SupportedVersionRange> supportedFeatures = new HashMap<>();
+                for (final SupportedFeatureKey key : response.data().supportedFeatures().valuesSet()) {
+                    supportedFeatures.put(key.name(), new SupportedVersionRange(key.minVersion(), key.maxVersion()));
+                }
+
+                return new FeatureMetadata(finalizedFeatures, finalizedFeaturesEpoch, supportedFeatures);
+            }
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(createFeatureMetadata(apiVersionsResponse));
+                } else if (options.sendRequestToController() &&
+                           apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) {
+                    handleNotControllerError(Errors.NOT_CONTROLLER);
+                } else {
+                    future.completeExceptionally(Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        runnable.call(call, now);
+        return new DescribeFeaturesResult(future);
+    }
+
+    @Override
+    public UpdateFeaturesResult updateFeatures(final Map<String, FeatureUpdate> featureUpdates,
+                                               final UpdateFeaturesOptions options) {
+        if (featureUpdates.isEmpty()) {
+            throw new IllegalArgumentException("Feature updates can not be null or empty.");
+        }
+
+        final Map<String, KafkaFutureImpl<Void>> updateFutures = new HashMap<>();
+        for (final Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+            updateFutures.put(entry.getKey(), new KafkaFutureImpl<>());
+        }
+
+        final long now = time.milliseconds();
+        final Call call = new Call("updateFeatures", calcDeadlineMs(now, options.timeoutMs()),
+            new ControllerNodeProvider()) {
+
+            @Override
+            UpdateFeaturesRequest.Builder createRequest(int timeoutMs) {
+                final UpdateFeaturesRequestData.FeatureUpdateKeyCollection featureUpdatesRequestData
+                    = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection();
+                for (Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+                    final String feature = entry.getKey();
+                    final FeatureUpdate update = entry.getValue();
+                    if (feature.trim().isEmpty()) {

Review comment:
       the error message says it can't be null but there is no null check.
   for another, this check can happen early (when creating ```updateFutures```)




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1865,192 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns an ApiError object containing Errors#INVALID_REQUEST.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+
+    val incompatibilityError = "Could not apply finalized feature update because" +
+      " brokers were found to have incompatible versions for the feature."
+
+    if (brokerFeatures.supportedFeatures.get(update.feature()) == null) {
+      Right(new ApiError(Errors.INVALID_REQUEST, incompatibilityError))
+    } else {
+      val defaultMinVersionLevel = brokerFeatures.defaultMinVersionLevel(update.feature)
+      val newVersionRange = new FinalizedVersionRange(defaultMinVersionLevel, update.maxVersionLevel)

Review comment:
       Yes, excellent point. I'll fix this.




----------------------------------------------------------------
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 pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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


   @kowshik : There are 27 system test failures with this PR. http://confluent-kafka-branch-builder-system-test-results.s3-us-west-2.amazonaws.com/2020-10-07--001.1602079476--kowshik--kip584_features_write_path--e1c79cee2/report.html
   
   Are they existing test failures compared with http://confluent-kafka-system-test-results.s3-us-west-2.amazonaws.com/2020-10-07--001.1602079305--apache--trunk--af27c2ddf/report.html ?


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/feature/FinalizedVersionRange.java
##########
@@ -40,14 +40,16 @@ public static FinalizedVersionRange fromMap(Map<String, Short> versionRangeMap)
 
     /**
      * Checks if the [min, max] version level range of this object does *NOT* fall within the
-     * [min, max] version range of the provided SupportedVersionRange parameter.
+     * [min, first_active_version, max] range of the provided SupportedVersionRange parameter.

Review comment:
       We need to keep the existing validation. Here is a case where `minVersionLevel < firstActiveVersion` is true, but still there are no incompatibilities:
   ```
   SupportedVersionRange={minVersion=1, firstActiveVersion=4, maxVersion=7}
   FinalizedVersionRange={minVersionLevel=2, maxVersionLevel=6}
   ```
   
   For example, the above can happen during step 1 of feature verison level deprecation. Imagine a feature exists in ZK already and is finalized at `{minVersionLevel=2, maxVersionLevel=6}`. Then imagine a new Kafka release is deployed that raises `firstActiveVersion` for the supported feature from 1 -> 4 (in order to deprecate versions: 1,2,3). In such a case, during Kafka server startup (where we check for feature incompatibilities), we would run into the comparison cited above between the new `SupportedVersionRange` and existing `FinalizedVersionRange`. But it is not considered to be a case of incompatibility.




----------------------------------------------------------------
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] chia7712 commented on pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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


   @kowshik sorry for bringing trivial comments after this is merged. I just noticed those nits in testing new APIs in 2.7.0.


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -977,14 +1179,30 @@ class KafkaController(val config: KafkaConfig,
 
   /**
    * Send the leader information for selected partitions to selected brokers so that they can correctly respond to
-   * metadata requests
+   * metadata requests. Particularly, when feature versioning is enabled, we filter out brokers with incompatible
+   * features from receiving the metadata requests. This is because we do not want to activate incompatible brokers,
+   * as these may have harmful consequences to the cluster.

Review comment:
       My understanding of the race condition is that the controller finalizes a feature while there is a pending broker registration in the controller event queue. When the controller starts to process the new broker registration, it will realize that its supported feature is not compatible. Here, it's seems that we will still process this new broker registration and only avoid sending UpdatateMetadataRequest to it. I am not sure if this helps since we already acted on this incompatible broker registration and some damage may already be done. The same UpdatateMetadataRequest will still be sent to other brokers and its metadata will be available to the clients.
   
   An alternative way is to just skip the handling of new broker registration if it's detected as incompatible.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java
##########
@@ -17,9 +17,16 @@
 package org.apache.kafka.common.feature;
 
 import java.util.Map;
+import java.util.Objects;
+import org.apache.kafka.common.utils.Utils;
 
 /**
- * An extended {@link BaseVersionRange} representing the min/max versions for supported features.
+ * An extended {@link BaseVersionRange} representing the min, max and first active versions for a
+ * supported feature:
+ *  - minVersion: This is the minimum supported version for the feature.
+ *  - maxVersion: This the maximum supported version for the feature.
+ *  - firstActiveVersion: This is the first active version for the feature. Versions in the range

Review comment:
       @junrao My response below.
   
   > I was thinking what if we relax the current check by just making sure that maxVersion of finalized is within the supported range. Basically in your example, if supported minVersion goes to 2, it's still allowed since it's less than maxVersion of finalized. However, if supported minVersion goes to 7, this fails the broker since it's more than maxVersion of finalized.
   
   If we relaxed the current check like you suggested, then, as of today the following will happen:
   The controller can not effectively finalize `minVersionLevel` for the feature, because, with a relaxed check we do not know whether all brokers in the cluster support a particular `minVersion` when the controller finalizes the `minVersionLevel` at a particular value.
   
   Supposing we want to keep the concept of `minVersionLevel` like the way it is now (i.e. it is the lowest version guaranteed to be supported by any broker in the cluster for a feature), then, `firstActiveVersion` provides the door to mutate it safely whenever the need arises.
   
   So when thinking about your suggestion, it is not clear to me if you are intending to alter the meaning of `minVersionLevel`? If so, please could you tell me what is the new meaning of `minVersionLevel` that you are proposing?
   
   > Your concern for the relaxed check seems to be around deploying a wrong version of the broker by mistake.
   
   Not exactly, my concern is a bit different. It is to be noted that `minVersionLevel` can not be changed through the `UPDATE_FEATURES` api. Assuming we want to maintain the meaning of `minVersionLevel` the way it is today, my concern is about how would one alter `minVersionLevel` for a feature when the need arises? This needs to be done without letting the versioning system misconstrue the intent with a feature version incompatibility. This is where usage of `firstActiveVersion` acts as a solution.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala
##########
@@ -0,0 +1,580 @@
+/**
+ * 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.server
+
+import java.util.{Optional, Properties}
+import java.util.concurrent.ExecutionException
+
+import kafka.api.KAFKA_2_7_IV0
+import kafka.utils.TestUtils
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion}
+import kafka.utils.TestUtils.waitUntilTrue
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureUpdate, UpdateFeaturesOptions, UpdateFeaturesResult}
+import org.apache.kafka.common.errors.InvalidRequestException
+import org.apache.kafka.common.feature.FinalizedVersionRange
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.message.UpdateFeaturesRequestData
+import org.apache.kafka.common.message.UpdateFeaturesRequestData.FeatureUpdateKeyCollection
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{UpdateFeaturesRequest, UpdateFeaturesResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.Test
+import org.junit.Assert.{assertEquals, assertFalse, assertNotEquals, assertNotNull, assertTrue}
+import org.scalatest.Assertions.intercept
+
+import scala.jdk.CollectionConverters._
+import scala.reflect.ClassTag
+import scala.util.matching.Regex
+
+class UpdateFeaturesTest extends BaseRequestTest {
+
+  override def brokerCount = 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))))
+  }
+
+  private def defaultFinalizedFeatures(): Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2))))
+  }
+
+  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)
+  }
+
+  private def updateFeatureZNode(features: Features[FinalizedVersionRange]): Int = {
+    val server = serverForId(0).get
+    val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, features)
+    val newVersion = server.zkClient.updateFeatureZNode(newNode)
+    servers.foreach(s => {
+      s.featureCache.waitUntilEpochOrThrow(newVersion, s.config.zkConnectionTimeoutMs)
+    })
+    newVersion
+  }
+
+  private def getFeatureZNode(): FeatureZNode = {
+    val (mayBeFeatureZNodeBytes, version) = serverForId(0).get.zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(version, ZkVersion.UnknownVersion)
+    FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+  }
+
+  private def finalizedFeatures(features: java.util.Map[String, org.apache.kafka.clients.admin.FinalizedVersionRange]): Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(features.asScala.map {
+      case(name, versionRange) =>
+        (name, new FinalizedVersionRange(versionRange.minVersionLevel(), versionRange.maxVersionLevel()))
+    }.asJava)
+  }
+
+  private def supportedFeatures(features: java.util.Map[String, org.apache.kafka.clients.admin.SupportedVersionRange]): Features[SupportedVersionRange] = {
+    Features.supportedFeatures(features.asScala.map {
+      case(name, versionRange) =>
+        (name, new SupportedVersionRange(versionRange.minVersion(), versionRange.maxVersion()))
+    }.asJava)
+  }
+
+  private def checkFeatures(client: Admin,
+                            expectedNode: FeatureZNode,
+                            expectedFinalizedFeatures: Features[FinalizedVersionRange],
+                            expectedFinalizedFeaturesEpoch: Long,
+                            expectedSupportedFeatures: Features[SupportedVersionRange]): Unit = {
+    assertEquals(expectedNode, getFeatureZNode())
+    val featureMetadata = client.describeFeatures(
+      new DescribeFeaturesOptions().sendRequestToController(true)).featureMetadata.get
+    assertEquals(expectedFinalizedFeatures, finalizedFeatures(featureMetadata.finalizedFeatures))
+    assertEquals(expectedSupportedFeatures, supportedFeatures(featureMetadata.supportedFeatures))
+    assertEquals(Optional.of(expectedFinalizedFeaturesEpoch), featureMetadata.finalizedFeaturesEpoch)
+  }
+
+  private def checkException[ExceptionType <: Throwable](result: UpdateFeaturesResult,
+                                                         featureExceptionMsgPatterns: Map[String, Regex])
+                                                        (implicit tag: ClassTag[ExceptionType]): Unit = {
+    featureExceptionMsgPatterns.foreach {
+      case (feature, exceptionMsgPattern) =>
+        val exception = intercept[ExecutionException] {
+          result.values().get(feature).get()
+        }
+        val cause = exception.getCause
+        assertNotNull(cause)
+        assertEquals(cause.getClass, tag.runtimeClass)
+        assertTrue(s"Received unexpected error message: ${cause.getMessage}",
+                   exceptionMsgPattern.findFirstIn(cause.getMessage).isDefined)
+    }
+  }
+
+  /**
+   * Tests whether an invalid feature update does not get processed on the server as expected,
+   * and raises the ExceptionType on the client side as expected.
+   *
+   * @param invalidUpdate         the invalid feature update to be sent in the
+   *                              updateFeatures request to the server
+   * @param exceptionMsgPattern   a pattern for the expected exception message
+   */
+  private def testWithInvalidFeatureUpdate[ExceptionType <: Throwable](feature: String,
+                                                                       invalidUpdate: FeatureUpdate,
+                                                                       exceptionMsgPattern: Regex)
+                                                                      (implicit tag: ClassTag[ExceptionType]): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+    val adminClient = createAdminClient()
+    val nodeBefore = getFeatureZNode()
+
+    val result = adminClient.updateFeatures(Utils.mkMap(Utils.mkEntry(feature, invalidUpdate)), new UpdateFeaturesOptions())
+
+    checkException[ExceptionType](result, Map(feature -> exceptionMsgPattern))
+    checkFeatures(
+      adminClient,
+      nodeBefore,
+      defaultFinalizedFeatures(),
+      versionBefore,
+      defaultSupportedFeatures())
+  }
+
+  /**
+   * Tests that an UpdateFeatures request sent to a non-Controller node fails as expected.
+   */
+  @Test
+  def testShouldFailRequestIfNotController(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val nodeBefore = getFeatureZNode()
+    val validUpdates = new FeatureUpdateKeyCollection()
+    val validUpdate = new UpdateFeaturesRequestData.FeatureUpdateKey();
+    validUpdate.setFeature("feature_1");
+    validUpdate.setMaxVersionLevel(defaultSupportedFeatures().get("feature_1").max())
+    validUpdate.setAllowDowngrade(false)
+    validUpdates.add(validUpdate)
+
+    val response = connectAndReceive[UpdateFeaturesResponse](
+      new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData().setFeatureUpdates(validUpdates)).build(),
+      notControllerSocketServer)
+
+    assertEquals(Errors.NOT_CONTROLLER, Errors.forCode(response.data.errorCode()))
+    assertNotNull(response.data.errorMessage())
+    assertEquals(0, response.data.results.size)
+    checkFeatures(
+      createAdminClient(),
+      nodeBefore,
+      defaultFinalizedFeatures(),
+      versionBefore,
+      defaultSupportedFeatures())
+  }
+
+  /**
+   * Tests that an UpdateFeatures request fails in the Controller, when, for a feature the
+   * allowDowngrade flag is not set during a downgrade request.
+   */
+  @Test
+  def testShouldFailRequestWhenDowngradeFlagIsNotSetDuringDowngrade(): Unit = {
+    val targetMaxVersionLevel = (defaultFinalizedFeatures().get("feature_1").max() - 1).asInstanceOf[Short]
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      "feature_1",
+      new FeatureUpdate(targetMaxVersionLevel,false),
+      ".*Can not downgrade finalized feature.*allowDowngrade.*".r)
+  }
+
+  /**
+   * Tests that an UpdateFeatures request fails in the Controller, when, for a feature the downgrade
+   * is attempted to a max version level thats higher than the existing max version level.
+   */
+  @Test
+  def testShouldFailRequestWhenDowngradeToHigherVersionLevelIsAttempted(): Unit = {
+    val targetMaxVersionLevel = (defaultFinalizedFeatures().get("feature_1").max() + 1).asInstanceOf[Short]
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      "feature_1",
+      new FeatureUpdate(targetMaxVersionLevel, true),
+      ".*When the allowDowngrade flag set in the request, the provided maxVersionLevel:3.*existing maxVersionLevel:2.*".r)
+  }
+
+  /**
+   * Tests that an UpdateFeatures request fails in the Controller, when, a feature deletion is
+   * attempted without setting the allowDowngrade flag.
+   */
+  @Test
+  def testShouldFailRequestInServerWhenDowngradeFlagIsNotSetDuringDeletion(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val adminClient = createAdminClient()
+    val nodeBefore = getFeatureZNode()
+
+    val invalidUpdates
+      = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection();
+    val invalidUpdate = new UpdateFeaturesRequestData.FeatureUpdateKey();
+    invalidUpdate.setFeature("feature_1")
+    invalidUpdate.setMaxVersionLevel(0)
+    invalidUpdate.setAllowDowngrade(false)
+    invalidUpdates.add(invalidUpdate);
+    val requestData = new UpdateFeaturesRequestData()
+    requestData.setFeatureUpdates(invalidUpdates);
+
+    val response = connectAndReceive[UpdateFeaturesResponse](
+      new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData().setFeatureUpdates(invalidUpdates)).build(),
+      controllerSocketServer)
+
+    assertEquals(1, response.data().results().size())
+    val result = response.data.results.asScala.head
+    assertEquals("feature_1", result.feature)
+    assertEquals(Errors.INVALID_REQUEST, Errors.forCode(result.errorCode))
+    assertNotNull(result.errorMessage)
+    assertFalse(result.errorMessage.isEmpty)
+    val exceptionMsgPattern = ".*Can not provide maxVersionLevel: 0 less than 1.*allowDowngrade.*".r
+    assertTrue(result.errorMessage, exceptionMsgPattern.findFirstIn(result.errorMessage).isDefined)
+    checkFeatures(
+      adminClient,
+      nodeBefore,
+      defaultFinalizedFeatures(),
+      versionBefore,
+      defaultSupportedFeatures())
+  }
+
+  /**
+   * Tests that an UpdateFeatures request fails in the Controller, when, a feature version level
+   * upgrade is attempted for a non-existing feature.
+   */
+  @Test
+  def testShouldFailRequestDuringDeletionOfNonExistingFeature(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      "feature_non_existing",
+      new FeatureUpdate(0, true),

Review comment:
       Done. Good point.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -20,26 +20,31 @@ package kafka.server
 import kafka.utils.Logging
 import org.apache.kafka.common.feature.{Features, FinalizedVersionRange}
 
+import scala.concurrent.TimeoutException
+import scala.math.max
+
 // Raised whenever there was an error in updating the FinalizedFeatureCache with features.
 class FeatureCacheUpdateException(message: String) extends RuntimeException(message) {
 }
 
 // Helper class that represents finalized features along with an epoch value.
-case class FinalizedFeaturesAndEpoch(features: Features[FinalizedVersionRange], epoch: Int) {
+case class FinalizedFeaturesAndEpoch(features: Features[FinalizedVersionRange], epoch: Long) {

Review comment:
       We would like to avoid overflow issues once ZK is gone in the future. This change is being done based on Colin's suggestion in the KIP-584 voting thread:
    - [Here](https://lists.apache.org/thread.html/r0dddbe01e2d6991310006b90aa5c9db011461f516f345f621c1f8171%40%3Cdev.kafka.apache.org%3E) is Colin's comment
    - [Here](https://lists.apache.org/thread.html/rf7fb6a033638c43a338be5cc316e9e69df6e2589fab66b69d8b67f0f%40%3Cdev.kafka.apache.org%3E) is my response




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1865,192 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns an ApiError object containing Errors#INVALID_REQUEST.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+
+    val incompatibilityError = "Could not apply finalized feature update because" +
+      " brokers were found to have incompatible versions for the feature."
+
+    if (brokerFeatures.supportedFeatures.get(update.feature()) == null) {

Review comment:
       It's required because `defaultMinVersionLevel` does not exist for a feature that's not in the supported list. However, I'll change the code to make the check more obvious to the reader (currently it's not).




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java
##########
@@ -17,9 +17,16 @@
 package org.apache.kafka.common.feature;
 
 import java.util.Map;
+import java.util.Objects;
+import org.apache.kafka.common.utils.Utils;
 
 /**
- * An extended {@link BaseVersionRange} representing the min/max versions for supported features.
+ * An extended {@link BaseVersionRange} representing the min, max and first active versions for a
+ * supported feature:
+ *  - minVersion: This is the minimum supported version for the feature.
+ *  - maxVersion: This the maximum supported version for the feature.
+ *  - firstActiveVersion: This is the first active version for the feature. Versions in the range

Review comment:
       @junrao My response below.
   
   > I was thinking what if we relax the current check by just making sure that maxVersion of finalized is within the supported range. Basically in your example, if supported minVersion goes to 2, it's still allowed since it's less than maxVersion of finalized. However, if supported minVersion goes to 7, this fails the broker since it's more than maxVersion of finalized.
   
   If we relaxed the current check like you suggested, then, as of today the following will happen:
   The controller can not effectively finalize `minVersionLevel` for the feature, because, with a relaxed check we do not know whether all brokers in the cluster support a particular `minVersion` when the controller finalizes the `minVersionLevel` at a particular value.
   
   Supposing we want to keep the concept of `minVersionLevel` like the way it is now (i.e. it tells you the lowest feature version level guaranteed to be supported by any broker in the cluster), then, `firstActiveVersion` provides the door to mutate it safely whenever the need arises.
   
   So when thinking about your suggestion, it is not clear to me if you are intending to alter the meaning of `minVersionLevel`? If so, please could you tell me what is the new meaning of `minVersionLevel` that you are proposing?
   
   > Your concern for the relaxed check seems to be around deploying a wrong version of the broker by mistake.
   
   Not exactly, my concern is a bit different. It is to be noted that `minVersionLevel` can not be changed through the `UPDATE_FEATURES` api. Assuming we want to maintain the meaning of `minVersionLevel` the way it is today, my concern is about how would one alter `minVersionLevel` for a feature when the need arises? This needs to be done without letting the versioning system misconstrue the intent with a feature version incompatibility. This is where usage of `firstActiveVersion` acts as a solution.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1865,192 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns an ApiError object containing Errors#INVALID_REQUEST.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+
+    val incompatibilityError = "Could not apply finalized feature update because" +
+      " brokers were found to have incompatible versions for the feature."
+
+    if (brokerFeatures.supportedFeatures.get(update.feature()) == null) {
+      Right(new ApiError(Errors.INVALID_REQUEST, incompatibilityError))
+    } else {
+      val defaultMinVersionLevel = brokerFeatures.defaultMinVersionLevel(update.feature)
+      val newVersionRange = new FinalizedVersionRange(defaultMinVersionLevel, update.maxVersionLevel)

Review comment:
       Done. This is fixed now.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1844,185 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns a suitable error.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+    // NOTE: Below we set the finalized min version level to be the default minimum version
+    // level. If the finalized feature already exists, then, this can cause deprecation of all
+    // version levels in the closed range:
+    // [existingVersionRange.min(), defaultMinVersionLevel - 1].
+    val defaultMinVersionLevel = brokerFeatures.defaultMinVersionLevel(update.feature)
+    val newVersionRange = new FinalizedVersionRange(defaultMinVersionLevel, update.maxVersionLevel)
+    val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+      val singleFinalizedFeature =
+        Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange)))
+      BrokerFeatures.hasIncompatibleFeatures(broker.features, singleFinalizedFeature)
+    })
+    if (numIncompatibleBrokers == 0) {
+      Left(newVersionRange)
+    } else {
+      Right(
+        new ApiError(Errors.INVALID_REQUEST,
+                     s"Could not apply finalized feature update because $numIncompatibleBrokers" +
+                     " brokers were found to have incompatible features."))
+    }
+  }
+
+  /**
+   * Validate and process a finalized feature update on an existing FinalizedVersionRange for the
+   * feature.
+   *
+   * If the processing is successful, then, the return value contains:
+   * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature.
+   * 2. Option.empty, if the feature update was meant to delete the feature.
+   *
+   * If the processing failed, then returned value contains a suitable ApiError.
+   *
+   * @param update                 the feature update to be processed.
+   * @param existingVersionRange   the existing FinalizedVersionRange which can be empty when no
+   *                               FinalizedVersionRange exists for the associated feature
+   *
+   * @return                       the new FinalizedVersionRange or error, as described above.
+   */
+  private def processFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey,
+                                   existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = {
+    def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = {
+      newFinalizedVersionRangeOrIncompatibilityError(update)
+        .fold(versionRange => Left(Some(versionRange)), error => Right(error))
+    }
+
+    if (update.feature.isEmpty) {
+      // Check that the feature name is not empty.
+      Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty."))
+    } else {
+      // We handle deletion requests separately from non-deletion requests.
+      if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+        if (existingVersionRange.isEmpty) {
+          // Disallow deletion of a non-existing finalized feature.
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             s"Can not delete non-existing finalized feature: '${update.feature}'"))
+        } else {
+          Left(Option.empty)
+        }
+      } else if (update.maxVersionLevel() < 1) {

Review comment:
       It is for returning the special error message that we handle it here explicitly: `...less than 1 for feature...`.
   A value < 1 is indicative of a deletion request (not purely a downgrade request).




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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


   @junrao Thanks for the review! I've addressed the latest comments in e55358fd1a00f12ef98fc4d2d649a297ddf146da .


----------------------------------------------------------------
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] chia7712 commented on a change in pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4335,6 +4343,150 @@ void handleFailure(Throwable throwable) {
                 .hi(password, salt, iterations);
     }
 
+    public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        final NodeProvider provider =
+            options.sendRequestToController() ? new ControllerNodeProvider() : new LeastLoadedNodeProvider();
+
+        final Call call = new Call(
+            "describeFeatures", calcDeadlineMs(now, options.timeoutMs()), provider) {
+
+            private FeatureMetadata createFeatureMetadata(final ApiVersionsResponse response) {
+                final Map<String, FinalizedVersionRange> finalizedFeatures = new HashMap<>();
+                for (final FinalizedFeatureKey key : response.data().finalizedFeatures().valuesSet()) {
+                    finalizedFeatures.put(key.name(), new FinalizedVersionRange(key.minVersionLevel(), key.maxVersionLevel()));
+                }
+
+                Optional<Long> finalizedFeaturesEpoch;
+                if (response.data().finalizedFeaturesEpoch() >= 0L) {
+                    finalizedFeaturesEpoch = Optional.of(response.data().finalizedFeaturesEpoch());
+                } else {
+                    finalizedFeaturesEpoch = Optional.empty();
+                }
+
+                final Map<String, SupportedVersionRange> supportedFeatures = new HashMap<>();
+                for (final SupportedFeatureKey key : response.data().supportedFeatures().valuesSet()) {
+                    supportedFeatures.put(key.name(), new SupportedVersionRange(key.minVersion(), key.maxVersion()));
+                }
+
+                return new FeatureMetadata(finalizedFeatures, finalizedFeaturesEpoch, supportedFeatures);
+            }
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(createFeatureMetadata(apiVersionsResponse));
+                } else if (options.sendRequestToController() &&
+                           apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) {
+                    handleNotControllerError(Errors.NOT_CONTROLLER);
+                } else {
+                    future.completeExceptionally(Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        runnable.call(call, now);
+        return new DescribeFeaturesResult(future);
+    }
+
+    @Override
+    public UpdateFeaturesResult updateFeatures(final Map<String, FeatureUpdate> featureUpdates,
+                                               final UpdateFeaturesOptions options) {
+        if (featureUpdates.isEmpty()) {
+            throw new IllegalArgumentException("Feature updates can not be null or empty.");
+        }
+
+        final Map<String, KafkaFutureImpl<Void>> updateFutures = new HashMap<>();
+        for (final Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+            updateFutures.put(entry.getKey(), new KafkaFutureImpl<>());
+        }
+
+        final long now = time.milliseconds();
+        final Call call = new Call("updateFeatures", calcDeadlineMs(now, options.timeoutMs()),
+            new ControllerNodeProvider()) {
+
+            @Override
+            UpdateFeaturesRequest.Builder createRequest(int timeoutMs) {
+                final UpdateFeaturesRequestData.FeatureUpdateKeyCollection featureUpdatesRequestData
+                    = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection();
+                for (Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+                    final String feature = entry.getKey();
+                    final FeatureUpdate update = entry.getValue();
+                    if (feature.trim().isEmpty()) {
+                        throw new IllegalArgumentException("Provided feature can not be null or empty.");
+                    }
+
+                    final UpdateFeaturesRequestData.FeatureUpdateKey requestItem =
+                        new UpdateFeaturesRequestData.FeatureUpdateKey();
+                    requestItem.setFeature(feature);
+                    requestItem.setMaxVersionLevel(update.maxVersionLevel());
+                    requestItem.setAllowDowngrade(update.allowDowngrade());
+                    featureUpdatesRequestData.add(requestItem);
+                }
+                return new UpdateFeaturesRequest.Builder(
+                    new UpdateFeaturesRequestData()
+                        .setTimeoutMs(timeoutMs)
+                        .setFeatureUpdates(featureUpdatesRequestData));
+            }
+
+            @Override
+            void handleResponse(AbstractResponse abstractResponse) {
+                final UpdateFeaturesResponse response =
+                    (UpdateFeaturesResponse) abstractResponse;
+
+                Errors topLevelError = Errors.forCode(response.data().errorCode());
+                switch (topLevelError) {
+                    case NONE:
+                        for (final UpdatableFeatureResult result : response.data().results()) {
+                            final KafkaFutureImpl<Void> future = updateFutures.get(result.feature());
+                            if (future == null) {
+                                log.warn("Server response mentioned unknown feature {}", result.feature());
+                            } else {
+                                final Errors error = Errors.forCode(result.errorCode());
+                                if (error == Errors.NONE) {
+                                    future.complete(null);
+                                } else {
+                                    future.completeExceptionally(error.exception(result.errorMessage()));
+                                }
+                            }
+                        }
+                        // The server should send back a response for every feature, but we do a sanity check anyway.
+                        completeUnrealizedFutures(updateFutures.entrySet().stream(),
+                            feature -> "The controller response did not contain a result for feature " + feature);
+                        break;
+                    case NOT_CONTROLLER:
+                        handleNotControllerError(topLevelError);
+                        break;
+                    default:
+                        for (final Map.Entry<String, KafkaFutureImpl<Void>> entry : updateFutures.entrySet()) {
+                            entry.getValue().completeExceptionally(topLevelError.exception());

Review comment:
       the top-level error message is not propagated.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4335,6 +4343,150 @@ void handleFailure(Throwable throwable) {
                 .hi(password, salt, iterations);
     }
 
+    public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        final NodeProvider provider =
+            options.sendRequestToController() ? new ControllerNodeProvider() : new LeastLoadedNodeProvider();
+
+        final Call call = new Call(
+            "describeFeatures", calcDeadlineMs(now, options.timeoutMs()), provider) {
+
+            private FeatureMetadata createFeatureMetadata(final ApiVersionsResponse response) {
+                final Map<String, FinalizedVersionRange> finalizedFeatures = new HashMap<>();
+                for (final FinalizedFeatureKey key : response.data().finalizedFeatures().valuesSet()) {
+                    finalizedFeatures.put(key.name(), new FinalizedVersionRange(key.minVersionLevel(), key.maxVersionLevel()));
+                }
+
+                Optional<Long> finalizedFeaturesEpoch;
+                if (response.data().finalizedFeaturesEpoch() >= 0L) {
+                    finalizedFeaturesEpoch = Optional.of(response.data().finalizedFeaturesEpoch());
+                } else {
+                    finalizedFeaturesEpoch = Optional.empty();
+                }
+
+                final Map<String, SupportedVersionRange> supportedFeatures = new HashMap<>();
+                for (final SupportedFeatureKey key : response.data().supportedFeatures().valuesSet()) {
+                    supportedFeatures.put(key.name(), new SupportedVersionRange(key.minVersion(), key.maxVersion()));
+                }
+
+                return new FeatureMetadata(finalizedFeatures, finalizedFeaturesEpoch, supportedFeatures);
+            }
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(createFeatureMetadata(apiVersionsResponse));
+                } else if (options.sendRequestToController() &&
+                           apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) {
+                    handleNotControllerError(Errors.NOT_CONTROLLER);
+                } else {
+                    future.completeExceptionally(Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        runnable.call(call, now);
+        return new DescribeFeaturesResult(future);
+    }
+
+    @Override
+    public UpdateFeaturesResult updateFeatures(final Map<String, FeatureUpdate> featureUpdates,
+                                               final UpdateFeaturesOptions options) {
+        if (featureUpdates.isEmpty()) {
+            throw new IllegalArgumentException("Feature updates can not be null or empty.");
+        }
+
+        final Map<String, KafkaFutureImpl<Void>> updateFutures = new HashMap<>();
+        for (final Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+            updateFutures.put(entry.getKey(), new KafkaFutureImpl<>());
+        }
+
+        final long now = time.milliseconds();
+        final Call call = new Call("updateFeatures", calcDeadlineMs(now, options.timeoutMs()),
+            new ControllerNodeProvider()) {
+
+            @Override
+            UpdateFeaturesRequest.Builder createRequest(int timeoutMs) {
+                final UpdateFeaturesRequestData.FeatureUpdateKeyCollection featureUpdatesRequestData
+                    = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection();
+                for (Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+                    final String feature = entry.getKey();
+                    final FeatureUpdate update = entry.getValue();
+                    if (feature.trim().isEmpty()) {

Review comment:
       the error message says it can't be null but there is no null check.
   for another, this check can happen early (when creating ```updateFutures```)




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1214,70 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker, but it can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the DescribeFeaturesResult containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);
+
+    /**
+     * Applies specified updates to finalized features. The API is atomic, meaning that if a single
+     * feature update in the request can't succeed on the controller, then none of the feature
+     * updates are carried out. This request is issued only to the controller since the API is
+     * only served by the controller.
+     * <p>
+     * The API takes as input a set of FinalizedFeatureUpdate that need to be applied. Each such
+     * update specifies the finalized feature to be added or updated or deleted, along with the new
+     * max feature version level value.
+     * <ul>
+     * <li>Downgrade of feature version level is not a regular operation/intent. It is only allowed
+     * in the controller if the feature update has the allowDowngrade flag set - setting this flag
+     * conveys user intent to attempt downgrade of a feature max version level. Note that despite
+     * the allowDowngrade flag being set, certain downgrades may be rejected by the controller if it
+     * is deemed impossible.</li>
+     * <li>Deletion of a finalized feature version is not a regular operation/intent. It is allowed
+     * only if the allowDowngrade flag is set in the feature update, and, if the max version level
+     * is set to a value less than 1.</li>
+     * </ul>
+     *<p>
+     * The following exceptions can be anticipated when calling {@code get()} on the futures
+     * obtained from the returned {@link UpdateFinalizedFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.ClusterAuthorizationException}
+     *   If the authenticated user didn't have alter access to the cluster.</li>
+     *   <li>{@link org.apache.kafka.common.errors.InvalidRequestException}
+     *   If the request details are invalid. e.g., a non-existing finalized feature is attempted
+     *   to be deleted or downgraded.</li>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the updates could finish. It cannot be guaranteed whether
+     *   the updates succeeded or not.</li>
+     *   <li>{@link org.apache.kafka.common.errors.FinalizedFeatureUpdateFailedException}
+     *   If the updates could not be applied on the controller, despite the request being valid.
+     *   This may be a temporary problem.</li>
+     * </ul>
+     * <p>
+     * This operation is supported by brokers with version 2.7.0 or higher.
+
+     * @param featureUpdates   the set of finalized feature updates
+     * @param options          the options to use
+     *
+     * @return                 the UpdateFinalizedFeaturesResult containing the result
+     */
+    UpdateFinalizedFeaturesResult updateFinalizedFeatures(

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java
##########
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+import org.apache.kafka.common.feature.Features;
+import org.apache.kafka.common.feature.FinalizedVersionRange;
+import org.apache.kafka.common.feature.SupportedVersionRange;
+
+/**
+ * Encapsulates details about finalized as well as supported features. This is particularly useful
+ * to hold the result returned by the {@link Admin#describeFeatures(DescribeFeaturesOptions)} API.
+ */
+public class FeatureMetadata {
+
+    private final Features<FinalizedVersionRange> finalizedFeatures;
+
+    private final int finalizedFeaturesEpoch;
+
+    private final Features<SupportedVersionRange> supportedFeatures;
+
+    public FeatureMetadata(
+        final Features<FinalizedVersionRange> finalizedFeatures,
+        final int finalizedFeaturesEpoch,
+        final Features<SupportedVersionRange> supportedFeatures
+    ) {
+        Objects.requireNonNull(finalizedFeatures, "Provided finalizedFeatures can not be null.");
+        Objects.requireNonNull(supportedFeatures, "Provided supportedFeatures can not be null.");
+        this.finalizedFeatures = finalizedFeatures;
+        this.finalizedFeaturesEpoch = finalizedFeaturesEpoch;
+        this.supportedFeatures = supportedFeatures;
+    }
+
+    /**
+     * A map of finalized feature versions, with key being finalized feature name and value
+     * containing the min/max version levels for the finalized feature.
+     */
+    public Features<FinalizedVersionRange> finalizedFeatures() {
+        return finalizedFeatures;
+    }
+
+    /**
+     * The epoch for the finalized features.
+     * Valid values are >= 0. A value < 0 means the finalized features are absent/unavailable.

Review comment:
       Done. Good point.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1214,70 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker, but it can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the DescribeFeaturesResult containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);
+
+    /**
+     * Applies specified updates to finalized features. The API is atomic, meaning that if a single
+     * feature update in the request can't succeed on the controller, then none of the feature
+     * updates are carried out. This request is issued only to the controller since the API is
+     * only served by the controller.
+     * <p>
+     * The API takes as input a set of FinalizedFeatureUpdate that need to be applied. Each such
+     * update specifies the finalized feature to be added or updated or deleted, along with the new
+     * max feature version level value.
+     * <ul>
+     * <li>Downgrade of feature version level is not a regular operation/intent. It is only allowed
+     * in the controller if the feature update has the allowDowngrade flag set - setting this flag
+     * conveys user intent to attempt downgrade of a feature max version level. Note that despite
+     * the allowDowngrade flag being set, certain downgrades may be rejected by the controller if it
+     * is deemed impossible.</li>
+     * <li>Deletion of a finalized feature version is not a regular operation/intent. It is allowed
+     * only if the allowDowngrade flag is set in the feature update, and, if the max version level
+     * is set to a value less than 1.</li>
+     * </ul>
+     *<p>
+     * The following exceptions can be anticipated when calling {@code get()} on the futures
+     * obtained from the returned {@link UpdateFinalizedFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.ClusterAuthorizationException}
+     *   If the authenticated user didn't have alter access to the cluster.</li>
+     *   <li>{@link org.apache.kafka.common.errors.InvalidRequestException}
+     *   If the request details are invalid. e.g., a non-existing finalized feature is attempted

Review comment:
       To your point, this information is available in the error message returned in the response.
   
   The feature updates are atomically applied to ZK by the controller i.e it is all or none. We don't have a use case (yet) where we have to programmatically learn which feature updates are incorrect. Instead an error message with details seems sufficient to us. Please let me know how you feel about it, and if you feel that we are better off in returning per-feature-update error code. This was discussed in the [KIP-584 thread](https://lists.apache.org/thread.html/r0b04fa38fb3c4a33bc6b6419284f557266e662c9840a2f7661401f30%40%3Cdev.kafka.apache.org%3E), search for the word "transaction".

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1214,70 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker, but it can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the DescribeFeaturesResult containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);
+
+    /**
+     * Applies specified updates to finalized features. The API is atomic, meaning that if a single
+     * feature update in the request can't succeed on the controller, then none of the feature
+     * updates are carried out. This request is issued only to the controller since the API is
+     * only served by the controller.
+     * <p>
+     * The API takes as input a set of FinalizedFeatureUpdate that need to be applied. Each such
+     * update specifies the finalized feature to be added or updated or deleted, along with the new
+     * max feature version level value.
+     * <ul>
+     * <li>Downgrade of feature version level is not a regular operation/intent. It is only allowed
+     * in the controller if the feature update has the allowDowngrade flag set - setting this flag
+     * conveys user intent to attempt downgrade of a feature max version level. Note that despite
+     * the allowDowngrade flag being set, certain downgrades may be rejected by the controller if it
+     * is deemed impossible.</li>
+     * <li>Deletion of a finalized feature version is not a regular operation/intent. It is allowed

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -177,15 +214,33 @@ public static ApiVersionsResponse createApiVersionsResponse(
             }
         }
 
+        return new ApiVersionsResponse(
+            createApiVersionsResponseData(
+                throttleTimeMs,
+                Errors.NONE,
+                apiKeys,
+                latestSupportedFeatures,
+                finalizedFeatures,
+                finalizedFeaturesEpoch));
+    }
+
+    public static ApiVersionsResponseData createApiVersionsResponseData(

Review comment:
       It calls into couple other helper functions.

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature
+ *    version level deprecation. This is how it works: in order to deprecate feature version levels,
+ *    in this map the default minimum version level of a feature can be set to a new value that's
+ *    higher than 1 (let's call this new_min_version_level). In doing so, the feature version levels
+ *    in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic
+ *    that applies this map to persistent finalized feature state in ZK (this mutation happens
+ *    during controller election and during finalized feature updates via the
+ *    APIKeys.UPDATE_FINALIZED_FEATURES api).
+ *
+ * This class also provides APIs to check for incompatibilities between the features supported by
+ * the Broker and finalized features.
+ *
+ * NOTE: the update*() and clear*() APIs of this class should be used only for testing purposes.
+ */
+class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange],
+                              @volatile var defaultFeatureMinVersionLevels: Map[String, Short]) {
+  require(BrokerFeatures.areFeatureMinVersionLevelsCompatible(
+    supportedFeatures, defaultFeatureMinVersionLevels))
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(newFeatures, defaultFeatureMinVersionLevels))
+    supportedFeatures = newFeatures
+  }
+
+  /**
+   * Returns the default minimum version level for a specific feature.
+   *
+   * @param feature   the name of the feature
+   *
+   * @return          the default minimum version level for the feature if its defined.
+   *                  otherwise, returns 1.
+   */
+  def defaultMinVersionLevel(feature: String): Short = {
+    defaultFeatureMinVersionLevels.getOrElse(feature, 1)
+  }
+
+  // For testing only.
+  def setDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(supportedFeatures, newMinVersionLevels))
+    defaultFeatureMinVersionLevels = newMinVersionLevels
+  }
+
+  /**
+   * Returns the default finalized features that a new Kafka cluster with IBP config >= KAFKA_2_7_IV0
+   * needs to be bootstrapped with.
+   */
+  def getDefaultFinalizedFeatures: Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(
+      supportedFeatures.features.asScala.map {
+        case(name, versionRange) => (
+          name, new FinalizedVersionRange(defaultMinVersionLevel(name), versionRange.max))
+      }.asJava)
+  }
+
+  /**
+   * Returns the set of feature names found to be 'incompatible'.
+   * A feature incompatibility is a version mismatch between the latest feature supported by the
+   * Broker, and the provided finalized feature. This can happen because a provided finalized
+   * feature:
+   *  1) Does not exist in the Broker (i.e. it is unknown to the Broker).
+   *           [OR]
+   *  2) Exists but the FinalizedVersionRange does not match with the
+   *     supported feature's SupportedVersionRange.
+   *
+   * @param finalized   The finalized features against which incompatibilities need to be checked for.
+   *
+   * @return            The subset of input features which are incompatible. If the returned object
+   *                    is empty, it means there were no feature incompatibilities found.
+   */
+  def incompatibleFeatures(finalized: Features[FinalizedVersionRange]): Features[FinalizedVersionRange] = {
+    BrokerFeatures.incompatibleFeatures(supportedFeatures, finalized, true)
+  }
+}
+
+object BrokerFeatures extends Logging {
+
+  def createDefault(): BrokerFeatures = {
+    // The arguments are currently empty, but, in the future as we define features we should
+    // populate the required values here.
+    new BrokerFeatures(emptySupportedFeatures, Map[String, Short]())
+  }
+
+  /**
+   * Returns true if any of the provided finalized features are incompatible with the provided
+   * supported features.
+   *
+   * @param supportedFeatures   The supported features to be compared
+   * @param finalizedFeatures   The finalized features to be compared
+   *
+   * @return                    - True if there are any feature incompatibilities found.
+   *                            - False otherwise.
+   */
+  def hasIncompatibleFeatures(supportedFeatures: Features[SupportedVersionRange],
+                              finalizedFeatures: Features[FinalizedVersionRange]): Boolean = {
+    !incompatibleFeatures(supportedFeatures, finalizedFeatures, false).empty
+  }
+
+  private def incompatibleFeatures(supportedFeatures: Features[SupportedVersionRange],
+                                   finalizedFeatures: Features[FinalizedVersionRange],
+                                   logIncompatibilities: Boolean): Features[FinalizedVersionRange] = {
+    val incompatibilities = finalizedFeatures.features.asScala.map {
+      case (feature, versionLevels) => {
+        val supportedVersions = supportedFeatures.get(feature)
+        if (supportedVersions == null) {
+          (feature, versionLevels, "{feature=%s, reason='Unsupported feature'}".format(feature))
+        } else if (versionLevels.isIncompatibleWith(supportedVersions)) {
+          (feature, versionLevels, "{feature=%s, reason='%s is incompatible with %s'}".format(
+            feature, versionLevels, supportedVersions))
+        } else {
+          (feature, versionLevels, null)
+        }
+      }
+    }.filter{ case(_, _, errorReason) => errorReason != null}.toList
+
+    if (logIncompatibilities && incompatibilities.nonEmpty) {
+      warn(
+        "Feature incompatibilities seen: " + incompatibilities.map{
+          case(_, _, errorReason) => errorReason })
+    }
+    Features.finalizedFeatures(incompatibilities.map{
+      case(feature, versionLevels, _) => (feature, versionLevels) }.toMap.asJava)
+  }
+
+  /**
+   * A check that ensures each feature defined with min version level is a supported feature, and
+   * the min version level value is valid (i.e. it is compatible with the supported version range).
+   *
+   * @param supportedFeatures         the supported features
+   * @param featureMinVersionLevels   the feature minimum version levels
+   *
+   * @return                          - true, if the above described check passes.
+   *                                  - false, otherwise.
+   */
+  private def areFeatureMinVersionLevelsCompatible(
+    supportedFeatures: Features[SupportedVersionRange],
+    featureMinVersionLevels: Map[String, Short]
+  ): Boolean = {
+    featureMinVersionLevels.forall {
+      case(featureName, minVersionLevel) => {

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,158 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * Enables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with enabled status. This status means the feature versioning system
+   * (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode are active. This
+   * status should be written by the controller to the FeatureZNode only when the broker IBP config
+   * is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    For a new Kafka cluster (i.e. it is deployed first time), we would like to start the cluster
+   *    with all the possible supported features finalized immediately. The new cluster will almost
+   *    never be started with an old IBP config that’s less than KAFKA_2_7_IV0. Assuming this is the
+   *    case, then here is how we it: the controller will start up and notice that the FeatureZNode

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature
+ *    version level deprecation. This is how it works: in order to deprecate feature version levels,
+ *    in this map the default minimum version level of a feature can be set to a new value that's
+ *    higher than 1 (let's call this new_min_version_level). In doing so, the feature version levels
+ *    in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic
+ *    that applies this map to persistent finalized feature state in ZK (this mutation happens
+ *    during controller election and during finalized feature updates via the
+ *    APIKeys.UPDATE_FINALIZED_FEATURES api).
+ *
+ * This class also provides APIs to check for incompatibilities between the features supported by
+ * the Broker and finalized features.
+ *
+ * NOTE: the update*() and clear*() APIs of this class should be used only for testing purposes.
+ */
+class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange],
+                              @volatile var defaultFeatureMinVersionLevels: Map[String, Short]) {
+  require(BrokerFeatures.areFeatureMinVersionLevelsCompatible(
+    supportedFeatures, defaultFeatureMinVersionLevels))
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(newFeatures, defaultFeatureMinVersionLevels))
+    supportedFeatures = newFeatures
+  }
+
+  /**
+   * Returns the default minimum version level for a specific feature.
+   *
+   * @param feature   the name of the feature
+   *
+   * @return          the default minimum version level for the feature if its defined.
+   *                  otherwise, returns 1.
+   */
+  def defaultMinVersionLevel(feature: String): Short = {
+    defaultFeatureMinVersionLevels.getOrElse(feature, 1)
+  }
+
+  // For testing only.
+  def setDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(supportedFeatures, newMinVersionLevels))
+    defaultFeatureMinVersionLevels = newMinVersionLevels
+  }
+
+  /**
+   * Returns the default finalized features that a new Kafka cluster with IBP config >= KAFKA_2_7_IV0
+   * needs to be bootstrapped with.
+   */
+  def getDefaultFinalizedFeatures: Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(
+      supportedFeatures.features.asScala.map {
+        case(name, versionRange) => (
+          name, new FinalizedVersionRange(defaultMinVersionLevel(name), versionRange.max))
+      }.asJava)
+  }
+
+  /**
+   * Returns the set of feature names found to be 'incompatible'.
+   * A feature incompatibility is a version mismatch between the latest feature supported by the
+   * Broker, and the provided finalized feature. This can happen because a provided finalized
+   * feature:
+   *  1) Does not exist in the Broker (i.e. it is unknown to the Broker).
+   *           [OR]
+   *  2) Exists but the FinalizedVersionRange does not match with the
+   *     supported feature's SupportedVersionRange.
+   *
+   * @param finalized   The finalized features against which incompatibilities need to be checked for.
+   *
+   * @return            The subset of input features which are incompatible. If the returned object
+   *                    is empty, it means there were no feature incompatibilities found.
+   */
+  def incompatibleFeatures(finalized: Features[FinalizedVersionRange]): Features[FinalizedVersionRange] = {
+    BrokerFeatures.incompatibleFeatures(supportedFeatures, finalized, true)
+  }
+}
+
+object BrokerFeatures extends Logging {
+
+  def createDefault(): BrokerFeatures = {
+    // The arguments are currently empty, but, in the future as we define features we should
+    // populate the required values here.
+    new BrokerFeatures(emptySupportedFeatures, Map[String, Short]())
+  }
+
+  /**
+   * Returns true if any of the provided finalized features are incompatible with the provided
+   * supported features.
+   *
+   * @param supportedFeatures   The supported features to be compared
+   * @param finalizedFeatures   The finalized features to be compared
+   *
+   * @return                    - True if there are any feature incompatibilities found.
+   *                            - False otherwise.
+   */
+  def hasIncompatibleFeatures(supportedFeatures: Features[SupportedVersionRange],
+                              finalizedFeatures: Features[FinalizedVersionRange]): Boolean = {
+    !incompatibleFeatures(supportedFeatures, finalizedFeatures, false).empty
+  }
+
+  private def incompatibleFeatures(supportedFeatures: Features[SupportedVersionRange],
+                                   finalizedFeatures: Features[FinalizedVersionRange],
+                                   logIncompatibilities: Boolean): Features[FinalizedVersionRange] = {
+    val incompatibilities = finalizedFeatures.features.asScala.map {
+      case (feature, versionLevels) => {

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1214,70 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker, but it can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the DescribeFeaturesResult containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);
+
+    /**
+     * Applies specified updates to finalized features. The API is atomic, meaning that if a single
+     * feature update in the request can't succeed on the controller, then none of the feature
+     * updates are carried out. This request is issued only to the controller since the API is
+     * only served by the controller.
+     * <p>
+     * The API takes as input a set of FinalizedFeatureUpdate that need to be applied. Each such

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FinalizedFeatureUpdate.java
##########
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+import java.util.Set;
+import org.apache.kafka.common.message.UpdateFinalizedFeaturesRequestData;
+
+/**
+ * Encapsulates details about an update to a finalized feature. This is particularly useful to
+ * define each feature update in the
+ * {@link Admin#updateFinalizedFeatures(Set, UpdateFinalizedFeaturesOptions)} API request.
+ */
+public class FinalizedFeatureUpdate {
+    private final String featureName;
+    private final short maxVersionLevel;
+    private final boolean allowDowngrade;
+
+    /**
+     * @param featureName       the name of the finalized feature to be updated.
+     * @param maxVersionLevel   the new maximum version level for the finalized feature.
+     *                          a value < 1 is special and indicates that the update is intended to
+     *                          delete the finalized feature, and should be accompanied by setting
+     *                          the allowDowngrade flag to true.
+     * @param allowDowngrade    - true, if this feature update was meant to downgrade the existing
+     *                            maximum version level of the finalized feature.
+     *                          - false, otherwise.
+     */
+    public FinalizedFeatureUpdate(

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1214,70 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any

Review comment:
       Done. Updated the doc now.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FinalizedFeatureUpdate.java
##########
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+import java.util.Set;
+import org.apache.kafka.common.message.UpdateFinalizedFeaturesRequestData;
+
+/**
+ * Encapsulates details about an update to a finalized feature. This is particularly useful to
+ * define each feature update in the
+ * {@link Admin#updateFinalizedFeatures(Set, UpdateFinalizedFeaturesOptions)} API request.
+ */
+public class FinalizedFeatureUpdate {
+    private final String featureName;
+    private final short maxVersionLevel;
+    private final boolean allowDowngrade;
+
+    /**
+     * @param featureName       the name of the finalized feature to be updated.
+     * @param maxVersionLevel   the new maximum version level for the finalized feature.
+     *                          a value < 1 is special and indicates that the update is intended to
+     *                          delete the finalized feature, and should be accompanied by setting
+     *                          the allowDowngrade flag to true.
+     * @param allowDowngrade    - true, if this feature update was meant to downgrade the existing
+     *                            maximum version level of the finalized feature.
+     *                          - false, otherwise.
+     */
+    public FinalizedFeatureUpdate(
+        final String featureName, final short maxVersionLevel, final boolean allowDowngrade) {
+        Objects.requireNonNull(featureName, "Provided feature name can not be null.");
+        if (maxVersionLevel < 1 && !allowDowngrade) {
+            throw new IllegalArgumentException(
+                String.format(
+                    "For featureName: %s, the allowDowngrade flag is not set when the" +
+                    " provided maxVersionLevel:%d is < 1.", featureName, maxVersionLevel));
+        }
+        this.featureName = featureName;
+        this.maxVersionLevel = maxVersionLevel;
+        this.allowDowngrade = allowDowngrade;
+    }
+
+    /**
+     * @return   the name of the finalized feature to be updated.
+     */
+    public String featureName() {
+        return featureName;
+    }
+
+    /**
+     * @return   the new maximum version level for the finalized feature.
+     */
+    public short maxVersionLevel() {
+        return maxVersionLevel;
+    }
+
+    /**
+     * @return   - true, if this feature update was meant to downgrade the maximum version level of
+     *             the finalized feature.
+     *           - false, otherwise.

Review comment:
       Done. Removed.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FinalizedFeatureUpdate.java
##########
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+import java.util.Set;
+import org.apache.kafka.common.message.UpdateFinalizedFeaturesRequestData;
+
+/**
+ * Encapsulates details about an update to a finalized feature. This is particularly useful to
+ * define each feature update in the
+ * {@link Admin#updateFinalizedFeatures(Set, UpdateFinalizedFeaturesOptions)} API request.
+ */
+public class FinalizedFeatureUpdate {
+    private final String featureName;
+    private final short maxVersionLevel;
+    private final boolean allowDowngrade;
+
+    /**
+     * @param featureName       the name of the finalized feature to be updated.
+     * @param maxVersionLevel   the new maximum version level for the finalized feature.
+     *                          a value < 1 is special and indicates that the update is intended to
+     *                          delete the finalized feature, and should be accompanied by setting
+     *                          the allowDowngrade flag to true.
+     * @param allowDowngrade    - true, if this feature update was meant to downgrade the existing
+     *                            maximum version level of the finalized feature.
+     *                          - false, otherwise.
+     */
+    public FinalizedFeatureUpdate(
+        final String featureName, final short maxVersionLevel, final boolean allowDowngrade) {
+        Objects.requireNonNull(featureName, "Provided feature name can not be null.");
+        if (maxVersionLevel < 1 && !allowDowngrade) {
+            throw new IllegalArgumentException(
+                String.format(
+                    "For featureName: %s, the allowDowngrade flag is not set when the" +
+                    " provided maxVersionLevel:%d is < 1.", featureName, maxVersionLevel));
+        }
+        this.featureName = featureName;
+        this.maxVersionLevel = maxVersionLevel;
+        this.allowDowngrade = allowDowngrade;
+    }
+
+    /**
+     * @return   the name of the finalized feature to be updated.

Review comment:
       Done. Removed.

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature
+ *    version level deprecation. This is how it works: in order to deprecate feature version levels,
+ *    in this map the default minimum version level of a feature can be set to a new value that's
+ *    higher than 1 (let's call this new_min_version_level). In doing so, the feature version levels
+ *    in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic
+ *    that applies this map to persistent finalized feature state in ZK (this mutation happens
+ *    during controller election and during finalized feature updates via the
+ *    APIKeys.UPDATE_FINALIZED_FEATURES api).
+ *
+ * This class also provides APIs to check for incompatibilities between the features supported by
+ * the Broker and finalized features.
+ *
+ * NOTE: the update*() and clear*() APIs of this class should be used only for testing purposes.
+ */
+class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange],

Review comment:
       It is already explained in the class level doc. This is also explained in the KIP-584 [in this section](https://cwiki.apache.org/confluence/display/KAFKA/KIP-584%3A+Versioning+scheme+for+features#KIP584:Versioningschemeforfeatures-Featureversiondeprecation).
   
   This is needed because `defaultFeatureMinVersionLevels` is mainly for feature version deprecation. When we deprecate feature version levels, we first bump the `defaultFeatureMinVersionLevels` in a broker release (after making an announcement to community). This will automatically mean clients have to stop using the finalized min version levels that have been deprecated (because upon startup the controller will write the `defaultFeatureMinVersionLevels` to ZK from within `KafkaController#setupFeatureVersioning` method). Once the write to ZK happens, clients that are using the finalized features are forced to stop using the deprecated version levels.
   
   Then, finally in the future when we remove the code for the deprecated version levels, that is when we will bump the min version for the supported feature in the broker. Thereby we will completely drop support for a feature version altogether.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1214,70 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker, but it can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the DescribeFeaturesResult containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);
+
+    /**
+     * Applies specified updates to finalized features. The API is atomic, meaning that if a single
+     * feature update in the request can't succeed on the controller, then none of the feature
+     * updates are carried out. This request is issued only to the controller since the API is
+     * only served by the controller.
+     * <p>
+     * The API takes as input a set of FinalizedFeatureUpdate that need to be applied. Each such
+     * update specifies the finalized feature to be added or updated or deleted, along with the new
+     * max feature version level value.
+     * <ul>
+     * <li>Downgrade of feature version level is not a regular operation/intent. It is only allowed
+     * in the controller if the feature update has the allowDowngrade flag set - setting this flag
+     * conveys user intent to attempt downgrade of a feature max version level. Note that despite
+     * the allowDowngrade flag being set, certain downgrades may be rejected by the controller if it
+     * is deemed impossible.</li>
+     * <li>Deletion of a finalized feature version is not a regular operation/intent. It is allowed
+     * only if the allowDowngrade flag is set in the feature update, and, if the max version level
+     * is set to a value less than 1.</li>
+     * </ul>
+     *<p>
+     * The following exceptions can be anticipated when calling {@code get()} on the futures
+     * obtained from the returned {@link UpdateFinalizedFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.ClusterAuthorizationException}
+     *   If the authenticated user didn't have alter access to the cluster.</li>
+     *   <li>{@link org.apache.kafka.common.errors.InvalidRequestException}
+     *   If the request details are invalid. e.g., a non-existing finalized feature is attempted
+     *   to be deleted or downgraded.</li>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the updates could finish. It cannot be guaranteed whether
+     *   the updates succeeded or not.</li>
+     *   <li>{@link org.apache.kafka.common.errors.FinalizedFeatureUpdateFailedException}
+     *   If the updates could not be applied on the controller, despite the request being valid.
+     *   This may be a temporary problem.</li>
+     * </ul>
+     * <p>
+     * This operation is supported by brokers with version 2.7.0 or higher.
+
+     * @param featureUpdates   the set of finalized feature updates
+     * @param options          the options to use
+     *
+     * @return                 the UpdateFinalizedFeaturesResult containing the result

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FinalizedFeatureUpdate.java
##########
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+import java.util.Set;
+import org.apache.kafka.common.message.UpdateFinalizedFeaturesRequestData;
+
+/**
+ * Encapsulates details about an update to a finalized feature. This is particularly useful to
+ * define each feature update in the
+ * {@link Admin#updateFinalizedFeatures(Set, UpdateFinalizedFeaturesOptions)} API request.
+ */
+public class FinalizedFeatureUpdate {
+    private final String featureName;
+    private final short maxVersionLevel;
+    private final boolean allowDowngrade;
+
+    /**
+     * @param featureName       the name of the finalized feature to be updated.
+     * @param maxVersionLevel   the new maximum version level for the finalized feature.
+     *                          a value < 1 is special and indicates that the update is intended to
+     *                          delete the finalized feature, and should be accompanied by setting
+     *                          the allowDowngrade flag to true.
+     * @param allowDowngrade    - true, if this feature update was meant to downgrade the existing
+     *                            maximum version level of the finalized feature.
+     *                          - false, otherwise.
+     */
+    public FinalizedFeatureUpdate(
+        final String featureName, final short maxVersionLevel, final boolean allowDowngrade) {
+        Objects.requireNonNull(featureName, "Provided feature name can not be null.");
+        if (maxVersionLevel < 1 && !allowDowngrade) {
+            throw new IllegalArgumentException(
+                String.format(
+                    "For featureName: %s, the allowDowngrade flag is not set when the" +
+                    " provided maxVersionLevel:%d is < 1.", featureName, maxVersionLevel));
+        }
+        this.featureName = featureName;
+        this.maxVersionLevel = maxVersionLevel;
+        this.allowDowngrade = allowDowngrade;
+    }
+
+    /**
+     * @return   the name of the finalized feature to be updated.
+     */
+    public String featureName() {
+        return featureName;
+    }
+
+    /**
+     * @return   the new maximum version level for the finalized feature.
+     */
+    public short maxVersionLevel() {
+        return maxVersionLevel;
+    }
+
+    /**
+     * @return   - true, if this feature update was meant to downgrade the maximum version level of
+     *             the finalized feature.
+     *           - false, otherwise.
+     */
+    public boolean allowDowngrade() {
+        return allowDowngrade;
+    }
+
+    /**
+     * Helper function that creates {@link UpdateFinalizedFeaturesRequestData} from a set of
+     * {@link FinalizedFeatureUpdate}.
+     *
+     * @param updates   the set of {@link FinalizedFeatureUpdate}
+     *
+     * @return          a newly constructed UpdateFinalizedFeaturesRequestData object
+     */
+    public static UpdateFinalizedFeaturesRequestData createRequest(Set<FinalizedFeatureUpdate> updates) {
+        final UpdateFinalizedFeaturesRequestData.FinalizedFeatureUpdateKeyCollection items
+            = new UpdateFinalizedFeaturesRequestData.FinalizedFeatureUpdateKeyCollection();
+        for (FinalizedFeatureUpdate update : updates) {
+            final UpdateFinalizedFeaturesRequestData.FinalizedFeatureUpdateKey item =
+                new UpdateFinalizedFeaturesRequestData.FinalizedFeatureUpdateKey();
+            item.setName(update.featureName());
+            item.setMaxVersionLevel(update.maxVersionLevel());
+            item.setAllowDowngrade(update.allowDowngrade());
+            items.add(item);
+        }
+        final UpdateFinalizedFeaturesRequestData data = new UpdateFinalizedFeaturesRequestData();
+        data.setFinalizedFeatureUpdates(items);

Review comment:
       Done. Good point.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1825,36 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def processUpdateFinalizedFeatures(newFeatures: Features[FinalizedVersionRange],
+                                             callback: UpdateFinalizedFeaturesCallback): Unit = {
+    if (isActive) {
+      val incompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.filter(broker => {
+        BrokerFeatures.hasIncompatibleFeatures(broker.features, newFeatures)
+      })
+      if (incompatibleBrokers.size > 0) {

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1214,70 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker, but it can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the DescribeFeaturesResult containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);
+
+    /**
+     * Applies specified updates to finalized features. The API is atomic, meaning that if a single
+     * feature update in the request can't succeed on the controller, then none of the feature
+     * updates are carried out. This request is issued only to the controller since the API is
+     * only served by the controller.
+     * <p>
+     * The API takes as input a set of FinalizedFeatureUpdate that need to be applied. Each such
+     * update specifies the finalized feature to be added or updated or deleted, along with the new
+     * max feature version level value.
+     * <ul>
+     * <li>Downgrade of feature version level is not a regular operation/intent. It is only allowed
+     * in the controller if the feature update has the allowDowngrade flag set - setting this flag
+     * conveys user intent to attempt downgrade of a feature max version level. Note that despite
+     * the allowDowngrade flag being set, certain downgrades may be rejected by the controller if it
+     * is deemed impossible.</li>
+     * <li>Deletion of a finalized feature version is not a regular operation/intent. It is allowed
+     * only if the allowDowngrade flag is set in the feature update, and, if the max version level
+     * is set to a value less than 1.</li>
+     * </ul>
+     *<p>
+     * The following exceptions can be anticipated when calling {@code get()} on the futures
+     * obtained from the returned {@link UpdateFinalizedFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.ClusterAuthorizationException}
+     *   If the authenticated user didn't have alter access to the cluster.</li>
+     *   <li>{@link org.apache.kafka.common.errors.InvalidRequestException}
+     *   If the request details are invalid. e.g., a non-existing finalized feature is attempted
+     *   to be deleted or downgraded.</li>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the updates could finish. It cannot be guaranteed whether
+     *   the updates succeeded or not.</li>
+     *   <li>{@link org.apache.kafka.common.errors.FinalizedFeatureUpdateFailedException}
+     *   If the updates could not be applied on the controller, despite the request being valid.
+     *   This may be a temporary problem.</li>
+     * </ul>
+     * <p>
+     * This operation is supported by brokers with version 2.7.0 or higher.
+
+     * @param featureUpdates   the set of finalized feature updates
+     * @param options          the options to use
+     *
+     * @return                 the UpdateFinalizedFeaturesResult containing the result
+     */
+    UpdateFinalizedFeaturesResult updateFinalizedFeatures(

Review comment:
       Done. Removed the word "finalized"in the context of this API.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,158 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * Enables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with enabled status. This status means the feature versioning system
+   * (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode are active. This
+   * status should be written by the controller to the FeatureZNode only when the broker IBP config
+   * is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    For a new Kafka cluster (i.e. it is deployed first time), we would like to start the cluster
+   *    with all the possible supported features finalized immediately. The new cluster will almost
+   *    never be started with an old IBP config that’s less than KAFKA_2_7_IV0. Assuming this is the
+   *    case, then here is how we it: the controller will start up and notice that the FeatureZNode
+   *    is absent in the new cluster, it will then create a FeatureZNode (with enabled status)
+   *    containing the entire list of default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. The reason to do this is that enabling all the possible features immediately after
+   *    an upgrade could be harmful to the cluster.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, then it
+   *        will react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, it's existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) => {
+            val updatedVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (updatedVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (existingVersionRange.max() >= updatedVersionRange.min()) {

Review comment:
       Done. This case is also handled now.
   To your point, the case  where `updated.max < existing.min` can never happen unless brokers get downgraded (after finalizing features at higher levels), and especially if the downgrade was done improperly (without applying feature tooling commands). It's a rare case. But even in that case, the broker will start crashing because of incompatibility in supported feature version max level, so the problem is found before it reaches this point.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -2945,6 +2950,137 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFinalizedFeatures(request: RequestChannel.Request): Unit = {
+    val updateFinalizedFeaturesRequest = request.body[UpdateFinalizedFeaturesRequest]
+    def sendResponseCallback(error: Errors, msgOverride: Option[String]): Unit = {
+      sendResponseExemptThrottle(request, new UpdateFinalizedFeaturesResponse(
+        new UpdateFinalizedFeaturesResponseData()
+          .setErrorCode(error.code())
+          .setErrorMessage(msgOverride.getOrElse(error.message()))))
+    }
+
+    if (!authorize(request.context, ALTER, CLUSTER, CLUSTER_NAME)) {
+      sendResponseCallback(Errors.CLUSTER_AUTHORIZATION_FAILED, Option.empty)
+    } else if (!controller.isActive) {
+      sendResponseCallback(Errors.NOT_CONTROLLER, Option.empty)
+    } else if (!config.isFeatureVersioningEnabled) {
+      sendResponseCallback(Errors.INVALID_REQUEST, Some("Feature versioning system is disabled."))
+    } else {
+      val targetFeaturesOrError = getTargetFinalizedFeaturesOrError(updateFinalizedFeaturesRequest)
+      targetFeaturesOrError match {
+        case Left(targetFeatures) =>
+          controller.updateFinalizedFeatures(targetFeatures, sendResponseCallback)
+        case Right(error) =>
+          sendResponseCallback(error.error, Some(error.message))
+      }
+    }
+  }
+
+  /**
+   * Validates the provided UpdateFinalizedFeaturesRequest, checking for various error cases.
+   * If the validation is successful, returns the target finalized features constructed from the
+   * request.
+   *
+   * @param request   the request to be validated
+   *
+   * @return          - the target finalized features, if request validation is successful
+   *                  - an ApiError if request validation fails
+   */
+  def getTargetFinalizedFeaturesOrError(request: UpdateFinalizedFeaturesRequest):
+    Either[Features[FinalizedVersionRange], ApiError] = {
+    val updates = request.data.finalizedFeatureUpdates
+    if (updates.isEmpty) {
+      return Right(new ApiError(Errors.INVALID_REQUEST,
+        "Can not provide empty FinalizedFeatureUpdates in the request."))
+    }
+
+    val latestFeatures = featureCache.get
+    val newFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]()
+    updates.asScala.foreach(
+      update => {
+        // Rule #1) Check that the feature name is not empty.
+        if (update.name.isEmpty) {
+          return Right(
+            new ApiError(Errors.INVALID_REQUEST,
+              "Can not contain empty feature name in the request."))
+        }
+
+        val cacheEntry = latestFeatures.map(lf => lf.features.get(update.name)).orNull
+
+        // We handle deletion requests separately from non-deletion requests.
+        if (UpdateFinalizedFeaturesRequest.isDeleteRequest(update)) { // Deletion request

Review comment:
       Done. Removed.

##########
File path: clients/src/main/resources/common/message/UpdateFinalizedFeaturesResponse.json
##########
@@ -0,0 +1,28 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "response",
+  "name": "UpdateFinalizedFeaturesResponse",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "ErrorCode", "type": "int16", "versions": "0+",
+      "about": "The error code or `0` if there was no error." },
+    { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+",

Review comment:
       The purpose of the error message is to sometimes describe with finer details on what is the error (such as which feature update is incorrect). To your point, it seems there are existing response types that do allow for an error message, examples are: `CreateTopicsResponse`, `CreatePartitionsResponse`, `DeleteAclsResponse` etc.
   There is ongoing related discussion under another PR review comment and we can continue the discussion there: https://github.com/apache/kafka/pull/9001/files#r456125212 . 

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -983,8 +1144,25 @@ class KafkaController(val config: KafkaConfig,
    */
   private[controller] def sendUpdateMetadataRequest(brokers: Seq[Int], partitions: Set[TopicPartition]): Unit = {
     try {
+      val filteredBrokers = scala.collection.mutable.Set[Int]() ++ brokers
+      if (config.isFeatureVersioningEnabled) {
+        def hasIncompatibleFeatures(broker: Broker): Boolean = {
+          val latestFinalizedFeatures = featureCache.get
+          if (latestFinalizedFeatures.isDefined) {
+            BrokerFeatures.hasIncompatibleFeatures(broker.features, latestFinalizedFeatures.get.features)
+          } else {
+            false
+          }
+        }
+        controllerContext.liveOrShuttingDownBrokers.foreach(broker => {
+          if (filteredBrokers.contains(broker.id) && hasIncompatibleFeatures(broker)) {

Review comment:
       This handles the race condition described in the KIP-584 [in this section](https://cwiki.apache.org/confluence/display/KAFKA/KIP-584%3A+Versioning+scheme+for+features#KIP584:Versioningschemeforfeatures-Incompatiblebrokerlifetimeracecondition). Please refer to the KIP for details. I have also added doc to this method.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -3984,6 +3988,108 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        Call callViaLeastLoadedNode = new Call("describeFeatures", calcDeadlineMs(now, options.timeoutMs()),
+            new LeastLoadedNodeProvider()) {
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(
+                        new FeatureMetadata(
+                            apiVersionsResponse.finalizedFeatures(),
+                            apiVersionsResponse.finalizedFeaturesEpoch(),
+                            apiVersionsResponse.supportedFeatures()));
+                } else {
+                    future.completeExceptionally(
+                        Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        Call call = callViaLeastLoadedNode;
+        if (options.sendRequestToController()) {
+            call = new Call("describeFeatures", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {
+
+                @Override
+                ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                    return (ApiVersionsRequest.Builder) callViaLeastLoadedNode.createRequest(timeoutMs);
+                }
+
+                @Override
+                void handleResponse(AbstractResponse response) {
+                    final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                    if (apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) {
+                        handleNotControllerError(Errors.NOT_CONTROLLER);
+                    } else {
+                        callViaLeastLoadedNode.handleResponse(response);

Review comment:
       Done. Good point.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,158 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * Enables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with enabled status. This status means the feature versioning system
+   * (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode are active. This
+   * status should be written by the controller to the FeatureZNode only when the broker IBP config
+   * is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    For a new Kafka cluster (i.e. it is deployed first time), we would like to start the cluster
+   *    with all the possible supported features finalized immediately. The new cluster will almost

Review comment:
       Done. Updated the doc.

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature
+ *    version level deprecation. This is how it works: in order to deprecate feature version levels,
+ *    in this map the default minimum version level of a feature can be set to a new value that's
+ *    higher than 1 (let's call this new_min_version_level). In doing so, the feature version levels
+ *    in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic
+ *    that applies this map to persistent finalized feature state in ZK (this mutation happens
+ *    during controller election and during finalized feature updates via the
+ *    APIKeys.UPDATE_FINALIZED_FEATURES api).

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,158 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * Enables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with enabled status. This status means the feature versioning system
+   * (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode are active. This
+   * status should be written by the controller to the FeatureZNode only when the broker IBP config
+   * is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    For a new Kafka cluster (i.e. it is deployed first time), we would like to start the cluster
+   *    with all the possible supported features finalized immediately. The new cluster will almost
+   *    never be started with an old IBP config that’s less than KAFKA_2_7_IV0. Assuming this is the
+   *    case, then here is how we it: the controller will start up and notice that the FeatureZNode
+   *    is absent in the new cluster, it will then create a FeatureZNode (with enabled status)
+   *    containing the entire list of default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. The reason to do this is that enabling all the possible features immediately after
+   *    an upgrade could be harmful to the cluster.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, then it
+   *        will react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, it's existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) => {
+            val updatedVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (updatedVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (existingVersionRange.max() >= updatedVersionRange.min()) {
+              // Through this change, we deprecate all version levels in the closed range:
+              // [existingVersionRange.min(), updatedVersionRange.min() - 1]
+              (featureName, new FinalizedVersionRange(updatedVersionRange.min(), existingVersionRange.max()))
+            } else {
+              // This is a special case: If the existing version levels fall completely outside the
+              // range of the default finalized version levels (i.e. no intersection), then, this
+              // case is not eligible for deprecation. This requires that the max version level be
+              // upgraded first to a value that's equal to the the default minimum version level.
+              info(s"Can not update minimum version level in finalized feature: $featureName,"
+              + s" since the existing $existingVersionRange does not intersect with the default"
+              + s" $updatedVersionRange.")
+              (featureName, existingVersionRange)
+            }
+          }
+        }.asJava)
+      }
+      val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures)
+      if (!newFeatureZNode.equals(existingFeatureZNode)) {
+        val newVersion = updateFeatureZNode(newFeatureZNode)
+        featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+      }
+    }
+  }
+
+  /**
+   * Disables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with disabled status. This status means the feature versioning system
+   * (KIP-584) is disabled, and, the finalized features stored in the FeatureZNode are not relevant.
+   * This status should be written by the controller to the FeatureZNode only when the broker
+   * IBP config is less than KAFKA_2_7_IV0.
+   *
+   * NOTE:
+   * 1. When this method returns, existing finalized features (if any) will be cleared from the
+   *    FeatureZNode.
+   * 2. This method, unlike enableFeatureVersioning() need not wait for the FinalizedFeatureCache
+   *    to be updated, because, such updates to the caceh (via FinalizedFeatureChangeListener)
+   *    are disabled when IBP config is < than KAFKA_2_7_IV0.
+   */
+  private def disableFeatureVersioning(): Unit = {

Review comment:
       Not sure I understood. We will only update the `FeatureZNode` if the status is not disabled currently (see the implementation below). What am I missing?

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/UpdateFinalizedFeaturesResponse.java
##########
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+import org.apache.kafka.common.message.UpdateFinalizedFeaturesResponseData;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.Struct;
+
+public class UpdateFinalizedFeaturesResponse extends AbstractResponse {

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1825,36 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def processUpdateFinalizedFeatures(newFeatures: Features[FinalizedVersionRange],
+                                             callback: UpdateFinalizedFeaturesCallback): Unit = {
+    if (isActive) {
+      val incompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.filter(broker => {
+        BrokerFeatures.hasIncompatibleFeatures(broker.features, newFeatures)
+      })
+      if (incompatibleBrokers.size > 0) {
+        callback(
+          Errors.INVALID_REQUEST,
+          Some(
+            s"Could not apply finalized feature updates because ${incompatibleBrokers.size} brokers"
+            + s" were found to have incompatible features. newFeatures: $newFeatures"
+           + s", incompatibleBrokers: $incompatibleBrokers."))
+      } else {
+        try {
+          val newVersion = zkClient.updateFeatureZNode(
+            new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures))
+          featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)

Review comment:
       I feel that there isn't a pressing reason to optimize this API path currently, and make it async.
   The API is not going to be frequently used, and an infrequent write to a ZK node with low write contention feels like a relatively inexpensive case that we could block the controller on.
   Please let me know how you feel.

##########
File path: clients/src/main/resources/common/message/UpdateFinalizedFeaturesRequest.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "request",
+  "name": "UpdateFinalizedFeaturesRequest",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "FinalizedFeatureUpdates", "type": "[]FinalizedFeatureUpdateKey", "versions": "0+",
+      "about": "The list of updates to finalized features.", "fields": [
+      {"name": "Name", "type":  "string", "versions":  "0+", "mapKey": true,
+        "about": "The name of the finalized feature to be updated."},
+      {"name":  "MaxVersionLevel", "type":  "int16", "versions":  "0+",

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -2945,6 +2950,137 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFinalizedFeatures(request: RequestChannel.Request): Unit = {
+    val updateFinalizedFeaturesRequest = request.body[UpdateFinalizedFeaturesRequest]
+    def sendResponseCallback(error: Errors, msgOverride: Option[String]): Unit = {
+      sendResponseExemptThrottle(request, new UpdateFinalizedFeaturesResponse(
+        new UpdateFinalizedFeaturesResponseData()
+          .setErrorCode(error.code())
+          .setErrorMessage(msgOverride.getOrElse(error.message()))))
+    }
+
+    if (!authorize(request.context, ALTER, CLUSTER, CLUSTER_NAME)) {
+      sendResponseCallback(Errors.CLUSTER_AUTHORIZATION_FAILED, Option.empty)
+    } else if (!controller.isActive) {
+      sendResponseCallback(Errors.NOT_CONTROLLER, Option.empty)
+    } else if (!config.isFeatureVersioningEnabled) {
+      sendResponseCallback(Errors.INVALID_REQUEST, Some("Feature versioning system is disabled."))
+    } else {
+      val targetFeaturesOrError = getTargetFinalizedFeaturesOrError(updateFinalizedFeaturesRequest)
+      targetFeaturesOrError match {
+        case Left(targetFeatures) =>
+          controller.updateFinalizedFeatures(targetFeatures, sendResponseCallback)
+        case Right(error) =>
+          sendResponseCallback(error.error, Some(error.message))
+      }
+    }
+  }
+
+  /**
+   * Validates the provided UpdateFinalizedFeaturesRequest, checking for various error cases.
+   * If the validation is successful, returns the target finalized features constructed from the
+   * request.
+   *
+   * @param request   the request to be validated
+   *
+   * @return          - the target finalized features, if request validation is successful
+   *                  - an ApiError if request validation fails
+   */
+  def getTargetFinalizedFeaturesOrError(request: UpdateFinalizedFeaturesRequest):
+    Either[Features[FinalizedVersionRange], ApiError] = {
+    val updates = request.data.finalizedFeatureUpdates
+    if (updates.isEmpty) {
+      return Right(new ApiError(Errors.INVALID_REQUEST,
+        "Can not provide empty FinalizedFeatureUpdates in the request."))
+    }
+
+    val latestFeatures = featureCache.get
+    val newFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]()
+    updates.asScala.foreach(
+      update => {
+        // Rule #1) Check that the feature name is not empty.
+        if (update.name.isEmpty) {
+          return Right(
+            new ApiError(Errors.INVALID_REQUEST,
+              "Can not contain empty feature name in the request."))
+        }
+
+        val cacheEntry = latestFeatures.map(lf => lf.features.get(update.name)).orNull
+
+        // We handle deletion requests separately from non-deletion requests.
+        if (UpdateFinalizedFeaturesRequest.isDeleteRequest(update)) { // Deletion request
+          // Rule #2) Disallow deletion of a finalized feature without allowDowngrade flag set.
+          if (!update.allowDowngrade) {
+            return Right(
+              new ApiError(Errors.INVALID_REQUEST,
+                s"Can not delete feature: '${update.name}' without setting the" +
+                  " allowDowngrade flag in the request."))

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -2945,6 +2950,137 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFinalizedFeatures(request: RequestChannel.Request): Unit = {
+    val updateFinalizedFeaturesRequest = request.body[UpdateFinalizedFeaturesRequest]
+    def sendResponseCallback(error: Errors, msgOverride: Option[String]): Unit = {
+      sendResponseExemptThrottle(request, new UpdateFinalizedFeaturesResponse(
+        new UpdateFinalizedFeaturesResponseData()
+          .setErrorCode(error.code())
+          .setErrorMessage(msgOverride.getOrElse(error.message()))))
+    }
+
+    if (!authorize(request.context, ALTER, CLUSTER, CLUSTER_NAME)) {
+      sendResponseCallback(Errors.CLUSTER_AUTHORIZATION_FAILED, Option.empty)
+    } else if (!controller.isActive) {
+      sendResponseCallback(Errors.NOT_CONTROLLER, Option.empty)
+    } else if (!config.isFeatureVersioningEnabled) {
+      sendResponseCallback(Errors.INVALID_REQUEST, Some("Feature versioning system is disabled."))
+    } else {
+      val targetFeaturesOrError = getTargetFinalizedFeaturesOrError(updateFinalizedFeaturesRequest)
+      targetFeaturesOrError match {
+        case Left(targetFeatures) =>
+          controller.updateFinalizedFeatures(targetFeatures, sendResponseCallback)
+        case Right(error) =>
+          sendResponseCallback(error.error, Some(error.message))
+      }
+    }
+  }
+
+  /**
+   * Validates the provided UpdateFinalizedFeaturesRequest, checking for various error cases.
+   * If the validation is successful, returns the target finalized features constructed from the
+   * request.
+   *
+   * @param request   the request to be validated
+   *
+   * @return          - the target finalized features, if request validation is successful
+   *                  - an ApiError if request validation fails
+   */
+  def getTargetFinalizedFeaturesOrError(request: UpdateFinalizedFeaturesRequest):

Review comment:
       Done. 

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -82,18 +108,55 @@ object FinalizedFeatureCache extends Logging {
         " The existing cache contents are %s").format(latest, oldFeatureAndEpoch)
       throw new FeatureCacheUpdateException(errorMsg)
     } else {
-      val incompatibleFeatures = SupportedFeatures.incompatibleFeatures(latest.features)
+      val incompatibleFeatures = brokerFeatures.incompatibleFeatures(latest.features)
       if (!incompatibleFeatures.empty) {
         val errorMsg = ("FinalizedFeatureCache update failed since feature compatibility" +
           " checks failed! Supported %s has incompatibilities with the latest %s."
-          ).format(SupportedFeatures.get, latest)
+          ).format(brokerFeatures.supportedFeatures, latest)
         throw new FeatureCacheUpdateException(errorMsg)
       } else {
-        val logMsg = "Updated cache from existing finalized %s to latest finalized %s".format(
+        val logMsg = "Updated cache from existing %s to latest %s".format(
           oldFeatureAndEpoch, latest)
-        featuresAndEpoch = Some(latest)
+        synchronized {
+          featuresAndEpoch = Some(latest)
+          notifyAll()
+        }
         info(logMsg)
       }
     }
   }
+
+  /**
+   * Causes the current thread to wait no more than timeoutMs for the specified condition to be met.
+   * It is guaranteed that the provided condition will always be invoked only from within a
+   * synchronized block.
+   *
+   * @param waitCondition   the condition to be waited upon:
+   *                         - if the condition returns true, then, the wait will stop.
+   *                         - if the condition returns false, it means the wait must continue until
+   *                           timeout.
+   *
+   * @param timeoutMs       the timeout (in milli seconds)
+   *
+   * @throws                TimeoutException if the condition is not met within timeoutMs.
+   */
+  private def waitUntilConditionOrThrow(waitCondition: () => Boolean, timeoutMs: Long): Unit = {
+    if(timeoutMs < 0L) {
+      throw new IllegalArgumentException(s"Expected timeoutMs >= 0, but $timeoutMs was provided.")
+    }
+    synchronized {
+      var sleptTimeMs = 0L
+      while (!waitCondition()) {
+        val timeoutLeftMs = timeoutMs - sleptTimeMs
+        if (timeoutLeftMs <= 0) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${timeoutMs}ms for required condition to be met." +
+              s" Current epoch: ${featuresAndEpoch.map(fe => fe.epoch).getOrElse("<none>")}.")
+        }
+        val timeBeforeNanos = System.nanoTime

Review comment:
       Done. Made it the way you suggested, pls take a look.
   Overall either way looked fine to me but the one you suggested is a bit simpler.

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature
+ *    version level deprecation. This is how it works: in order to deprecate feature version levels,
+ *    in this map the default minimum version level of a feature can be set to a new value that's
+ *    higher than 1 (let's call this new_min_version_level). In doing so, the feature version levels
+ *    in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic
+ *    that applies this map to persistent finalized feature state in ZK (this mutation happens
+ *    during controller election and during finalized feature updates via the
+ *    APIKeys.UPDATE_FINALIZED_FEATURES api).
+ *
+ * This class also provides APIs to check for incompatibilities between the features supported by
+ * the Broker and finalized features.
+ *
+ * NOTE: the update*() and clear*() APIs of this class should be used only for testing purposes.
+ */
+class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange],
+                              @volatile var defaultFeatureMinVersionLevels: Map[String, Short]) {
+  require(BrokerFeatures.areFeatureMinVersionLevelsCompatible(
+    supportedFeatures, defaultFeatureMinVersionLevels))
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(newFeatures, defaultFeatureMinVersionLevels))
+    supportedFeatures = newFeatures
+  }
+
+  /**
+   * Returns the default minimum version level for a specific feature.
+   *
+   * @param feature   the name of the feature
+   *
+   * @return          the default minimum version level for the feature if its defined.
+   *                  otherwise, returns 1.
+   */
+  def defaultMinVersionLevel(feature: String): Short = {
+    defaultFeatureMinVersionLevels.getOrElse(feature, 1)
+  }
+
+  // For testing only.
+  def setDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(supportedFeatures, newMinVersionLevels))
+    defaultFeatureMinVersionLevels = newMinVersionLevels
+  }
+
+  /**
+   * Returns the default finalized features that a new Kafka cluster with IBP config >= KAFKA_2_7_IV0
+   * needs to be bootstrapped with.
+   */
+  def getDefaultFinalizedFeatures: Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(
+      supportedFeatures.features.asScala.map {
+        case(name, versionRange) => (
+          name, new FinalizedVersionRange(defaultMinVersionLevel(name), versionRange.max))
+      }.asJava)
+  }
+
+  /**
+   * Returns the set of feature names found to be 'incompatible'.
+   * A feature incompatibility is a version mismatch between the latest feature supported by the
+   * Broker, and the provided finalized feature. This can happen because a provided finalized
+   * feature:
+   *  1) Does not exist in the Broker (i.e. it is unknown to the Broker).
+   *           [OR]
+   *  2) Exists but the FinalizedVersionRange does not match with the
+   *     supported feature's SupportedVersionRange.
+   *
+   * @param finalized   The finalized features against which incompatibilities need to be checked for.
+   *
+   * @return            The subset of input features which are incompatible. If the returned object
+   *                    is empty, it means there were no feature incompatibilities found.
+   */
+  def incompatibleFeatures(finalized: Features[FinalizedVersionRange]): Features[FinalizedVersionRange] = {
+    BrokerFeatures.incompatibleFeatures(supportedFeatures, finalized, true)
+  }
+}
+
+object BrokerFeatures extends Logging {
+
+  def createDefault(): BrokerFeatures = {
+    // The arguments are currently empty, but, in the future as we define features we should
+    // populate the required values here.
+    new BrokerFeatures(emptySupportedFeatures, Map[String, Short]())
+  }
+
+  /**
+   * Returns true if any of the provided finalized features are incompatible with the provided
+   * supported features.
+   *
+   * @param supportedFeatures   The supported features to be compared
+   * @param finalizedFeatures   The finalized features to be compared
+   *
+   * @return                    - True if there are any feature incompatibilities found.
+   *                            - False otherwise.
+   */
+  def hasIncompatibleFeatures(supportedFeatures: Features[SupportedVersionRange],
+                              finalizedFeatures: Features[FinalizedVersionRange]): Boolean = {
+    !incompatibleFeatures(supportedFeatures, finalizedFeatures, false).empty
+  }
+
+  private def incompatibleFeatures(supportedFeatures: Features[SupportedVersionRange],
+                                   finalizedFeatures: Features[FinalizedVersionRange],
+                                   logIncompatibilities: Boolean): Features[FinalizedVersionRange] = {
+    val incompatibilities = finalizedFeatures.features.asScala.map {
+      case (feature, versionLevels) => {
+        val supportedVersions = supportedFeatures.get(feature)
+        if (supportedVersions == null) {
+          (feature, versionLevels, "{feature=%s, reason='Unsupported feature'}".format(feature))
+        } else if (versionLevels.isIncompatibleWith(supportedVersions)) {
+          (feature, versionLevels, "{feature=%s, reason='%s is incompatible with %s'}".format(
+            feature, versionLevels, supportedVersions))
+        } else {
+          (feature, versionLevels, null)
+        }
+      }
+    }.filter{ case(_, _, errorReason) => errorReason != null}.toList
+
+    if (logIncompatibilities && incompatibilities.nonEmpty) {
+      warn(
+        "Feature incompatibilities seen: " + incompatibilities.map{

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature
+ *    version level deprecation. This is how it works: in order to deprecate feature version levels,
+ *    in this map the default minimum version level of a feature can be set to a new value that's
+ *    higher than 1 (let's call this new_min_version_level). In doing so, the feature version levels
+ *    in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic
+ *    that applies this map to persistent finalized feature state in ZK (this mutation happens
+ *    during controller election and during finalized feature updates via the
+ *    APIKeys.UPDATE_FINALIZED_FEATURES api).
+ *
+ * This class also provides APIs to check for incompatibilities between the features supported by
+ * the Broker and finalized features.
+ *
+ * NOTE: the update*() and clear*() APIs of this class should be used only for testing purposes.
+ */
+class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange],
+                              @volatile var defaultFeatureMinVersionLevels: Map[String, Short]) {
+  require(BrokerFeatures.areFeatureMinVersionLevelsCompatible(
+    supportedFeatures, defaultFeatureMinVersionLevels))
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(newFeatures, defaultFeatureMinVersionLevels))
+    supportedFeatures = newFeatures
+  }
+
+  /**
+   * Returns the default minimum version level for a specific feature.
+   *
+   * @param feature   the name of the feature
+   *
+   * @return          the default minimum version level for the feature if its defined.
+   *                  otherwise, returns 1.
+   */
+  def defaultMinVersionLevel(feature: String): Short = {
+    defaultFeatureMinVersionLevels.getOrElse(feature, 1)
+  }
+
+  // For testing only.
+  def setDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(supportedFeatures, newMinVersionLevels))
+    defaultFeatureMinVersionLevels = newMinVersionLevels
+  }
+
+  /**
+   * Returns the default finalized features that a new Kafka cluster with IBP config >= KAFKA_2_7_IV0
+   * needs to be bootstrapped with.
+   */
+  def getDefaultFinalizedFeatures: Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(
+      supportedFeatures.features.asScala.map {
+        case(name, versionRange) => (
+          name, new FinalizedVersionRange(defaultMinVersionLevel(name), versionRange.max))
+      }.asJava)
+  }
+
+  /**
+   * Returns the set of feature names found to be 'incompatible'.
+   * A feature incompatibility is a version mismatch between the latest feature supported by the
+   * Broker, and the provided finalized feature. This can happen because a provided finalized
+   * feature:
+   *  1) Does not exist in the Broker (i.e. it is unknown to the Broker).
+   *           [OR]
+   *  2) Exists but the FinalizedVersionRange does not match with the
+   *     supported feature's SupportedVersionRange.
+   *
+   * @param finalized   The finalized features against which incompatibilities need to be checked for.
+   *
+   * @return            The subset of input features which are incompatible. If the returned object
+   *                    is empty, it means there were no feature incompatibilities found.
+   */
+  def incompatibleFeatures(finalized: Features[FinalizedVersionRange]): Features[FinalizedVersionRange] = {
+    BrokerFeatures.incompatibleFeatures(supportedFeatures, finalized, true)
+  }
+}
+
+object BrokerFeatures extends Logging {
+
+  def createDefault(): BrokerFeatures = {
+    // The arguments are currently empty, but, in the future as we define features we should
+    // populate the required values here.
+    new BrokerFeatures(emptySupportedFeatures, Map[String, Short]())
+  }
+
+  /**
+   * Returns true if any of the provided finalized features are incompatible with the provided
+   * supported features.
+   *
+   * @param supportedFeatures   The supported features to be compared
+   * @param finalizedFeatures   The finalized features to be compared
+   *
+   * @return                    - True if there are any feature incompatibilities found.
+   *                            - False otherwise.
+   */
+  def hasIncompatibleFeatures(supportedFeatures: Features[SupportedVersionRange],
+                              finalizedFeatures: Features[FinalizedVersionRange]): Boolean = {
+    !incompatibleFeatures(supportedFeatures, finalizedFeatures, false).empty
+  }
+
+  private def incompatibleFeatures(supportedFeatures: Features[SupportedVersionRange],
+                                   finalizedFeatures: Features[FinalizedVersionRange],
+                                   logIncompatibilities: Boolean): Features[FinalizedVersionRange] = {
+    val incompatibilities = finalizedFeatures.features.asScala.map {

Review comment:
       Done. Calling it `incompatibleFeaturesInfo` now.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -2945,6 +2950,137 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFinalizedFeatures(request: RequestChannel.Request): Unit = {
+    val updateFinalizedFeaturesRequest = request.body[UpdateFinalizedFeaturesRequest]
+    def sendResponseCallback(error: Errors, msgOverride: Option[String]): Unit = {
+      sendResponseExemptThrottle(request, new UpdateFinalizedFeaturesResponse(
+        new UpdateFinalizedFeaturesResponseData()
+          .setErrorCode(error.code())
+          .setErrorMessage(msgOverride.getOrElse(error.message()))))

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature
+ *    version level deprecation. This is how it works: in order to deprecate feature version levels,
+ *    in this map the default minimum version level of a feature can be set to a new value that's
+ *    higher than 1 (let's call this new_min_version_level). In doing so, the feature version levels
+ *    in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic
+ *    that applies this map to persistent finalized feature state in ZK (this mutation happens
+ *    during controller election and during finalized feature updates via the
+ *    APIKeys.UPDATE_FINALIZED_FEATURES api).
+ *
+ * This class also provides APIs to check for incompatibilities between the features supported by
+ * the Broker and finalized features.
+ *
+ * NOTE: the update*() and clear*() APIs of this class should be used only for testing purposes.
+ */
+class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange],
+                              @volatile var defaultFeatureMinVersionLevels: Map[String, Short]) {
+  require(BrokerFeatures.areFeatureMinVersionLevelsCompatible(
+    supportedFeatures, defaultFeatureMinVersionLevels))
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(newFeatures, defaultFeatureMinVersionLevels))
+    supportedFeatures = newFeatures
+  }
+
+  /**
+   * Returns the default minimum version level for a specific feature.
+   *
+   * @param feature   the name of the feature
+   *
+   * @return          the default minimum version level for the feature if its defined.
+   *                  otherwise, returns 1.
+   */
+  def defaultMinVersionLevel(feature: String): Short = {
+    defaultFeatureMinVersionLevels.getOrElse(feature, 1)
+  }
+
+  // For testing only.
+  def setDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(supportedFeatures, newMinVersionLevels))
+    defaultFeatureMinVersionLevels = newMinVersionLevels
+  }
+
+  /**
+   * Returns the default finalized features that a new Kafka cluster with IBP config >= KAFKA_2_7_IV0
+   * needs to be bootstrapped with.
+   */
+  def getDefaultFinalizedFeatures: Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(
+      supportedFeatures.features.asScala.map {
+        case(name, versionRange) => (
+          name, new FinalizedVersionRange(defaultMinVersionLevel(name), versionRange.max))
+      }.asJava)
+  }
+
+  /**
+   * Returns the set of feature names found to be 'incompatible'.
+   * A feature incompatibility is a version mismatch between the latest feature supported by the
+   * Broker, and the provided finalized feature. This can happen because a provided finalized
+   * feature:
+   *  1) Does not exist in the Broker (i.e. it is unknown to the Broker).
+   *           [OR]
+   *  2) Exists but the FinalizedVersionRange does not match with the
+   *     supported feature's SupportedVersionRange.
+   *
+   * @param finalized   The finalized features against which incompatibilities need to be checked for.
+   *
+   * @return            The subset of input features which are incompatible. If the returned object
+   *                    is empty, it means there were no feature incompatibilities found.
+   */
+  def incompatibleFeatures(finalized: Features[FinalizedVersionRange]): Features[FinalizedVersionRange] = {
+    BrokerFeatures.incompatibleFeatures(supportedFeatures, finalized, true)

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,158 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * Enables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with enabled status. This status means the feature versioning system
+   * (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode are active. This
+   * status should be written by the controller to the FeatureZNode only when the broker IBP config
+   * is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    For a new Kafka cluster (i.e. it is deployed first time), we would like to start the cluster
+   *    with all the possible supported features finalized immediately. The new cluster will almost
+   *    never be started with an old IBP config that’s less than KAFKA_2_7_IV0. Assuming this is the
+   *    case, then here is how we it: the controller will start up and notice that the FeatureZNode
+   *    is absent in the new cluster, it will then create a FeatureZNode (with enabled status)
+   *    containing the entire list of default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. The reason to do this is that enabling all the possible features immediately after
+   *    an upgrade could be harmful to the cluster.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, then it
+   *        will react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, it's existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) => {
+            val updatedVersionRange = defaultFinalizedFeatures.get(featureName)

Review comment:
       Done. I'm calling it `brokerDefaultVersionRange` now.

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature
+ *    version level deprecation. This is how it works: in order to deprecate feature version levels,
+ *    in this map the default minimum version level of a feature can be set to a new value that's
+ *    higher than 1 (let's call this new_min_version_level). In doing so, the feature version levels
+ *    in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic
+ *    that applies this map to persistent finalized feature state in ZK (this mutation happens
+ *    during controller election and during finalized feature updates via the
+ *    APIKeys.UPDATE_FINALIZED_FEATURES api).
+ *
+ * This class also provides APIs to check for incompatibilities between the features supported by
+ * the Broker and finalized features.
+ *
+ * NOTE: the update*() and clear*() APIs of this class should be used only for testing purposes.

Review comment:
       Done. Made the comment better. Pls take a look.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -2945,6 +2950,137 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFinalizedFeatures(request: RequestChannel.Request): Unit = {
+    val updateFinalizedFeaturesRequest = request.body[UpdateFinalizedFeaturesRequest]
+    def sendResponseCallback(error: Errors, msgOverride: Option[String]): Unit = {
+      sendResponseExemptThrottle(request, new UpdateFinalizedFeaturesResponse(
+        new UpdateFinalizedFeaturesResponseData()
+          .setErrorCode(error.code())
+          .setErrorMessage(msgOverride.getOrElse(error.message()))))
+    }
+
+    if (!authorize(request.context, ALTER, CLUSTER, CLUSTER_NAME)) {
+      sendResponseCallback(Errors.CLUSTER_AUTHORIZATION_FAILED, Option.empty)
+    } else if (!controller.isActive) {
+      sendResponseCallback(Errors.NOT_CONTROLLER, Option.empty)
+    } else if (!config.isFeatureVersioningEnabled) {
+      sendResponseCallback(Errors.INVALID_REQUEST, Some("Feature versioning system is disabled."))
+    } else {
+      val targetFeaturesOrError = getTargetFinalizedFeaturesOrError(updateFinalizedFeaturesRequest)
+      targetFeaturesOrError match {
+        case Left(targetFeatures) =>
+          controller.updateFinalizedFeatures(targetFeatures, sendResponseCallback)
+        case Right(error) =>
+          sendResponseCallback(error.error, Some(error.message))
+      }
+    }
+  }
+
+  /**
+   * Validates the provided UpdateFinalizedFeaturesRequest, checking for various error cases.
+   * If the validation is successful, returns the target finalized features constructed from the
+   * request.
+   *
+   * @param request   the request to be validated
+   *
+   * @return          - the target finalized features, if request validation is successful
+   *                  - an ApiError if request validation fails
+   */
+  def getTargetFinalizedFeaturesOrError(request: UpdateFinalizedFeaturesRequest):
+    Either[Features[FinalizedVersionRange], ApiError] = {
+    val updates = request.data.finalizedFeatureUpdates
+    if (updates.isEmpty) {
+      return Right(new ApiError(Errors.INVALID_REQUEST,
+        "Can not provide empty FinalizedFeatureUpdates in the request."))
+    }
+
+    val latestFeatures = featureCache.get
+    val newFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]()
+    updates.asScala.foreach(
+      update => {
+        // Rule #1) Check that the feature name is not empty.
+        if (update.name.isEmpty) {

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 #9001: KAFKA-10028: Implement KIP-584 write path

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java
##########
@@ -0,0 +1,26 @@
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+@InterfaceStability.Evolving
+public class DescribeFeaturesOptions extends AbstractOptions<DescribeFeaturesOptions> {
+    private boolean shouldUseControllerAsDestination = false;
+
+    public DescribeFeaturesOptions shouldUseControllerAsDestination(boolean shouldUse) {
+        shouldUseControllerAsDestination = shouldUse;
+        return this;
+    }
+
+    public boolean shouldUseControllerAsDestination() {

Review comment:
       remove word 'should'

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1215,10 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);
+
+    UpdateFinalizedFeaturesResult updateFinalizedFeatures(Set<FeatureUpdate> featureUpdates, UpdateFinalizedFeaturesOptions options);

Review comment:
       add doc

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java
##########
@@ -0,0 +1,26 @@
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+@InterfaceStability.Evolving
+public class DescribeFeaturesOptions extends AbstractOptions<DescribeFeaturesOptions> {
+    private boolean shouldUseControllerAsDestination = false;
+
+    public DescribeFeaturesOptions shouldUseControllerAsDestination(boolean shouldUse) {

Review comment:
       add doc

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -3979,6 +3986,98 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options) {

Review comment:
       1. add test code in `KafkaAdminClientTest`
   2. final variable names

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -3979,6 +3986,98 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        Call callViaLeastLoadedNode = new Call("describeFeatures", calcDeadlineMs(now, options.timeoutMs()),
+            new LeastLoadedNodeProvider()) {
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(
+                        new FeatureMetadata(
+                            apiVersionsResponse.finalizedFeatures(),
+                            apiVersionsResponse.finalizedFeaturesEpoch(),
+                            apiVersionsResponse.supportedFeatures()));
+                } else {
+                    future.completeExceptionally(
+                        Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        Call call = callViaLeastLoadedNode;
+        if (options.shouldUseControllerAsDestination()) {
+            call = new Call("describeFeatures", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {
+
+                @Override
+                ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                    return (ApiVersionsRequest.Builder) callViaLeastLoadedNode.createRequest(timeoutMs);
+                }
+
+                @Override
+                void handleResponse(AbstractResponse response) {
+                    callViaLeastLoadedNode.handleResponse(response);
+                }
+
+                @Override
+                void handleFailure(Throwable throwable) {
+                    callViaLeastLoadedNode.handleFailure(throwable);
+                }
+            };
+        }
+        runnable.call(call, now);
+        return new DescribeFeaturesResult(future);
+    }
+
+    @Override
+    public UpdateFinalizedFeaturesResult updateFinalizedFeatures(
+        Set<FeatureUpdate> featureUpdates, UpdateFinalizedFeaturesOptions options) {
+        final KafkaFutureImpl<Void> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        Call call = new Call("updateFinalizedFeatures", calcDeadlineMs(now, options.timeoutMs()),

Review comment:
       1 line gap before `cal`

##########
File path: clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
##########
@@ -319,7 +322,10 @@
     GROUP_SUBSCRIBED_TO_TOPIC(86, "Deleting offsets of a topic is forbidden while the consumer group is actively subscribed to it.",
         GroupSubscribedToTopicException::new),
     INVALID_RECORD(87, "This record has failed the validation on broker and hence will be rejected.", InvalidRecordException::new),
-    UNSTABLE_OFFSET_COMMIT(88, "There are unstable offsets that need to be cleared.", UnstableOffsetCommitException::new);
+    UNSTABLE_OFFSET_COMMIT(88, "There are unstable offsets that need to be cleared.", UnstableOffsetCommitException::new),
+    INCOMPATIBLE_FEATURES(89, "Could not apply finalized feature updates due to incompatible features.", IncompatibleFeaturesException::new),

Review comment:
       Eliminate and use INVALID_REQUEST

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +277,24 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def setupFeatureZNode(newNode: FeatureZNode): Unit = {

Review comment:
       call the variable as `nodeContents` ?

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/UpdateFinalizedFeaturesResponse.java
##########
@@ -0,0 +1,54 @@
+package org.apache.kafka.common.requests;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData;
+import org.apache.kafka.common.message.UpdateFinalizedFeaturesResponseData;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.Struct;
+
+public class UpdateFinalizedFeaturesResponse extends AbstractResponse {
+
+    public final UpdateFinalizedFeaturesResponseData data;
+
+    public UpdateFinalizedFeaturesResponse(UpdateFinalizedFeaturesResponseData data) {
+        this.data = data;
+    }
+
+    public UpdateFinalizedFeaturesResponse(Struct struct) {
+        short latestVersion = (short) (UpdateFinalizedFeaturesResponseData.SCHEMAS.length - 1);
+        this.data = new UpdateFinalizedFeaturesResponseData(struct, latestVersion);
+    }
+
+    public UpdateFinalizedFeaturesResponse(Struct struct, short version) {
+        this.data = new UpdateFinalizedFeaturesResponseData(struct, version);
+    }
+
+    public Errors error() {
+        return Errors.forCode(data.errorCode());
+    }
+
+    @Override
+    public Map<Errors, Integer> errorCounts() {
+        return errorCounts(Errors.forCode(data.errorCode()));
+    }
+
+    @Override
+    protected Struct toStruct(short version) {
+        return data.toStruct(version);
+    }
+
+    @Override
+    public String toString() {
+        return data.toString();
+    }
+
+    public UpdateFinalizedFeaturesResponseData data() {
+        return data;
+    }
+
+    public static UpdateFinalizedFeaturesResponse parse(ByteBuffer buffer, short version) {
+        return new UpdateFinalizedFeaturesResponse(ApiKeys.SYNC_GROUP.parseResponse(version, buffer), version);

Review comment:
       Fix ApiKeys

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1693,33 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def processUpdateFinalizedFeatures(newFeatures: Features[FinalizedVersionRange],
+                                             callback: UpdateFinalizedFeaturesCallback): Unit = {
+    if (isActive) {
+      val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+        BrokerFeatures.hasIncompatibleFeatures(broker.features, newFeatures)
+      })
+      if (numIncompatibleBrokers > 0) {
+        callback(
+          Errors.INCOMPATIBLE_FEATURES,
+          Some(
+            s"Could not apply finalized feature updates because $numIncompatibleBrokers brokers" +

Review comment:
       Perhaps add info about newFeatures and incompatibleBrokers.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java
##########
@@ -0,0 +1,70 @@
+package org.apache.kafka.clients.admin;
+
+import static java.util.stream.Collectors.joining;
+
+import java.util.Objects;
+import org.apache.kafka.common.feature.Features;
+import org.apache.kafka.common.feature.FinalizedVersionRange;
+import org.apache.kafka.common.feature.SupportedVersionRange;
+
+public class FeatureMetadata {

Review comment:
       add doc to entire class

##########
File path: core/src/test/resources/log4j.properties
##########
@@ -12,14 +12,80 @@
 # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.

Review comment:
       revert the file eventually

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java
##########
@@ -0,0 +1,51 @@
+package org.apache.kafka.clients.admin;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.kafka.common.message.UpdateFinalizedFeaturesRequestData;
+
+public class FeatureUpdate {

Review comment:
       add doc to entire class

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1215,10 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);

Review comment:
       add doc

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java
##########
@@ -0,0 +1,51 @@
+package org.apache.kafka.clients.admin;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.kafka.common.message.UpdateFinalizedFeaturesRequestData;
+
+public class FeatureUpdate {
+    private String name;

Review comment:
       attributes can be final

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/UpdateFinalizedFeaturesRequest.java
##########
@@ -0,0 +1,72 @@
+package org.apache.kafka.common.requests;
+
+import java.nio.ByteBuffer;
+import org.apache.kafka.common.message.UpdateFinalizedFeaturesResponseData;
+import org.apache.kafka.common.message.UpdateFinalizedFeaturesRequestData;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.types.Struct;
+
+public class UpdateFinalizedFeaturesRequest extends AbstractRequest {

Review comment:
       make variables final throught class
   add doc

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -3979,6 +3986,98 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        Call callViaLeastLoadedNode = new Call("describeFeatures", calcDeadlineMs(now, options.timeoutMs()),
+            new LeastLoadedNodeProvider()) {
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(
+                        new FeatureMetadata(
+                            apiVersionsResponse.finalizedFeatures(),
+                            apiVersionsResponse.finalizedFeaturesEpoch(),
+                            apiVersionsResponse.supportedFeatures()));
+                } else {
+                    future.completeExceptionally(
+                        Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        Call call = callViaLeastLoadedNode;
+        if (options.shouldUseControllerAsDestination()) {
+            call = new Call("describeFeatures", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {
+
+                @Override
+                ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                    return (ApiVersionsRequest.Builder) callViaLeastLoadedNode.createRequest(timeoutMs);
+                }
+
+                @Override
+                void handleResponse(AbstractResponse response) {
+                    callViaLeastLoadedNode.handleResponse(response);
+                }
+
+                @Override
+                void handleFailure(Throwable throwable) {
+                    callViaLeastLoadedNode.handleFailure(throwable);
+                }
+            };
+        }
+        runnable.call(call, now);
+        return new DescribeFeaturesResult(future);
+    }
+
+    @Override
+    public UpdateFinalizedFeaturesResult updateFinalizedFeatures(

Review comment:
       add test code in `KafkaAdminClientTest`

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -71,9 +90,37 @@ object SupportedFeatures extends Logging {
    *                    is empty, it means there were no feature incompatibilities found.
    */
   def incompatibleFeatures(finalized: Features[FinalizedVersionRange]): Features[FinalizedVersionRange] = {
+    BrokerFeatures.incompatibleFeatures(getSupportedFeatures, finalized, true)
+  }
+
+}
+
+object BrokerFeatures extends Logging {
+  /**
+   * Returns true if any of the provided finalized features are incompatible with the provided
+   * supported features.
+   *
+   * @param supported   The supported features to be compared
+   * @param finalized   The finalized features to be compared
+   *
+   * @return            - True if there are any incompatibilities.

Review comment:
       say "if there are any feature incompatibilities found."

##########
File path: clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
##########
@@ -319,7 +322,10 @@
     GROUP_SUBSCRIBED_TO_TOPIC(86, "Deleting offsets of a topic is forbidden while the consumer group is actively subscribed to it.",
         GroupSubscribedToTopicException::new),
     INVALID_RECORD(87, "This record has failed the validation on broker and hence will be rejected.", InvalidRecordException::new),
-    UNSTABLE_OFFSET_COMMIT(88, "There are unstable offsets that need to be cleared.", UnstableOffsetCommitException::new);
+    UNSTABLE_OFFSET_COMMIT(88, "There are unstable offsets that need to be cleared.", UnstableOffsetCommitException::new),
+    INCOMPATIBLE_FEATURES(89, "Could not apply finalized feature updates due to incompatible features.", IncompatibleFeaturesException::new),
+    FEATURE_VERSIONING_DISABLED(90, "Feature versioning system is disabled.", FeatureVersioningDisabledException::new),

Review comment:
       Eliminate and use INVALID_REQUEST

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -68,6 +69,36 @@ public ApiVersionsResponse(Struct struct, short version) {
         this(new ApiVersionsResponseData(struct, version));
     }
 
+    public ApiVersionsResponseData data() {
+        return data;
+    }
+
+    public Features<SupportedVersionRange> supportedFeatures() {
+        Map<String, SupportedVersionRange> features = new HashMap<>();

Review comment:
       final

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -68,6 +69,36 @@ public ApiVersionsResponse(Struct struct, short version) {
         this(new ApiVersionsResponseData(struct, version));
     }
 
+    public ApiVersionsResponseData data() {
+        return data;
+    }
+
+    public Features<SupportedVersionRange> supportedFeatures() {
+        Map<String, SupportedVersionRange> features = new HashMap<>();
+
+        for (SupportedFeatureKey key : data.supportedFeatures().valuesSet()) {
+            features.put(key.name(), new SupportedVersionRange(key.minVersion(),key.maxVersion()));
+        }
+
+        return Features.supportedFeatures(features);
+    }
+
+    public Features<FinalizedVersionRange> finalizedFeatures() {
+        Map<String, FinalizedVersionRange> features = new HashMap<>();

Review comment:
       final

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -68,6 +69,36 @@ public ApiVersionsResponse(Struct struct, short version) {
         this(new ApiVersionsResponseData(struct, version));
     }
 
+    public ApiVersionsResponseData data() {
+        return data;
+    }
+
+    public Features<SupportedVersionRange> supportedFeatures() {
+        Map<String, SupportedVersionRange> features = new HashMap<>();
+
+        for (SupportedFeatureKey key : data.supportedFeatures().valuesSet()) {
+            features.put(key.name(), new SupportedVersionRange(key.minVersion(),key.maxVersion()));

Review comment:
       space between "," and "key"

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -71,9 +90,37 @@ object SupportedFeatures extends Logging {
    *                    is empty, it means there were no feature incompatibilities found.
    */
   def incompatibleFeatures(finalized: Features[FinalizedVersionRange]): Features[FinalizedVersionRange] = {
+    BrokerFeatures.incompatibleFeatures(getSupportedFeatures, finalized, true)
+  }
+
+}
+
+object BrokerFeatures extends Logging {
+  /**
+   * Returns true if any of the provided finalized features are incompatible with the provided
+   * supported features.
+   *
+   * @param supported   The supported features to be compared
+   * @param finalized   The finalized features to be compared
+   *
+   * @return            - True if there are any incompatibilities.
+   *                    - False otherwise.
+   */
+  def hasIncompatibleFeatures(
+    supported: Features[SupportedVersionRange],

Review comment:
       s/supported/supportedFeatures
   same for other one

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +277,24 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def setupFeatureZNode(newNode: FeatureZNode): Unit = {

Review comment:
       add doc and explain various cases

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -24,13 +24,13 @@ import org.apache.kafka.common.feature.Features._
 import scala.jdk.CollectionConverters._
 
 /**
- * A common immutable object used in the Broker to define the latest features supported by the
- * Broker. Also provides API to check for incompatibilities between the latest features supported
- * by the Broker and cluster-wide finalized features.
+ * A class that defines the latest features supported by the Broker, and the finalized cluster-wide

Review comment:
       can improve by splitting into few lines

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -2945,6 +2950,113 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFinalizedFeatures(request: RequestChannel.Request): Unit = {
+    val updateFinalizedFeaturesRequest = request.body[UpdateFinalizedFeaturesRequest]
+    def sendResponseCallback(error: Errors, msgOverride: Option[String]): Unit = {
+      sendResponseExemptThrottle(request, new UpdateFinalizedFeaturesResponse(
+        new UpdateFinalizedFeaturesResponseData()
+          .setErrorCode(error.code())
+          .setErrorMessage(msgOverride.getOrElse(error.message()))))
+    }
+    if (!authorize(request.context, ALTER, CLUSTER, CLUSTER_NAME)) {
+      sendResponseCallback(Errors.CLUSTER_AUTHORIZATION_FAILED, Option.empty)
+    } else if (!controller.isActive) {
+      sendResponseCallback(Errors.NOT_CONTROLLER, Option.empty)
+    } else if (!config.isFeatureVersioningEnabled) {
+      sendResponseCallback(Errors.FEATURE_VERSIONING_DISABLED, Option.empty)
+    } else {
+      val targetFeaturesOrError = getTargetFinalizedFeaturesOrError(updateFinalizedFeaturesRequest)
+      targetFeaturesOrError match {
+        case Left(targetFeatures) =>
+          controller.updateFinalizedFeatures(targetFeatures, sendResponseCallback)
+        case Right(error) =>
+          sendResponseCallback(error.error, Some(error.message))
+      }
+    }
+  }
+
+  def getTargetFinalizedFeaturesOrError(request: UpdateFinalizedFeaturesRequest):

Review comment:
       add doc

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -39,21 +39,40 @@ object SupportedFeatures extends Logging {
    */
   @volatile private var supportedFeatures = emptySupportedFeatures
 
+  /**
+   * This is the cluster-wide finalized minimum version levels.
+   * This is currently empty, but in the future as we define supported features, this map can be
+   * populated in cases where minimum version level of a finalized feature is advanced beyond 1.
+   */
+  @volatile private var finalizedFeatureMinVersionLevels = Map[String, Short]()
+
   /**
    * Returns a reference to the latest features supported by the Broker.
    */
-  def get: Features[SupportedVersionRange] = {
+  def getSupportedFeatures: Features[SupportedVersionRange] = {
     supportedFeatures
   }
 
   // For testing only.
-  def update(newFeatures: Features[SupportedVersionRange]): Unit = {
+  def updateSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
     supportedFeatures = newFeatures
   }
 
+  def getFinalizedMinVersionLevel(feature: String): Short = {
+    finalizedFeatureMinVersionLevels.getOrElse(feature, 1)
+  }
+
   // For testing only.
-  def clear(): Unit = {
-    supportedFeatures = emptySupportedFeatures
+  def updateMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = {
+    finalizedFeatureMinVersionLevels = newMinVersionLevels
+  }
+
+  def getDefaultFinalizedFeatures: Features[FinalizedVersionRange] = {

Review comment:
       check braces ()

##########
File path: core/src/main/scala/kafka/server/KafkaConfig.scala
##########
@@ -1480,6 +1480,8 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean, dynamicConfigO
   def logMessageTimestampDifferenceMaxMs: Long = getLong(KafkaConfig.LogMessageTimestampDifferenceMaxMsProp)
   def logMessageDownConversionEnable: Boolean = getBoolean(KafkaConfig.LogMessageDownConversionEnableProp)
 
+  /** Feature  configuration*/

Review comment:
       remove these 2 lines

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1693,33 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def processUpdateFinalizedFeatures(newFeatures: Features[FinalizedVersionRange],
+                                             callback: UpdateFinalizedFeaturesCallback): Unit = {
+    if (isActive) {
+      val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+        BrokerFeatures.hasIncompatibleFeatures(broker.features, newFeatures)
+      })
+      if (numIncompatibleBrokers > 0) {
+        callback(
+          Errors.INCOMPATIBLE_FEATURES,
+          Some(
+            s"Could not apply finalized feature updates because $numIncompatibleBrokers brokers" +
+            " were found to have incompatible features."))
+      } else {
+        try {
+          zkClient.updateFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures))

Review comment:
       Shouldn't the code be waiting here?

##########
File path: clients/src/main/resources/common/message/UpdateFinalizedFeaturesRequest.json
##########
@@ -0,0 +1,35 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "request",
+  "name": "UpdateFinalizedFeaturesRequest",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "timeoutMs", "type": "int32", "versions": "0+", "default": "60000",

Review comment:
       Eliminate timeout?

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -53,6 +56,26 @@ object FinalizedFeatureCache extends Logging {
     featuresAndEpoch.isEmpty
   }
 
+  def waitUntilEpochOrThrow(expectedMinEpoch: Int, timeoutMs: Long): Unit = {

Review comment:
       Add unit test
   Add doc




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java
##########
@@ -17,9 +17,16 @@
 package org.apache.kafka.common.feature;
 
 import java.util.Map;
+import java.util.Objects;
+import org.apache.kafka.common.utils.Utils;
 
 /**
- * An extended {@link BaseVersionRange} representing the min/max versions for supported features.
+ * An extended {@link BaseVersionRange} representing the min, max and first active versions for a
+ * supported feature:
+ *  - minVersion: This is the minimum supported version for the feature.
+ *  - maxVersion: This the maximum supported version for the feature.
+ *  - firstActiveVersion: This is the first active version for the feature. Versions in the range

Review comment:
       @junrao Awesome. This is a very good point. The approach you proposed is very elegant, and we should shoot for it, when we’re giving the benefit of the doubt on deprecation to the broker binary version. I’ll update the KIP with details and share with community for feedback. As soon as that is done, I'll follow up in separate PR implementing this logic.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java
##########
@@ -17,9 +17,16 @@
 package org.apache.kafka.common.feature;
 
 import java.util.Map;
+import java.util.Objects;
+import org.apache.kafka.common.utils.Utils;
 
 /**
- * An extended {@link BaseVersionRange} representing the min/max versions for supported features.
+ * An extended {@link BaseVersionRange} representing the min, max and first active versions for a
+ * supported feature:
+ *  - minVersion: This is the minimum supported version for the feature.
+ *  - maxVersion: This the maximum supported version for the feature.
+ *  - firstActiveVersion: This is the first active version for the feature. Versions in the range

Review comment:
       @junrao :
   
   I'd like to discuss an example that cites a problem I'm concerned about.
   
   > In general, we tend to deprecate a version very slowly in AK. So, if the mistake is to deploy a new release that actually deprecates a supported version. Old clients are likely all gone. So, moving finalized min version to supported min version may not cause a big problem. We can just document that people should make sure old versions are no longer used before deploying new releases.
   
   Let's say we have some feature `F` whose:
    * Supported version range is: `[minVersion=1, maxVersion=6]`
    * Existing finalized version range in the cluster is: `[minVersionLevel=1, maxVersionLevel=6]`
   
   Now, let us say a point in time arrives when we need to deprecate the feature version `1`.
   Let us say we bump up supported `minVersion` to `2` in a subsequent major Kafka release.
   Before this new release is deployed, let us assume the cluster operator knows 100% that old clients that were using the feature at version `1` are gone, so this is not a problem.
   
   **PROBLEM:** Still, if we deploy this new release, the broker will consider the following as a feature version incompatibility.
    * Supported version range is: `[minVersion=2, maxVersion=6]`
    * Existing finalized version range in the cluster is: `[minVersionLevel=1, maxVersionLevel=6]`
   
   Upon startup of a broker thats using the new release binary, the above combination will crash the broker since supported `minVersion=2` is greater than `minVersionLevel=1`. Basically the versioning system thinks that there is now a broker that does not support `minVersionLevel=1`, which does not adhere to the rules of the system.
   
   Here is my thought: This is where `firstActiveVersion` becomes useful. By bumping it up during a release (instead of the supported feature's `minVersion`), we are able to get past this situation. When `firstActiveVersion`is advanced in the code, and the cluster is deployed, the controller (and all brokers) know that the advancement acts a request to the controller to act upon the feature deprecation (by writing the advanced value to the `FeatureZNode`). So, in this case we would release the broker with the supported feature version range: `[minVersion=1, firstActiveVersion=2, maxVersion=6]`, and the broker release wouldn't fail (because the intent is clearly expressed to the versioning system).
   
   What are your thoughts on the above?




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesOptions.java
##########
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Map;
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+/**
+ * Options for {@link AdminClient#updateFeatures(Map, UpdateFeaturesOptions)}.
+ *
+ * The API of this class is evolving. See {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class UpdateFeaturesOptions extends AbstractOptions<UpdateFeaturesOptions> {

Review comment:
       Yes, it is already added. The base class: `AbstractOptions` contains a `timeoutMs` attribute.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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


   @junrao Thanks for the review comments! I have done the change proposed in https://github.com/apache/kafka/pull/9001#discussion_r498574911 in the most recent commit: 4218f95904989028a469930d0c266362bf173ece . 


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -272,6 +281,161 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (string) and a range of versions (defined by a
+   * SupportedVersionRange). It refers to a feature that a particular broker advertises support for.
+   * Each broker advertises the version ranges of its own supported features in its own
+   * BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (string) and a range of version levels (defined
+   * by a FinalizedVersionRange). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there was an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has now been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). But the IBP config is still set to lower than KAFKA_2_7_IV0, and may be
+   *    set to a higher value later. In this case, we want to start with no finalized features and
+   *    allow the user to finalize them whenever they are ready i.e. in the future whenever the
+   *    user sets IBP config to be greater than or equal to KAFKA_2_7_IV0, then the user could start
+   *    finalizing the features. This process ensures we do not enable all the possible features
+   *    immediately after an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent.
+   *        - If the node is absent, it will react by creating a FeatureZNode with disabled status
+   *          and empty finalized features.
+   *        - Otherwise, if a node already exists in enabled status then the controller will just
+   *          flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled.
+   *         - If the node is in disabled status, the controller won’t upgrade all features immediately.
+   *           Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *           user finalize the features later.
+   *         - Otherwise, if a node already exists in enabled status then the controller will leave
+   *           the node umodified.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine there was an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker
+   *    binary has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and
+   *    higher). The controller will start up and find that a FeatureZNode is already present with
+   *    enabled status and existing finalized features. In such a case, the controller leaves the node
+   *    unmodified.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled,
+                                          brokerFeatures.defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      val newFeatures = existingFeatureZNode.status match {
+        case FeatureZNodeStatus.Enabled => existingFeatureZNode.features
+        case FeatureZNodeStatus.Disabled =>
+          if (!existingFeatureZNode.features.empty()) {
+            warn(s"FeatureZNode at path: ${FeatureZNode.path} with disabled status" +
+              " contains non-empty features.")

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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -43,7 +43,7 @@
  */
 public class ApiVersionsResponse extends AbstractResponse {
 
-    public static final int UNKNOWN_FINALIZED_FEATURES_EPOCH = -1;
+    public static final long UNKNOWN_FINALIZED_FEATURES_EPOCH = -1;

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -3109,6 +3110,37 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFeatures(request: RequestChannel.Request): Unit = {
+    val updateFeaturesRequest = request.body[UpdateFeaturesRequest]
+
+    def sendResponseCallback(errors: Either[ApiError, Map[String, ApiError]]): Unit = {
+      def createResponse(throttleTimeMs: Int): UpdateFeaturesResponse = {
+        errors match {
+          case Left(topLevelError) => {
+            val featureUpdateNoErrors = updateFeaturesRequest
+              .data().featureUpdates().asScala

Review comment:
       Done. Great point.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -977,14 +1179,30 @@ class KafkaController(val config: KafkaConfig,
 
   /**
    * Send the leader information for selected partitions to selected brokers so that they can correctly respond to
-   * metadata requests
+   * metadata requests. Particularly, when feature versioning is enabled, we filter out brokers with incompatible
+   * features from receiving the metadata requests. This is because we do not want to activate incompatible brokers,
+   * as these may have harmful consequences to the cluster.

Review comment:
       Done. I've changed the code such that we skip the broker registration if it's detected as incompatible.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1216,71 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker. It can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter. This is particularly useful if the user requires strongly consistent reads of
+     * finalized features.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the {@link DescribeFeaturesResult} containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);
+
+    /**
+     * Applies specified updates to finalized features. This operation is not transactional so it
+     * may succeed for some features while fail for others.
+     * <p>
+     * The API takes in a map of finalized feature names to {@link FeatureUpdate} that needs to be
+     * applied. Each entry in the map specifies the finalized feature to be added or updated or
+     * deleted, along with the new max feature version level value. This request is issued only to
+     * the controller since the API is only served by the controller. The return value contains an
+     * error code for each supplied {@link FeatureUpdate}, and the code indicates if the update
+     * succeeded or failed in the controller.
+     * <ul>
+     * <li>Downgrade of feature version level is not a regular operation/intent. It is only allowed
+     * in the controller if the {@link FeatureUpdate} has the allowDowngrade flag set - setting this
+     * flag conveys user intent to attempt downgrade of a feature max version level. Note that
+     * despite the allowDowngrade flag being set, certain downgrades may be rejected by the
+     * controller if it is deemed impossible.</li>
+     * <li>Deletion of a finalized feature version is not a regular operation/intent. It could be
+     * done by setting the allowDowngrade flag to true in the {@link FeatureUpdate}, and, setting
+     * the max version level to be less than 1.</li>
+     * </ul>
+     *<p>
+     * The following exceptions can be anticipated when calling {@code get()} on the futures
+     * obtained from the returned {@link UpdateFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.ClusterAuthorizationException}
+     *   If the authenticated user didn't have alter access to the cluster.</li>
+     *   <li>{@link org.apache.kafka.common.errors.InvalidRequestException}
+     *   If the request details are invalid. e.g., a non-existing finalized feature is attempted
+     *   to be deleted or downgraded.</li>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the updates could finish. It cannot be guaranteed whether
+     *   the updates succeeded or not.</li>
+     *   <li>{@link FeatureUpdateFailedException}
+     *   If the updates could not be applied on the controller, despite the request being valid.
+     *   This may be a temporary problem.</li>
+     * </ul>
+     * <p>
+     * This operation is supported by brokers with version 2.7.0 or higher.
+
+     * @param featureUpdates   the map of finalized feature name to {@link FeatureUpdate}
+     * @param options          the options to use
+     *
+     * @return                 the {@link UpdateFeaturesResult} containing the result
+     */
+    UpdateFeaturesResult updateFeatures(Map<String, FeatureUpdate> featureUpdates, UpdateFeaturesOptions options);

Review comment:
       Done. I've updated the KIP to align with whats used here.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1216,71 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker. It can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter. This is particularly useful if the user requires strongly consistent reads of
+     * finalized features.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the {@link DescribeFeaturesResult} containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);
+
+    /**
+     * Applies specified updates to finalized features. This operation is not transactional so it
+     * may succeed for some features while fail for others.
+     * <p>
+     * The API takes in a map of finalized feature names to {@link FeatureUpdate} that needs to be
+     * applied. Each entry in the map specifies the finalized feature to be added or updated or
+     * deleted, along with the new max feature version level value. This request is issued only to
+     * the controller since the API is only served by the controller. The return value contains an
+     * error code for each supplied {@link FeatureUpdate}, and the code indicates if the update
+     * succeeded or failed in the controller.
+     * <ul>
+     * <li>Downgrade of feature version level is not a regular operation/intent. It is only allowed
+     * in the controller if the {@link FeatureUpdate} has the allowDowngrade flag set - setting this
+     * flag conveys user intent to attempt downgrade of a feature max version level. Note that
+     * despite the allowDowngrade flag being set, certain downgrades may be rejected by the
+     * controller if it is deemed impossible.</li>
+     * <li>Deletion of a finalized feature version is not a regular operation/intent. It could be
+     * done by setting the allowDowngrade flag to true in the {@link FeatureUpdate}, and, setting
+     * the max version level to be less than 1.</li>
+     * </ul>
+     *<p>
+     * The following exceptions can be anticipated when calling {@code get()} on the futures
+     * obtained from the returned {@link UpdateFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.ClusterAuthorizationException}
+     *   If the authenticated user didn't have alter access to the cluster.</li>
+     *   <li>{@link org.apache.kafka.common.errors.InvalidRequestException}
+     *   If the request details are invalid. e.g., a non-existing finalized feature is attempted
+     *   to be deleted or downgraded.</li>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the updates could finish. It cannot be guaranteed whether
+     *   the updates succeeded or not.</li>
+     *   <li>{@link FeatureUpdateFailedException}
+     *   If the updates could not be applied on the controller, despite the request being valid.
+     *   This may be a temporary problem.</li>
+     * </ul>
+     * <p>
+     * This operation is supported by brokers with version 2.7.0 or higher.
+
+     * @param featureUpdates   the map of finalized feature name to {@link FeatureUpdate}
+     * @param options          the options to use
+     *
+     * @return                 the {@link UpdateFeaturesResult} containing the result
+     */
+    UpdateFeaturesResult updateFeatures(Map<String, FeatureUpdate> featureUpdates, UpdateFeaturesOptions options);

Review comment:
       Done. I've updated the KIP to align with whats used here, so both are the same now.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FeatureUpdate.java
##########
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;

Review comment:
       Done. I have now moved it to the package: `org.apache.kafka.clients.admin`.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/resources/common/message/ApiVersionsResponse.json
##########
@@ -55,8 +55,8 @@
           "about": "The maximum supported version for the feature." }
       ]
     },
-    {"name": "FinalizedFeaturesEpoch", "type": "int32", "versions": "3+",
-      "tag": 1, "taggedVersions": "3+", "default": "-1",
+    {"name": "FinalizedFeaturesEpoch", "type": "int64", "versions": "3+",

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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FinalizedVersionRange.java
##########
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+
+/**
+ * Represents a range of version levels supported by every broker in a cluster for some feature.
+ */
+public class FinalizedVersionRange {
+    private final short minVersionLevel;
+
+    private final short maxVersionLevel;
+
+    /**
+     * Raises an exception unless the following condition is met:
+     * minVersionLevel >= 1 and maxVersionLevel >= 1 and maxVersionLevel >= minVersionLevel.
+     *
+     * @param minVersionLevel   The minimum version level value.
+     * @param maxVersionLevel   The maximum version level value.
+     *
+     * @throws IllegalArgumentException   Raised when the condition described above is not met.
+     */
+    public FinalizedVersionRange(final short minVersionLevel, final short maxVersionLevel) {

Review comment:
       It is instantiated from `kafka.server.UpdateFeaturesTest`, so have to keep the c'tor public.
   Also this class is removed now, and we are just using the `VersionRange` class.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -272,6 +281,147 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (string) and a range of versions (defined by a
+   * SupportedVersionRange). It refers to a feature that a particular broker advertises support for.
+   * Each broker advertises the version ranges of its own supported features in its own
+   * BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (string) and a range of version levels (defined
+   * by a FinalizedVersionRange). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there was an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has now been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). But the IBP config is still set to lower than KAFKA_2_7_IV0, and may be
+   *    set to a higher value later. In this case, we want to start with no finalized features and
+   *    allow the user to finalize them whenever they are ready i.e. in the future whenever the
+   *    user sets IBP config to be greater than or equal to KAFKA_2_7_IV0, then the user could start
+   *    finalizing the features. This process ensures we do not enable all the possible features
+   *    immediately after an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent.
+   *        - If the node is absent, it will react by creating a FeatureZNode with disabled status
+   *          and empty finalized features.
+   *        - Otherwise, if a node already exists in enabled status then the controller will just
+   *          flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled.
+   *         - If the node is in disabled status, the controller won’t upgrade all features immediately.
+   *           Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *           user finalize the features later.
+   *         - Otherwise, if a node already exists in enabled status then the controller will leave
+   *           the node umodified.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine there was an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker
+   *    binary has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and
+   *    higher). The controller will start up and find that a FeatureZNode is already present with
+   *    enabled status and existing finalized features. In such a case, the controller leaves the node
+   *    unmodified.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled,
+                                          brokerFeatures.defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      if (!existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        val newVersion = updateFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled,

Review comment:
       This this case, existingFeatureZNode.features is expected to be empty? Could we log a warn if this is not the case and always set finalized to empty?

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java
##########
@@ -17,9 +17,16 @@
 package org.apache.kafka.common.feature;
 
 import java.util.Map;
+import java.util.Objects;
+import org.apache.kafka.common.utils.Utils;
 
 /**
- * An extended {@link BaseVersionRange} representing the min/max versions for supported features.
+ * An extended {@link BaseVersionRange} representing the min, max and first active versions for a
+ * supported feature:
+ *  - minVersion: This is the minimum supported version for the feature.
+ *  - maxVersion: This the maximum supported version for the feature.
+ *  - firstActiveVersion: This is the first active version for the feature. Versions in the range

Review comment:
       "we do not know whether all brokers in the cluster support a particular minVersion when the controller finalizes the minVersionLevel at a particular value." The controller knows the minSupportedVersion for all brokers, right? What if we do the following? When finalizing a feature, the controllers uses the highest minSupportedVersion across all brokers as finalizedMinVersion, as long as it's <= finalizedMaxVersion. On broker restart, we also advance finalizedMinVersion if the new broker's minSupportedVersion has advanced (assuming still <= finalizedMaxVersion). 

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -272,6 +281,147 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (string) and a range of versions (defined by a
+   * SupportedVersionRange). It refers to a feature that a particular broker advertises support for.
+   * Each broker advertises the version ranges of its own supported features in its own
+   * BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (string) and a range of version levels (defined
+   * by a FinalizedVersionRange). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there was an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has now been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). But the IBP config is still set to lower than KAFKA_2_7_IV0, and may be
+   *    set to a higher value later. In this case, we want to start with no finalized features and
+   *    allow the user to finalize them whenever they are ready i.e. in the future whenever the
+   *    user sets IBP config to be greater than or equal to KAFKA_2_7_IV0, then the user could start
+   *    finalizing the features. This process ensures we do not enable all the possible features
+   *    immediately after an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent.
+   *        - If the node is absent, it will react by creating a FeatureZNode with disabled status
+   *          and empty finalized features.
+   *        - Otherwise, if a node already exists in enabled status then the controller will just
+   *          flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled.
+   *         - If the node is in disabled status, the controller won’t upgrade all features immediately.
+   *           Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *           user finalize the features later.
+   *         - Otherwise, if a node already exists in enabled status then the controller will leave
+   *           the node umodified.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine there was an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker
+   *    binary has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and
+   *    higher). The controller will start up and find that a FeatureZNode is already present with
+   *    enabled status and existing finalized features. In such a case, the controller leaves the node
+   *    unmodified.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled,
+                                          brokerFeatures.defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      if (!existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {

Review comment:
       This test is not enough. The issue is that when a controller fails over, it's possible that new brokers have joined the cluster during the failover. So, if existingFeatureZNode is enabled, it may not be reflecting the state in those newly joined brokers. So, it seems that we need to do the validation for every broker during controller failover in that case.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1656,6 +1840,204 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns an ApiError object containing Errors.INVALID_REQUEST.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+
+    val supportedVersionRange = brokerFeatures.supportedFeatures.get(update.feature)
+    if (supportedVersionRange == null) {
+      Right(new ApiError(Errors.INVALID_REQUEST,
+                         "Could not apply finalized feature update because the provided feature" +
+                         " is not supported."))
+    } else {
+      var newVersionRange: FinalizedVersionRange = null
+      try {
+        newVersionRange = new FinalizedVersionRange(supportedVersionRange.min, update.maxVersionLevel)
+      } catch {
+        case _: IllegalArgumentException => {
+          // This exception means the provided maxVersionLevel is invalid. It is handled below
+          // outside of this catch clause.
+        }
+      }
+      if (newVersionRange == null) {
+        Right(new ApiError(Errors.INVALID_REQUEST,
+          "Could not apply finalized feature update because the provided" +
+          s" maxVersionLevel:${update.maxVersionLevel} is lower than the" +
+          s" supported minVersion:${supportedVersionRange.min}."))
+      } else {
+        val newFinalizedFeature =
+          Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange)))
+        val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+          BrokerFeatures.hasIncompatibleFeatures(broker.features, newFinalizedFeature)
+        })
+        if (numIncompatibleBrokers == 0) {
+          Left(newVersionRange)
+        } else {
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Could not apply finalized feature update because" +
+                             " brokers were found to have incompatible versions for the feature."))
+        }
+      }
+    }
+  }
+
+  /**
+   * Validates a feature update on an existing FinalizedVersionRange.
+   * If the validation succeeds, then, the return value contains:
+   * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature.
+   * 2. Option.empty, if the feature update was meant to delete the feature.
+   *
+   * If the validation fails, then returned value contains a suitable ApiError.
+   *
+   * @param update                 the feature update to be processed.
+   * @param existingVersionRange   the existing FinalizedVersionRange which can be empty when no
+   *                               FinalizedVersionRange exists for the associated feature
+   *
+   * @return                       the new FinalizedVersionRange to be updated into ZK or error
+   *                               as described above.
+   */
+  private def validateFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey,
+                                    existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = {
+    def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = {
+      newFinalizedVersionRangeOrIncompatibilityError(update)
+        .fold(versionRange => Left(Some(versionRange)), error => Right(error))
+    }
+
+    if (update.feature.isEmpty) {
+      // Check that the feature name is not empty.
+      Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty."))
+    } else {
+      // We handle deletion requests separately from non-deletion requests.
+      if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+        if (existingVersionRange.isEmpty) {
+          // Disallow deletion of a non-existing finalized feature.
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Can not delete non-existing finalized feature."))
+        } else {
+          Left(Option.empty)
+        }
+      } else if (update.maxVersionLevel() < 1) {
+        // Disallow deletion of a finalized feature without allowDowngrade flag set.
+        Right(new ApiError(Errors.INVALID_REQUEST,
+                           s"Can not provide maxVersionLevel: ${update.maxVersionLevel} less" +
+                           s" than 1 without setting the allowDowngrade flag to true in the request."))
+      } else {
+        existingVersionRange.map(existing =>
+          if (update.maxVersionLevel == existing.max) {
+            // Disallow a case where target maxVersionLevel matches existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not ${if (update.allowDowngrade) "downgrade" else "upgrade"}" +
+                               s" a finalized feature from existing maxVersionLevel:${existing.max}" +
+                               " to the same value."))
+          } else if (update.maxVersionLevel < existing.max && !update.allowDowngrade) {
+            // Disallow downgrade of a finalized feature without the allowDowngrade flag set.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature from existing" +
+                               s" maxVersionLevel:${existing.max} to provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} without setting the" +
+                               " allowDowngrade flag in the request."))
+          } else if (update.allowDowngrade && update.maxVersionLevel > existing.max) {
+            // Disallow a request that sets allowDowngrade flag without specifying a
+            // maxVersionLevel that's lower than the existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"When the allowDowngrade flag set in the request, the provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} can not be greater than" +
+                               s" existing maxVersionLevel:${existing.max}."))
+          } else if (update.maxVersionLevel < existing.min) {
+            // Disallow downgrade of a finalized feature below the existing finalized
+            // minVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature to maxVersionLevel:${update.maxVersionLevel}" +
+                               s" because it's lower than the existing minVersionLevel:${existing.min}."))
+          } else {
+            newVersionRangeOrError(update)
+          }
+        ).getOrElse(newVersionRangeOrError(update))
+      }
+    }
+  }
+
+  private def processFeatureUpdates(request: UpdateFeaturesRequest,
+                                    callback: UpdateFeaturesCallback): Unit = {
+    if (isActive) {
+      processFeatureUpdatesWithActiveController(request, callback)
+    } else {
+      callback(Left(new ApiError(Errors.NOT_CONTROLLER)))
+    }
+  }
+
+  private def processFeatureUpdatesWithActiveController(request: UpdateFeaturesRequest,
+                                                        callback: UpdateFeaturesCallback): Unit = {
+    val updates = request.data.featureUpdates
+    val existingFeatures = featureCache.get
+      .map(featuresAndEpoch => featuresAndEpoch.features.features().asScala)
+      .getOrElse(Map[String, FinalizedVersionRange]())
+    // A map with key being feature name and value being FinalizedVersionRange.
+    // This contains the target features to be eventually written to FeatureZNode.
+    val targetFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]() ++ existingFeatures
+    // A map with key being feature name and value being error encountered when the FeatureUpdate
+    // was applied.
+    val errors = scala.collection.mutable.Map[String, ApiError]()
+
+    // Below we process each FeatureUpdate using the following logic:
+    //  - If a FeatureUpdate is found to be valid, then:
+    //    - The corresponding entry in errors map would be updated to contain Errors.NONE.
+    //    - If the FeatureUpdate is an add or update request, then the targetFeatures map is updated
+    //      to contain the new FinalizedVersionRange for the feature.
+    //    - Otherwise if the FeatureUpdate is a delete request, then the feature is removed from the
+    //      targetFeatures map.
+    //  - Otherwise if a FeatureUpdate is found to be invalid, then:
+    //    - The corresponding entry in errors map would be updated with the appropriate ApiError.
+    //    - The entry in targetFeatures map is left untouched.
+    updates.asScala.iterator.foreach { update =>
+      validateFeatureUpdate(update, existingFeatures.get(update.feature())) match {
+        case Left(newVersionRangeOrNone) =>
+          newVersionRangeOrNone match {
+            case Some(newVersionRange) => targetFeatures += (update.feature() -> newVersionRange)
+            case None => targetFeatures -= update.feature()
+          }
+          errors += (update.feature() -> new ApiError(Errors.NONE))
+        case Right(featureUpdateFailureReason) =>
+          errors += (update.feature() -> featureUpdateFailureReason)
+      }
+    }
+
+    // If the existing and target features are the same, then, we skip the update to the
+    // FeatureZNode as no changes to the node are required. Otherwise, we replace the contents
+    // of the FeatureZNode with the new features. This may result in partial or full modification
+    // of the existing finalized features in ZK.
+    try {
+      if (!existingFeatures.equals(targetFeatures)) {
+        val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, Features.finalizedFeatures(targetFeatures.asJava))
+        val newVersion = zkClient.updateFeatureZNode(newNode)

Review comment:
       Should we call updateFeatureZNode() so that we can get the logging?




----------------------------------------------------------------
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 edited a comment on pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #9001:
URL: https://github.com/apache/kafka/pull/9001#issuecomment-702023940


   > Just looked through the change for firstActiveVersion, which makes sense. The only question I have is that if I'm doing a bold release by removing the deprecated code completely, such that previously I have version range [1, 6] and now the code only supports [4, 6] for a pair [min, max], would my deployment be successful automatically?
   
   @abbccdda The deployment will fail if the feature was finalized at say [1, 6] previously, but the new broker only supports [4, 6]. This is where `firstActiveVersion` becomes useful. By bumping it up during a release (instead of the supported feature's `minVersion`), we are able to get past this situation. When `firstActiveVersion` is advanced in the code, and the cluster is deployed, the controller (and all brokers) know that the advancement acts a request to the controller to act upon the feature deprecation (by writing the advanced value to `FeatureZNode`).
   
   Thanks for the review! I've addressed the other comments from your most recent pass.


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,179 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the latest features supported by the Broker and also provides APIs to
+ * check for incompatibilities between the features supported by the Broker and finalized features.
+ * The class also enables feature version level deprecation, as explained below. This class is
+ * immutable in production. It provides few APIs to mutate state only for the purpose of testing.
+ *
+ * Feature version level deprecation:
+ * ==================================
+ *
+ * Deprecation of certain version levels of a feature is a process to stop supporting the
+ * functionality offered by the feature at a those version levels, across the entire Kafka cluster.
+ * Feature version deprecation is a simple 2-step process explained below. In each step below, an
+ * example is provided to help understand the process better:
+ *
+ * STEP 1:

Review comment:
       Yes, correct. I have updated the doc mentioning the same.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -272,6 +281,199 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (string) and a range of versions (defined by a
+   * SupportedVersionRange). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of its own supported features in its
+   * own BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (string) and a range of version levels (defined
+   * by a FinalizedVersionRange). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. This process ensures we do not enable all the possible features immediately after
+   *    an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, it will
+   *        react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, its existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.defaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) =>
+            val brokerDefaultVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (brokerDefaultVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (brokerDefaultVersionRange.max() >= existingVersionRange.max() &&
+                       brokerDefaultVersionRange.min() <= existingVersionRange.max()) {
+              // Using the change below, we deprecate all version levels in the range:
+              // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1].
+              //
+              // NOTE: if existingVersionRange.min() equals brokerDefaultVersionRange.min(), then
+              // we do not deprecate any version levels (since there is none to be deprecated).
+              //
+              // Examples:
+              // 1. brokerDefaultVersionRange = [4, 7] and existingVersionRange = [1, 5].
+              //    In this case, we deprecate all version levels in the range: [1, 3].
+              // 2. brokerDefaultVersionRange = [4, 7] and existingVersionRange = [4, 5].
+              //    In this case, we do not deprecate any version levels since
+              //    brokerDefaultVersionRange.min() equals existingVersionRange.min().
+              (featureName, new FinalizedVersionRange(brokerDefaultVersionRange.min(), existingVersionRange.max()))
+            } else {
+              // This is a serious error. We should never be reaching here, since we already
+              // verify once during KafkaServer startup that existing finalized feature versions in
+              // the FeatureZNode contained no incompatibilities. If we are here, it means that one
+              // of the following is true:
+              // 1. The existing version levels fall completely outside the range of the default
+              // finalized version levels (i.e. no intersection), or
+              // 2. The existing version levels are incompatible with default finalized version
+              // levels.
+              //
+              // Examples of invalid cases that can cause this exception to be triggered:
+              // 1. No intersection      : brokerDefaultVersionRange = [4, 7] and existingVersionRange = [2, 3].
+              // 2. No intersection      : brokerDefaultVersionRange = [2, 3] and existingVersionRange = [4, 7].
+              // 3. Incompatible versions: brokerDefaultVersionRange = [2, 3] and existingVersionRange = [1, 7].
+              throw new IllegalStateException(
+                s"Can not update minimum version level in finalized feature: $featureName,"
+                + s" since the existing $existingVersionRange is not eligible for a change"
+                + s" based on the default $brokerDefaultVersionRange. This should never happen"
+                + s" since feature version incompatibilities are already checked during"
+                + s" Kafka server startup.")
+            }
+        }.asJava)
+      }
+      val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures)
+      if (!newFeatureZNode.equals(existingFeatureZNode)) {
+        val newVersion = updateFeatureZNode(newFeatureZNode)
+        featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+      }
+    }
+  }
+
+  /**
+   * Disables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with disabled status. This status means the feature versioning system
+   * (KIP-584) is disabled, and, the finalized features stored in the FeatureZNode are not relevant.
+   * This status should be written by the controller to the FeatureZNode only when the broker
+   * IBP config is less than KAFKA_2_7_IV0.
+   *
+   * NOTE:
+   * 1. When this method returns, existing finalized features (if any) will be cleared from the
+   *    FeatureZNode.
+   * 2. This method, unlike enableFeatureVersioning() need not wait for the FinalizedFeatureCache
+   *    to be updated, because, such updates to the cache (via FinalizedFeatureChangeListener)
+   *    are disabled when IBP config is < than KAFKA_2_7_IV0.
+   */
+  private def disableFeatureVersioning(): Unit = {
+    val newNode = FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures())
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      createFeatureZNode(newNode)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      if (!existingFeatureZNode.status.equals(FeatureZNodeStatus.Disabled)) {
+        updateFeatureZNode(newNode)
+      }
+    }
+  }
+
+  private def setupFeatureVersioning(): Unit = {
+    if (config.isFeatureVersioningEnabled) {

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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1656,6 +1893,203 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns an ApiError object containing Errors.INVALID_REQUEST.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+
+    val supportedVersionRange = brokerFeatures.supportedFeatures.get(update.feature)
+    if (supportedVersionRange == null) {
+      Right(new ApiError(Errors.INVALID_REQUEST,
+                         "Could not apply finalized feature update because the provided feature" +
+                         " is not supported."))
+    } else {
+      var newVersionRange: FinalizedVersionRange = null
+      try {
+        newVersionRange = new FinalizedVersionRange(supportedVersionRange.firstActiveVersion, update.maxVersionLevel)
+      } catch {
+        case _: IllegalArgumentException => {
+          // This exception means the provided maxVersionLevel is invalid. It is handled below
+          // outside of this catch clause.
+        }
+      }
+      if (newVersionRange == null) {
+        Right(new ApiError(Errors.INVALID_REQUEST,
+          "Could not apply finalized feature update because the provided" +
+          s" maxVersionLevel:${update.maxVersionLevel} is lower than the" +
+          s" first active version:${supportedVersionRange.firstActiveVersion}."))
+      } else {
+        val newFinalizedFeature =
+          Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange)))
+        val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+          BrokerFeatures.hasIncompatibleFeatures(broker.features, newFinalizedFeature)
+        })
+        if (numIncompatibleBrokers == 0) {
+          Left(newVersionRange)
+        } else {
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Could not apply finalized feature update because" +
+                             " brokers were found to have incompatible versions for the feature."))
+        }
+      }
+    }
+  }
+
+  /**
+   * Validates a feature update on an existing FinalizedVersionRange.
+   * If the validation succeeds, then, the return value contains:
+   * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature.
+   * 2. Option.empty, if the feature update was meant to delete the feature.
+   *
+   * If the validation fails, then returned value contains a suitable ApiError.
+   *
+   * @param update                 the feature update to be processed.
+   * @param existingVersionRange   the existing FinalizedVersionRange which can be empty when no
+   *                               FinalizedVersionRange exists for the associated feature
+   *
+   * @return                       the new FinalizedVersionRange to be updated into ZK or error
+   *                               as described above.
+   */
+  private def validateFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey,
+                                   existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = {
+    def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = {
+      newFinalizedVersionRangeOrIncompatibilityError(update)
+        .fold(versionRange => Left(Some(versionRange)), error => Right(error))
+    }
+
+    if (update.feature.isEmpty) {
+      // Check that the feature name is not empty.
+      Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty."))
+    } else {
+      // We handle deletion requests separately from non-deletion requests.
+      if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+        if (existingVersionRange.isEmpty) {
+          // Disallow deletion of a non-existing finalized feature.
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Can not delete non-existing finalized feature."))
+        } else {
+          Left(Option.empty)
+        }
+      } else if (update.maxVersionLevel() < 1) {
+        // Disallow deletion of a finalized feature without allowDowngrade flag set.
+        Right(new ApiError(Errors.INVALID_REQUEST,
+                           s"Can not provide maxVersionLevel: ${update.maxVersionLevel} less" +
+                           s" than 1 without setting the allowDowngrade flag to true in the request."))
+      } else {
+        existingVersionRange.map(existing =>
+          if (update.maxVersionLevel == existing.max) {
+            // Disallow a case where target maxVersionLevel matches existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not ${if (update.allowDowngrade) "downgrade" else "upgrade"}" +
+                               s" a finalized feature from existing maxVersionLevel:${existing.max}" +
+                               " to the same value."))
+          } else if (update.maxVersionLevel < existing.max && !update.allowDowngrade) {
+            // Disallow downgrade of a finalized feature without the allowDowngrade flag set.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature from existing" +
+                               s" maxVersionLevel:${existing.max} to provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} without setting the" +
+                               " allowDowngrade flag in the request."))
+          } else if (update.allowDowngrade && update.maxVersionLevel > existing.max) {
+            // Disallow a request that sets allowDowngrade flag without specifying a
+            // maxVersionLevel that's lower than the existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"When the allowDowngrade flag set in the request, the provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} can not be greater than" +
+                               s" existing maxVersionLevel:${existing.max}."))
+          } else if (update.maxVersionLevel < existing.min) {
+            // Disallow downgrade of a finalized feature below the existing finalized
+            // minVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature to maxVersionLevel:${update.maxVersionLevel}" +
+                               s" because it's lower than the existing minVersionLevel:${existing.min}."))
+          } else {
+            newVersionRangeOrError(update)
+          }
+        ).getOrElse(newVersionRangeOrError(update))
+      }
+    }
+  }
+
+  private def processFeatureUpdates(request: UpdateFeaturesRequest,
+                                    callback: UpdateFeaturesCallback): Unit = {
+    if (isActive) {
+      processFeatureUpdatesWithActiveController(request, callback)
+    } else {
+      callback(Left(new ApiError(Errors.NOT_CONTROLLER)))
+    }
+  }
+
+  private def processFeatureUpdatesWithActiveController(request: UpdateFeaturesRequest,
+                                                        callback: UpdateFeaturesCallback): Unit = {
+    val updates = request.data.featureUpdates
+    val existingFeatures = featureCache.get
+      .map(featuresAndEpoch => featuresAndEpoch.features.features().asScala)
+      .getOrElse(Map[String, FinalizedVersionRange]())
+    // A map with key being feature name and value being FinalizedVersionRange.
+    // This contains the target features to be eventually written to FeatureZNode.
+    val targetFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]() ++ existingFeatures
+    // A map with key being feature name and value being error encountered when the FeatureUpdate
+    // was applied.
+    val errors = scala.collection.mutable.Map[String, ApiError]()
+
+    // Below we process each FeatureUpdate using the following logic:
+    //  - If a FeatureUpdate is found to be valid, then:
+    //    - The corresponding entry in errors map would be updated to contain ApiError(Errors.NONE).
+    //    - If the FeatureUpdate is an add or update request, then the targetFeatures map is updated
+    //      to contain the new FinalizedVersionRange for the feature.
+    //    - Otherwise if the FeatureUpdate is a delete request, then the feature is removed from the
+    //      targetFeatures map.
+    //  - Otherwise if a FeatureUpdate is found to be invalid, then:
+    //    - The corresponding entry in errors map would be updated with the appropriate ApiError.
+    //    - The entry in targetFeatures map is left untouched.
+    updates.asScala.iterator.foreach { update =>
+      validateFeatureUpdate(update, existingFeatures.get(update.feature())) match {
+        case Left(newVersionRangeOrNone) =>
+          newVersionRangeOrNone
+            .map(newVersionRange => targetFeatures += (update.feature() -> newVersionRange))
+            .getOrElse(targetFeatures -= update.feature())
+          errors += (update.feature() -> new ApiError(Errors.NONE))
+        case Right(featureUpdateFailureReason) =>
+          errors += (update.feature() -> featureUpdateFailureReason)
+      }
+    }
+
+    // If the existing and target features are the same, then, we skip the update to the
+    // FeatureZNode as no changes to the node are required. Otherwise, we replace the contents
+    // of the FeatureZNode with the new features. This may result in partial or full modification
+    // of the existing finalized features in ZK.
+    try {
+      if (!existingFeatures.equals(targetFeatures)) {
+        val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, Features.finalizedFeatures(targetFeatures.asJava))
+        val newVersion = zkClient.updateFeatureZNode(newNode)
+        featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+      }
+    } catch {
+      // For all features that correspond to valid FeatureUpdate (i.e. error is Errors.NONE),
+      // we set the error as Errors.FEATURE_UPDATE_FAILED since the FeatureZNode update has failed
+      // for these. For the rest, the existing error is left untouched.
+      case e: Exception =>
+        errors.foreach { case (feature, apiError) =>
+          if (apiError.error() == Errors.NONE) {
+            errors(feature) = new ApiError(Errors.FEATURE_UPDATE_FAILED,
+                                           Errors.FEATURE_UPDATE_FAILED.message() + " Error: " + e)

Review comment:
       Done. Good point. I'm now logging just a warning and I've removed the stacktrace from the return value.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -272,6 +281,199 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (string) and a range of versions (defined by a
+   * SupportedVersionRange). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of its own supported features in its
+   * own BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (string) and a range of version levels (defined
+   * by a FinalizedVersionRange). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. This process ensures we do not enable all the possible features immediately after
+   *    an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, it will
+   *        react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, its existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.defaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)

Review comment:
       I agree. But note that in this method, we do not process an `UpdateFeaturesRequest`. This method is only called during controller election to setup feature versioning. So, I have incorporated your suggestion at the point where we process the request, look for `def processFeatureUpdatesWithActiveController` in this file where now I set the ZK write timeout to be `min(timeoutMs, config.zkConnectionTimeoutMs)`.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -34,7 +34,7 @@ import scala.concurrent.TimeoutException
  *
  * @param zkClient     the Zookeeper client
  */
-class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+class FinalizedFeatureChangeListener(private val featureCache: FinalizedFeatureCache, private val zkClient: KafkaZkClient) extends Logging {

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 edited a comment on pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #9001:
URL: https://github.com/apache/kafka/pull/9001#issuecomment-702023940


   > Just looked through the change for firstActiveVersion, which makes sense. The only question I have is that if I'm doing a bold release by removing the deprecated code completely, such that previously I have version range [1, 6] and now the code only supports [4, 6] for a pair [min, max], would my deployment be successful automatically?
   
   @abbccdda Thanks for the review! I've addressed the comments from your most recent pass in a7f4860f5f8bb87cfb01452e208ff8f4e45bcd8b. To answer your question, the deployment will fail if the feature was finalized at say `[minVersionLevel=1, maxVersionLevel=6]` previously, but the new broker only supports version range: `[minVersion=4, maxVersion=6]`. This is where `firstActiveVersion` becomes useful. By bumping it up during a release (instead of the supported feature's `minVersion`), we are able to get past this situation. When `firstActiveVersion` is advanced in the code, and the cluster is deployed, the controller (and all brokers) know that the advancement acts a request to the controller to act upon the feature deprecation (by writing the advanced value to `FeatureZNode`). So, in this case we would release the broker with the supported feature version range: `[minVersion=1, firstActiveVersion=4, maxVersion=6]`.
   
   


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/resources/common/message/UpdateFeaturesResponse.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "response",
+  "name": "UpdateFeaturesResponse",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [

Review comment:
       I don't see that we consistently use a top level error code, so I will leave it as it is. It feels OK for this api to not use it as it makes little difference.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,187 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the latest features supported by the Broker and also provides APIs to
+ * check for incompatibilities between the features supported by the Broker and finalized features.
+ * The class also enables feature version level deprecation, as explained below. This class is
+ * immutable in production. It provides few APIs to mutate state only for the purpose of testing.
+ *
+ * Feature version level deprecation:
+ * ==================================
+ *
+ * Deprecation of certain version levels of a feature is a process to stop supporting the
+ * functionality offered by the feature at those version levels, across the entire Kafka cluster.
+ * Feature version deprecation is a simple 2-step process explained below. In each step below, an
+ * example is provided to help understand the process better:
+ *
+ * STEP 1:
+ * =======
+ *
+ * In the first step, a major Kafka release is made with a Broker code change (explained later
+ * below) that establishes the intent to deprecate certain versions of one or more features
+ * cluster-wide. When this new Kafka release is deployed to the cluster, deprecated finalized
+ * feature versions are no longer advertised to the client, but they can still be used by existing
+ * connections. The way it works is that the feature versioning system (via the controller) will
+ * automatically persist the new minVersionLevel for the feature in ZK to propagate the deprecation
+ * of certain versions. After this happens, any external client that queries the Broker to learn the
+ * feature versions will at some point start to see the new value for the finalized minVersionLevel
+ * for the feature. The external clients are expected to stop using the deprecated versions at least
+ * by the time that they learn about it.
+ *
+ * Here is how the above code change needs to be done:
+ * In order to deprecate feature version levels, in the supportedFeatures map you need to supply a
+ * specific firstActiveVersion value that's higher than the minVersion for the feature. The
+ * value for firstActiveVersion should be 1 beyond the highest version that you intend to deprecate
+ * for that feature. Whenever the controller is elected or the features are finalized via the
+ * ApiKeys.UPDATE_FEATURES api, the feature version levels in the closed range:
+ * [minVersion, firstActiveVersion - 1] are automatically deprecated in ZK by the controller logic.
+ *
+ * Example:
+ * - Let us assume the existing finalized feature in ZK:
+ *   {
+ *      "feature" -> FinalizedVersionRange(minVersionLevel=1, maxVersionLevel=5)
+ *   }
+ *   Now, supposing you would like to deprecate feature version levels: [1, 2].
+ *   Then, in the supportedFeatures map you should supply the following:
+ *   supportedFeatures = {
+ *     "feature" -> SupportedVersionRange(minVersion=1, firstActiveVersion=3, maxVersion=5)
+ *   }
+ * - If you do NOT want to deprecate a version level for a feature, then, in the supportedFeatures

Review comment:
       Done. I've provided an overloaded c'tor now in `org.apache.kafka.common.feature.SupportedVersionRange` that only takes `minVersion` and `maxVersion` as parameters.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java
##########
@@ -17,9 +17,16 @@
 package org.apache.kafka.common.feature;
 
 import java.util.Map;
+import java.util.Objects;
+import org.apache.kafka.common.utils.Utils;
 
 /**
- * An extended {@link BaseVersionRange} representing the min/max versions for supported features.
+ * An extended {@link BaseVersionRange} representing the min, max and first active versions for a
+ * supported feature:
+ *  - minVersion: This is the minimum supported version for the feature.
+ *  - maxVersion: This the maximum supported version for the feature.
+ *  - firstActiveVersion: This is the first active version for the feature. Versions in the range

Review comment:
       @junrao Does the below feel right to you?
   
   The key thing seems to be that you feel it is rare to deprecate feature versions in AK. I agree with the same. So, I propose we just do not have to solve the deprecation problem in this PR, until we find a clear route that the AK community agrees with. In this PR I propose to revert the `firstActiveVersion` change, leaving the rest of the things the way they are. In the future, we can develop a concrete solution for version deprecation i.e. the part on how to advance `minVersion` of supported feature, may be (or may not be) using `firstActiveVersion` or other ways (it is up for discussion, maybe in a separate KIP). I have made this change in the most recent commit: 4218f95904989028a469930d0c266362bf173ece.
   
   Going back to your point:
   > I was thinking what if we relax the current check by just making sure that maxVersion of finalized is within the supported range. Basically in your example, if supported minVersion goes to 2, it's still allowed since it's less than maxVersion of finalized. However, if supported minVersion goes to 7, this fails the broker since it's more than maxVersion of finalized.
   
   There is a consequence to relaxing the current check:
   The controller can not effectively finalize `minVersionLevel` for the feature, because, with a relaxed check we do not know whether all brokers in the cluster support a particular `minVersion` when the controller finalizes the `minVersionLevel` at a particular value. It seems useful to keep the concept of `minVersionLevel` like the way it is now (i.e. it is the lowest version guaranteed to be supported by any broker in the cluster for a feature). And as I said above, in the future, we can decide on ways to mutate it safely (maybe through `firstActiveVersion` or other means).
   
   




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java
##########
@@ -17,9 +17,16 @@
 package org.apache.kafka.common.feature;
 
 import java.util.Map;
+import java.util.Objects;
+import org.apache.kafka.common.utils.Utils;
 
 /**
- * An extended {@link BaseVersionRange} representing the min/max versions for supported features.
+ * An extended {@link BaseVersionRange} representing the min, max and first active versions for a
+ * supported feature:
+ *  - minVersion: This is the minimum supported version for the feature.
+ *  - maxVersion: This the maximum supported version for the feature.
+ *  - firstActiveVersion: This is the first active version for the feature. Versions in the range

Review comment:
       @junrao :
   
   I'd like to discuss an example that cites a problem I'm concerned about.
   
   > In general, we tend to deprecate a version very slowly in AK. So, if the mistake is to deploy a new release that actually deprecates a supported version. Old clients are likely all gone. So, moving finalized min version to supported min version may not cause a big problem. We can just document that people should make sure old versions are no longer used before deploying new releases.
   
   Let's say we have some feature `F` whose:
    * Supported version range is: `[minVersion=1, maxVersion=6]`
    * Existing finalized version range in the cluster is: `[minVersionLevel=1, maxVersionLevel=6]`
   
   Now, let us say a point in time arrives when we need to deprecate the feature version `1`.
   Let us say we bump up supported `minVersion` to `2` in a subsequent major Kafka release.
   Before this new release is deployed, let us assume the cluster operator knows 100% that old clients that were using the feature at version `1` are gone, so this is not a problem.
   
   **PROBLEM:** Still, if we deploy this new release, the broker will consider the following as a feature version incompatibility.
    * Supported version range is: `[minVersion=2, maxVersion=6]`
    * Existing finalized version range in the cluster is: `[minVersionLevel=1, maxVersionLevel=6]`
   
   Upon startup of a broker thats using the new release binary, the above combination will crash the broker since supported `minVersion=2` is greater than `minVersionLevel=1`. Basically the versioning system thinks that there is now a broker that does not support `minVersionLevel=1`, which does not adhere to the rules of the system. We currently do feature version incompatibility checks during KafkaServer startup sequence, [here is the code](https://github.com/confluentinc/ce-kafka/blob/master/core/src/main/scala/kafka/server/KafkaServer.scala#L398).
   
   Here is my thought: This is where `firstActiveVersion` becomes useful. By bumping it up during a release (instead of the supported feature's `minVersion`), we are able to get past this situation. When `firstActiveVersion`is advanced in the code, and the cluster is deployed, the controller (and all brokers) know that the advancement acts a request to the controller to act upon the feature deprecation (by writing the advanced value to the `FeatureZNode`). So, in this case we would release the broker with the supported feature version range: `[minVersion=1, firstActiveVersion=2, maxVersion=6]`, and the broker release wouldn't fail (because the intent is clearly expressed to the versioning system).
   
   What are your thoughts on the above?
   Is there a different way to solve it better that I'm missing, without compromising the versioning checks enforced by the system?




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/resources/common/message/UpdateFeaturesRequest.json
##########
@@ -0,0 +1,35 @@
+// 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.
+
+{
+  "apiKey": 57,
+  "type": "request",
+  "name": "UpdateFeaturesRequest",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "timeoutMs", "type": "int32", "versions": "0+", "default": "60000",

Review comment:
       Done. Updated the KIP.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java
##########
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+/**
+ * Options for {@link AdminClient#describeFeatures(DescribeFeaturesOptions)}.
+ *
+ * The API of this class is evolving. See {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeFeaturesOptions extends AbstractOptions<DescribeFeaturesOptions> {

Review comment:
       Done. Updated the KIP.
   
   




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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


   @junrao Thanks for the review! I've addressed the comments, the PR is ready for another pass.


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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


   @junrao Thanks a lot for the review! I've addressed the comments in the recent commit: 06d8b47131f168db88e4f7d5bda3dd025ba9a2a2.


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,179 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the latest features supported by the Broker and also provides APIs to
+ * check for incompatibilities between the features supported by the Broker and finalized features.
+ * The class also enables feature version level deprecation, as explained below. This class is
+ * immutable in production. It provides few APIs to mutate state only for the purpose of testing.
+ *
+ * Feature version level deprecation:
+ * ==================================
+ *
+ * Deprecation of certain version levels of a feature is a process to stop supporting the
+ * functionality offered by the feature at a those version levels, across the entire Kafka cluster.
+ * Feature version deprecation is a simple 2-step process explained below. In each step below, an
+ * example is provided to help understand the process better:
+ *
+ * STEP 1:
+ * In the first step, a major Kafka release is made with a Broker code change (explained later
+ * below) that establishes the intent to deprecate certain versions of one or more features
+ * cluster-wide. When this new Kafka release is deployed to the cluster, the feature versioning
+ * system (via the controller) will automatically persist the new minVersionLevel for the feature in
+ * Zk to propagate the deprecation of certain versions. After this happens, any external client that
+ * queries the Broker to learn the feature versions will at some point start to see the new value
+ * for the finalized minVersionLevel for the feature. This makes the version deprecation permanent.
+ *
+ * Here is how the above code change needs to be done:
+ * In order to deprecate feature version levels, in the supportedFeatures map you need to supply a
+ * specific firstActiveVersion value that's higher than the minVersion for the feature. The
+ * value for firstActiveVersion should be 1 beyond the highest version that you intend to deprecate
+ * for that feature. When features are finalized via the ApiKeys.UPDATE_FEATURES api, the feature
+ * version levels in the closed range: [minVersion, firstActiveVersion - 1] are automatically
+ * deprecated in ZK by the controller logic.
+ * Example:
+ * - Let us assume the existing finalized feature in ZK:
+ *   {
+ *      "feature_1" -> FinalizedVersionRange(minVersionLevel=1, maxVersionLevel=5)
+ *   }
+ *   Now, supposing you would like to deprecate feature version levels: [1, 2].
+ *   Then, in the supportedFeatures map you should supply the following:
+ *   supportedFeatures = {
+ *     "feature1" -> SupportedVersionRange(minVersion=1, firstActiveVersion=3, maxVersion=5)
+ *   }
+ * - If you do NOT want to deprecate a version level for a feature, then in the supportedFeatures
+ *   map you should supply the firstActiveVersion to be the same as the minVersion supplied for that
+ *   feature.
+ *   Example:
+ *   supportedFeatures = {
+ *     "feature1" -> SupportedVersionRange(minVersion=1, firstActiveVersion=1, maxVersion=5)
+ *   }
+ *   This indicates no intent to deprecate any version levels for the feature.
+ *
+ * STEP 2:
+ * After the first step is over, you may (at some point) want to permanently remove the code/logic
+ * for the functionality offered by the deprecated feature versions. This is the second step. Here a
+ * subsequent major Kafka release is made with another Broker code change that removes the code for
+ * the functionality offered by the deprecated feature versions. This would completely drop support
+ * for the deprecated versions. Such a code change needs to be supplemented by supplying a
+ * suitable higher minVersion value for the feature in the supportedFeatures map.
+ * Example:
+ * - In the example above in step 1, we showed how to deprecate version levels [1, 2] for
+ *   "feature_1". Now let us assume the following finalized feature in ZK (after the deprecation
+ *   has been carried out):
+ *   {
+ *     "feature_1" -> FinalizedVersionRange(minVersionLevel=3, maxVersionLevel=5)
+ *   }
+ *   Now, supposing you would like to permanently remove support for feature versions: [1, 2].
+ *   Then, in the supportedFeatures map you should now supply the following:
+ *   supportedFeatures = {
+ *     "feature1" -> SupportedVersionRange(minVersion=3, firstActiveVersion=3, maxVersion=5)

Review comment:
       Done. The `firstActiveVersion` is now part of `ApiVersionsResponse`. I added it in the recent commit: a7f4860f5f8bb87cfb01452e208ff8f4e45bcd8b.




----------------------------------------------------------------
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 edited a comment on pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #9001:
URL: https://github.com/apache/kafka/pull/9001#issuecomment-702023940


   > Just looked through the change for firstActiveVersion, which makes sense. The only question I have is that if I'm doing a bold release by removing the deprecated code completely, such that previously I have version range [1, 6] and now the code only supports [4, 6] for a pair [min, max], would my deployment be successful automatically?
   
   @abbccdda Thanks for the review! I've addressed the comments from your most recent pass in a7f4860f5f8bb87cfb01452e208ff8f4e45bcd8b. To answer your question, the deployment will fail if the feature was finalized at say `[minVersionLevel=1, maxVersionLevel=6]` previously, but the new broker only supports version range: `[minVersion=4, maxVersion=6]`. This is where `firstActiveVersion` becomes useful. By bumping it up during a release (instead of the supported feature's `minVersion`), we are able to get past this situation. When `firstActiveVersion` is advanced in the code, and the cluster is deployed, the controller (and all brokers) know that the advancement acts a request to the controller to act upon the feature deprecation (by writing the advanced value to `FeatureZNode`). So, in this case we would release the broker with the supported feature version range: `[minVersion=1, firstActiveVersion=4, maxVersion=6]`, and the deployment wouldn't fail.
   
   


----------------------------------------------------------------
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] chia7712 commented on a change in pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4335,6 +4343,150 @@ void handleFailure(Throwable throwable) {
                 .hi(password, salt, iterations);
     }
 
+    public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        final NodeProvider provider =
+            options.sendRequestToController() ? new ControllerNodeProvider() : new LeastLoadedNodeProvider();
+
+        final Call call = new Call(
+            "describeFeatures", calcDeadlineMs(now, options.timeoutMs()), provider) {
+
+            private FeatureMetadata createFeatureMetadata(final ApiVersionsResponse response) {
+                final Map<String, FinalizedVersionRange> finalizedFeatures = new HashMap<>();
+                for (final FinalizedFeatureKey key : response.data().finalizedFeatures().valuesSet()) {
+                    finalizedFeatures.put(key.name(), new FinalizedVersionRange(key.minVersionLevel(), key.maxVersionLevel()));
+                }
+
+                Optional<Long> finalizedFeaturesEpoch;
+                if (response.data().finalizedFeaturesEpoch() >= 0L) {
+                    finalizedFeaturesEpoch = Optional.of(response.data().finalizedFeaturesEpoch());
+                } else {
+                    finalizedFeaturesEpoch = Optional.empty();
+                }
+
+                final Map<String, SupportedVersionRange> supportedFeatures = new HashMap<>();
+                for (final SupportedFeatureKey key : response.data().supportedFeatures().valuesSet()) {
+                    supportedFeatures.put(key.name(), new SupportedVersionRange(key.minVersion(), key.maxVersion()));
+                }
+
+                return new FeatureMetadata(finalizedFeatures, finalizedFeaturesEpoch, supportedFeatures);
+            }
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(createFeatureMetadata(apiVersionsResponse));
+                } else if (options.sendRequestToController() &&
+                           apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) {
+                    handleNotControllerError(Errors.NOT_CONTROLLER);
+                } else {
+                    future.completeExceptionally(Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        runnable.call(call, now);
+        return new DescribeFeaturesResult(future);
+    }
+
+    @Override
+    public UpdateFeaturesResult updateFeatures(final Map<String, FeatureUpdate> featureUpdates,
+                                               final UpdateFeaturesOptions options) {
+        if (featureUpdates.isEmpty()) {
+            throw new IllegalArgumentException("Feature updates can not be null or empty.");
+        }
+
+        final Map<String, KafkaFutureImpl<Void>> updateFutures = new HashMap<>();
+        for (final Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+            updateFutures.put(entry.getKey(), new KafkaFutureImpl<>());
+        }
+
+        final long now = time.milliseconds();
+        final Call call = new Call("updateFeatures", calcDeadlineMs(now, options.timeoutMs()),
+            new ControllerNodeProvider()) {
+
+            @Override
+            UpdateFeaturesRequest.Builder createRequest(int timeoutMs) {
+                final UpdateFeaturesRequestData.FeatureUpdateKeyCollection featureUpdatesRequestData
+                    = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection();
+                for (Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+                    final String feature = entry.getKey();
+                    final FeatureUpdate update = entry.getValue();
+                    if (feature.trim().isEmpty()) {

Review comment:
       sorry for bring delayed comments :(
   
   the error message says it can't be null but there is no null check.
   for another, this check can happen early (when creating ```updateFutures```)




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,192 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature

Review comment:
       Sure, I'll update the PR documenting it.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/test/scala/unit/kafka/server/UpdateFinalizedFeaturesTest.scala
##########
@@ -0,0 +1,450 @@
+package kafka.server

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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/resources/common/message/UpdateFeaturesResponse.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "response",
+  "name": "UpdateFeaturesResponse",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [

Review comment:
       Done. I have added a top-level error code now.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1215,71 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker. It can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter. This is particularly useful if the user requires strongly consistent reads of
+     * finalized features.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the {@link DescribeFeaturesResult} containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);
+
+    /**
+     * Applies specified updates to finalized features. This operation is not transactional so it
+     * may succeed for some features while fail for others.
+     * <p>
+     * The API takes in a map of finalized feature name to {@link FeatureUpdate} that needs to be
+     * applied. Each entry in the map specifies the finalized feature to be added or updated or
+     * deleted, along with the new max feature version level value. This request is issued only to
+     * the controller since the API is only served by the controller. The return value contains an
+     * error code for each supplied {@link FeatureUpdate}, and the code indicates if the update
+     * succeeded or failed in the controller.
+     * <ul>
+     * <li>Downgrade of feature version level is not a regular operation/intent. It is only allowed
+     * in the controller if the {@link FeatureUpdate} has the allowDowngrade flag set - setting this
+     * flag conveys user intent to attempt downgrade of a feature max version level. Note that
+     * despite the allowDowngrade flag being set, certain downgrades may be rejected by the
+     * controller if it is deemed impossible.</li>
+     * <li>Deletion of a finalized feature version is not a regular operation/intent. It could be
+     * done by setting the allowDowngrade flag to true in the {@link FeatureUpdate}, and, setting
+     * the max version level to be less than 1.</li>
+     * </ul>
+     *<p>
+     * The following exceptions can be anticipated when calling {@code get()} on the futures
+     * obtained from the returned {@link UpdateFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.ClusterAuthorizationException}
+     *   If the authenticated user didn't have alter access to the cluster.</li>
+     *   <li>{@link org.apache.kafka.common.errors.InvalidRequestException}

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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java
##########
@@ -17,9 +17,16 @@
 package org.apache.kafka.common.feature;
 
 import java.util.Map;
+import java.util.Objects;
+import org.apache.kafka.common.utils.Utils;
 
 /**
- * An extended {@link BaseVersionRange} representing the min/max versions for supported features.
+ * An extended {@link BaseVersionRange} representing the min, max and first active versions for a
+ * supported feature:
+ *  - minVersion: This is the minimum supported version for the feature.
+ *  - maxVersion: This the maximum supported version for the feature.
+ *  - firstActiveVersion: This is the first active version for the feature. Versions in the range

Review comment:
       @junrao My response below.
   
   > I was thinking what if we relax the current check by just making sure that maxVersion of finalized is within the supported range. Basically in your example, if supported minVersion goes to 2, it's still allowed since it's less than maxVersion of finalized. However, if supported minVersion goes to 7, this fails the broker since it's more than maxVersion of finalized.
   
   If we relaxed the current check like you suggested, then, as of today the following will happen:
   The controller can not effectively finalize `minVersionLevel` for the feature, because, with a relaxed check we do not know whether all brokers in the cluster support a particular `minVersion` when the controller finalizes the `minVersionLevel` at a particular value.
   
   Supposing we want to keep the concept of `minVersionLevel` like the way it is now (i.e. it is the lowest version guaranteed to be supported by any broker in the cluster for a feature), then, `firstActiveVersion` provides the door to mutate it safely whenever the need arises.
   
   So when thinking about your suggestion, it is not clear to me if you are intending to alter the meaning of `minVersionLevel`? If so, please could you tell me what is the new meaning of `minVersionLevel` that you are proposing?
   
   > Your concern for the relaxed check seems to be around deploying a wrong version of the broker by mistake.
   
   Not exactly, my concern is a bit different. It is to be noted that `minVersionLevel` can not be changed through the `UPDATE_FEATURES` api. Assuming we want to maintain the meaning of `minVersionLevel` the way it is today, my concern is about how would one alter `minVersionLevel` for a feature when the need arises? This needs to be done without letting the versioning system misconstrue the intent with a feature version incompatibility. This is where usage of `firstActiveVersion` acts as a solution. It is a community driven change to advance `firstActiveVersion` in an AK release.




----------------------------------------------------------------
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 edited a comment on pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #9001:
URL: https://github.com/apache/kafka/pull/9001#issuecomment-700585584


   @junrao Thanks a lot for the review! I've addressed the comments in the recent commit: 06d8b47131f168db88e4f7d5bda3dd025ba9a2a2. I've provided a response to all of your comments. There are few I couldn't address, and 1-2 comments I'll address in the near 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] kowshik commented on a change in pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1656,6 +1910,204 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns an ApiError object containing Errors.INVALID_REQUEST.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+
+    val supportedVersionRange = brokerFeatures.supportedFeatures.get(update.feature)
+    if (supportedVersionRange == null) {
+      Right(new ApiError(Errors.INVALID_REQUEST,
+                         "Could not apply finalized feature update because the provided feature" +
+                         " is not supported."))
+    } else {
+      var newVersionRange: FinalizedVersionRange = null
+      try {
+        newVersionRange = new FinalizedVersionRange(supportedVersionRange.firstActiveVersion, update.maxVersionLevel)
+      } catch {
+        case _: IllegalArgumentException => {
+          // This exception means the provided maxVersionLevel is invalid. It is handled below
+          // outside of this catch clause.
+        }
+      }
+      if (newVersionRange == null) {
+        Right(new ApiError(Errors.INVALID_REQUEST,
+          "Could not apply finalized feature update because the provided" +
+          s" maxVersionLevel:${update.maxVersionLevel} is lower than the" +
+          s" first active version:${supportedVersionRange.firstActiveVersion}."))
+      } else {
+        val newFinalizedFeature =
+          Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange)))
+        val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+          BrokerFeatures.hasIncompatibleFeatures(broker.features, newFinalizedFeature)
+        })
+        if (numIncompatibleBrokers == 0) {
+          Left(newVersionRange)
+        } else {
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Could not apply finalized feature update because" +
+                             " brokers were found to have incompatible versions for the feature."))
+        }
+      }
+    }
+  }
+
+  /**
+   * Validates a feature update on an existing FinalizedVersionRange.
+   * If the validation succeeds, then, the return value contains:
+   * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature.
+   * 2. Option.empty, if the feature update was meant to delete the feature.
+   *
+   * If the validation fails, then returned value contains a suitable ApiError.
+   *
+   * @param update                 the feature update to be processed.
+   * @param existingVersionRange   the existing FinalizedVersionRange which can be empty when no
+   *                               FinalizedVersionRange exists for the associated feature
+   *
+   * @return                       the new FinalizedVersionRange to be updated into ZK or error
+   *                               as described above.
+   */
+  private def validateFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey,
+                                    existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = {
+    def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = {
+      newFinalizedVersionRangeOrIncompatibilityError(update)
+        .fold(versionRange => Left(Some(versionRange)), error => Right(error))
+    }
+
+    if (update.feature.isEmpty) {
+      // Check that the feature name is not empty.
+      Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty."))
+    } else {
+      // We handle deletion requests separately from non-deletion requests.
+      if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+        if (existingVersionRange.isEmpty) {
+          // Disallow deletion of a non-existing finalized feature.
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Can not delete non-existing finalized feature."))
+        } else {
+          Left(Option.empty)
+        }
+      } else if (update.maxVersionLevel() < 1) {
+        // Disallow deletion of a finalized feature without allowDowngrade flag set.
+        Right(new ApiError(Errors.INVALID_REQUEST,
+                           s"Can not provide maxVersionLevel: ${update.maxVersionLevel} less" +
+                           s" than 1 without setting the allowDowngrade flag to true in the request."))
+      } else {
+        existingVersionRange.map(existing =>
+          if (update.maxVersionLevel == existing.max) {
+            // Disallow a case where target maxVersionLevel matches existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not ${if (update.allowDowngrade) "downgrade" else "upgrade"}" +
+                               s" a finalized feature from existing maxVersionLevel:${existing.max}" +
+                               " to the same value."))
+          } else if (update.maxVersionLevel < existing.max && !update.allowDowngrade) {
+            // Disallow downgrade of a finalized feature without the allowDowngrade flag set.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature from existing" +
+                               s" maxVersionLevel:${existing.max} to provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} without setting the" +
+                               " allowDowngrade flag in the request."))
+          } else if (update.allowDowngrade && update.maxVersionLevel > existing.max) {
+            // Disallow a request that sets allowDowngrade flag without specifying a
+            // maxVersionLevel that's lower than the existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"When the allowDowngrade flag set in the request, the provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} can not be greater than" +
+                               s" existing maxVersionLevel:${existing.max}."))
+          } else if (update.maxVersionLevel < existing.min) {
+            // Disallow downgrade of a finalized feature below the existing finalized
+            // minVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature to maxVersionLevel:${update.maxVersionLevel}" +
+                               s" because it's lower than the existing minVersionLevel:${existing.min}."))
+          } else {
+            newVersionRangeOrError(update)
+          }
+        ).getOrElse(newVersionRangeOrError(update))
+      }
+    }
+  }
+
+  private def processFeatureUpdates(request: UpdateFeaturesRequest,
+                                    callback: UpdateFeaturesCallback): Unit = {
+    if (isActive) {
+      processFeatureUpdatesWithActiveController(request, callback)
+    } else {
+      callback(Left(new ApiError(Errors.NOT_CONTROLLER)))
+    }
+  }
+
+  private def processFeatureUpdatesWithActiveController(request: UpdateFeaturesRequest,
+                                                        callback: UpdateFeaturesCallback): Unit = {
+    val updates = request.data.featureUpdates
+    val existingFeatures = featureCache.get
+      .map(featuresAndEpoch => featuresAndEpoch.features.features().asScala)
+      .getOrElse(Map[String, FinalizedVersionRange]())
+    // A map with key being feature name and value being FinalizedVersionRange.
+    // This contains the target features to be eventually written to FeatureZNode.
+    val targetFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]() ++ existingFeatures
+    // A map with key being feature name and value being error encountered when the FeatureUpdate
+    // was applied.
+    val errors = scala.collection.mutable.Map[String, ApiError]()
+
+    // Below we process each FeatureUpdate using the following logic:
+    //  - If a FeatureUpdate is found to be valid, then:
+    //    - The corresponding entry in errors map would be updated to contain Errors.NONE.
+    //    - If the FeatureUpdate is an add or update request, then the targetFeatures map is updated
+    //      to contain the new FinalizedVersionRange for the feature.
+    //    - Otherwise if the FeatureUpdate is a delete request, then the feature is removed from the
+    //      targetFeatures map.
+    //  - Otherwise if a FeatureUpdate is found to be invalid, then:
+    //    - The corresponding entry in errors map would be updated with the appropriate ApiError.
+    //    - The entry in targetFeatures map is left untouched.
+    updates.asScala.iterator.foreach { update =>
+      validateFeatureUpdate(update, existingFeatures.get(update.feature())) match {
+        case Left(newVersionRangeOrNone) =>
+          newVersionRangeOrNone match {
+            case Some(newVersionRange) => targetFeatures += (update.feature() -> newVersionRange)
+            case None => targetFeatures -= update.feature()
+          }
+          errors += (update.feature() -> new ApiError(Errors.NONE))
+        case Right(featureUpdateFailureReason) =>
+          errors += (update.feature() -> featureUpdateFailureReason)
+      }
+    }
+
+    // If the existing and target features are the same, then, we skip the update to the
+    // FeatureZNode as no changes to the node are required. Otherwise, we replace the contents
+    // of the FeatureZNode with the new features. This may result in partial or full modification
+    // of the existing finalized features in ZK.
+    try {
+      if (!existingFeatures.equals(targetFeatures)) {
+        val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, Features.finalizedFeatures(targetFeatures.asJava))
+        val newVersion = zkClient.updateFeatureZNode(newNode)
+        featureCache.waitUntilEpochOrThrow(newVersion, request.data().timeoutMs().min(config.zkConnectionTimeoutMs))
+      }
+    } catch {
+      // For all features that correspond to valid FeatureUpdate (i.e. error is Errors.NONE),
+      // we set the error as Errors.FEATURE_UPDATE_FAILED since the FeatureZNode update has failed
+      // for these. For the rest, the existing error is left untouched.
+      case e: Exception =>
+        warn(s"Processing of feature updates: $request failed due to error: $e")
+        errors.foreach { case (feature, apiError) =>
+          if (apiError.error() == Errors.NONE) {
+            errors(feature) = new ApiError(Errors.FEATURE_UPDATE_FAILED)

Review comment:
       The default error message reads: `Unable to update finalized features due to an unexpected server error.`.
   Would that suffice?

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1656,6 +1910,204 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns an ApiError object containing Errors.INVALID_REQUEST.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+
+    val supportedVersionRange = brokerFeatures.supportedFeatures.get(update.feature)
+    if (supportedVersionRange == null) {
+      Right(new ApiError(Errors.INVALID_REQUEST,
+                         "Could not apply finalized feature update because the provided feature" +
+                         " is not supported."))
+    } else {
+      var newVersionRange: FinalizedVersionRange = null
+      try {
+        newVersionRange = new FinalizedVersionRange(supportedVersionRange.firstActiveVersion, update.maxVersionLevel)
+      } catch {
+        case _: IllegalArgumentException => {
+          // This exception means the provided maxVersionLevel is invalid. It is handled below
+          // outside of this catch clause.
+        }
+      }
+      if (newVersionRange == null) {
+        Right(new ApiError(Errors.INVALID_REQUEST,
+          "Could not apply finalized feature update because the provided" +
+          s" maxVersionLevel:${update.maxVersionLevel} is lower than the" +
+          s" first active version:${supportedVersionRange.firstActiveVersion}."))
+      } else {
+        val newFinalizedFeature =
+          Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange)))
+        val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+          BrokerFeatures.hasIncompatibleFeatures(broker.features, newFinalizedFeature)
+        })
+        if (numIncompatibleBrokers == 0) {
+          Left(newVersionRange)
+        } else {
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Could not apply finalized feature update because" +
+                             " brokers were found to have incompatible versions for the feature."))
+        }
+      }
+    }
+  }
+
+  /**
+   * Validates a feature update on an existing FinalizedVersionRange.
+   * If the validation succeeds, then, the return value contains:
+   * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature.
+   * 2. Option.empty, if the feature update was meant to delete the feature.
+   *
+   * If the validation fails, then returned value contains a suitable ApiError.
+   *
+   * @param update                 the feature update to be processed.
+   * @param existingVersionRange   the existing FinalizedVersionRange which can be empty when no
+   *                               FinalizedVersionRange exists for the associated feature
+   *
+   * @return                       the new FinalizedVersionRange to be updated into ZK or error
+   *                               as described above.
+   */
+  private def validateFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey,
+                                    existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = {
+    def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = {
+      newFinalizedVersionRangeOrIncompatibilityError(update)
+        .fold(versionRange => Left(Some(versionRange)), error => Right(error))
+    }
+
+    if (update.feature.isEmpty) {
+      // Check that the feature name is not empty.
+      Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty."))
+    } else {
+      // We handle deletion requests separately from non-deletion requests.
+      if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+        if (existingVersionRange.isEmpty) {
+          // Disallow deletion of a non-existing finalized feature.
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Can not delete non-existing finalized feature."))
+        } else {
+          Left(Option.empty)
+        }
+      } else if (update.maxVersionLevel() < 1) {
+        // Disallow deletion of a finalized feature without allowDowngrade flag set.
+        Right(new ApiError(Errors.INVALID_REQUEST,
+                           s"Can not provide maxVersionLevel: ${update.maxVersionLevel} less" +
+                           s" than 1 without setting the allowDowngrade flag to true in the request."))
+      } else {
+        existingVersionRange.map(existing =>
+          if (update.maxVersionLevel == existing.max) {
+            // Disallow a case where target maxVersionLevel matches existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not ${if (update.allowDowngrade) "downgrade" else "upgrade"}" +
+                               s" a finalized feature from existing maxVersionLevel:${existing.max}" +
+                               " to the same value."))
+          } else if (update.maxVersionLevel < existing.max && !update.allowDowngrade) {
+            // Disallow downgrade of a finalized feature without the allowDowngrade flag set.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature from existing" +
+                               s" maxVersionLevel:${existing.max} to provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} without setting the" +
+                               " allowDowngrade flag in the request."))
+          } else if (update.allowDowngrade && update.maxVersionLevel > existing.max) {
+            // Disallow a request that sets allowDowngrade flag without specifying a
+            // maxVersionLevel that's lower than the existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"When the allowDowngrade flag set in the request, the provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} can not be greater than" +
+                               s" existing maxVersionLevel:${existing.max}."))
+          } else if (update.maxVersionLevel < existing.min) {
+            // Disallow downgrade of a finalized feature below the existing finalized
+            // minVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature to maxVersionLevel:${update.maxVersionLevel}" +
+                               s" because it's lower than the existing minVersionLevel:${existing.min}."))
+          } else {
+            newVersionRangeOrError(update)
+          }
+        ).getOrElse(newVersionRangeOrError(update))
+      }
+    }
+  }
+
+  private def processFeatureUpdates(request: UpdateFeaturesRequest,
+                                    callback: UpdateFeaturesCallback): Unit = {
+    if (isActive) {
+      processFeatureUpdatesWithActiveController(request, callback)
+    } else {
+      callback(Left(new ApiError(Errors.NOT_CONTROLLER)))
+    }
+  }
+
+  private def processFeatureUpdatesWithActiveController(request: UpdateFeaturesRequest,
+                                                        callback: UpdateFeaturesCallback): Unit = {
+    val updates = request.data.featureUpdates
+    val existingFeatures = featureCache.get
+      .map(featuresAndEpoch => featuresAndEpoch.features.features().asScala)
+      .getOrElse(Map[String, FinalizedVersionRange]())
+    // A map with key being feature name and value being FinalizedVersionRange.
+    // This contains the target features to be eventually written to FeatureZNode.
+    val targetFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]() ++ existingFeatures
+    // A map with key being feature name and value being error encountered when the FeatureUpdate
+    // was applied.
+    val errors = scala.collection.mutable.Map[String, ApiError]()
+
+    // Below we process each FeatureUpdate using the following logic:
+    //  - If a FeatureUpdate is found to be valid, then:
+    //    - The corresponding entry in errors map would be updated to contain Errors.NONE.
+    //    - If the FeatureUpdate is an add or update request, then the targetFeatures map is updated
+    //      to contain the new FinalizedVersionRange for the feature.
+    //    - Otherwise if the FeatureUpdate is a delete request, then the feature is removed from the
+    //      targetFeatures map.
+    //  - Otherwise if a FeatureUpdate is found to be invalid, then:
+    //    - The corresponding entry in errors map would be updated with the appropriate ApiError.
+    //    - The entry in targetFeatures map is left untouched.
+    updates.asScala.iterator.foreach { update =>
+      validateFeatureUpdate(update, existingFeatures.get(update.feature())) match {
+        case Left(newVersionRangeOrNone) =>
+          newVersionRangeOrNone match {
+            case Some(newVersionRange) => targetFeatures += (update.feature() -> newVersionRange)
+            case None => targetFeatures -= update.feature()
+          }
+          errors += (update.feature() -> new ApiError(Errors.NONE))
+        case Right(featureUpdateFailureReason) =>
+          errors += (update.feature() -> featureUpdateFailureReason)
+      }
+    }
+
+    // If the existing and target features are the same, then, we skip the update to the
+    // FeatureZNode as no changes to the node are required. Otherwise, we replace the contents
+    // of the FeatureZNode with the new features. This may result in partial or full modification
+    // of the existing finalized features in ZK.
+    try {
+      if (!existingFeatures.equals(targetFeatures)) {
+        val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, Features.finalizedFeatures(targetFeatures.asJava))
+        val newVersion = zkClient.updateFeatureZNode(newNode)
+        featureCache.waitUntilEpochOrThrow(newVersion, request.data().timeoutMs().min(config.zkConnectionTimeoutMs))
+      }
+    } catch {
+      // For all features that correspond to valid FeatureUpdate (i.e. error is Errors.NONE),
+      // we set the error as Errors.FEATURE_UPDATE_FAILED since the FeatureZNode update has failed
+      // for these. For the rest, the existing error is left untouched.
+      case e: Exception =>
+        warn(s"Processing of feature updates: $request failed due to error: $e")
+        errors.foreach { case (feature, apiError) =>
+          if (apiError.error() == Errors.NONE) {
+            errors(feature) = new ApiError(Errors.FEATURE_UPDATE_FAILED)

Review comment:
       The default error message reads: `Unable to update finalized features due to an unexpected server error.`. Would that suffice?




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -272,6 +281,199 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (string) and a range of versions (defined by a
+   * SupportedVersionRange). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of its own supported features in its
+   * own BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (string) and a range of version levels (defined
+   * by a FinalizedVersionRange). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. This process ensures we do not enable all the possible features immediately after
+   *    an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, it will
+   *        react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, its existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.defaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)

Review comment:
       I agree. But note that in this method, we do not process an `UpdateFeaturesRequest`. This method is only called during controller election to setup feature versioning. So, I have incorporated your suggestion at the point where we process the request, look for `def processFeatureUpdatesWithActiveController` in this file where I set the ZK write timeout to be `min(timeoutMs, config.zkConnectionTimeoutMs)`.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java
##########
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+import java.util.Optional;
+import org.apache.kafka.common.feature.Features;
+import org.apache.kafka.common.feature.FinalizedVersionRange;
+import org.apache.kafka.common.feature.SupportedVersionRange;
+
+/**
+ * Encapsulates details about finalized as well as supported features. This is particularly useful
+ * to hold the result returned by the {@link Admin#describeFeatures(DescribeFeaturesOptions)} API.
+ */
+public class FeatureMetadata {
+
+    private final Features<FinalizedVersionRange> finalizedFeatures;
+
+    private final Optional<Integer> finalizedFeaturesEpoch;
+
+    private final Features<SupportedVersionRange> supportedFeatures;
+
+    public FeatureMetadata(final Features<FinalizedVersionRange> finalizedFeatures,
+                           final int finalizedFeaturesEpoch,
+                           final Features<SupportedVersionRange> supportedFeatures) {
+        Objects.requireNonNull(finalizedFeatures, "Provided finalizedFeatures can not be null.");
+        Objects.requireNonNull(supportedFeatures, "Provided supportedFeatures can not be null.");
+        this.finalizedFeatures = finalizedFeatures;
+        if (finalizedFeaturesEpoch >= 0) {
+            this.finalizedFeaturesEpoch = Optional.of(finalizedFeaturesEpoch);
+        } else {
+            this.finalizedFeaturesEpoch = Optional.empty();
+        }
+        this.supportedFeatures = supportedFeatures;
+    }
+
+    /**
+     * A map of finalized feature versions, with key being finalized feature name and value
+     * containing the min/max version levels for the finalized feature.
+     */
+    public Features<FinalizedVersionRange> finalizedFeatures() {

Review comment:
       Done. I've fixed this now to align with the KIP.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/resources/common/message/UpdateFinalizedFeaturesResponse.json
##########
@@ -0,0 +1,28 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "response",
+  "name": "UpdateFinalizedFeaturesResponse",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "ErrorCode", "type": "int16", "versions": "0+",
+      "about": "The error code or `0` if there was no error." },
+    { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+",

Review comment:
       Done now.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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


   The latest test failures do not seem related to this PR:
    * JDK 8: `org.apache.kafka.connect.integration.ExampleConnectIntegrationTest.testSourceConnector`
    * JDK 11: `org.apache.kafka.streams.integration.EosBetaUpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosBeta`
   


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -82,18 +110,54 @@ object FinalizedFeatureCache extends Logging {
         " The existing cache contents are %s").format(latest, oldFeatureAndEpoch)
       throw new FeatureCacheUpdateException(errorMsg)
     } else {
-      val incompatibleFeatures = SupportedFeatures.incompatibleFeatures(latest.features)
+      val incompatibleFeatures = brokerFeatures.incompatibleFeatures(latest.features)
       if (!incompatibleFeatures.empty) {
         val errorMsg = ("FinalizedFeatureCache update failed since feature compatibility" +
           " checks failed! Supported %s has incompatibilities with the latest %s."
-          ).format(SupportedFeatures.get, latest)
+          ).format(brokerFeatures.supportedFeatures, latest)

Review comment:
       Good question. The existing behavior is that it shuts itself down, as triggered by this LOC: https://github.com/apache/kafka/blob/89a3ba69e03acbe9635ee1039abb567bf0c6631b/core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala#L154-L156. The reason to do it is that an incompatible broker can potentially do harmful things to a cluster (because max version level upgrades are used for breaking changes).




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java
##########
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import static java.util.stream.Collectors.joining;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * Encapsulates details about finalized as well as supported features. This is particularly useful
+ * to hold the result returned by the {@link Admin#describeFeatures(DescribeFeaturesOptions)} API.
+ */
+public class FeatureMetadata {
+
+    private final Map<String, FinalizedVersionRange> finalizedFeatures;
+
+    private final Optional<Long> finalizedFeaturesEpoch;
+
+    private final Map<String, SupportedVersionRange> supportedFeatures;
+
+    public FeatureMetadata(final Map<String, FinalizedVersionRange> finalizedFeatures,

Review comment:
       I was looking at existing classes fro the return value. For example, CreateAclsResult deliberately makes the constructor non-public.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -3109,6 +3109,36 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFeatures(request: RequestChannel.Request): Unit = {
+    val updateFeaturesRequest = request.body[UpdateFeaturesRequest]
+
+    def sendResponseCallback(errors: Either[ApiError, Map[String, ApiError]]): Unit = {
+      def createResponse(throttleTimeMs: Int): UpdateFeaturesResponse = {
+        errors match {
+          case Left(topLevelError) =>
+            UpdateFeaturesResponse.createWithErrors(
+              topLevelError,
+              new util.HashMap[String, ApiError](),

Review comment:
       Could we use Collections.emptyMap()?

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1656,6 +1910,204 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns an ApiError object containing Errors.INVALID_REQUEST.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+
+    val supportedVersionRange = brokerFeatures.supportedFeatures.get(update.feature)
+    if (supportedVersionRange == null) {
+      Right(new ApiError(Errors.INVALID_REQUEST,
+                         "Could not apply finalized feature update because the provided feature" +
+                         " is not supported."))
+    } else {
+      var newVersionRange: FinalizedVersionRange = null
+      try {
+        newVersionRange = new FinalizedVersionRange(supportedVersionRange.firstActiveVersion, update.maxVersionLevel)
+      } catch {
+        case _: IllegalArgumentException => {
+          // This exception means the provided maxVersionLevel is invalid. It is handled below
+          // outside of this catch clause.
+        }
+      }
+      if (newVersionRange == null) {
+        Right(new ApiError(Errors.INVALID_REQUEST,
+          "Could not apply finalized feature update because the provided" +
+          s" maxVersionLevel:${update.maxVersionLevel} is lower than the" +
+          s" first active version:${supportedVersionRange.firstActiveVersion}."))
+      } else {
+        val newFinalizedFeature =
+          Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange)))
+        val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+          BrokerFeatures.hasIncompatibleFeatures(broker.features, newFinalizedFeature)
+        })
+        if (numIncompatibleBrokers == 0) {
+          Left(newVersionRange)
+        } else {
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Could not apply finalized feature update because" +
+                             " brokers were found to have incompatible versions for the feature."))
+        }
+      }
+    }
+  }
+
+  /**
+   * Validates a feature update on an existing FinalizedVersionRange.
+   * If the validation succeeds, then, the return value contains:
+   * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature.
+   * 2. Option.empty, if the feature update was meant to delete the feature.
+   *
+   * If the validation fails, then returned value contains a suitable ApiError.
+   *
+   * @param update                 the feature update to be processed.
+   * @param existingVersionRange   the existing FinalizedVersionRange which can be empty when no
+   *                               FinalizedVersionRange exists for the associated feature
+   *
+   * @return                       the new FinalizedVersionRange to be updated into ZK or error
+   *                               as described above.
+   */
+  private def validateFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey,
+                                    existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = {
+    def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = {
+      newFinalizedVersionRangeOrIncompatibilityError(update)
+        .fold(versionRange => Left(Some(versionRange)), error => Right(error))
+    }
+
+    if (update.feature.isEmpty) {
+      // Check that the feature name is not empty.
+      Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty."))
+    } else {
+      // We handle deletion requests separately from non-deletion requests.
+      if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+        if (existingVersionRange.isEmpty) {
+          // Disallow deletion of a non-existing finalized feature.
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Can not delete non-existing finalized feature."))
+        } else {
+          Left(Option.empty)
+        }
+      } else if (update.maxVersionLevel() < 1) {
+        // Disallow deletion of a finalized feature without allowDowngrade flag set.
+        Right(new ApiError(Errors.INVALID_REQUEST,
+                           s"Can not provide maxVersionLevel: ${update.maxVersionLevel} less" +
+                           s" than 1 without setting the allowDowngrade flag to true in the request."))
+      } else {
+        existingVersionRange.map(existing =>
+          if (update.maxVersionLevel == existing.max) {
+            // Disallow a case where target maxVersionLevel matches existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not ${if (update.allowDowngrade) "downgrade" else "upgrade"}" +
+                               s" a finalized feature from existing maxVersionLevel:${existing.max}" +
+                               " to the same value."))
+          } else if (update.maxVersionLevel < existing.max && !update.allowDowngrade) {
+            // Disallow downgrade of a finalized feature without the allowDowngrade flag set.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature from existing" +
+                               s" maxVersionLevel:${existing.max} to provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} without setting the" +
+                               " allowDowngrade flag in the request."))
+          } else if (update.allowDowngrade && update.maxVersionLevel > existing.max) {
+            // Disallow a request that sets allowDowngrade flag without specifying a
+            // maxVersionLevel that's lower than the existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"When the allowDowngrade flag set in the request, the provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} can not be greater than" +
+                               s" existing maxVersionLevel:${existing.max}."))
+          } else if (update.maxVersionLevel < existing.min) {
+            // Disallow downgrade of a finalized feature below the existing finalized
+            // minVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature to maxVersionLevel:${update.maxVersionLevel}" +
+                               s" because it's lower than the existing minVersionLevel:${existing.min}."))
+          } else {
+            newVersionRangeOrError(update)
+          }
+        ).getOrElse(newVersionRangeOrError(update))
+      }
+    }
+  }
+
+  private def processFeatureUpdates(request: UpdateFeaturesRequest,
+                                    callback: UpdateFeaturesCallback): Unit = {
+    if (isActive) {
+      processFeatureUpdatesWithActiveController(request, callback)
+    } else {
+      callback(Left(new ApiError(Errors.NOT_CONTROLLER)))
+    }
+  }
+
+  private def processFeatureUpdatesWithActiveController(request: UpdateFeaturesRequest,
+                                                        callback: UpdateFeaturesCallback): Unit = {
+    val updates = request.data.featureUpdates
+    val existingFeatures = featureCache.get
+      .map(featuresAndEpoch => featuresAndEpoch.features.features().asScala)
+      .getOrElse(Map[String, FinalizedVersionRange]())
+    // A map with key being feature name and value being FinalizedVersionRange.
+    // This contains the target features to be eventually written to FeatureZNode.
+    val targetFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]() ++ existingFeatures
+    // A map with key being feature name and value being error encountered when the FeatureUpdate
+    // was applied.
+    val errors = scala.collection.mutable.Map[String, ApiError]()
+
+    // Below we process each FeatureUpdate using the following logic:
+    //  - If a FeatureUpdate is found to be valid, then:
+    //    - The corresponding entry in errors map would be updated to contain Errors.NONE.
+    //    - If the FeatureUpdate is an add or update request, then the targetFeatures map is updated
+    //      to contain the new FinalizedVersionRange for the feature.
+    //    - Otherwise if the FeatureUpdate is a delete request, then the feature is removed from the
+    //      targetFeatures map.
+    //  - Otherwise if a FeatureUpdate is found to be invalid, then:
+    //    - The corresponding entry in errors map would be updated with the appropriate ApiError.
+    //    - The entry in targetFeatures map is left untouched.
+    updates.asScala.iterator.foreach { update =>
+      validateFeatureUpdate(update, existingFeatures.get(update.feature())) match {
+        case Left(newVersionRangeOrNone) =>
+          newVersionRangeOrNone match {
+            case Some(newVersionRange) => targetFeatures += (update.feature() -> newVersionRange)
+            case None => targetFeatures -= update.feature()
+          }
+          errors += (update.feature() -> new ApiError(Errors.NONE))
+        case Right(featureUpdateFailureReason) =>
+          errors += (update.feature() -> featureUpdateFailureReason)
+      }
+    }
+
+    // If the existing and target features are the same, then, we skip the update to the
+    // FeatureZNode as no changes to the node are required. Otherwise, we replace the contents
+    // of the FeatureZNode with the new features. This may result in partial or full modification
+    // of the existing finalized features in ZK.
+    try {
+      if (!existingFeatures.equals(targetFeatures)) {
+        val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, Features.finalizedFeatures(targetFeatures.asJava))
+        val newVersion = zkClient.updateFeatureZNode(newNode)
+        featureCache.waitUntilEpochOrThrow(newVersion, request.data().timeoutMs().min(config.zkConnectionTimeoutMs))

Review comment:
       Hmm, why do we need to take the min? If the ZK data is propagated quickly, waitUntilEpochOrThrow() will just return early.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1656,6 +1910,204 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns an ApiError object containing Errors.INVALID_REQUEST.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+
+    val supportedVersionRange = brokerFeatures.supportedFeatures.get(update.feature)
+    if (supportedVersionRange == null) {
+      Right(new ApiError(Errors.INVALID_REQUEST,
+                         "Could not apply finalized feature update because the provided feature" +
+                         " is not supported."))
+    } else {
+      var newVersionRange: FinalizedVersionRange = null
+      try {
+        newVersionRange = new FinalizedVersionRange(supportedVersionRange.firstActiveVersion, update.maxVersionLevel)
+      } catch {
+        case _: IllegalArgumentException => {
+          // This exception means the provided maxVersionLevel is invalid. It is handled below
+          // outside of this catch clause.
+        }
+      }
+      if (newVersionRange == null) {
+        Right(new ApiError(Errors.INVALID_REQUEST,
+          "Could not apply finalized feature update because the provided" +
+          s" maxVersionLevel:${update.maxVersionLevel} is lower than the" +
+          s" first active version:${supportedVersionRange.firstActiveVersion}."))
+      } else {
+        val newFinalizedFeature =
+          Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange)))
+        val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+          BrokerFeatures.hasIncompatibleFeatures(broker.features, newFinalizedFeature)
+        })
+        if (numIncompatibleBrokers == 0) {
+          Left(newVersionRange)
+        } else {
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Could not apply finalized feature update because" +
+                             " brokers were found to have incompatible versions for the feature."))
+        }
+      }
+    }
+  }
+
+  /**
+   * Validates a feature update on an existing FinalizedVersionRange.
+   * If the validation succeeds, then, the return value contains:
+   * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature.
+   * 2. Option.empty, if the feature update was meant to delete the feature.
+   *
+   * If the validation fails, then returned value contains a suitable ApiError.
+   *
+   * @param update                 the feature update to be processed.
+   * @param existingVersionRange   the existing FinalizedVersionRange which can be empty when no
+   *                               FinalizedVersionRange exists for the associated feature
+   *
+   * @return                       the new FinalizedVersionRange to be updated into ZK or error
+   *                               as described above.
+   */
+  private def validateFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey,
+                                    existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = {
+    def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = {
+      newFinalizedVersionRangeOrIncompatibilityError(update)
+        .fold(versionRange => Left(Some(versionRange)), error => Right(error))
+    }
+
+    if (update.feature.isEmpty) {
+      // Check that the feature name is not empty.
+      Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty."))
+    } else {
+      // We handle deletion requests separately from non-deletion requests.
+      if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+        if (existingVersionRange.isEmpty) {
+          // Disallow deletion of a non-existing finalized feature.
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Can not delete non-existing finalized feature."))
+        } else {
+          Left(Option.empty)
+        }
+      } else if (update.maxVersionLevel() < 1) {
+        // Disallow deletion of a finalized feature without allowDowngrade flag set.
+        Right(new ApiError(Errors.INVALID_REQUEST,
+                           s"Can not provide maxVersionLevel: ${update.maxVersionLevel} less" +
+                           s" than 1 without setting the allowDowngrade flag to true in the request."))
+      } else {
+        existingVersionRange.map(existing =>
+          if (update.maxVersionLevel == existing.max) {
+            // Disallow a case where target maxVersionLevel matches existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not ${if (update.allowDowngrade) "downgrade" else "upgrade"}" +
+                               s" a finalized feature from existing maxVersionLevel:${existing.max}" +
+                               " to the same value."))
+          } else if (update.maxVersionLevel < existing.max && !update.allowDowngrade) {
+            // Disallow downgrade of a finalized feature without the allowDowngrade flag set.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature from existing" +
+                               s" maxVersionLevel:${existing.max} to provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} without setting the" +
+                               " allowDowngrade flag in the request."))
+          } else if (update.allowDowngrade && update.maxVersionLevel > existing.max) {
+            // Disallow a request that sets allowDowngrade flag without specifying a
+            // maxVersionLevel that's lower than the existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"When the allowDowngrade flag set in the request, the provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} can not be greater than" +
+                               s" existing maxVersionLevel:${existing.max}."))
+          } else if (update.maxVersionLevel < existing.min) {
+            // Disallow downgrade of a finalized feature below the existing finalized
+            // minVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature to maxVersionLevel:${update.maxVersionLevel}" +
+                               s" because it's lower than the existing minVersionLevel:${existing.min}."))
+          } else {
+            newVersionRangeOrError(update)
+          }
+        ).getOrElse(newVersionRangeOrError(update))
+      }
+    }
+  }
+
+  private def processFeatureUpdates(request: UpdateFeaturesRequest,
+                                    callback: UpdateFeaturesCallback): Unit = {
+    if (isActive) {
+      processFeatureUpdatesWithActiveController(request, callback)
+    } else {
+      callback(Left(new ApiError(Errors.NOT_CONTROLLER)))
+    }
+  }
+
+  private def processFeatureUpdatesWithActiveController(request: UpdateFeaturesRequest,
+                                                        callback: UpdateFeaturesCallback): Unit = {
+    val updates = request.data.featureUpdates
+    val existingFeatures = featureCache.get
+      .map(featuresAndEpoch => featuresAndEpoch.features.features().asScala)
+      .getOrElse(Map[String, FinalizedVersionRange]())
+    // A map with key being feature name and value being FinalizedVersionRange.
+    // This contains the target features to be eventually written to FeatureZNode.
+    val targetFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]() ++ existingFeatures
+    // A map with key being feature name and value being error encountered when the FeatureUpdate
+    // was applied.
+    val errors = scala.collection.mutable.Map[String, ApiError]()
+
+    // Below we process each FeatureUpdate using the following logic:
+    //  - If a FeatureUpdate is found to be valid, then:
+    //    - The corresponding entry in errors map would be updated to contain Errors.NONE.
+    //    - If the FeatureUpdate is an add or update request, then the targetFeatures map is updated
+    //      to contain the new FinalizedVersionRange for the feature.
+    //    - Otherwise if the FeatureUpdate is a delete request, then the feature is removed from the
+    //      targetFeatures map.
+    //  - Otherwise if a FeatureUpdate is found to be invalid, then:
+    //    - The corresponding entry in errors map would be updated with the appropriate ApiError.
+    //    - The entry in targetFeatures map is left untouched.
+    updates.asScala.iterator.foreach { update =>
+      validateFeatureUpdate(update, existingFeatures.get(update.feature())) match {
+        case Left(newVersionRangeOrNone) =>
+          newVersionRangeOrNone match {
+            case Some(newVersionRange) => targetFeatures += (update.feature() -> newVersionRange)
+            case None => targetFeatures -= update.feature()
+          }
+          errors += (update.feature() -> new ApiError(Errors.NONE))
+        case Right(featureUpdateFailureReason) =>
+          errors += (update.feature() -> featureUpdateFailureReason)
+      }
+    }
+
+    // If the existing and target features are the same, then, we skip the update to the
+    // FeatureZNode as no changes to the node are required. Otherwise, we replace the contents
+    // of the FeatureZNode with the new features. This may result in partial or full modification
+    // of the existing finalized features in ZK.
+    try {
+      if (!existingFeatures.equals(targetFeatures)) {
+        val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, Features.finalizedFeatures(targetFeatures.asJava))
+        val newVersion = zkClient.updateFeatureZNode(newNode)
+        featureCache.waitUntilEpochOrThrow(newVersion, request.data().timeoutMs().min(config.zkConnectionTimeoutMs))
+      }
+    } catch {
+      // For all features that correspond to valid FeatureUpdate (i.e. error is Errors.NONE),
+      // we set the error as Errors.FEATURE_UPDATE_FAILED since the FeatureZNode update has failed
+      // for these. For the rest, the existing error is left untouched.
+      case e: Exception =>
+        warn(s"Processing of feature updates: $request failed due to error: $e")
+        errors.foreach { case (feature, apiError) =>
+          if (apiError.error() == Errors.NONE) {
+            errors(feature) = new ApiError(Errors.FEATURE_UPDATE_FAILED)

Review comment:
       It's useful to return an error message too.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -219,6 +226,8 @@ class KafkaController(val config: KafkaConfig,
    * This ensures another controller election will be triggered and there will always be an actively serving controller
    */
   private def onControllerFailover(): Unit = {
+    maybeSetupFeatureVersioning()

Review comment:
       This can throw an exception due to feature mismatch. Currently, this forces the controller to move but keeps the broker alive. Should we force the broker to exit in this case?




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1656,6 +1893,203 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns an ApiError object containing Errors.INVALID_REQUEST.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+
+    val supportedVersionRange = brokerFeatures.supportedFeatures.get(update.feature)
+    if (supportedVersionRange == null) {
+      Right(new ApiError(Errors.INVALID_REQUEST,
+                         "Could not apply finalized feature update because the provided feature" +
+                         " is not supported."))
+    } else {
+      var newVersionRange: FinalizedVersionRange = null
+      try {
+        newVersionRange = new FinalizedVersionRange(supportedVersionRange.firstActiveVersion, update.maxVersionLevel)
+      } catch {
+        case _: IllegalArgumentException => {
+          // This exception means the provided maxVersionLevel is invalid. It is handled below
+          // outside of this catch clause.
+        }
+      }
+      if (newVersionRange == null) {
+        Right(new ApiError(Errors.INVALID_REQUEST,
+          "Could not apply finalized feature update because the provided" +
+          s" maxVersionLevel:${update.maxVersionLevel} is lower than the" +
+          s" first active version:${supportedVersionRange.firstActiveVersion}."))
+      } else {
+        val newFinalizedFeature =
+          Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange)))
+        val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+          BrokerFeatures.hasIncompatibleFeatures(broker.features, newFinalizedFeature)
+        })
+        if (numIncompatibleBrokers == 0) {
+          Left(newVersionRange)
+        } else {
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Could not apply finalized feature update because" +
+                             " brokers were found to have incompatible versions for the feature."))
+        }
+      }
+    }
+  }
+
+  /**
+   * Validates a feature update on an existing FinalizedVersionRange.
+   * If the validation succeeds, then, the return value contains:
+   * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature.
+   * 2. Option.empty, if the feature update was meant to delete the feature.
+   *
+   * If the validation fails, then returned value contains a suitable ApiError.
+   *
+   * @param update                 the feature update to be processed.
+   * @param existingVersionRange   the existing FinalizedVersionRange which can be empty when no
+   *                               FinalizedVersionRange exists for the associated feature
+   *
+   * @return                       the new FinalizedVersionRange to be updated into ZK or error
+   *                               as described above.
+   */
+  private def validateFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey,
+                                   existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = {

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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.kafka.common.message.UpdateFeaturesResponseData;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.Struct;
+
+
+/**
+ * Possible error codes:
+ *
+ *   - {@link Errors#CLUSTER_AUTHORIZATION_FAILED}
+ *   - {@link Errors#NOT_CONTROLLER}
+ *   - {@link Errors#INVALID_REQUEST}
+ *   - {@link Errors#FEATURE_UPDATE_FAILED}
+ */
+public class UpdateFeaturesResponse extends AbstractResponse {
+
+    private final UpdateFeaturesResponseData data;
+
+    public UpdateFeaturesResponse(UpdateFeaturesResponseData data) {
+        this.data = data;
+    }
+
+    public UpdateFeaturesResponse(Struct struct) {
+        final short latestVersion = (short) (UpdateFeaturesResponseData.SCHEMAS.length - 1);
+        this.data = new UpdateFeaturesResponseData(struct, latestVersion);
+    }
+
+    public UpdateFeaturesResponse(Struct struct, short version) {
+        this.data = new UpdateFeaturesResponseData(struct, version);
+    }
+
+    public Map<String, ApiError> errors() {
+        return data.results().valuesSet().stream().collect(
+            Collectors.toMap(
+                result -> result.feature(),

Review comment:
       Like how? I don't understand.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -977,14 +1179,30 @@ class KafkaController(val config: KafkaConfig,
 
   /**
    * Send the leader information for selected partitions to selected brokers so that they can correctly respond to
-   * metadata requests
+   * metadata requests. Particularly, when feature versioning is enabled, we filter out brokers with incompatible
+   * features from receiving the metadata requests. This is because we do not want to activate incompatible brokers,
+   * as these may have harmful consequences to the cluster.

Review comment:
       Good question. Yes, the broker will shut itself down. But still there is a possible race condition that needs to be handled to prevent an incompatible broker from causing damage to cluster. The race condition is described in the KIP-584 [in this section](https://cwiki.apache.org/confluence/display/KAFKA/KIP-584%3A+Versioning+scheme+for+features#KIP584:Versioningschemeforfeatures-Incompatiblebrokerlifetimeracecondition). Please let me know your thoughts.




----------------------------------------------------------------
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 edited a comment on pull request #9001: KAFKA-10028: Implement KIP-584 write path

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #9001:
URL: https://github.com/apache/kafka/pull/9001#issuecomment-656902905


   @dajac Thank you for taking a look! IIUC you are referring to these lines:
   
   https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java#L550-L553
   
   My requirement is that under the hood of the newly added API: `org.apache.kafka.clients.Admin#describeFeatures`, the `ApiVersionsResponse` returned to the `AdminClient` needs to contain the features information. Note that this new API issues an explicit `ApiVersionsRequest` under the hood. In such a case do you think I should populate the features information in the above lines 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] junrao merged pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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


   


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4071,6 +4078,113 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        final NodeProvider provider =
+            options.sendRequestToController() ? new ControllerNodeProvider() : new LeastLoadedNodeProvider();
+
+        Call call = new Call(
+            "describeFeatures", calcDeadlineMs(now, options.timeoutMs()), provider) {
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(
+                        new FeatureMetadata(
+                            apiVersionsResponse.finalizedFeatures(),
+                            apiVersionsResponse.finalizedFeaturesEpoch(),
+                            apiVersionsResponse.supportedFeatures()));
+                } else if (options.sendRequestToController() && apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) {
+                    handleNotControllerError(Errors.NOT_CONTROLLER);
+                } else {
+                    future.completeExceptionally(
+                        Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        runnable.call(call, now);
+        return new DescribeFeaturesResult(future);
+    }
+
+    @Override
+    public UpdateFeaturesResult updateFeatures(
+        final Map<String, FeatureUpdate> featureUpdates, final UpdateFeaturesOptions options) {
+        if (featureUpdates == null || featureUpdates.isEmpty()) {
+            throw new IllegalArgumentException("Feature updates can not be null or empty.");
+        }
+        Objects.requireNonNull(options, "UpdateFeaturesOptions can not be null");
+
+        final UpdateFeaturesRequestData request = UpdateFeaturesRequest.create(featureUpdates);
+        final Map<String, KafkaFutureImpl<Void>> updateFutures = new HashMap<>();
+        for (Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+            updateFutures.put(entry.getKey(), new KafkaFutureImpl<>());
+        }
+        final long now = time.milliseconds();
+        final Call call = new Call("updateFeatures", calcDeadlineMs(now, options.timeoutMs()),
+            new ControllerNodeProvider()) {
+
+            @Override
+            UpdateFeaturesRequest.Builder createRequest(int timeoutMs) {
+                return new UpdateFeaturesRequest.Builder(request);
+            }
+
+            @Override
+            void handleResponse(AbstractResponse abstractResponse) {
+                final UpdateFeaturesResponse response =
+                    (UpdateFeaturesResponse) abstractResponse;
+
+                // Check for controller change.
+                for (UpdatableFeatureResult result : response.data().results()) {
+                    final Errors error = Errors.forCode(result.errorCode());
+                    if (error == Errors.NOT_CONTROLLER) {
+                        handleNotControllerError(error);
+                        throw error.exception();

Review comment:
       > handleNotControllerError() already throws an exception.
   Done. Fixed the code to not throw exception again when handling NOT_CONTROLLER error.
   
   > Should other errors like CLUSTER_AUTHORIZATION_FAILED be treated in the same way?
   I'm not sure how could we treat it the same way. In the case of the NOT_CONTROLLER error, the admin client code would retry the request once again when the exception is raised. But when cluster authorization fails, would a retry help?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4071,6 +4078,113 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        final NodeProvider provider =
+            options.sendRequestToController() ? new ControllerNodeProvider() : new LeastLoadedNodeProvider();
+
+        Call call = new Call(
+            "describeFeatures", calcDeadlineMs(now, options.timeoutMs()), provider) {
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(
+                        new FeatureMetadata(
+                            apiVersionsResponse.finalizedFeatures(),
+                            apiVersionsResponse.finalizedFeaturesEpoch(),
+                            apiVersionsResponse.supportedFeatures()));
+                } else if (options.sendRequestToController() && apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) {
+                    handleNotControllerError(Errors.NOT_CONTROLLER);
+                } else {
+                    future.completeExceptionally(
+                        Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        runnable.call(call, now);
+        return new DescribeFeaturesResult(future);
+    }
+
+    @Override
+    public UpdateFeaturesResult updateFeatures(
+        final Map<String, FeatureUpdate> featureUpdates, final UpdateFeaturesOptions options) {
+        if (featureUpdates == null || featureUpdates.isEmpty()) {
+            throw new IllegalArgumentException("Feature updates can not be null or empty.");
+        }
+        Objects.requireNonNull(options, "UpdateFeaturesOptions can not be null");
+
+        final UpdateFeaturesRequestData request = UpdateFeaturesRequest.create(featureUpdates);
+        final Map<String, KafkaFutureImpl<Void>> updateFutures = new HashMap<>();
+        for (Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+            updateFutures.put(entry.getKey(), new KafkaFutureImpl<>());
+        }
+        final long now = time.milliseconds();
+        final Call call = new Call("updateFeatures", calcDeadlineMs(now, options.timeoutMs()),
+            new ControllerNodeProvider()) {
+
+            @Override
+            UpdateFeaturesRequest.Builder createRequest(int timeoutMs) {
+                return new UpdateFeaturesRequest.Builder(request);
+            }
+
+            @Override
+            void handleResponse(AbstractResponse abstractResponse) {
+                final UpdateFeaturesResponse response =
+                    (UpdateFeaturesResponse) abstractResponse;
+
+                // Check for controller change.
+                for (UpdatableFeatureResult result : response.data().results()) {
+                    final Errors error = Errors.forCode(result.errorCode());
+                    if (error == Errors.NOT_CONTROLLER) {
+                        handleNotControllerError(error);
+                        throw error.exception();

Review comment:
       > handleNotControllerError() already throws an exception.
   
   Done. Fixed the code to not throw exception again when handling NOT_CONTROLLER error.
   
   > Should other errors like CLUSTER_AUTHORIZATION_FAILED be treated in the same way?
   
   I'm not sure how could we treat it the same way. In the case of the NOT_CONTROLLER error, the admin client code would retry the request once again when the exception is raised. But when cluster authorization fails, would a retry help?




----------------------------------------------------------------
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 pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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


   retest this


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/resources/common/message/UpdateFeaturesRequest.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "request",
+  "name": "UpdateFeaturesRequest",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "FeatureUpdates", "type": "[]FeatureUpdateKey", "versions": "0+",
+      "about": "The list of updates to finalized features.", "fields": [
+      {"name": "Feature", "type": "string", "versions": "0+", "mapKey": true,
+        "about": "The name of the finalized feature to be updated."},
+      {"name": "MaxVersionLevel", "type": "int16", "versions": "0+",
+        "about": "The new maximum version level for the finalized feature. A value >= 1 is valid. A value < 1, is special, and can be used to request the deletion of the finalized feature."},
+      {"name": "AllowDowngrade", "type": "bool", "versions": "0+",

Review comment:
       Done. Fixed the KIP and the code, so that they align with each other now.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java
##########
@@ -17,9 +17,16 @@
 package org.apache.kafka.common.feature;
 
 import java.util.Map;
+import java.util.Objects;
+import org.apache.kafka.common.utils.Utils;
 
 /**
- * An extended {@link BaseVersionRange} representing the min/max versions for supported features.
+ * An extended {@link BaseVersionRange} representing the min, max and first active versions for a
+ * supported feature:
+ *  - minVersion: This is the minimum supported version for the feature.
+ *  - maxVersion: This the maximum supported version for the feature.
+ *  - firstActiveVersion: This is the first active version for the feature. Versions in the range

Review comment:
       Thinking about this a bit more. It seems that the intention of firstActiveVersion is to avoid deploying a wrong version of the broker that causes the deprecation of a finalized feature version unexpectedly. However, the same mistake can happen with firstActiveVersion since the deprecation of a finalized feature version is based on firstActiveVersion. So, I am not sure if firstActiveVersion addresses a real problem.
   
   In general, we tend to deprecate a version very slowly in AK. So, if the mistake is to deploy a new release that actually deprecates a supported version. Old clients are likely all gone. So, moving finalized min version to supported min version may not cause a big problem. We can just document that people should make sure old versions are no longer used before deploying new releases.
   
   If the mistake is to deploy an old version of the broker whose maxSupportedVersion is < maxFinalizedVersion, we will fail the broker. So, this mistake can be prevented.




----------------------------------------------------------------
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 pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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


   retest this please
   
   


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FinalizedVersionRange.java
##########
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+
+/**
+ * Represents a range of version levels supported by every broker in a cluster for some feature.
+ */
+public class FinalizedVersionRange {

Review comment:
       I considered this, however if we plan to expose `firstActiveVersion` to the client, then, it is better to have 2 separate classes like we do now. This is because `firstActiveVersion` will become an attribute only in `SupportedVersionRange` class. 




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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


   @junrao Thanks for the links! I had a look at the links. I found 27 test failures in both links. I compared the individual test failures and found that they have all failed on the same tests. Would that mean we are OK to merge this PR?


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4071,6 +4078,113 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        final NodeProvider provider =
+            options.sendRequestToController() ? new ControllerNodeProvider() : new LeastLoadedNodeProvider();
+
+        Call call = new Call(
+            "describeFeatures", calcDeadlineMs(now, options.timeoutMs()), provider) {
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(
+                        new FeatureMetadata(
+                            apiVersionsResponse.finalizedFeatures(),
+                            apiVersionsResponse.finalizedFeaturesEpoch(),
+                            apiVersionsResponse.supportedFeatures()));
+                } else if (options.sendRequestToController() && apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) {
+                    handleNotControllerError(Errors.NOT_CONTROLLER);
+                } else {
+                    future.completeExceptionally(
+                        Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        runnable.call(call, now);
+        return new DescribeFeaturesResult(future);
+    }
+
+    @Override
+    public UpdateFeaturesResult updateFeatures(
+        final Map<String, FeatureUpdate> featureUpdates, final UpdateFeaturesOptions options) {
+        if (featureUpdates == null || featureUpdates.isEmpty()) {
+            throw new IllegalArgumentException("Feature updates can not be null or empty.");
+        }
+        Objects.requireNonNull(options, "UpdateFeaturesOptions can not be null");
+
+        final UpdateFeaturesRequestData request = UpdateFeaturesRequest.create(featureUpdates);
+        final Map<String, KafkaFutureImpl<Void>> updateFutures = new HashMap<>();
+        for (Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+            updateFutures.put(entry.getKey(), new KafkaFutureImpl<>());
+        }
+        final long now = time.milliseconds();
+        final Call call = new Call("updateFeatures", calcDeadlineMs(now, options.timeoutMs()),
+            new ControllerNodeProvider()) {
+
+            @Override
+            UpdateFeaturesRequest.Builder createRequest(int timeoutMs) {
+                return new UpdateFeaturesRequest.Builder(request);
+            }
+
+            @Override
+            void handleResponse(AbstractResponse abstractResponse) {
+                final UpdateFeaturesResponse response =
+                    (UpdateFeaturesResponse) abstractResponse;
+
+                // Check for controller change.
+                for (UpdatableFeatureResult result : response.data().results()) {
+                    final Errors error = Errors.forCode(result.errorCode());
+                    if (error == Errors.NOT_CONTROLLER) {
+                        handleNotControllerError(error);
+                        throw error.exception();

Review comment:
       1) Fixed the code to not throw exception again.
   2) For CLUSTER_AUTHORIZATION_FAILED, I'm not sure how could we treat it the same way. In the case of the NOT_CONTROLLER error, the admin client code would retry the request once again when the exception is raised. But when cluster authorization fails, would a retry help?




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala
##########
@@ -185,7 +185,7 @@ class BrokerEndPointTest {
       "endpoints":["CLIENT://host1:9092", "REPLICATION://host1:9093"],
       "listener_security_protocol_map":{"CLIENT":"SSL", "REPLICATION":"PLAINTEXT"},
       "rack":"dc1",
-      "features": {"feature1": {"min_version": 1, "max_version": 2}, "feature2": {"min_version": 2, "max_version": 4}}
+      "features": {"feature1": {"min_version": 1, "first_active_version": 1, "max_version": 2}, "feature2": {"min_version": 2, "first_active_version": 2, "max_version": 4}}

Review comment:
       Should we revert the changes here?

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -272,6 +281,161 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (string) and a range of versions (defined by a
+   * SupportedVersionRange). It refers to a feature that a particular broker advertises support for.
+   * Each broker advertises the version ranges of its own supported features in its own
+   * BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (string) and a range of version levels (defined
+   * by a FinalizedVersionRange). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there was an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has now been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). But the IBP config is still set to lower than KAFKA_2_7_IV0, and may be
+   *    set to a higher value later. In this case, we want to start with no finalized features and
+   *    allow the user to finalize them whenever they are ready i.e. in the future whenever the
+   *    user sets IBP config to be greater than or equal to KAFKA_2_7_IV0, then the user could start
+   *    finalizing the features. This process ensures we do not enable all the possible features
+   *    immediately after an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent.
+   *        - If the node is absent, it will react by creating a FeatureZNode with disabled status
+   *          and empty finalized features.
+   *        - Otherwise, if a node already exists in enabled status then the controller will just
+   *          flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled.
+   *         - If the node is in disabled status, the controller won’t upgrade all features immediately.
+   *           Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *           user finalize the features later.
+   *         - Otherwise, if a node already exists in enabled status then the controller will leave
+   *           the node umodified.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine there was an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker
+   *    binary has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and
+   *    higher). The controller will start up and find that a FeatureZNode is already present with
+   *    enabled status and existing finalized features. In such a case, the controller leaves the node
+   *    unmodified.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled,
+                                          brokerFeatures.defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      val newFeatures = existingFeatureZNode.status match {
+        case FeatureZNodeStatus.Enabled => existingFeatureZNode.features
+        case FeatureZNodeStatus.Disabled =>
+          if (!existingFeatureZNode.features.empty()) {
+            warn(s"FeatureZNode at path: ${FeatureZNode.path} with disabled status" +
+              " contains non-empty features.")
+          }
+          Features.emptyFinalizedFeatures
+      }
+      val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures)

Review comment:
       It's a bit weird that FeatureZNode.status is defined as FeatureZNodeStatus.Value. It seems that it should be defined as just FeatureZNodeStatus?

##########
File path: core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala
##########
@@ -0,0 +1,580 @@
+/**
+ * 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.server
+
+import java.util.{Optional, Properties}
+import java.util.concurrent.ExecutionException
+
+import kafka.api.KAFKA_2_7_IV0
+import kafka.utils.TestUtils
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion}
+import kafka.utils.TestUtils.waitUntilTrue
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureUpdate, UpdateFeaturesOptions, UpdateFeaturesResult}
+import org.apache.kafka.common.errors.InvalidRequestException
+import org.apache.kafka.common.feature.FinalizedVersionRange
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.message.UpdateFeaturesRequestData
+import org.apache.kafka.common.message.UpdateFeaturesRequestData.FeatureUpdateKeyCollection
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{UpdateFeaturesRequest, UpdateFeaturesResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.Test
+import org.junit.Assert.{assertEquals, assertFalse, assertNotEquals, assertNotNull, assertTrue}
+import org.scalatest.Assertions.intercept
+
+import scala.jdk.CollectionConverters._
+import scala.reflect.ClassTag
+import scala.util.matching.Regex
+
+class UpdateFeaturesTest extends BaseRequestTest {
+
+  override def brokerCount = 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))))
+  }
+
+  private def defaultFinalizedFeatures(): Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2))))
+  }
+
+  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)
+  }
+
+  private def updateFeatureZNode(features: Features[FinalizedVersionRange]): Int = {
+    val server = serverForId(0).get
+    val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, features)
+    val newVersion = server.zkClient.updateFeatureZNode(newNode)
+    servers.foreach(s => {
+      s.featureCache.waitUntilEpochOrThrow(newVersion, s.config.zkConnectionTimeoutMs)
+    })
+    newVersion
+  }
+
+  private def getFeatureZNode(): FeatureZNode = {
+    val (mayBeFeatureZNodeBytes, version) = serverForId(0).get.zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(version, ZkVersion.UnknownVersion)
+    FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+  }
+
+  private def finalizedFeatures(features: java.util.Map[String, org.apache.kafka.clients.admin.FinalizedVersionRange]): Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(features.asScala.map {
+      case(name, versionRange) =>
+        (name, new FinalizedVersionRange(versionRange.minVersionLevel(), versionRange.maxVersionLevel()))
+    }.asJava)
+  }
+
+  private def supportedFeatures(features: java.util.Map[String, org.apache.kafka.clients.admin.SupportedVersionRange]): Features[SupportedVersionRange] = {
+    Features.supportedFeatures(features.asScala.map {
+      case(name, versionRange) =>
+        (name, new SupportedVersionRange(versionRange.minVersion(), versionRange.maxVersion()))
+    }.asJava)
+  }
+
+  private def checkFeatures(client: Admin,
+                            expectedNode: FeatureZNode,
+                            expectedFinalizedFeatures: Features[FinalizedVersionRange],
+                            expectedFinalizedFeaturesEpoch: Long,
+                            expectedSupportedFeatures: Features[SupportedVersionRange]): Unit = {
+    assertEquals(expectedNode, getFeatureZNode())
+    val featureMetadata = client.describeFeatures(
+      new DescribeFeaturesOptions().sendRequestToController(true)).featureMetadata.get
+    assertEquals(expectedFinalizedFeatures, finalizedFeatures(featureMetadata.finalizedFeatures))
+    assertEquals(expectedSupportedFeatures, supportedFeatures(featureMetadata.supportedFeatures))
+    assertEquals(Optional.of(expectedFinalizedFeaturesEpoch), featureMetadata.finalizedFeaturesEpoch)
+  }
+
+  private def checkException[ExceptionType <: Throwable](result: UpdateFeaturesResult,
+                                                         featureExceptionMsgPatterns: Map[String, Regex])
+                                                        (implicit tag: ClassTag[ExceptionType]): Unit = {
+    featureExceptionMsgPatterns.foreach {
+      case (feature, exceptionMsgPattern) =>
+        val exception = intercept[ExecutionException] {
+          result.values().get(feature).get()
+        }
+        val cause = exception.getCause
+        assertNotNull(cause)
+        assertEquals(cause.getClass, tag.runtimeClass)
+        assertTrue(s"Received unexpected error message: ${cause.getMessage}",
+                   exceptionMsgPattern.findFirstIn(cause.getMessage).isDefined)
+    }
+  }
+
+  /**
+   * Tests whether an invalid feature update does not get processed on the server as expected,
+   * and raises the ExceptionType on the client side as expected.
+   *
+   * @param invalidUpdate         the invalid feature update to be sent in the
+   *                              updateFeatures request to the server
+   * @param exceptionMsgPattern   a pattern for the expected exception message
+   */
+  private def testWithInvalidFeatureUpdate[ExceptionType <: Throwable](feature: String,

Review comment:
       Could we add feature to the javadoc above?

##########
File path: core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala
##########
@@ -0,0 +1,580 @@
+/**
+ * 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.server
+
+import java.util.{Optional, Properties}
+import java.util.concurrent.ExecutionException
+
+import kafka.api.KAFKA_2_7_IV0
+import kafka.utils.TestUtils
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion}
+import kafka.utils.TestUtils.waitUntilTrue
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureUpdate, UpdateFeaturesOptions, UpdateFeaturesResult}
+import org.apache.kafka.common.errors.InvalidRequestException
+import org.apache.kafka.common.feature.FinalizedVersionRange
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.message.UpdateFeaturesRequestData
+import org.apache.kafka.common.message.UpdateFeaturesRequestData.FeatureUpdateKeyCollection
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{UpdateFeaturesRequest, UpdateFeaturesResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.Test
+import org.junit.Assert.{assertEquals, assertFalse, assertNotEquals, assertNotNull, assertTrue}
+import org.scalatest.Assertions.intercept
+
+import scala.jdk.CollectionConverters._
+import scala.reflect.ClassTag
+import scala.util.matching.Regex
+
+class UpdateFeaturesTest extends BaseRequestTest {
+
+  override def brokerCount = 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))))
+  }
+
+  private def defaultFinalizedFeatures(): Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2))))
+  }
+
+  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)
+  }
+
+  private def updateFeatureZNode(features: Features[FinalizedVersionRange]): Int = {
+    val server = serverForId(0).get
+    val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, features)
+    val newVersion = server.zkClient.updateFeatureZNode(newNode)
+    servers.foreach(s => {
+      s.featureCache.waitUntilEpochOrThrow(newVersion, s.config.zkConnectionTimeoutMs)
+    })
+    newVersion
+  }
+
+  private def getFeatureZNode(): FeatureZNode = {
+    val (mayBeFeatureZNodeBytes, version) = serverForId(0).get.zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(version, ZkVersion.UnknownVersion)
+    FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+  }
+
+  private def finalizedFeatures(features: java.util.Map[String, org.apache.kafka.clients.admin.FinalizedVersionRange]): Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(features.asScala.map {
+      case(name, versionRange) =>
+        (name, new FinalizedVersionRange(versionRange.minVersionLevel(), versionRange.maxVersionLevel()))
+    }.asJava)
+  }
+
+  private def supportedFeatures(features: java.util.Map[String, org.apache.kafka.clients.admin.SupportedVersionRange]): Features[SupportedVersionRange] = {
+    Features.supportedFeatures(features.asScala.map {
+      case(name, versionRange) =>
+        (name, new SupportedVersionRange(versionRange.minVersion(), versionRange.maxVersion()))
+    }.asJava)
+  }
+
+  private def checkFeatures(client: Admin,
+                            expectedNode: FeatureZNode,
+                            expectedFinalizedFeatures: Features[FinalizedVersionRange],
+                            expectedFinalizedFeaturesEpoch: Long,
+                            expectedSupportedFeatures: Features[SupportedVersionRange]): Unit = {
+    assertEquals(expectedNode, getFeatureZNode())
+    val featureMetadata = client.describeFeatures(
+      new DescribeFeaturesOptions().sendRequestToController(true)).featureMetadata.get
+    assertEquals(expectedFinalizedFeatures, finalizedFeatures(featureMetadata.finalizedFeatures))
+    assertEquals(expectedSupportedFeatures, supportedFeatures(featureMetadata.supportedFeatures))
+    assertEquals(Optional.of(expectedFinalizedFeaturesEpoch), featureMetadata.finalizedFeaturesEpoch)
+  }
+
+  private def checkException[ExceptionType <: Throwable](result: UpdateFeaturesResult,
+                                                         featureExceptionMsgPatterns: Map[String, Regex])
+                                                        (implicit tag: ClassTag[ExceptionType]): Unit = {
+    featureExceptionMsgPatterns.foreach {
+      case (feature, exceptionMsgPattern) =>
+        val exception = intercept[ExecutionException] {
+          result.values().get(feature).get()
+        }
+        val cause = exception.getCause
+        assertNotNull(cause)
+        assertEquals(cause.getClass, tag.runtimeClass)
+        assertTrue(s"Received unexpected error message: ${cause.getMessage}",
+                   exceptionMsgPattern.findFirstIn(cause.getMessage).isDefined)
+    }
+  }
+
+  /**
+   * Tests whether an invalid feature update does not get processed on the server as expected,
+   * and raises the ExceptionType on the client side as expected.
+   *
+   * @param invalidUpdate         the invalid feature update to be sent in the
+   *                              updateFeatures request to the server
+   * @param exceptionMsgPattern   a pattern for the expected exception message
+   */
+  private def testWithInvalidFeatureUpdate[ExceptionType <: Throwable](feature: String,
+                                                                       invalidUpdate: FeatureUpdate,
+                                                                       exceptionMsgPattern: Regex)
+                                                                      (implicit tag: ClassTag[ExceptionType]): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+    val adminClient = createAdminClient()
+    val nodeBefore = getFeatureZNode()
+
+    val result = adminClient.updateFeatures(Utils.mkMap(Utils.mkEntry(feature, invalidUpdate)), new UpdateFeaturesOptions())
+
+    checkException[ExceptionType](result, Map(feature -> exceptionMsgPattern))
+    checkFeatures(
+      adminClient,
+      nodeBefore,
+      defaultFinalizedFeatures(),
+      versionBefore,
+      defaultSupportedFeatures())
+  }
+
+  /**
+   * Tests that an UpdateFeatures request sent to a non-Controller node fails as expected.
+   */
+  @Test
+  def testShouldFailRequestIfNotController(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val nodeBefore = getFeatureZNode()
+    val validUpdates = new FeatureUpdateKeyCollection()
+    val validUpdate = new UpdateFeaturesRequestData.FeatureUpdateKey();
+    validUpdate.setFeature("feature_1");
+    validUpdate.setMaxVersionLevel(defaultSupportedFeatures().get("feature_1").max())
+    validUpdate.setAllowDowngrade(false)
+    validUpdates.add(validUpdate)
+
+    val response = connectAndReceive[UpdateFeaturesResponse](
+      new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData().setFeatureUpdates(validUpdates)).build(),
+      notControllerSocketServer)
+
+    assertEquals(Errors.NOT_CONTROLLER, Errors.forCode(response.data.errorCode()))
+    assertNotNull(response.data.errorMessage())
+    assertEquals(0, response.data.results.size)
+    checkFeatures(
+      createAdminClient(),
+      nodeBefore,
+      defaultFinalizedFeatures(),
+      versionBefore,
+      defaultSupportedFeatures())
+  }
+
+  /**
+   * Tests that an UpdateFeatures request fails in the Controller, when, for a feature the
+   * allowDowngrade flag is not set during a downgrade request.
+   */
+  @Test
+  def testShouldFailRequestWhenDowngradeFlagIsNotSetDuringDowngrade(): Unit = {
+    val targetMaxVersionLevel = (defaultFinalizedFeatures().get("feature_1").max() - 1).asInstanceOf[Short]
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      "feature_1",
+      new FeatureUpdate(targetMaxVersionLevel,false),
+      ".*Can not downgrade finalized feature.*allowDowngrade.*".r)
+  }
+
+  /**
+   * Tests that an UpdateFeatures request fails in the Controller, when, for a feature the downgrade
+   * is attempted to a max version level thats higher than the existing max version level.
+   */
+  @Test
+  def testShouldFailRequestWhenDowngradeToHigherVersionLevelIsAttempted(): Unit = {
+    val targetMaxVersionLevel = (defaultFinalizedFeatures().get("feature_1").max() + 1).asInstanceOf[Short]
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      "feature_1",
+      new FeatureUpdate(targetMaxVersionLevel, true),
+      ".*When the allowDowngrade flag set in the request, the provided maxVersionLevel:3.*existing maxVersionLevel:2.*".r)
+  }
+
+  /**
+   * Tests that an UpdateFeatures request fails in the Controller, when, a feature deletion is
+   * attempted without setting the allowDowngrade flag.
+   */
+  @Test
+  def testShouldFailRequestInServerWhenDowngradeFlagIsNotSetDuringDeletion(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val adminClient = createAdminClient()
+    val nodeBefore = getFeatureZNode()
+
+    val invalidUpdates
+      = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection();
+    val invalidUpdate = new UpdateFeaturesRequestData.FeatureUpdateKey();
+    invalidUpdate.setFeature("feature_1")
+    invalidUpdate.setMaxVersionLevel(0)
+    invalidUpdate.setAllowDowngrade(false)
+    invalidUpdates.add(invalidUpdate);
+    val requestData = new UpdateFeaturesRequestData()
+    requestData.setFeatureUpdates(invalidUpdates);
+
+    val response = connectAndReceive[UpdateFeaturesResponse](
+      new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData().setFeatureUpdates(invalidUpdates)).build(),
+      controllerSocketServer)
+
+    assertEquals(1, response.data().results().size())
+    val result = response.data.results.asScala.head
+    assertEquals("feature_1", result.feature)
+    assertEquals(Errors.INVALID_REQUEST, Errors.forCode(result.errorCode))
+    assertNotNull(result.errorMessage)
+    assertFalse(result.errorMessage.isEmpty)
+    val exceptionMsgPattern = ".*Can not provide maxVersionLevel: 0 less than 1.*allowDowngrade.*".r
+    assertTrue(result.errorMessage, exceptionMsgPattern.findFirstIn(result.errorMessage).isDefined)
+    checkFeatures(
+      adminClient,
+      nodeBefore,
+      defaultFinalizedFeatures(),
+      versionBefore,
+      defaultSupportedFeatures())
+  }
+
+  /**
+   * Tests that an UpdateFeatures request fails in the Controller, when, a feature version level
+   * upgrade is attempted for a non-existing feature.
+   */
+  @Test
+  def testShouldFailRequestDuringDeletionOfNonExistingFeature(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      "feature_non_existing",
+      new FeatureUpdate(0, true),
+      ".*Can not delete non-existing finalized feature.*".r)
+  }
+
+  /**
+   * Tests that an UpdateFeatures request fails in the Controller, when, a feature version level
+   * upgrade is attempted to a version level thats the same as the existing max version level.

Review comment:
       typo thats

##########
File path: core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala
##########
@@ -715,7 +747,58 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness {
     doAnswer((_: InvocationOnMock) => {
       latch.countDown()
     }).doCallRealMethod().when(spyThread).awaitShutdown()
-    controller.shutdown() 
+    controller.shutdown()
+  }
+
+  private def testControllerFeatureZNodeSetup(initialZNode: Option[FeatureZNode],
+                                              interBrokerProtocolVersion: ApiVersion): Unit = {
+    val versionBeforeOpt = initialZNode match {
+      case Some(node) =>
+        zkClient.createFeatureZNode(node)
+        Some(zkClient.getDataAndVersion(FeatureZNode.path)._2)
+      case None =>
+        Option.empty
+    }
+    servers = makeServers(1, interBrokerProtocolVersion = Some(interBrokerProtocolVersion))
+    TestUtils.waitUntilControllerElected(zkClient)

Review comment:
       This is probably not enough since it only waits for the controller path to be created in ZK, which happens before the processing of the finalized features.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -272,6 +281,161 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (string) and a range of versions (defined by a
+   * SupportedVersionRange). It refers to a feature that a particular broker advertises support for.
+   * Each broker advertises the version ranges of its own supported features in its own
+   * BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (string) and a range of version levels (defined
+   * by a FinalizedVersionRange). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there was an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has now been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). But the IBP config is still set to lower than KAFKA_2_7_IV0, and may be
+   *    set to a higher value later. In this case, we want to start with no finalized features and
+   *    allow the user to finalize them whenever they are ready i.e. in the future whenever the
+   *    user sets IBP config to be greater than or equal to KAFKA_2_7_IV0, then the user could start
+   *    finalizing the features. This process ensures we do not enable all the possible features
+   *    immediately after an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent.
+   *        - If the node is absent, it will react by creating a FeatureZNode with disabled status
+   *          and empty finalized features.
+   *        - Otherwise, if a node already exists in enabled status then the controller will just
+   *          flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled.
+   *         - If the node is in disabled status, the controller won’t upgrade all features immediately.
+   *           Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *           user finalize the features later.
+   *         - Otherwise, if a node already exists in enabled status then the controller will leave
+   *           the node umodified.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine there was an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker
+   *    binary has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and
+   *    higher). The controller will start up and find that a FeatureZNode is already present with
+   *    enabled status and existing finalized features. In such a case, the controller leaves the node
+   *    unmodified.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled,
+                                          brokerFeatures.defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      val newFeatures = existingFeatureZNode.status match {
+        case FeatureZNodeStatus.Enabled => existingFeatureZNode.features
+        case FeatureZNodeStatus.Disabled =>
+          if (!existingFeatureZNode.features.empty()) {
+            warn(s"FeatureZNode at path: ${FeatureZNode.path} with disabled status" +
+              " contains non-empty features.")

Review comment:
       Should we log the non-empty features too?

##########
File path: core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala
##########
@@ -0,0 +1,580 @@
+/**
+ * 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.server
+
+import java.util.{Optional, Properties}
+import java.util.concurrent.ExecutionException
+
+import kafka.api.KAFKA_2_7_IV0
+import kafka.utils.TestUtils
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion}
+import kafka.utils.TestUtils.waitUntilTrue
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureUpdate, UpdateFeaturesOptions, UpdateFeaturesResult}
+import org.apache.kafka.common.errors.InvalidRequestException
+import org.apache.kafka.common.feature.FinalizedVersionRange
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.message.UpdateFeaturesRequestData
+import org.apache.kafka.common.message.UpdateFeaturesRequestData.FeatureUpdateKeyCollection
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{UpdateFeaturesRequest, UpdateFeaturesResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.Test
+import org.junit.Assert.{assertEquals, assertFalse, assertNotEquals, assertNotNull, assertTrue}
+import org.scalatest.Assertions.intercept
+
+import scala.jdk.CollectionConverters._
+import scala.reflect.ClassTag
+import scala.util.matching.Regex
+
+class UpdateFeaturesTest extends BaseRequestTest {
+
+  override def brokerCount = 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))))
+  }
+
+  private def defaultFinalizedFeatures(): Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2))))
+  }
+
+  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)
+  }
+
+  private def updateFeatureZNode(features: Features[FinalizedVersionRange]): Int = {
+    val server = serverForId(0).get
+    val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, features)
+    val newVersion = server.zkClient.updateFeatureZNode(newNode)
+    servers.foreach(s => {
+      s.featureCache.waitUntilEpochOrThrow(newVersion, s.config.zkConnectionTimeoutMs)
+    })
+    newVersion
+  }
+
+  private def getFeatureZNode(): FeatureZNode = {
+    val (mayBeFeatureZNodeBytes, version) = serverForId(0).get.zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(version, ZkVersion.UnknownVersion)
+    FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+  }
+
+  private def finalizedFeatures(features: java.util.Map[String, org.apache.kafka.clients.admin.FinalizedVersionRange]): Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(features.asScala.map {
+      case(name, versionRange) =>
+        (name, new FinalizedVersionRange(versionRange.minVersionLevel(), versionRange.maxVersionLevel()))
+    }.asJava)
+  }
+
+  private def supportedFeatures(features: java.util.Map[String, org.apache.kafka.clients.admin.SupportedVersionRange]): Features[SupportedVersionRange] = {
+    Features.supportedFeatures(features.asScala.map {
+      case(name, versionRange) =>
+        (name, new SupportedVersionRange(versionRange.minVersion(), versionRange.maxVersion()))
+    }.asJava)
+  }
+
+  private def checkFeatures(client: Admin,
+                            expectedNode: FeatureZNode,
+                            expectedFinalizedFeatures: Features[FinalizedVersionRange],
+                            expectedFinalizedFeaturesEpoch: Long,
+                            expectedSupportedFeatures: Features[SupportedVersionRange]): Unit = {
+    assertEquals(expectedNode, getFeatureZNode())
+    val featureMetadata = client.describeFeatures(
+      new DescribeFeaturesOptions().sendRequestToController(true)).featureMetadata.get
+    assertEquals(expectedFinalizedFeatures, finalizedFeatures(featureMetadata.finalizedFeatures))
+    assertEquals(expectedSupportedFeatures, supportedFeatures(featureMetadata.supportedFeatures))
+    assertEquals(Optional.of(expectedFinalizedFeaturesEpoch), featureMetadata.finalizedFeaturesEpoch)
+  }
+
+  private def checkException[ExceptionType <: Throwable](result: UpdateFeaturesResult,
+                                                         featureExceptionMsgPatterns: Map[String, Regex])
+                                                        (implicit tag: ClassTag[ExceptionType]): Unit = {
+    featureExceptionMsgPatterns.foreach {
+      case (feature, exceptionMsgPattern) =>
+        val exception = intercept[ExecutionException] {
+          result.values().get(feature).get()
+        }
+        val cause = exception.getCause
+        assertNotNull(cause)
+        assertEquals(cause.getClass, tag.runtimeClass)
+        assertTrue(s"Received unexpected error message: ${cause.getMessage}",
+                   exceptionMsgPattern.findFirstIn(cause.getMessage).isDefined)
+    }
+  }
+
+  /**
+   * Tests whether an invalid feature update does not get processed on the server as expected,
+   * and raises the ExceptionType on the client side as expected.
+   *
+   * @param invalidUpdate         the invalid feature update to be sent in the
+   *                              updateFeatures request to the server
+   * @param exceptionMsgPattern   a pattern for the expected exception message
+   */
+  private def testWithInvalidFeatureUpdate[ExceptionType <: Throwable](feature: String,
+                                                                       invalidUpdate: FeatureUpdate,
+                                                                       exceptionMsgPattern: Regex)
+                                                                      (implicit tag: ClassTag[ExceptionType]): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+    val adminClient = createAdminClient()
+    val nodeBefore = getFeatureZNode()
+
+    val result = adminClient.updateFeatures(Utils.mkMap(Utils.mkEntry(feature, invalidUpdate)), new UpdateFeaturesOptions())
+
+    checkException[ExceptionType](result, Map(feature -> exceptionMsgPattern))
+    checkFeatures(
+      adminClient,
+      nodeBefore,
+      defaultFinalizedFeatures(),
+      versionBefore,
+      defaultSupportedFeatures())
+  }
+
+  /**
+   * Tests that an UpdateFeatures request sent to a non-Controller node fails as expected.
+   */
+  @Test
+  def testShouldFailRequestIfNotController(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val nodeBefore = getFeatureZNode()
+    val validUpdates = new FeatureUpdateKeyCollection()
+    val validUpdate = new UpdateFeaturesRequestData.FeatureUpdateKey();
+    validUpdate.setFeature("feature_1");
+    validUpdate.setMaxVersionLevel(defaultSupportedFeatures().get("feature_1").max())
+    validUpdate.setAllowDowngrade(false)
+    validUpdates.add(validUpdate)
+
+    val response = connectAndReceive[UpdateFeaturesResponse](
+      new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData().setFeatureUpdates(validUpdates)).build(),
+      notControllerSocketServer)
+
+    assertEquals(Errors.NOT_CONTROLLER, Errors.forCode(response.data.errorCode()))
+    assertNotNull(response.data.errorMessage())
+    assertEquals(0, response.data.results.size)
+    checkFeatures(
+      createAdminClient(),
+      nodeBefore,
+      defaultFinalizedFeatures(),
+      versionBefore,
+      defaultSupportedFeatures())
+  }
+
+  /**
+   * Tests that an UpdateFeatures request fails in the Controller, when, for a feature the
+   * allowDowngrade flag is not set during a downgrade request.
+   */
+  @Test
+  def testShouldFailRequestWhenDowngradeFlagIsNotSetDuringDowngrade(): Unit = {
+    val targetMaxVersionLevel = (defaultFinalizedFeatures().get("feature_1").max() - 1).asInstanceOf[Short]
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      "feature_1",
+      new FeatureUpdate(targetMaxVersionLevel,false),
+      ".*Can not downgrade finalized feature.*allowDowngrade.*".r)
+  }
+
+  /**
+   * Tests that an UpdateFeatures request fails in the Controller, when, for a feature the downgrade
+   * is attempted to a max version level thats higher than the existing max version level.
+   */
+  @Test
+  def testShouldFailRequestWhenDowngradeToHigherVersionLevelIsAttempted(): Unit = {
+    val targetMaxVersionLevel = (defaultFinalizedFeatures().get("feature_1").max() + 1).asInstanceOf[Short]
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      "feature_1",
+      new FeatureUpdate(targetMaxVersionLevel, true),
+      ".*When the allowDowngrade flag set in the request, the provided maxVersionLevel:3.*existing maxVersionLevel:2.*".r)
+  }
+
+  /**
+   * Tests that an UpdateFeatures request fails in the Controller, when, a feature deletion is
+   * attempted without setting the allowDowngrade flag.
+   */
+  @Test
+  def testShouldFailRequestInServerWhenDowngradeFlagIsNotSetDuringDeletion(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val adminClient = createAdminClient()
+    val nodeBefore = getFeatureZNode()
+
+    val invalidUpdates
+      = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection();
+    val invalidUpdate = new UpdateFeaturesRequestData.FeatureUpdateKey();
+    invalidUpdate.setFeature("feature_1")
+    invalidUpdate.setMaxVersionLevel(0)
+    invalidUpdate.setAllowDowngrade(false)
+    invalidUpdates.add(invalidUpdate);
+    val requestData = new UpdateFeaturesRequestData()
+    requestData.setFeatureUpdates(invalidUpdates);
+
+    val response = connectAndReceive[UpdateFeaturesResponse](
+      new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData().setFeatureUpdates(invalidUpdates)).build(),
+      controllerSocketServer)
+
+    assertEquals(1, response.data().results().size())
+    val result = response.data.results.asScala.head
+    assertEquals("feature_1", result.feature)
+    assertEquals(Errors.INVALID_REQUEST, Errors.forCode(result.errorCode))
+    assertNotNull(result.errorMessage)
+    assertFalse(result.errorMessage.isEmpty)
+    val exceptionMsgPattern = ".*Can not provide maxVersionLevel: 0 less than 1.*allowDowngrade.*".r
+    assertTrue(result.errorMessage, exceptionMsgPattern.findFirstIn(result.errorMessage).isDefined)
+    checkFeatures(
+      adminClient,
+      nodeBefore,
+      defaultFinalizedFeatures(),
+      versionBefore,
+      defaultSupportedFeatures())
+  }
+
+  /**
+   * Tests that an UpdateFeatures request fails in the Controller, when, a feature version level
+   * upgrade is attempted for a non-existing feature.
+   */
+  @Test
+  def testShouldFailRequestDuringDeletionOfNonExistingFeature(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      "feature_non_existing",
+      new FeatureUpdate(0, true),

Review comment:
       Should we use a version > 0?




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4335,6 +4343,150 @@ void handleFailure(Throwable throwable) {
                 .hi(password, salt, iterations);
     }
 
+    public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        final NodeProvider provider =
+            options.sendRequestToController() ? new ControllerNodeProvider() : new LeastLoadedNodeProvider();
+
+        final Call call = new Call(
+            "describeFeatures", calcDeadlineMs(now, options.timeoutMs()), provider) {
+
+            private FeatureMetadata createFeatureMetadata(final ApiVersionsResponse response) {
+                final Map<String, FinalizedVersionRange> finalizedFeatures = new HashMap<>();
+                for (final FinalizedFeatureKey key : response.data().finalizedFeatures().valuesSet()) {
+                    finalizedFeatures.put(key.name(), new FinalizedVersionRange(key.minVersionLevel(), key.maxVersionLevel()));
+                }
+
+                Optional<Long> finalizedFeaturesEpoch;
+                if (response.data().finalizedFeaturesEpoch() >= 0L) {
+                    finalizedFeaturesEpoch = Optional.of(response.data().finalizedFeaturesEpoch());
+                } else {
+                    finalizedFeaturesEpoch = Optional.empty();
+                }
+
+                final Map<String, SupportedVersionRange> supportedFeatures = new HashMap<>();
+                for (final SupportedFeatureKey key : response.data().supportedFeatures().valuesSet()) {
+                    supportedFeatures.put(key.name(), new SupportedVersionRange(key.minVersion(), key.maxVersion()));
+                }
+
+                return new FeatureMetadata(finalizedFeatures, finalizedFeaturesEpoch, supportedFeatures);
+            }
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(createFeatureMetadata(apiVersionsResponse));
+                } else if (options.sendRequestToController() &&
+                           apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) {
+                    handleNotControllerError(Errors.NOT_CONTROLLER);
+                } else {
+                    future.completeExceptionally(Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        runnable.call(call, now);
+        return new DescribeFeaturesResult(future);
+    }
+
+    @Override
+    public UpdateFeaturesResult updateFeatures(final Map<String, FeatureUpdate> featureUpdates,
+                                               final UpdateFeaturesOptions options) {
+        if (featureUpdates.isEmpty()) {
+            throw new IllegalArgumentException("Feature updates can not be null or empty.");
+        }
+
+        final Map<String, KafkaFutureImpl<Void>> updateFutures = new HashMap<>();
+        for (final Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+            updateFutures.put(entry.getKey(), new KafkaFutureImpl<>());
+        }
+
+        final long now = time.milliseconds();
+        final Call call = new Call("updateFeatures", calcDeadlineMs(now, options.timeoutMs()),
+            new ControllerNodeProvider()) {
+
+            @Override
+            UpdateFeaturesRequest.Builder createRequest(int timeoutMs) {
+                final UpdateFeaturesRequestData.FeatureUpdateKeyCollection featureUpdatesRequestData
+                    = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection();
+                for (Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+                    final String feature = entry.getKey();
+                    final FeatureUpdate update = entry.getValue();
+                    if (feature.trim().isEmpty()) {

Review comment:
       Done. Addressed in #9393.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1306,6 +1307,73 @@ default AlterUserScramCredentialsResult alterUserScramCredentials(List<UserScram
     AlterUserScramCredentialsResult alterUserScramCredentials(List<UserScramCredentialAlteration> alterations,
                                                               AlterUserScramCredentialsOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker. It can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter. This is particularly useful if the user requires strongly consistent reads of
+     * finalized features.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the {@link DescribeFeaturesResult} containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);

Review comment:
       Done. Addressed in #9393.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4335,6 +4343,150 @@ void handleFailure(Throwable throwable) {
                 .hi(password, salt, iterations);
     }
 
+    public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        final NodeProvider provider =
+            options.sendRequestToController() ? new ControllerNodeProvider() : new LeastLoadedNodeProvider();
+
+        final Call call = new Call(
+            "describeFeatures", calcDeadlineMs(now, options.timeoutMs()), provider) {
+
+            private FeatureMetadata createFeatureMetadata(final ApiVersionsResponse response) {
+                final Map<String, FinalizedVersionRange> finalizedFeatures = new HashMap<>();
+                for (final FinalizedFeatureKey key : response.data().finalizedFeatures().valuesSet()) {
+                    finalizedFeatures.put(key.name(), new FinalizedVersionRange(key.minVersionLevel(), key.maxVersionLevel()));
+                }
+
+                Optional<Long> finalizedFeaturesEpoch;
+                if (response.data().finalizedFeaturesEpoch() >= 0L) {
+                    finalizedFeaturesEpoch = Optional.of(response.data().finalizedFeaturesEpoch());
+                } else {
+                    finalizedFeaturesEpoch = Optional.empty();
+                }
+
+                final Map<String, SupportedVersionRange> supportedFeatures = new HashMap<>();
+                for (final SupportedFeatureKey key : response.data().supportedFeatures().valuesSet()) {
+                    supportedFeatures.put(key.name(), new SupportedVersionRange(key.minVersion(), key.maxVersion()));
+                }
+
+                return new FeatureMetadata(finalizedFeatures, finalizedFeaturesEpoch, supportedFeatures);
+            }
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(createFeatureMetadata(apiVersionsResponse));
+                } else if (options.sendRequestToController() &&
+                           apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) {
+                    handleNotControllerError(Errors.NOT_CONTROLLER);
+                } else {
+                    future.completeExceptionally(Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        runnable.call(call, now);
+        return new DescribeFeaturesResult(future);
+    }
+
+    @Override
+    public UpdateFeaturesResult updateFeatures(final Map<String, FeatureUpdate> featureUpdates,
+                                               final UpdateFeaturesOptions options) {
+        if (featureUpdates.isEmpty()) {
+            throw new IllegalArgumentException("Feature updates can not be null or empty.");
+        }
+
+        final Map<String, KafkaFutureImpl<Void>> updateFutures = new HashMap<>();
+        for (final Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+            updateFutures.put(entry.getKey(), new KafkaFutureImpl<>());
+        }
+
+        final long now = time.milliseconds();
+        final Call call = new Call("updateFeatures", calcDeadlineMs(now, options.timeoutMs()),
+            new ControllerNodeProvider()) {
+
+            @Override
+            UpdateFeaturesRequest.Builder createRequest(int timeoutMs) {
+                final UpdateFeaturesRequestData.FeatureUpdateKeyCollection featureUpdatesRequestData
+                    = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection();
+                for (Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+                    final String feature = entry.getKey();
+                    final FeatureUpdate update = entry.getValue();
+                    if (feature.trim().isEmpty()) {
+                        throw new IllegalArgumentException("Provided feature can not be null or empty.");
+                    }
+
+                    final UpdateFeaturesRequestData.FeatureUpdateKey requestItem =
+                        new UpdateFeaturesRequestData.FeatureUpdateKey();
+                    requestItem.setFeature(feature);
+                    requestItem.setMaxVersionLevel(update.maxVersionLevel());
+                    requestItem.setAllowDowngrade(update.allowDowngrade());
+                    featureUpdatesRequestData.add(requestItem);
+                }
+                return new UpdateFeaturesRequest.Builder(
+                    new UpdateFeaturesRequestData()
+                        .setTimeoutMs(timeoutMs)
+                        .setFeatureUpdates(featureUpdatesRequestData));
+            }
+
+            @Override
+            void handleResponse(AbstractResponse abstractResponse) {
+                final UpdateFeaturesResponse response =
+                    (UpdateFeaturesResponse) abstractResponse;
+
+                Errors topLevelError = Errors.forCode(response.data().errorCode());
+                switch (topLevelError) {
+                    case NONE:
+                        for (final UpdatableFeatureResult result : response.data().results()) {
+                            final KafkaFutureImpl<Void> future = updateFutures.get(result.feature());
+                            if (future == null) {
+                                log.warn("Server response mentioned unknown feature {}", result.feature());
+                            } else {
+                                final Errors error = Errors.forCode(result.errorCode());
+                                if (error == Errors.NONE) {
+                                    future.complete(null);
+                                } else {
+                                    future.completeExceptionally(error.exception(result.errorMessage()));
+                                }
+                            }
+                        }
+                        // The server should send back a response for every feature, but we do a sanity check anyway.
+                        completeUnrealizedFutures(updateFutures.entrySet().stream(),
+                            feature -> "The controller response did not contain a result for feature " + feature);
+                        break;
+                    case NOT_CONTROLLER:
+                        handleNotControllerError(topLevelError);
+                        break;
+                    default:
+                        for (final Map.Entry<String, KafkaFutureImpl<Void>> entry : updateFutures.entrySet()) {
+                            entry.getValue().completeExceptionally(topLevelError.exception());

Review comment:
       Done. Addressed in #9393.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java
##########
@@ -17,9 +17,16 @@
 package org.apache.kafka.common.feature;
 
 import java.util.Map;
+import java.util.Objects;
+import org.apache.kafka.common.utils.Utils;
 
 /**
- * An extended {@link BaseVersionRange} representing the min/max versions for supported features.
+ * An extended {@link BaseVersionRange} representing the min, max and first active versions for a
+ * supported feature:
+ *  - minVersion: This is the minimum supported version for the feature.
+ *  - maxVersion: This the maximum supported version for the feature.
+ *  - firstActiveVersion: This is the first active version for the feature. Versions in the range

Review comment:
       @junrao Does the below feel right to you?
   
   The key thing seems to be that you feel it is rare to deprecate feature versions in AK. I agree with the same. So, I propose we just do not have to solve the deprecation problem in this PR, until we find a clear route that the AK community agrees with. In this PR I propose to revert the `firstActiveVersion` change, leaving the rest of the things the way they are. In the future, we can develop a concrete solution for version deprecation i.e. the part on how to advance `minVersion` of supported feature, may be (or may not be) using `firstActiveVersion` or other ways (it is up for discussion, maybe in a separate KIP). I have made this proposed change in the most recent commit: 4218f95904989028a469930d0c266362bf173ece.
   
   Regarding your thought:
   > I was thinking what if we relax the current check by just making sure that maxVersion of finalized is within the supported range. Basically in your example, if supported minVersion goes to 2, it's still allowed since it's less than maxVersion of finalized. However, if supported minVersion goes to 7, this fails the broker since it's more than maxVersion of finalized.
   
   There is a consequence to relaxing the current check:
   The controller can not effectively finalize `minVersionLevel` for the feature, because, with a relaxed check we do not know whether all brokers in the cluster support a particular `minVersion` when the controller finalizes the `minVersionLevel` at a particular value. It seems useful to keep the concept of `minVersionLevel` like the way it is now (i.e. it is the lowest version guaranteed to be supported by any broker in the cluster for a feature). And as I said above, in the future, we can decide on ways to mutate it safely (maybe through `firstActiveVersion` or other means).
   
   




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4052,6 +4058,128 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        final NodeProvider provider =
+            options.sendRequestToController() ? new ControllerNodeProvider() : new LeastLoadedNodeProvider();
+
+        Call call = new Call(
+            "describeFeatures", calcDeadlineMs(now, options.timeoutMs()), provider) {
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(
+                        new FeatureMetadata(
+                            apiVersionsResponse.finalizedFeatures(),
+                            apiVersionsResponse.finalizedFeaturesEpoch(),
+                            apiVersionsResponse.supportedFeatures()));
+                } else if (options.sendRequestToController() && apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) {
+                    handleNotControllerError(Errors.NOT_CONTROLLER);
+                } else {
+                    future.completeExceptionally(
+                        Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        runnable.call(call, now);
+        return new DescribeFeaturesResult(future);
+    }
+
+    @Override
+    public UpdateFeaturesResult updateFeatures(
+        final Map<String, FeatureUpdate> featureUpdates, final UpdateFeaturesOptions options) {
+        if (featureUpdates == null || featureUpdates.isEmpty()) {
+            throw new IllegalArgumentException("Feature updates can not be null or empty.");
+        }
+        Objects.requireNonNull(options, "UpdateFeaturesOptions can not be null");
+
+        final Map<String, KafkaFutureImpl<Void>> updateFutures = new HashMap<>();
+        final UpdateFeaturesRequestData.FeatureUpdateKeyCollection featureUpdatesRequestData
+            = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection();
+        for (Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+            final String feature = entry.getKey();
+            final FeatureUpdate update = entry.getValue();
+            if (feature.trim().isEmpty()) {
+                throw new IllegalArgumentException("Provided feature can not be null or empty.");
+            }
+
+            updateFutures.put(feature, new KafkaFutureImpl<>());
+            final UpdateFeaturesRequestData.FeatureUpdateKey requestItem =
+                new UpdateFeaturesRequestData.FeatureUpdateKey();
+            requestItem.setFeature(feature);
+            requestItem.setMaxVersionLevel(update.maxVersionLevel());
+            requestItem.setAllowDowngrade(update.allowDowngrade());
+            featureUpdatesRequestData.add(requestItem);
+        }
+        final UpdateFeaturesRequestData request = new UpdateFeaturesRequestData().setFeatureUpdates(featureUpdatesRequestData);
+
+        final long now = time.milliseconds();
+        final Call call = new Call("updateFeatures", calcDeadlineMs(now, options.timeoutMs()),
+            new ControllerNodeProvider()) {
+
+            @Override
+            UpdateFeaturesRequest.Builder createRequest(int timeoutMs) {
+                return new UpdateFeaturesRequest.Builder(request);
+            }
+
+            @Override
+            void handleResponse(AbstractResponse abstractResponse) {
+                final UpdateFeaturesResponse response =
+                    (UpdateFeaturesResponse) abstractResponse;
+
+                // Check for controller change.
+                for (UpdatableFeatureResult result : response.data().results()) {
+                    final Errors error = Errors.forCode(result.errorCode());
+                    if (error == Errors.NOT_CONTROLLER) {
+                        handleNotControllerError(error);
+                        throw error.exception();
+                    }
+                }
+
+                for (UpdatableFeatureResult result : response.data().results()) {
+                    final KafkaFutureImpl<Void> future = updateFutures.get(result.feature());
+                    if (future == null) {

Review comment:
       It does not overlap. This checks for unexpected responses for features that we never intended to update. `completeUnrealizedFutures` checks for futures that we never got a response for from the server.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1215,70 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker. It can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter. This is particularly useful if the user requires strongly consistent reads of
+     * finalized features.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the DescribeFeaturesResult containing the result

Review comment:
       nit: do `{@link DescribeFeaturesResult}`

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java
##########
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+import java.util.Set;
+import org.apache.kafka.common.message.UpdateFeaturesRequestData;
+
+/**
+ * Encapsulates details about an update to a finalized feature. This is particularly useful to
+ * define each feature update in the {@link Admin#updateFeatures(Set, UpdateFeaturesOptions)} API.
+ */
+public class FeatureUpdate {
+    private final String featureName;
+    private final short maxVersionLevel;
+    private final boolean allowDowngrade;
+
+    /**
+     * @param featureName       the name of the finalized feature to be updated.
+     * @param maxVersionLevel   the new maximum version level for the finalized feature.
+     *                          a value < 1 is special and indicates that the update is intended to
+     *                          delete the finalized feature, and should be accompanied by setting
+     *                          the allowDowngrade flag to true.
+     * @param allowDowngrade    - true, if this feature update was meant to downgrade the existing
+     *                            maximum version level of the finalized feature.
+     *                          - false, otherwise.
+     */
+    public FeatureUpdate(final String featureName, final short maxVersionLevel, final boolean allowDowngrade) {
+        Objects.requireNonNull(featureName, "Provided feature name can not be null.");
+        if (maxVersionLevel < 1 && !allowDowngrade) {
+            throw new IllegalArgumentException(
+                String.format(
+                    "For featureName: %s, the allowDowngrade flag is not set when the" +

Review comment:
       s/featureName/feature

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/FinalizedVersionRange.java
##########
@@ -50,4 +50,8 @@ public static FinalizedVersionRange fromMap(Map<String, Short> versionRangeMap)
     public boolean isIncompatibleWith(SupportedVersionRange supportedVersionRange) {
         return min() < supportedVersionRange.min() || max() > supportedVersionRange.max();
     }
+
+    public boolean isCompatibleWith(SupportedVersionRange supportedVersionRange) {

Review comment:
       Couldn't we just use `isIncompatibleWith`?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java
##########
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+@InterfaceStability.Evolving
+public class DescribeFeaturesOptions extends AbstractOptions<DescribeFeaturesOptions> {
+    /**
+     * - True means the {@link Admin#describeFeatures(DescribeFeaturesOptions)} request can be
+     *   issued only to the controller.
+     * - False means the {@link Admin#describeFeatures(DescribeFeaturesOptions)} request can be
+     *   issued to any random broker.
+     */
+    private boolean shouldSendRequestToController = false;
+
+    /**
+     * Sets a flag indicating that the describe features request should be issued to the controller.
+     */
+    public DescribeFeaturesOptions sendRequestToController(boolean shouldSendRequestToController) {
+        this.shouldSendRequestToController = shouldSendRequestToController;
+        return this;
+    }
+
+    public boolean sendRequestToController() {
+        return shouldSendRequestToController;
+    }
+
+    /**
+     * Sets the timeout in milliseconds for this operation or {@code null} if the default API
+     * timeout for the AdminClient should be used.
+     */
+    public DescribeFeaturesOptions timeoutMs(Integer timeoutMs) {

Review comment:
       Why do we need this override, which seems to be exactly the same with super class?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -3984,6 +3988,120 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+
+        Call callViaLeastLoadedNode = new Call("describeFeatures", calcDeadlineMs(now, options.timeoutMs()),
+            new LeastLoadedNodeProvider()) {
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(
+                        new FeatureMetadata(
+                            apiVersionsResponse.finalizedFeatures(),
+                            apiVersionsResponse.finalizedFeaturesEpoch(),
+                            apiVersionsResponse.supportedFeatures()));
+                } else {
+                    future.completeExceptionally(
+                        Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        Call callViaControllerNode = new Call("describeFeatures", calcDeadlineMs(now, options.timeoutMs()),

Review comment:
       The two `Call` structs only have two differences:
   
   1. Used different node provider
   2. One would handle not controller, one not
   
   So I would suggest a bit refactoring to reduce the code redundancy, by providing a helper as:
   ```
   getDescribeFeaturesCall(boolean sendToController) {
     NodeProvider provider = sendToController ? new ControllerNodeProvider()  : new 
     LeastLoadedNodeProvider();
     return new Call("describeFeatures", calcDeadlineMs(now, options.timeoutMs()), 
   provider, ...
     ...
        void handleResponse(AbstractResponse response) {
         ....
         if ( apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code() && sendToController) 
        {
           handleNotControllerError(Errors.NOT_CONTROLLER);
        }
     };
   }
   ```
   

##########
File path: clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
##########
@@ -319,7 +320,8 @@
     GROUP_SUBSCRIBED_TO_TOPIC(86, "Deleting offsets of a topic is forbidden while the consumer group is actively subscribed to it.",
         GroupSubscribedToTopicException::new),
     INVALID_RECORD(87, "This record has failed the validation on broker and hence will be rejected.", InvalidRecordException::new),
-    UNSTABLE_OFFSET_COMMIT(88, "There are unstable offsets that need to be cleared.", UnstableOffsetCommitException::new);
+    UNSTABLE_OFFSET_COMMIT(88, "There are unstable offsets that need to be cleared.", UnstableOffsetCommitException::new),
+    FEATURE_UPDATE_FAILED(91, "Unable to update finalized features.", FeatureUpdateFailedException::new);

Review comment:
       Why do we jump from code 88 to 91?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java
##########
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+@InterfaceStability.Evolving
+public class DescribeFeaturesOptions extends AbstractOptions<DescribeFeaturesOptions> {
+    /**
+     * - True means the {@link Admin#describeFeatures(DescribeFeaturesOptions)} request can be
+     *   issued only to the controller.
+     * - False means the {@link Admin#describeFeatures(DescribeFeaturesOptions)} request can be
+     *   issued to any random broker.
+     */
+    private boolean shouldSendRequestToController = false;

Review comment:
       Could be simplified as `sendToController`

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import java.nio.ByteBuffer;
+import org.apache.kafka.common.message.UpdateFeaturesResponseData;
+import org.apache.kafka.common.message.UpdateFeaturesRequestData;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.types.Struct;
+
+public class UpdateFeaturesRequest extends AbstractRequest {
+
+    public static class Builder extends AbstractRequest.Builder<UpdateFeaturesRequest> {
+
+        private final UpdateFeaturesRequestData data;
+
+        public Builder(UpdateFeaturesRequestData data) {
+            super(ApiKeys.UPDATE_FEATURES);
+            this.data = data;
+        }
+
+        @Override
+        public UpdateFeaturesRequest build(short version) {
+            return new UpdateFeaturesRequest(data, version);
+        }
+
+        @Override
+        public String toString() {
+            return data.toString();
+        }
+    }
+
+    public final UpdateFeaturesRequestData data;
+
+    public UpdateFeaturesRequest(UpdateFeaturesRequestData data, short version) {
+        super(ApiKeys.UPDATE_FEATURES, version);
+        this.data = data;
+    }
+
+    public UpdateFeaturesRequest(Struct struct, short version) {
+        super(ApiKeys.UPDATE_FEATURES, version);
+        this.data = new UpdateFeaturesRequestData(struct, version);
+    }
+
+    @Override
+    public AbstractResponse getErrorResponse(int throttleTimeMsIgnored, Throwable e) {
+        final ApiError apiError = ApiError.fromThrowable(e);
+        return new UpdateFeaturesResponse(
+            new UpdateFeaturesResponseData()
+                .setErrorCode(apiError.error().code())
+                .setErrorMessage(apiError.message()));
+    }
+
+    @Override
+    protected Struct toStruct() {
+        return data.toStruct(version());
+    }
+
+    public UpdateFeaturesRequestData data() {
+        return data;
+    }
+
+    public static UpdateFeaturesRequest parse(ByteBuffer buffer, short version) {
+        return new UpdateFeaturesRequest(
+            ApiKeys.UPDATE_FEATURES.parseRequest(version, buffer), version);
+    }
+
+    public static boolean isDeleteRequest(UpdateFeaturesRequestData.FeatureUpdateKey update) {
+        return update.maxVersionLevel() < 1;

Review comment:
       Do we also need to check `allowAutoDowngrade` here?

##########
File path: clients/src/main/resources/common/message/UpdateFeaturesRequest.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "request",
+  "name": "UpdateFeaturesRequest",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "FeatureUpdates", "type": "[]FeatureUpdateKey", "versions": "0+",
+      "about": "The list of updates to finalized features.", "fields": [
+      {"name": "Name", "type":  "string", "versions":  "0+", "mapKey": true,
+        "about": "The name of the finalized feature to be updated."},
+      {"name":  "MaxVersionLevel", "type": "int16", "versions":  "0+",

Review comment:
       Spaces look weird, let's try to remove all `two space` cases in this file.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.
+   *
+   * This method sets up the FeatureZNode with enabled status. This status means the feature
+   * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode

Review comment:
       Could we just remove ` the feature versioning system (KIP-584) is enabled, and`? It does not provide any useful information.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)

Review comment:
       nit: `zkClient.getDataAndVersion(FeatureZNode.path)._2` should be suffice

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying

Review comment:
       remove `one and`

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and

Review comment:
       s/Znode/ZNode

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -2945,6 +2948,130 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFeatures(request: RequestChannel.Request): Unit = {
+    val updateFeaturesRequest = request.body[UpdateFeaturesRequest]
+    def sendResponseCallback(error: Errors, msgOverride: Option[String]): Unit = {
+      val data = new UpdateFeaturesResponseData().setErrorCode(error.code())
+      msgOverride.map(msg => data.setErrorMessage(msg))
+      sendResponseExemptThrottle(request, new UpdateFeaturesResponse(data))
+    }
+
+    if (!authorize(request.context, ALTER, CLUSTER, CLUSTER_NAME)) {
+      sendResponseCallback(Errors.CLUSTER_AUTHORIZATION_FAILED, Option.empty)
+    } else if (!controller.isActive) {
+      sendResponseCallback(Errors.NOT_CONTROLLER, Option.empty)
+    } else if (!config.isFeatureVersioningEnabled) {
+      sendResponseCallback(Errors.INVALID_REQUEST, Some("Feature versioning system is disabled."))
+    } else if (updateFeaturesRequest.data.featureUpdates.isEmpty) {
+      sendResponseCallback(Errors.INVALID_REQUEST, Some("Can not provide empty FinalizedFeatureUpdates in the request."))
+    } else {
+      val targetFeaturesOrError = getTargetFinalizedFeaturesOrError(updateFeaturesRequest)
+      targetFeaturesOrError match {
+        case Left(targetFeatures) =>
+          controller.updateFeatures(targetFeatures, sendResponseCallback)
+        case Right(error) =>
+          sendResponseCallback(error.error, Some(error.message))
+      }
+    }
+  }
+
+  /**
+   * Validates the provided UpdateFinalizedFeaturesRequest, checking for various error cases.
+   * If the validation is successful, returns the target finalized features constructed from the
+   * request.
+   *
+   * @param request   the request to be validated
+   *
+   * @return          - the target finalized features, if request validation is successful

Review comment:
       We could explicitly mention this is `either or` result.

##########
File path: core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala
##########
@@ -0,0 +1,83 @@
+/**
+ * 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.server
+
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.junit.Assert.{assertEquals, assertThrows, assertTrue}
+import org.junit.Test
+
+import scala.jdk.CollectionConverters._
+
+class BrokerFeaturesTest {
+  @Test
+  def testEmpty(): Unit = {
+    assertTrue(BrokerFeatures.createDefault().supportedFeatures.empty)
+  }
+
+  @Test
+  def testIncompatibleFeatures(): Unit = {
+    val brokerFeatures = BrokerFeatures.createDefault()
+    val supportedFeatures = Map[String, SupportedVersionRange](
+      "test_feature_1" -> new SupportedVersionRange(1, 4),
+      "test_feature_2" -> new SupportedVersionRange(1, 3))
+    brokerFeatures.setSupportedFeatures(Features.supportedFeatures(supportedFeatures.asJava))
+
+    val compatibleFeatures = Map[String, FinalizedVersionRange](
+      "test_feature_1" -> new FinalizedVersionRange(2, 3))
+    val inCompatibleFeatures = Map[String, FinalizedVersionRange](
+      "test_feature_2" -> new FinalizedVersionRange(1, 4),
+      "test_feature_3" -> new FinalizedVersionRange(3, 4))
+    val features = compatibleFeatures++inCompatibleFeatures
+    val finalizedFeatures = Features.finalizedFeatures(features.asJava)
+
+    assertEquals(
+      Features.finalizedFeatures(inCompatibleFeatures.asJava),
+      brokerFeatures.incompatibleFeatures(finalizedFeatures))
+  }
+
+  @Test
+  def testFeatureVersionAssertions(): Unit = {
+    val brokerFeatures = BrokerFeatures.createDefault()
+    val supportedFeatures = Features.supportedFeatures(Map[String, SupportedVersionRange](
+      "test_feature_1" -> new SupportedVersionRange(1, 4),
+      "test_feature_2" -> new SupportedVersionRange(1, 3)).asJava)
+    brokerFeatures.setSupportedFeatures(supportedFeatures)
+
+    val defaultMinVersionLevelsWithNonExistingFeature = Map[String, Short](
+      "test_feature_1" -> 2,
+      "test_feature_2" -> 2,
+      "test_feature_non_existing" -> 5)
+    assertThrows(
+      classOf[IllegalArgumentException],
+      () => brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevelsWithNonExistingFeature))
+
+    val defaultMinVersionLevelsWithInvalidValue1 = Map[String, Short](

Review comment:
       ...`WithInvalidSmallValue`

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.
+   *
+   * This method sets up the FeatureZNode with enabled status. This status means the feature
+   * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode
+   * are active. This status should be written by the controller to the FeatureZNode only when the
+   * broker IBP config is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. The reason to do this is that enabling all the possible features immediately after
+   *    an upgrade could be harmful to the cluster.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, then it
+   *        will react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, it's existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) => {
+            val brokerDefaultVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (brokerDefaultVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (existingVersionRange.max() >= brokerDefaultVersionRange.min() &&
+                       brokerDefaultVersionRange.max() >= existingVersionRange.max()) {
+              // Through this change, we deprecate all version levels in the closed range:
+              // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1]
+              (featureName, new FinalizedVersionRange(brokerDefaultVersionRange.min(), existingVersionRange.max()))
+            } else {
+              // If the existing version levels fall completely outside the
+              // range of the default finalized version levels (i.e. no intersection), or, if the
+              // existing version levels are ineligible for a modification since they are
+              // incompatible with default finalized version levels, then we skip the update.
+              warn(s"Can not update minimum version level in finalized feature: $featureName,"
+                + s" since the existing $existingVersionRange is not eligible for a change"
+                + s" based on the default $brokerDefaultVersionRange.")
+              (featureName, existingVersionRange)
+            }
+          }
+        }.asJava)
+      }
+      val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures)
+      if (!newFeatureZNode.equals(existingFeatureZNode)) {
+        val newVersion = updateFeatureZNode(newFeatureZNode)
+        featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+      }
+    }
+  }
+
+  /**
+   * Disables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with disabled status. This status means the feature versioning system
+   * (KIP-584) is disabled, and, the finalized features stored in the FeatureZNode are not relevant.
+   * This status should be written by the controller to the FeatureZNode only when the broker
+   * IBP config is less than KAFKA_2_7_IV0.
+   *
+   * NOTE:
+   * 1. When this method returns, existing finalized features (if any) will be cleared from the
+   *    FeatureZNode.
+   * 2. This method, unlike enableFeatureVersioning() need not wait for the FinalizedFeatureCache
+   *    to be updated, because, such updates to the caceh (via FinalizedFeatureChangeListener)

Review comment:
       cache

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesResult.java
##########
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.common.KafkaFuture;
+
+public class UpdateFeaturesResult {

Review comment:
       As discussed offline, we need to extend the result as per feature.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -177,15 +214,33 @@ public static ApiVersionsResponse createApiVersionsResponse(
             }
         }
 
+        return new ApiVersionsResponse(
+            createApiVersionsResponseData(
+                throttleTimeMs,
+                Errors.NONE,
+                apiKeys,
+                latestSupportedFeatures,
+                finalizedFeatures,
+                finalizedFeaturesEpoch));
+    }
+
+    public static ApiVersionsResponseData createApiVersionsResponseData(

Review comment:
       Make sense, after looking further I realized that we also did some data format conversion.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java
##########
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+@InterfaceStability.Evolving
+public class DescribeFeaturesOptions extends AbstractOptions<DescribeFeaturesOptions> {

Review comment:
       nit: new line

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesOptions.java
##########
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+@InterfaceStability.Evolving
+public class UpdateFeaturesOptions extends AbstractOptions<UpdateFeaturesOptions> {
+    /**
+     * Sets the timeout in milliseconds for this operation or {@code null} if the default API
+     * timeout for the AdminClient should be used.
+     */
+    public UpdateFeaturesOptions timeoutMs(final Integer timeoutMs) {

Review comment:
       Same here, why do we need this extension?

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import java.nio.ByteBuffer;
+import org.apache.kafka.common.message.UpdateFeaturesResponseData;
+import org.apache.kafka.common.message.UpdateFeaturesRequestData;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.types.Struct;
+
+public class UpdateFeaturesRequest extends AbstractRequest {
+
+    public static class Builder extends AbstractRequest.Builder<UpdateFeaturesRequest> {
+
+        private final UpdateFeaturesRequestData data;
+
+        public Builder(UpdateFeaturesRequestData data) {
+            super(ApiKeys.UPDATE_FEATURES);
+            this.data = data;
+        }
+
+        @Override
+        public UpdateFeaturesRequest build(short version) {
+            return new UpdateFeaturesRequest(data, version);
+        }
+
+        @Override
+        public String toString() {
+            return data.toString();
+        }
+    }
+
+    public final UpdateFeaturesRequestData data;
+
+    public UpdateFeaturesRequest(UpdateFeaturesRequestData data, short version) {
+        super(ApiKeys.UPDATE_FEATURES, version);
+        this.data = data;
+    }
+
+    public UpdateFeaturesRequest(Struct struct, short version) {
+        super(ApiKeys.UPDATE_FEATURES, version);
+        this.data = new UpdateFeaturesRequestData(struct, version);
+    }
+
+    @Override
+    public AbstractResponse getErrorResponse(int throttleTimeMsIgnored, Throwable e) {
+        final ApiError apiError = ApiError.fromThrowable(e);
+        return new UpdateFeaturesResponse(
+            new UpdateFeaturesResponseData()
+                .setErrorCode(apiError.error().code())
+                .setErrorMessage(apiError.message()));
+    }
+
+    @Override
+    protected Struct toStruct() {
+        return data.toStruct(version());
+    }
+
+    public UpdateFeaturesRequestData data() {

Review comment:
       We could consider either making `data` to be private or remove this unnecessary accessor. I would prefer making it private.

##########
File path: clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
##########
@@ -3193,6 +3238,104 @@ public void testListOffsetsNonRetriableErrors() throws Exception {
         }
     }
 
+    @Test
+    public void testUpdateFeaturesDuringSuccess() throws Exception {
+        testUpdateFeaturesDuringError(Errors.NONE);
+    }
+
+    @Test
+    public void testUpdateFeaturesInvalidRequestError() throws Exception {
+        testUpdateFeaturesDuringError(Errors.INVALID_REQUEST);
+    }
+
+    @Test
+    public void testUpdateFeaturesUpdateFailedError() throws Exception {
+        testUpdateFeaturesDuringError(Errors.FEATURE_UPDATE_FAILED);
+    }
+
+    private void testUpdateFeaturesDuringError(Errors error) throws Exception {

Review comment:
       `testUpdateFeatures` should be suffice, as we sometimes are not passing in a real error.

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,178 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature
+ *    version level deprecation. This is how it works: in order to deprecate feature version levels,
+ *    in this map the default minimum version level of a feature can be set to a new value that's
+ *    higher than 1 (let's call this new_min_version_level). In doing so, the feature version levels
+ *    in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic
+ *    that applies this map to persistent finalized feature state in ZK (this mutation happens
+ *    during controller election and during finalized feature updates via the
+ *    ApiKeys.UPDATE_FINALIZED_FEATURES api). This will automatically mean clients have to stop

Review comment:
       what `clients` are we referring to here?

##########
File path: clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
##########
@@ -3193,6 +3238,104 @@ public void testListOffsetsNonRetriableErrors() throws Exception {
         }
     }
 
+    @Test
+    public void testUpdateFeaturesDuringSuccess() throws Exception {
+        testUpdateFeaturesDuringError(Errors.NONE);
+    }
+
+    @Test
+    public void testUpdateFeaturesInvalidRequestError() throws Exception {
+        testUpdateFeaturesDuringError(Errors.INVALID_REQUEST);
+    }
+
+    @Test
+    public void testUpdateFeaturesUpdateFailedError() throws Exception {
+        testUpdateFeaturesDuringError(Errors.FEATURE_UPDATE_FAILED);
+    }
+
+    private void testUpdateFeaturesDuringError(Errors error) throws Exception {
+        try (final AdminClientUnitTestEnv env = mockClientEnv()) {
+            env.kafkaClient().prepareResponse(
+                body -> body instanceof UpdateFeaturesRequest,
+                prepareUpdateFeaturesResponse(error));
+            final KafkaFuture<Void> future = env.adminClient().updateFeatures(
+                new HashSet<>(
+                    Arrays.asList(
+                        new FeatureUpdate(
+                            "test_feature_1", (short) 2, false),
+                        new FeatureUpdate(
+                            "test_feature_2", (short) 3, true))),
+                new UpdateFeaturesOptions().timeoutMs(10000)).result();
+            if (error.exception() == null) {
+                future.get();
+            } else {
+                final ExecutionException e = assertThrows(ExecutionException.class,
+                    () -> future.get());
+                assertEquals(e.getCause().getClass(), error.exception().getClass());
+            }
+        }
+    }
+
+    @Test
+    public void testUpdateFeaturesHandleNotControllerException() throws Exception {
+        try (final AdminClientUnitTestEnv env = mockClientEnv()) {
+            env.kafkaClient().prepareResponseFrom(
+                prepareUpdateFeaturesResponse(Errors.NOT_CONTROLLER),
+                env.cluster().nodeById(0));
+            env.kafkaClient().prepareResponse(MetadataResponse.prepareResponse(env.cluster().nodes(),
+                env.cluster().clusterResource().clusterId(),
+                1,
+                Collections.<MetadataResponse.TopicMetadata>emptyList()));

Review comment:
       `Collections.emptyList()` should be suffice. 

##########
File path: clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
##########
@@ -3193,6 +3238,104 @@ public void testListOffsetsNonRetriableErrors() throws Exception {
         }
     }
 
+    @Test
+    public void testUpdateFeaturesDuringSuccess() throws Exception {
+        testUpdateFeaturesDuringError(Errors.NONE);
+    }
+
+    @Test
+    public void testUpdateFeaturesInvalidRequestError() throws Exception {
+        testUpdateFeaturesDuringError(Errors.INVALID_REQUEST);
+    }
+
+    @Test
+    public void testUpdateFeaturesUpdateFailedError() throws Exception {
+        testUpdateFeaturesDuringError(Errors.FEATURE_UPDATE_FAILED);
+    }
+
+    private void testUpdateFeaturesDuringError(Errors error) throws Exception {
+        try (final AdminClientUnitTestEnv env = mockClientEnv()) {
+            env.kafkaClient().prepareResponse(
+                body -> body instanceof UpdateFeaturesRequest,
+                prepareUpdateFeaturesResponse(error));
+            final KafkaFuture<Void> future = env.adminClient().updateFeatures(
+                new HashSet<>(
+                    Arrays.asList(
+                        new FeatureUpdate(
+                            "test_feature_1", (short) 2, false),
+                        new FeatureUpdate(
+                            "test_feature_2", (short) 3, true))),
+                new UpdateFeaturesOptions().timeoutMs(10000)).result();
+            if (error.exception() == null) {
+                future.get();
+            } else {
+                final ExecutionException e = assertThrows(ExecutionException.class,
+                    () -> future.get());
+                assertEquals(e.getCause().getClass(), error.exception().getClass());
+            }
+        }
+    }
+
+    @Test
+    public void testUpdateFeaturesHandleNotControllerException() throws Exception {
+        try (final AdminClientUnitTestEnv env = mockClientEnv()) {
+            env.kafkaClient().prepareResponseFrom(
+                prepareUpdateFeaturesResponse(Errors.NOT_CONTROLLER),
+                env.cluster().nodeById(0));
+            env.kafkaClient().prepareResponse(MetadataResponse.prepareResponse(env.cluster().nodes(),
+                env.cluster().clusterResource().clusterId(),
+                1,
+                Collections.<MetadataResponse.TopicMetadata>emptyList()));
+            env.kafkaClient().prepareResponseFrom(

Review comment:
       We should have a matcher checking whether the sent request is pointing at the correct controller id.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -977,14 +1159,33 @@ class KafkaController(val config: KafkaConfig,
 
   /**
    * Send the leader information for selected partitions to selected brokers so that they can correctly respond to
-   * metadata requests
+   * metadata requests. Particularly, when feature versioning is enabled, we filter out brokers with incompatible
+   * features from receiving the metadata requests. This is because we do not want to activate incompatible brokers,
+   * as these may have harmful consequences to the cluster.
    *
    * @param brokers The brokers that the update metadata request should be sent to
    */
   private[controller] def sendUpdateMetadataRequest(brokers: Seq[Int], partitions: Set[TopicPartition]): Unit = {
     try {
+      val filteredBrokers = scala.collection.mutable.Set[Int]() ++ brokers
+      if (config.isFeatureVersioningEnabled) {
+        def hasIncompatibleFeatures(broker: Broker): Boolean = {
+          val latestFinalizedFeatures = featureCache.get
+          if (latestFinalizedFeatures.isDefined) {
+            BrokerFeatures.hasIncompatibleFeatures(broker.features, latestFinalizedFeatures.get.features)
+          } else {
+            false
+          }
+        }
+        controllerContext.liveOrShuttingDownBrokers.foreach(broker => {
+          if (filteredBrokers.contains(broker.id) && hasIncompatibleFeatures(broker)) {
+            warn(s"Ignoring UpdateMetadataRequest to broker: ${broker.id} due to incompatible features")

Review comment:
       `No UpdateMetadataRequest will be sent to broker...`

##########
File path: clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
##########
@@ -3193,6 +3238,104 @@ public void testListOffsetsNonRetriableErrors() throws Exception {
         }
     }
 
+    @Test
+    public void testUpdateFeaturesDuringSuccess() throws Exception {
+        testUpdateFeaturesDuringError(Errors.NONE);
+    }
+
+    @Test
+    public void testUpdateFeaturesInvalidRequestError() throws Exception {
+        testUpdateFeaturesDuringError(Errors.INVALID_REQUEST);
+    }
+
+    @Test
+    public void testUpdateFeaturesUpdateFailedError() throws Exception {
+        testUpdateFeaturesDuringError(Errors.FEATURE_UPDATE_FAILED);
+    }
+
+    private void testUpdateFeaturesDuringError(Errors error) throws Exception {
+        try (final AdminClientUnitTestEnv env = mockClientEnv()) {
+            env.kafkaClient().prepareResponse(
+                body -> body instanceof UpdateFeaturesRequest,
+                prepareUpdateFeaturesResponse(error));
+            final KafkaFuture<Void> future = env.adminClient().updateFeatures(
+                new HashSet<>(
+                    Arrays.asList(
+                        new FeatureUpdate(
+                            "test_feature_1", (short) 2, false),
+                        new FeatureUpdate(
+                            "test_feature_2", (short) 3, true))),
+                new UpdateFeaturesOptions().timeoutMs(10000)).result();
+            if (error.exception() == null) {
+                future.get();
+            } else {
+                final ExecutionException e = assertThrows(ExecutionException.class,
+                    () -> future.get());
+                assertEquals(e.getCause().getClass(), error.exception().getClass());
+            }
+        }
+    }
+
+    @Test
+    public void testUpdateFeaturesHandleNotControllerException() throws Exception {
+        try (final AdminClientUnitTestEnv env = mockClientEnv()) {
+            env.kafkaClient().prepareResponseFrom(
+                prepareUpdateFeaturesResponse(Errors.NOT_CONTROLLER),
+                env.cluster().nodeById(0));
+            env.kafkaClient().prepareResponse(MetadataResponse.prepareResponse(env.cluster().nodes(),
+                env.cluster().clusterResource().clusterId(),
+                1,

Review comment:
       make it a variable, as `int controllerId = 1`

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java
##########
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+import org.apache.kafka.common.message.UpdateFeaturesResponseData;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.Struct;
+
+
+/**
+ * Possible error codes:
+ *
+ *   - {@link Errors#CLUSTER_AUTHORIZATION_FAILED}
+ *   - {@link Errors#NOT_CONTROLLER}
+ *   - {@link Errors#INVALID_REQUEST}
+ *   - {@link Errors#FEATURE_UPDATE_FAILED}
+ */
+public class UpdateFeaturesResponse extends AbstractResponse {
+
+    public final UpdateFeaturesResponseData data;
+
+    public UpdateFeaturesResponse(UpdateFeaturesResponseData data) {
+        this.data = data;
+    }
+
+    public UpdateFeaturesResponse(Struct struct) {
+        final short latestVersion = (short) (UpdateFeaturesResponseData.SCHEMAS.length - 1);
+        this.data = new UpdateFeaturesResponseData(struct, latestVersion);
+    }
+
+    public UpdateFeaturesResponse(Struct struct, short version) {
+        this.data = new UpdateFeaturesResponseData(struct, version);
+    }
+
+    public Errors error() {
+        return Errors.forCode(data.errorCode());
+    }
+
+    @Override
+    public Map<Errors, Integer> errorCounts() {
+        return errorCounts(Errors.forCode(data.errorCode()));
+    }
+
+    @Override
+    protected Struct toStruct(short version) {
+        return data.toStruct(version);
+    }
+
+    @Override
+    public String toString() {
+        return data.toString();
+    }
+
+    public UpdateFeaturesResponseData data() {

Review comment:
       Same here for consistency.

##########
File path: clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
##########
@@ -3193,6 +3238,104 @@ public void testListOffsetsNonRetriableErrors() throws Exception {
         }
     }
 
+    @Test
+    public void testUpdateFeaturesDuringSuccess() throws Exception {
+        testUpdateFeaturesDuringError(Errors.NONE);
+    }
+
+    @Test
+    public void testUpdateFeaturesInvalidRequestError() throws Exception {
+        testUpdateFeaturesDuringError(Errors.INVALID_REQUEST);
+    }
+
+    @Test
+    public void testUpdateFeaturesUpdateFailedError() throws Exception {
+        testUpdateFeaturesDuringError(Errors.FEATURE_UPDATE_FAILED);
+    }
+
+    private void testUpdateFeaturesDuringError(Errors error) throws Exception {
+        try (final AdminClientUnitTestEnv env = mockClientEnv()) {
+            env.kafkaClient().prepareResponse(
+                body -> body instanceof UpdateFeaturesRequest,
+                prepareUpdateFeaturesResponse(error));
+            final KafkaFuture<Void> future = env.adminClient().updateFeatures(
+                new HashSet<>(
+                    Arrays.asList(
+                        new FeatureUpdate(
+                            "test_feature_1", (short) 2, false),
+                        new FeatureUpdate(
+                            "test_feature_2", (short) 3, true))),
+                new UpdateFeaturesOptions().timeoutMs(10000)).result();
+            if (error.exception() == null) {

Review comment:
       nit: prefer using `error == Errors.NONE`

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -68,6 +69,36 @@ public ApiVersionsResponse(Struct struct, short version) {
         this(new ApiVersionsResponseData(struct, version));
     }
 
+    public ApiVersionsResponseData data() {
+        return data;
+    }
+
+    public Features<SupportedVersionRange> supportedFeatures() {
+        final Map<String, SupportedVersionRange> features = new HashMap<>();
+
+        for (SupportedFeatureKey key : data.supportedFeatures().valuesSet()) {
+            features.put(key.name(), new SupportedVersionRange(key.minVersion(), key.maxVersion()));
+        }
+
+        return Features.supportedFeatures(features);
+    }
+
+    public Features<FinalizedVersionRange> finalizedFeatures() {
+        final Map<String, FinalizedVersionRange> features = new HashMap<>();
+
+        for (FinalizedFeatureKey key : data.finalizedFeatures().valuesSet()) {
+            features.put(

Review comment:
       Parameters could be on the same line to be consistent with L80

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.

Review comment:
       remove `in ZK`

##########
File path: clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
##########
@@ -413,6 +422,42 @@ private static DescribeGroupsResponseData prepareDescribeGroupsResponseData(Stri
                 Collections.emptySet()));
         return data;
     }
+
+    private static UpdateFeaturesResponse prepareUpdateFeaturesResponse(Errors error) {
+        final UpdateFeaturesResponseData data = new UpdateFeaturesResponseData();
+        data.setErrorCode(error.code());
+        return new UpdateFeaturesResponse(data);
+    }
+
+    private static FeatureMetadata getDefaultFeatureMetadata() {
+        return new FeatureMetadata(
+            Features.finalizedFeatures(new HashMap<String, FinalizedVersionRange>() {
+                {

Review comment:
       nit: could use `Utils.mkMap` to simplify here.

##########
File path: clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
##########
@@ -413,6 +422,42 @@ private static DescribeGroupsResponseData prepareDescribeGroupsResponseData(Stri
                 Collections.emptySet()));
         return data;
     }
+
+    private static UpdateFeaturesResponse prepareUpdateFeaturesResponse(Errors error) {
+        final UpdateFeaturesResponseData data = new UpdateFeaturesResponseData();
+        data.setErrorCode(error.code());
+        return new UpdateFeaturesResponse(data);
+    }
+
+    private static FeatureMetadata getDefaultFeatureMetadata() {

Review comment:
       `defaultFeatureMetadata` should be suffice. AK repo normally tries to avoid using `get` as func prefix.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its

Review comment:
       nit: s/it's/its

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.
+   *
+   * This method sets up the FeatureZNode with enabled status. This status means the feature
+   * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode
+   * are active. This status should be written by the controller to the FeatureZNode only when the

Review comment:
       s/This status/The enabled status 

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.
+   *
+   * This method sets up the FeatureZNode with enabled status. This status means the feature
+   * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode
+   * are active. This status should be written by the controller to the FeatureZNode only when the
+   * broker IBP config is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker

Review comment:
       same here

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,158 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * Enables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with enabled status. This status means the feature versioning system
+   * (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode are active. This
+   * status should be written by the controller to the FeatureZNode only when the broker IBP config
+   * is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    For a new Kafka cluster (i.e. it is deployed first time), we would like to start the cluster
+   *    with all the possible supported features finalized immediately. The new cluster will almost
+   *    never be started with an old IBP config that’s less than KAFKA_2_7_IV0. Assuming this is the
+   *    case, then here is how we it: the controller will start up and notice that the FeatureZNode
+   *    is absent in the new cluster, it will then create a FeatureZNode (with enabled status)
+   *    containing the entire list of default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. The reason to do this is that enabling all the possible features immediately after
+   *    an upgrade could be harmful to the cluster.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, then it
+   *        will react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, it's existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) => {
+            val updatedVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (updatedVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (existingVersionRange.max() >= updatedVersionRange.min()) {
+              // Through this change, we deprecate all version levels in the closed range:
+              // [existingVersionRange.min(), updatedVersionRange.min() - 1]
+              (featureName, new FinalizedVersionRange(updatedVersionRange.min(), existingVersionRange.max()))
+            } else {
+              // This is a special case: If the existing version levels fall completely outside the
+              // range of the default finalized version levels (i.e. no intersection), then, this
+              // case is not eligible for deprecation. This requires that the max version level be
+              // upgraded first to a value that's equal to the the default minimum version level.
+              info(s"Can not update minimum version level in finalized feature: $featureName,"
+              + s" since the existing $existingVersionRange does not intersect with the default"
+              + s" $updatedVersionRange.")
+              (featureName, existingVersionRange)
+            }
+          }
+        }.asJava)
+      }
+      val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures)
+      if (!newFeatureZNode.equals(existingFeatureZNode)) {
+        val newVersion = updateFeatureZNode(newFeatureZNode)
+        featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+      }
+    }
+  }
+
+  /**
+   * Disables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with disabled status. This status means the feature versioning system
+   * (KIP-584) is disabled, and, the finalized features stored in the FeatureZNode are not relevant.
+   * This status should be written by the controller to the FeatureZNode only when the broker
+   * IBP config is less than KAFKA_2_7_IV0.
+   *
+   * NOTE:
+   * 1. When this method returns, existing finalized features (if any) will be cleared from the
+   *    FeatureZNode.
+   * 2. This method, unlike enableFeatureVersioning() need not wait for the FinalizedFeatureCache
+   *    to be updated, because, such updates to the caceh (via FinalizedFeatureChangeListener)
+   *    are disabled when IBP config is < than KAFKA_2_7_IV0.
+   */
+  private def disableFeatureVersioning(): Unit = {

Review comment:
       My pt is that since we know the outcome (feature versioning will be disabled), we don't need to do one more lookup but just try to push the update. Anyway, I think this is a nit.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined

Review comment:
       s/ is is / is

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.
+   *
+   * This method sets up the FeatureZNode with enabled status. This status means the feature
+   * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode
+   * are active. This status should be written by the controller to the FeatureZNode only when the
+   * broker IBP config is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. The reason to do this is that enabling all the possible features immediately after
+   *    an upgrade could be harmful to the cluster.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, then it

Review comment:
       remove `then`

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.

Review comment:
       remove `and their version levels` or restructure as `the information about finalized features' version levels`

##########
File path: core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala
##########
@@ -0,0 +1,467 @@
+/**
+ * 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.server
+
+import java.util
+import java.util.Arrays
+import java.util.Collections
+import java.util.HashSet
+import java.util.Properties
+import java.util.concurrent.ExecutionException
+
+import kafka.api.KAFKA_2_7_IV0
+import kafka.utils.TestUtils
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion}
+import kafka.utils.TestUtils.waitUntilTrue
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions, UpdateFeaturesResult}
+import org.apache.kafka.common.errors.InvalidRequestException
+import org.apache.kafka.common.feature.FinalizedVersionRange
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.message.UpdateFeaturesRequestData
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{UpdateFeaturesRequest, UpdateFeaturesResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.Test
+import org.junit.Assert.{assertEquals, assertNotEquals, assertNotNull, assertTrue}
+import org.scalatest.Assertions.{assertThrows, intercept}
+
+import scala.reflect.ClassTag
+import scala.util.matching.Regex
+
+class UpdateFeaturesTest extends BaseRequestTest {
+
+  override def brokerCount = 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))))
+  }
+
+  private def defaultFinalizedFeatures(): Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2))))
+  }
+
+  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 updateSupportedFeatures(features: Features[SupportedVersionRange]): Unit = {
+    updateSupportedFeatures(features, Set[KafkaServer]() ++ servers)
+  }
+
+  private def updateDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = {
+    servers.foreach(s => {
+      s.brokerFeatures.setDefaultMinVersionLevels(newMinVersionLevels)
+    })
+  }
+
+  private def updateFeatureZNode(features: Features[FinalizedVersionRange]): Int = {
+    val server = serverForId(0).get
+    val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, features)
+    val newVersion = server.zkClient.updateFeatureZNode(newNode)
+    servers.foreach(s => {
+      s.featureCache.waitUntilEpochOrThrow(newVersion, s.config.zkConnectionTimeoutMs)
+    })
+    newVersion
+  }
+
+  private def getFeatureZNode(): FeatureZNode = {
+    val (mayBeFeatureZNodeBytes, version) = serverForId(0).get.zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(version, ZkVersion.UnknownVersion)
+    FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+  }
+
+  private def checkFeatures(client: Admin, expectedNode: FeatureZNode, expectedMetadata: FeatureMetadata): Unit = {
+    assertEquals(expectedNode, getFeatureZNode())
+    val featureMetadata = client.describeFeatures(
+      new DescribeFeaturesOptions().sendRequestToController(true)).featureMetadata().get()
+    assertEquals(expectedMetadata, featureMetadata)
+  }
+
+  private def checkException[ExceptionType <: Throwable](
+                                                          result: UpdateFeaturesResult,
+                                                          exceptionMsgPattern: Regex
+  )(implicit tag: ClassTag[ExceptionType]): Unit = {
+    val exception = intercept[ExecutionException] {
+      result.result().get()
+    }
+    assertNotNull(exception.getCause)
+    assertEquals(exception.getCause.getClass, tag.runtimeClass)
+    assertTrue(exceptionMsgPattern.findFirstIn(exception.getCause.getMessage).isDefined)
+  }
+
+  /**
+   * Tests whether an invalid feature update does not get processed on the server as expected,
+   * and raises the ExceptionType on the client side as expected.
+   *
+   * @param invalidUpdate         the invalid feature update to be sent in the
+   *                              updateFeatures request to the server
+   * @param exceptionMsgPattern   a pattern for the expected exception message
+   */
+  private def testWithInvalidFeatureUpdate[ExceptionType <: Throwable](

Review comment:
       format

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.
+   *
+   * This method sets up the FeatureZNode with enabled status. This status means the feature
+   * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode
+   * are active. This status should be written by the controller to the FeatureZNode only when the
+   * broker IBP config is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. The reason to do this is that enabling all the possible features immediately after
+   *    an upgrade could be harmful to the cluster.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, then it
+   *        will react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, it's existing minimum version level is updated to the default

Review comment:
       nit: s/it's/its

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1825,36 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def processUpdateFinalizedFeatures(newFeatures: Features[FinalizedVersionRange],
+                                             callback: UpdateFinalizedFeaturesCallback): Unit = {
+    if (isActive) {
+      val incompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.filter(broker => {
+        BrokerFeatures.hasIncompatibleFeatures(broker.features, newFeatures)
+      })
+      if (incompatibleBrokers.size > 0) {
+        callback(
+          Errors.INVALID_REQUEST,
+          Some(
+            s"Could not apply finalized feature updates because ${incompatibleBrokers.size} brokers"
+            + s" were found to have incompatible features. newFeatures: $newFeatures"
+           + s", incompatibleBrokers: $incompatibleBrokers."))
+      } else {
+        try {
+          val newVersion = zkClient.updateFeatureZNode(
+            new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures))
+          featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)

Review comment:
       Yea, I'm a bit worried about such a blocking call here as we don't have a precedence for relying on zk connect timeout (18 seconds), besides the result doesn't matter to the controller (since client will do the retry). cc @cmccabe @junrao to see if they have a different opinion on this.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.
+   *
+   * This method sets up the FeatureZNode with enabled status. This status means the feature
+   * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode
+   * are active. This status should be written by the controller to the FeatureZNode only when the
+   * broker IBP config is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. The reason to do this is that enabling all the possible features immediately after
+   *    an upgrade could be harmful to the cluster.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, then it
+   *        will react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, it's existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) => {

Review comment:
       {} could be removed.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.
+   *
+   * This method sets up the FeatureZNode with enabled status. This status means the feature
+   * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode
+   * are active. This status should be written by the controller to the FeatureZNode only when the
+   * broker IBP config is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. The reason to do this is that enabling all the possible features immediately after
+   *    an upgrade could be harmful to the cluster.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, then it
+   *        will react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, it's existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) => {
+            val brokerDefaultVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (brokerDefaultVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (existingVersionRange.max() >= brokerDefaultVersionRange.min() &&
+                       brokerDefaultVersionRange.max() >= existingVersionRange.max()) {
+              // Through this change, we deprecate all version levels in the closed range:
+              // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1]
+              (featureName, new FinalizedVersionRange(brokerDefaultVersionRange.min(), existingVersionRange.max()))
+            } else {
+              // If the existing version levels fall completely outside the
+              // range of the default finalized version levels (i.e. no intersection), or, if the
+              // existing version levels are ineligible for a modification since they are
+              // incompatible with default finalized version levels, then we skip the update.
+              warn(s"Can not update minimum version level in finalized feature: $featureName,"
+                + s" since the existing $existingVersionRange is not eligible for a change"
+                + s" based on the default $brokerDefaultVersionRange.")
+              (featureName, existingVersionRange)
+            }
+          }
+        }.asJava)
+      }
+      val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures)
+      if (!newFeatureZNode.equals(existingFeatureZNode)) {

Review comment:
       I think we need to override `equals` here.

##########
File path: core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala
##########
@@ -0,0 +1,467 @@
+/**
+ * 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.server
+
+import java.util
+import java.util.Arrays
+import java.util.Collections
+import java.util.HashSet
+import java.util.Properties
+import java.util.concurrent.ExecutionException
+
+import kafka.api.KAFKA_2_7_IV0
+import kafka.utils.TestUtils
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion}
+import kafka.utils.TestUtils.waitUntilTrue
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions, UpdateFeaturesResult}
+import org.apache.kafka.common.errors.InvalidRequestException
+import org.apache.kafka.common.feature.FinalizedVersionRange
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.message.UpdateFeaturesRequestData
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{UpdateFeaturesRequest, UpdateFeaturesResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.Test
+import org.junit.Assert.{assertEquals, assertNotEquals, assertNotNull, assertTrue}
+import org.scalatest.Assertions.{assertThrows, intercept}
+
+import scala.reflect.ClassTag
+import scala.util.matching.Regex
+
+class UpdateFeaturesTest extends BaseRequestTest {
+
+  override def brokerCount = 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))))
+  }
+
+  private def defaultFinalizedFeatures(): Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2))))
+  }
+
+  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 updateSupportedFeatures(features: Features[SupportedVersionRange]): Unit = {
+    updateSupportedFeatures(features, Set[KafkaServer]() ++ servers)
+  }
+
+  private def updateDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = {
+    servers.foreach(s => {
+      s.brokerFeatures.setDefaultMinVersionLevels(newMinVersionLevels)
+    })
+  }
+
+  private def updateFeatureZNode(features: Features[FinalizedVersionRange]): Int = {
+    val server = serverForId(0).get
+    val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, features)
+    val newVersion = server.zkClient.updateFeatureZNode(newNode)
+    servers.foreach(s => {
+      s.featureCache.waitUntilEpochOrThrow(newVersion, s.config.zkConnectionTimeoutMs)
+    })
+    newVersion
+  }
+
+  private def getFeatureZNode(): FeatureZNode = {
+    val (mayBeFeatureZNodeBytes, version) = serverForId(0).get.zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(version, ZkVersion.UnknownVersion)
+    FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+  }
+
+  private def checkFeatures(client: Admin, expectedNode: FeatureZNode, expectedMetadata: FeatureMetadata): Unit = {
+    assertEquals(expectedNode, getFeatureZNode())
+    val featureMetadata = client.describeFeatures(
+      new DescribeFeaturesOptions().sendRequestToController(true)).featureMetadata().get()
+    assertEquals(expectedMetadata, featureMetadata)
+  }
+
+  private def checkException[ExceptionType <: Throwable](

Review comment:
       Format

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.
+   *
+   * This method sets up the FeatureZNode with enabled status. This status means the feature
+   * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode
+   * are active. This status should be written by the controller to the FeatureZNode only when the
+   * broker IBP config is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. The reason to do this is that enabling all the possible features immediately after
+   *    an upgrade could be harmful to the cluster.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, then it
+   *        will react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, it's existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) => {
+            val brokerDefaultVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (brokerDefaultVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (existingVersionRange.max() >= brokerDefaultVersionRange.min() &&

Review comment:
       nit: would be easier to read if we always compare `existingVersionRange` towards `brokerDefaultVersionRange` instead of flipping in this statement.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1848,36 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def processUpdateFeatures(newFeatures: Features[FinalizedVersionRange],
+                                             callback: UpdateFeaturesCallback): Unit = {

Review comment:
       format

##########
File path: core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala
##########
@@ -0,0 +1,83 @@
+/**
+ * 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.server
+
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.junit.Assert.{assertEquals, assertThrows, assertTrue}
+import org.junit.Test
+
+import scala.jdk.CollectionConverters._
+
+class BrokerFeaturesTest {
+  @Test
+  def testEmpty(): Unit = {
+    assertTrue(BrokerFeatures.createDefault().supportedFeatures.empty)
+  }
+
+  @Test
+  def testIncompatibleFeatures(): Unit = {
+    val brokerFeatures = BrokerFeatures.createDefault()
+    val supportedFeatures = Map[String, SupportedVersionRange](
+      "test_feature_1" -> new SupportedVersionRange(1, 4),
+      "test_feature_2" -> new SupportedVersionRange(1, 3))
+    brokerFeatures.setSupportedFeatures(Features.supportedFeatures(supportedFeatures.asJava))
+
+    val compatibleFeatures = Map[String, FinalizedVersionRange](
+      "test_feature_1" -> new FinalizedVersionRange(2, 3))
+    val inCompatibleFeatures = Map[String, FinalizedVersionRange](
+      "test_feature_2" -> new FinalizedVersionRange(1, 4),
+      "test_feature_3" -> new FinalizedVersionRange(3, 4))
+    val features = compatibleFeatures++inCompatibleFeatures
+    val finalizedFeatures = Features.finalizedFeatures(features.asJava)
+
+    assertEquals(
+      Features.finalizedFeatures(inCompatibleFeatures.asJava),
+      brokerFeatures.incompatibleFeatures(finalizedFeatures))
+  }
+
+  @Test
+  def testFeatureVersionAssertions(): Unit = {
+    val brokerFeatures = BrokerFeatures.createDefault()
+    val supportedFeatures = Features.supportedFeatures(Map[String, SupportedVersionRange](
+      "test_feature_1" -> new SupportedVersionRange(1, 4),
+      "test_feature_2" -> new SupportedVersionRange(1, 3)).asJava)
+    brokerFeatures.setSupportedFeatures(supportedFeatures)
+
+    val defaultMinVersionLevelsWithNonExistingFeature = Map[String, Short](
+      "test_feature_1" -> 2,
+      "test_feature_2" -> 2,
+      "test_feature_non_existing" -> 5)
+    assertThrows(
+      classOf[IllegalArgumentException],
+      () => brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevelsWithNonExistingFeature))
+
+    val defaultMinVersionLevelsWithInvalidValue1 = Map[String, Short](
+      "test_feature_1" -> 2,
+      "test_feature_2" -> (supportedFeatures.get("test_feature_2").min() - 1).asInstanceOf[Short])
+    assertThrows(
+      classOf[IllegalArgumentException],
+      () => brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevelsWithInvalidValue1))
+
+    val defaultMinVersionLevelsWithInvalidValue2 = Map[String, Short](

Review comment:
       ...`WithInvalidLargeValue`

##########
File path: core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala
##########
@@ -0,0 +1,467 @@
+/**
+ * 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.server
+
+import java.util
+import java.util.Arrays
+import java.util.Collections
+import java.util.HashSet
+import java.util.Properties
+import java.util.concurrent.ExecutionException
+
+import kafka.api.KAFKA_2_7_IV0
+import kafka.utils.TestUtils
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion}
+import kafka.utils.TestUtils.waitUntilTrue
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions, UpdateFeaturesResult}
+import org.apache.kafka.common.errors.InvalidRequestException
+import org.apache.kafka.common.feature.FinalizedVersionRange
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.message.UpdateFeaturesRequestData
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{UpdateFeaturesRequest, UpdateFeaturesResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.Test
+import org.junit.Assert.{assertEquals, assertNotEquals, assertNotNull, assertTrue}
+import org.scalatest.Assertions.{assertThrows, intercept}
+
+import scala.reflect.ClassTag
+import scala.util.matching.Regex
+
+class UpdateFeaturesTest extends BaseRequestTest {
+
+  override def brokerCount = 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))))
+  }
+
+  private def defaultFinalizedFeatures(): Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2))))
+  }
+
+  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 updateSupportedFeatures(features: Features[SupportedVersionRange]): Unit = {
+    updateSupportedFeatures(features, Set[KafkaServer]() ++ servers)
+  }
+
+  private def updateDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = {
+    servers.foreach(s => {
+      s.brokerFeatures.setDefaultMinVersionLevels(newMinVersionLevels)
+    })
+  }
+
+  private def updateFeatureZNode(features: Features[FinalizedVersionRange]): Int = {
+    val server = serverForId(0).get
+    val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, features)
+    val newVersion = server.zkClient.updateFeatureZNode(newNode)
+    servers.foreach(s => {
+      s.featureCache.waitUntilEpochOrThrow(newVersion, s.config.zkConnectionTimeoutMs)
+    })
+    newVersion
+  }
+
+  private def getFeatureZNode(): FeatureZNode = {
+    val (mayBeFeatureZNodeBytes, version) = serverForId(0).get.zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(version, ZkVersion.UnknownVersion)
+    FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+  }
+
+  private def checkFeatures(client: Admin, expectedNode: FeatureZNode, expectedMetadata: FeatureMetadata): Unit = {
+    assertEquals(expectedNode, getFeatureZNode())
+    val featureMetadata = client.describeFeatures(
+      new DescribeFeaturesOptions().sendRequestToController(true)).featureMetadata().get()
+    assertEquals(expectedMetadata, featureMetadata)
+  }
+
+  private def checkException[ExceptionType <: Throwable](
+                                                          result: UpdateFeaturesResult,
+                                                          exceptionMsgPattern: Regex
+  )(implicit tag: ClassTag[ExceptionType]): Unit = {
+    val exception = intercept[ExecutionException] {
+      result.result().get()
+    }
+    assertNotNull(exception.getCause)
+    assertEquals(exception.getCause.getClass, tag.runtimeClass)
+    assertTrue(exceptionMsgPattern.findFirstIn(exception.getCause.getMessage).isDefined)
+  }
+
+  /**
+   * Tests whether an invalid feature update does not get processed on the server as expected,
+   * and raises the ExceptionType on the client side as expected.
+   *
+   * @param invalidUpdate         the invalid feature update to be sent in the
+   *                              updateFeatures request to the server
+   * @param exceptionMsgPattern   a pattern for the expected exception message
+   */
+  private def testWithInvalidFeatureUpdate[ExceptionType <: Throwable](
+                                                                        invalidUpdate: FeatureUpdate,
+                                                                        exceptionMsgPattern: Regex
+  )(implicit tag: ClassTag[ExceptionType]): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeatures(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val adminClient = createAdminClient()
+    val nodeBefore = getFeatureZNode()
+
+    val result = adminClient.updateFeatures(
+      new HashSet[FeatureUpdate](Collections.singletonList(invalidUpdate)), new UpdateFeaturesOptions())
+
+    checkException[ExceptionType](result, exceptionMsgPattern)
+    checkFeatures(
+      adminClient,
+      nodeBefore,
+      new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures()))
+  }
+
+  @Test
+  def testShouldFailRequestIfNotController(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeatures(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val nodeBefore = getFeatureZNode()
+    val requestData = FeatureUpdate.createRequest(
+      new util.HashSet[FeatureUpdate](
+        Collections.singletonList(new FeatureUpdate("feature_1",
+          defaultSupportedFeatures().get("feature_1").max(),
+        false))))
+
+    val response = connectAndReceive[UpdateFeaturesResponse](
+      new UpdateFeaturesRequest.Builder(requestData).build(), notControllerSocketServer)
+
+    assertEquals(Errors.NOT_CONTROLLER, response.error())
+    checkFeatures(
+      createAdminClient(),
+      nodeBefore,
+      new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures()))
+  }
+
+  @Test
+  def testShouldFailRequestForInvalidFeatureName(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](new FeatureUpdate(
+      "",
+      defaultSupportedFeatures().get("feature_1").max(),
+      false),
+      ".*empty feature name.*".r)
+  }
+
+  @Test
+  def testShouldFailRequestWhenDowngradeFlagIsNotSetDuringDowngrade(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](new FeatureUpdate(
+      "feature_1",
+      (defaultFinalizedFeatures().get("feature_1").max() - 1).asInstanceOf[Short],
+      false),
+      ".*Can not downgrade finalized feature: 'feature_1'.*allowDowngrade.*".r)
+  }
+
+  @Test
+  def testShouldFailRequestWhenDowngradeToHigherVersionLevelIsAttempted(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      new FeatureUpdate(
+        "feature_1",
+        defaultSupportedFeatures().get("feature_1").max(),
+        true),
+      ".*finalized feature: 'feature_1'.*allowDowngrade.* provided maxVersionLevel:3.*existing maxVersionLevel:2.*".r)
+  }
+
+  @Test
+  def testShouldFailRequestInClientWhenDowngradeFlagIsNotSetDuringDeletion(): Unit = {
+    assertThrows[IllegalArgumentException] {
+      new FeatureUpdate("feature_1", 0, false)
+    }
+  }
+
+  @Test
+  def testShouldFailRequestInServerWhenDowngradeFlagIsNotSetDuringDeletion(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeatures(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val adminClient = createAdminClient()
+    val nodeBefore = getFeatureZNode()
+
+    val featureUpdates
+      = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection();
+    val featureUpdate = new UpdateFeaturesRequestData.FeatureUpdateKey();
+    featureUpdate.setName("feature_1")
+    featureUpdate.setMaxVersionLevel(0)
+    featureUpdate.setAllowDowngrade(false)
+    featureUpdates.add(featureUpdate);
+    val requestData = new UpdateFeaturesRequestData()
+    requestData.setFeatureUpdates(featureUpdates);
+
+    val response = connectAndReceive[UpdateFeaturesResponse](
+      new UpdateFeaturesRequest.Builder(requestData).build(), controllerSocketServer)
+
+    assertEquals(Errors.INVALID_REQUEST, response.error)
+    val exceptionMsgPattern = ".*Can not delete feature: 'feature_1'.*allowDowngrade.*".r
+    assertTrue(exceptionMsgPattern.findFirstIn(response.data.errorMessage).isDefined)
+    checkFeatures(
+      adminClient,
+      nodeBefore,
+      new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures()))
+  }
+
+  @Test
+  def testShouldFailRequestDuringDeletionOfNonExistingFeature(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      new FeatureUpdate("feature_non_existing", 0, true),
+      ".*Can not delete non-existing finalized feature: 'feature_non_existing'.*".r)
+  }
+
+  @Test
+  def testShouldFailRequestWhenUpgradingToSameVersionLevel(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      new FeatureUpdate(
+        "feature_1", defaultFinalizedFeatures().get("feature_1").max(), false),
+      ".*Can not upgrade a finalized feature: 'feature_1'.*to the same value.*".r)
+  }
+
+  @Test
+  def testShouldFailRequestWhenDowngradingBelowMinVersionLevel(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeatures(defaultSupportedFeatures())
+    val minVersionLevel = 2.asInstanceOf[Short]
+    updateDefaultMinVersionLevels(Map[String, Short]("feature_1" -> minVersionLevel))
+    val initialFinalizedFeatures = Features.finalizedFeatures(
+      Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(minVersionLevel, 2))))
+    val versionBefore = updateFeatureZNode(initialFinalizedFeatures)
+
+    val update = new FeatureUpdate(
+      "feature_1", (minVersionLevel - 1).asInstanceOf[Short], true)
+    val adminClient = createAdminClient()
+    val nodeBefore = getFeatureZNode()
+
+    val result = adminClient.updateFeatures(
+      new HashSet[FeatureUpdate](Collections.singletonList(update)), new UpdateFeaturesOptions())
+
+    checkException[InvalidRequestException](
+      result, ".*Can not downgrade finalized feature: 'feature_1' to maxVersionLevel:1.*existing minVersionLevel:2.*".r)
+    checkFeatures(
+      adminClient,
+      nodeBefore,
+      new FeatureMetadata(initialFinalizedFeatures, versionBefore, defaultSupportedFeatures()))
+  }
+
+  @Test
+  def testShouldFailRequestDuringBrokerMaxVersionLevelIncompatibility(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    val controller = servers.filter { server => server.kafkaController.isActive}.head
+    val nonControllerServers = servers.filter { server => !server.kafkaController.isActive}
+    val unsupportedBrokers = Set[KafkaServer](nonControllerServers(0))

Review comment:
       nit: replace with `nonControllerServers.head`

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.
+   *
+   * This method sets up the FeatureZNode with enabled status. This status means the feature
+   * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode
+   * are active. This status should be written by the controller to the FeatureZNode only when the
+   * broker IBP config is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning

Review comment:
       We don't need to capitalize `Broker` here

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,178 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature
+ *    version level deprecation. This is how it works: in order to deprecate feature version levels,
+ *    in this map the default minimum version level of a feature can be set to a new value that's
+ *    higher than 1 (let's call this new_min_version_level). In doing so, the feature version levels
+ *    in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic
+ *    that applies this map to persistent finalized feature state in ZK (this mutation happens
+ *    during controller election and during finalized feature updates via the
+ *    ApiKeys.UPDATE_FINALIZED_FEATURES api). This will automatically mean clients have to stop
+ *    using the finalized min version levels that have been deprecated.
+ *
+ * This class also provides APIs to check for incompatibilities between the features supported by
+ * the Broker and finalized features. The class is generally immutable. It provides few APIs to
+ * mutate state only for the purpose of testing.
+ */
+class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange],
+                              @volatile var defaultFeatureMinVersionLevels: Map[String, Short]) {
+  require(BrokerFeatures.areFeatureMinVersionLevelsCompatible(
+    supportedFeatures, defaultFeatureMinVersionLevels))
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(newFeatures, defaultFeatureMinVersionLevels))
+    supportedFeatures = newFeatures
+  }
+
+  /**
+   * Returns the default minimum version level for a specific feature.
+   *
+   * @param feature   the name of the feature
+   *
+   * @return          the default minimum version level for the feature if its defined.
+   *                  otherwise, returns 1.
+   */
+  def defaultMinVersionLevel(feature: String): Short = {
+    defaultFeatureMinVersionLevels.getOrElse(feature, 1)
+  }
+
+  // For testing only.
+  def setDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(supportedFeatures, newMinVersionLevels))
+    defaultFeatureMinVersionLevels = newMinVersionLevels
+  }
+
+  /**
+   * Returns the default finalized features that a new Kafka cluster with IBP config >= KAFKA_2_7_IV0
+   * needs to be bootstrapped with.
+   */
+  def getDefaultFinalizedFeatures: Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(
+      supportedFeatures.features.asScala.map {
+        case(name, versionRange) => (
+          name, new FinalizedVersionRange(defaultMinVersionLevel(name), versionRange.max))
+      }.asJava)
+  }
+
+  /**
+   * Returns the set of feature names found to be 'incompatible'.

Review comment:
       Is it necessary to quote `incompatible`?

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.
+   *
+   * This method sets up the FeatureZNode with enabled status. This status means the feature
+   * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode
+   * are active. This status should be written by the controller to the FeatureZNode only when the
+   * broker IBP config is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. The reason to do this is that enabling all the possible features immediately after

Review comment:
       s/`The reason to do this is that...`/`This process ensures we do not enable all the possible features immediately after an upgrade, which could be harmful to the application.`

##########
File path: core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala
##########
@@ -0,0 +1,467 @@
+/**
+ * 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.server
+
+import java.util
+import java.util.Arrays
+import java.util.Collections
+import java.util.HashSet
+import java.util.Properties
+import java.util.concurrent.ExecutionException
+
+import kafka.api.KAFKA_2_7_IV0
+import kafka.utils.TestUtils
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion}
+import kafka.utils.TestUtils.waitUntilTrue
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions, UpdateFeaturesResult}
+import org.apache.kafka.common.errors.InvalidRequestException
+import org.apache.kafka.common.feature.FinalizedVersionRange
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.message.UpdateFeaturesRequestData
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{UpdateFeaturesRequest, UpdateFeaturesResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.Test
+import org.junit.Assert.{assertEquals, assertNotEquals, assertNotNull, assertTrue}
+import org.scalatest.Assertions.{assertThrows, intercept}
+
+import scala.reflect.ClassTag
+import scala.util.matching.Regex
+
+class UpdateFeaturesTest extends BaseRequestTest {
+
+  override def brokerCount = 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))))
+  }
+
+  private def defaultFinalizedFeatures(): Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2))))
+  }
+
+  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 updateSupportedFeatures(features: Features[SupportedVersionRange]): Unit = {
+    updateSupportedFeatures(features, Set[KafkaServer]() ++ servers)
+  }
+
+  private def updateDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = {
+    servers.foreach(s => {
+      s.brokerFeatures.setDefaultMinVersionLevels(newMinVersionLevels)
+    })
+  }
+
+  private def updateFeatureZNode(features: Features[FinalizedVersionRange]): Int = {
+    val server = serverForId(0).get
+    val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, features)
+    val newVersion = server.zkClient.updateFeatureZNode(newNode)
+    servers.foreach(s => {
+      s.featureCache.waitUntilEpochOrThrow(newVersion, s.config.zkConnectionTimeoutMs)
+    })
+    newVersion
+  }
+
+  private def getFeatureZNode(): FeatureZNode = {
+    val (mayBeFeatureZNodeBytes, version) = serverForId(0).get.zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(version, ZkVersion.UnknownVersion)
+    FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+  }
+
+  private def checkFeatures(client: Admin, expectedNode: FeatureZNode, expectedMetadata: FeatureMetadata): Unit = {
+    assertEquals(expectedNode, getFeatureZNode())
+    val featureMetadata = client.describeFeatures(
+      new DescribeFeaturesOptions().sendRequestToController(true)).featureMetadata().get()
+    assertEquals(expectedMetadata, featureMetadata)
+  }
+
+  private def checkException[ExceptionType <: Throwable](
+                                                          result: UpdateFeaturesResult,
+                                                          exceptionMsgPattern: Regex
+  )(implicit tag: ClassTag[ExceptionType]): Unit = {
+    val exception = intercept[ExecutionException] {
+      result.result().get()
+    }
+    assertNotNull(exception.getCause)
+    assertEquals(exception.getCause.getClass, tag.runtimeClass)
+    assertTrue(exceptionMsgPattern.findFirstIn(exception.getCause.getMessage).isDefined)
+  }
+
+  /**
+   * Tests whether an invalid feature update does not get processed on the server as expected,
+   * and raises the ExceptionType on the client side as expected.
+   *
+   * @param invalidUpdate         the invalid feature update to be sent in the
+   *                              updateFeatures request to the server
+   * @param exceptionMsgPattern   a pattern for the expected exception message
+   */
+  private def testWithInvalidFeatureUpdate[ExceptionType <: Throwable](
+                                                                        invalidUpdate: FeatureUpdate,
+                                                                        exceptionMsgPattern: Regex
+  )(implicit tag: ClassTag[ExceptionType]): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeatures(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val adminClient = createAdminClient()
+    val nodeBefore = getFeatureZNode()
+
+    val result = adminClient.updateFeatures(
+      new HashSet[FeatureUpdate](Collections.singletonList(invalidUpdate)), new UpdateFeaturesOptions())
+
+    checkException[ExceptionType](result, exceptionMsgPattern)
+    checkFeatures(
+      adminClient,
+      nodeBefore,
+      new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures()))
+  }
+
+  @Test
+  def testShouldFailRequestIfNotController(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeatures(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val nodeBefore = getFeatureZNode()
+    val requestData = FeatureUpdate.createRequest(
+      new util.HashSet[FeatureUpdate](
+        Collections.singletonList(new FeatureUpdate("feature_1",
+          defaultSupportedFeatures().get("feature_1").max(),
+        false))))
+
+    val response = connectAndReceive[UpdateFeaturesResponse](
+      new UpdateFeaturesRequest.Builder(requestData).build(), notControllerSocketServer)
+
+    assertEquals(Errors.NOT_CONTROLLER, response.error())
+    checkFeatures(
+      createAdminClient(),
+      nodeBefore,
+      new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures()))
+  }
+
+  @Test
+  def testShouldFailRequestForInvalidFeatureName(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](new FeatureUpdate(
+      "",
+      defaultSupportedFeatures().get("feature_1").max(),
+      false),
+      ".*empty feature name.*".r)
+  }
+
+  @Test
+  def testShouldFailRequestWhenDowngradeFlagIsNotSetDuringDowngrade(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](new FeatureUpdate(
+      "feature_1",
+      (defaultFinalizedFeatures().get("feature_1").max() - 1).asInstanceOf[Short],
+      false),
+      ".*Can not downgrade finalized feature: 'feature_1'.*allowDowngrade.*".r)
+  }
+
+  @Test
+  def testShouldFailRequestWhenDowngradeToHigherVersionLevelIsAttempted(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      new FeatureUpdate(
+        "feature_1",
+        defaultSupportedFeatures().get("feature_1").max(),
+        true),
+      ".*finalized feature: 'feature_1'.*allowDowngrade.* provided maxVersionLevel:3.*existing maxVersionLevel:2.*".r)
+  }
+
+  @Test
+  def testShouldFailRequestInClientWhenDowngradeFlagIsNotSetDuringDeletion(): Unit = {
+    assertThrows[IllegalArgumentException] {
+      new FeatureUpdate("feature_1", 0, false)
+    }
+  }
+
+  @Test
+  def testShouldFailRequestInServerWhenDowngradeFlagIsNotSetDuringDeletion(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeatures(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val adminClient = createAdminClient()
+    val nodeBefore = getFeatureZNode()
+
+    val featureUpdates
+      = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection();
+    val featureUpdate = new UpdateFeaturesRequestData.FeatureUpdateKey();

Review comment:
       We could refactor out a helper in `UpdateFeaturesRequest` to create `FeatureUpdateKey`

##########
File path: core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala
##########
@@ -0,0 +1,83 @@
+/**
+ * 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.server
+
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.junit.Assert.{assertEquals, assertThrows, assertTrue}
+import org.junit.Test
+
+import scala.jdk.CollectionConverters._
+
+class BrokerFeaturesTest {

Review comment:
       nit: new line

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,178 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature
+ *    version level deprecation. This is how it works: in order to deprecate feature version levels,
+ *    in this map the default minimum version level of a feature can be set to a new value that's
+ *    higher than 1 (let's call this new_min_version_level). In doing so, the feature version levels
+ *    in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic
+ *    that applies this map to persistent finalized feature state in ZK (this mutation happens
+ *    during controller election and during finalized feature updates via the
+ *    ApiKeys.UPDATE_FINALIZED_FEATURES api). This will automatically mean clients have to stop
+ *    using the finalized min version levels that have been deprecated.
+ *
+ * This class also provides APIs to check for incompatibilities between the features supported by
+ * the Broker and finalized features. The class is generally immutable. It provides few APIs to

Review comment:
       `The class is immutable in production`

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1395,7 +1596,7 @@ class KafkaController(val config: KafkaConfig,
     if (newMetadataOpt.nonEmpty && oldMetadataOpt.nonEmpty) {
       val oldMetadata = oldMetadataOpt.get
       val newMetadata = newMetadataOpt.get
-      if (newMetadata.endPoints != oldMetadata.endPoints) {
+      if (newMetadata.endPoints != oldMetadata.endPoints || !oldMetadata.features.equals(newMetadata.features)) {

Review comment:
       does `features` guarantee to be non-null?

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -2945,6 +2948,130 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFeatures(request: RequestChannel.Request): Unit = {
+    val updateFeaturesRequest = request.body[UpdateFeaturesRequest]
+    def sendResponseCallback(error: Errors, msgOverride: Option[String]): Unit = {
+      val data = new UpdateFeaturesResponseData().setErrorCode(error.code())
+      msgOverride.map(msg => data.setErrorMessage(msg))
+      sendResponseExemptThrottle(request, new UpdateFeaturesResponse(data))
+    }
+
+    if (!authorize(request.context, ALTER, CLUSTER, CLUSTER_NAME)) {
+      sendResponseCallback(Errors.CLUSTER_AUTHORIZATION_FAILED, Option.empty)
+    } else if (!controller.isActive) {
+      sendResponseCallback(Errors.NOT_CONTROLLER, Option.empty)
+    } else if (!config.isFeatureVersioningEnabled) {
+      sendResponseCallback(Errors.INVALID_REQUEST, Some("Feature versioning system is disabled."))
+    } else if (updateFeaturesRequest.data.featureUpdates.isEmpty) {
+      sendResponseCallback(Errors.INVALID_REQUEST, Some("Can not provide empty FinalizedFeatureUpdates in the request."))
+    } else {
+      val targetFeaturesOrError = getTargetFinalizedFeaturesOrError(updateFeaturesRequest)
+      targetFeaturesOrError match {
+        case Left(targetFeatures) =>
+          controller.updateFeatures(targetFeatures, sendResponseCallback)
+        case Right(error) =>
+          sendResponseCallback(error.error, Some(error.message))
+      }
+    }
+  }
+
+  /**
+   * Validates the provided UpdateFinalizedFeaturesRequest, checking for various error cases.
+   * If the validation is successful, returns the target finalized features constructed from the
+   * request.
+   *
+   * @param request   the request to be validated
+   *
+   * @return          - the target finalized features, if request validation is successful
+   *                  - an ApiError if request validation fails
+   */
+  private def getTargetFinalizedFeaturesOrError(request: UpdateFeaturesRequest): Either[Features[FinalizedVersionRange], ApiError] = {
+    val updates = request.data.featureUpdates

Review comment:
       Could be initialized closer to L3005

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -85,25 +85,25 @@ class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
       //                                           a case.
       if (version == ZkVersion.UnknownVersion) {
         info(s"Feature ZK node at path: $featureZkNodePath does not exist")
-        FinalizedFeatureCache.clear()
+        featureCache.clear()
       } else {
         var maybeFeatureZNode: Option[FeatureZNode] = Option.empty
         try {
           maybeFeatureZNode = Some(FeatureZNode.decode(mayBeFeatureZNodeBytes.get))
         } catch {
           case e: IllegalArgumentException => {
             error(s"Unable to deserialize feature ZK node at path: $featureZkNodePath", e)
-            FinalizedFeatureCache.clear()
+            featureCache.clear()
           }
         }
-        maybeFeatureZNode.map(featureZNode => {
+        maybeFeatureZNode.foreach(featureZNode => {
           featureZNode.status match {
             case FeatureZNodeStatus.Disabled => {
               info(s"Feature ZK node at path: $featureZkNodePath is in disabled status.")
-              FinalizedFeatureCache.clear()
+              featureCache.clear()

Review comment:
       Could you explain a bit why we no longer use singletons for feature cache?

##########
File path: core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala
##########
@@ -598,6 +608,21 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness {
     }, "Broker fail to initialize after restart")
   }
 
+  private def testControllerFeatureZNodeSetup(interBrokerProtocolVersion: ApiVersion): Unit = {
+    servers = makeServers(1, interBrokerProtocolVersion = Some(interBrokerProtocolVersion))
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(version, ZkVersion.UnknownVersion)

Review comment:
       Could we assert the expected version here?

##########
File path: core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala
##########
@@ -78,25 +76,37 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness {
   /**
    * Tests that the listener can be initialized, and that it can listen to ZK notifications
    * successfully from an "Enabled" FeatureZNode (the ZK data has no feature incompatibilities).
+   * Particularly the test checks if multiple notifications can be processed in ZK
+   * (i.e. whether the FeatureZNode watch can be re-established).
    */
   @Test
   def testInitSuccessAndNotificationSuccess(): Unit = {
-    createSupportedFeatures()
     val initialFinalizedFeatures = createFinalizedFeatures()
-    val listener = createListener(Some(initialFinalizedFeatures))
+    val brokerFeatures = createBrokerFeatures()
+    val cache = new FinalizedFeatureCache(brokerFeatures)
+    val listener = createListener(cache, Some(initialFinalizedFeatures))
 
-    val updatedFinalizedFeaturesMap = Map[String, FinalizedVersionRange](
-      "feature_1" -> new FinalizedVersionRange(2, 4))
-    val updatedFinalizedFeatures = Features.finalizedFeatures(updatedFinalizedFeaturesMap.asJava)
-    zkClient.updateFeatureZNode(FeatureZNode(FeatureZNodeStatus.Enabled, updatedFinalizedFeatures))
-    val (mayBeFeatureZNodeNewBytes, updatedVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
-    assertNotEquals(updatedVersion, ZkVersion.UnknownVersion)
-    assertFalse(mayBeFeatureZNodeNewBytes.isEmpty)
-    assertTrue(updatedVersion > initialFinalizedFeatures.epoch)
-    TestUtils.waitUntilTrue(() => {
-      FinalizedFeatureCache.get.get.equals(FinalizedFeaturesAndEpoch(updatedFinalizedFeatures, updatedVersion))
-    }, "Timed out waiting for FinalizedFeatureCache to be updated with new features")
-    assertTrue(listener.isListenerInitiated)
+    def updateAndCheckCache(finalizedFeatures: Features[FinalizedVersionRange]): Unit = {
+      zkClient.updateFeatureZNode(FeatureZNode(FeatureZNodeStatus.Enabled, finalizedFeatures))
+      val (mayBeFeatureZNodeNewBytes, updatedVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+      assertNotEquals(updatedVersion, ZkVersion.UnknownVersion)
+      assertFalse(mayBeFeatureZNodeNewBytes.isEmpty)
+      assertTrue(updatedVersion > initialFinalizedFeatures.epoch)
+
+      cache.waitUntilEpochOrThrow(updatedVersion, JTestUtils.DEFAULT_MAX_WAIT_MS)
+      assertEquals(FinalizedFeaturesAndEpoch(finalizedFeatures, updatedVersion), cache.get.get)
+      assertTrue(listener.isListenerInitiated)
+    }
+
+    updateAndCheckCache(
+      Features.finalizedFeatures(
+        Map[String, FinalizedVersionRange](
+      "feature_1" -> new FinalizedVersionRange(2, 4)).asJava))
+    updateAndCheckCache(

Review comment:
       What's the purpose of this second test?

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -2945,6 +2948,130 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFeatures(request: RequestChannel.Request): Unit = {
+    val updateFeaturesRequest = request.body[UpdateFeaturesRequest]
+    def sendResponseCallback(error: Errors, msgOverride: Option[String]): Unit = {
+      val data = new UpdateFeaturesResponseData().setErrorCode(error.code())
+      msgOverride.map(msg => data.setErrorMessage(msg))
+      sendResponseExemptThrottle(request, new UpdateFeaturesResponse(data))
+    }
+
+    if (!authorize(request.context, ALTER, CLUSTER, CLUSTER_NAME)) {
+      sendResponseCallback(Errors.CLUSTER_AUTHORIZATION_FAILED, Option.empty)
+    } else if (!controller.isActive) {
+      sendResponseCallback(Errors.NOT_CONTROLLER, Option.empty)
+    } else if (!config.isFeatureVersioningEnabled) {
+      sendResponseCallback(Errors.INVALID_REQUEST, Some("Feature versioning system is disabled."))
+    } else if (updateFeaturesRequest.data.featureUpdates.isEmpty) {
+      sendResponseCallback(Errors.INVALID_REQUEST, Some("Can not provide empty FinalizedFeatureUpdates in the request."))
+    } else {
+      val targetFeaturesOrError = getTargetFinalizedFeaturesOrError(updateFeaturesRequest)
+      targetFeaturesOrError match {
+        case Left(targetFeatures) =>
+          controller.updateFeatures(targetFeatures, sendResponseCallback)
+        case Right(error) =>
+          sendResponseCallback(error.error, Some(error.message))
+      }
+    }
+  }
+
+  /**
+   * Validates the provided UpdateFinalizedFeaturesRequest, checking for various error cases.
+   * If the validation is successful, returns the target finalized features constructed from the
+   * request.
+   *
+   * @param request   the request to be validated
+   *
+   * @return          - the target finalized features, if request validation is successful
+   *                  - an ApiError if request validation fails
+   */
+  private def getTargetFinalizedFeaturesOrError(request: UpdateFeaturesRequest): Either[Features[FinalizedVersionRange], ApiError] = {
+    val updates = request.data.featureUpdates
+    val newFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]()
+
+    def addFeature(update: UpdateFeaturesRequestData.FeatureUpdateKey): Unit = {
+      // NOTE: Below we set the finalized min version level to be the default minimum version
+      // level. If the finalized feature already exists, then, this can cause deprecation of all
+      // version levels in the closed range:
+      // [existingVersionRange.min(), defaultMinVersionLevel - 1].
+      val defaultMinVersionLevel = brokerFeatures.defaultMinVersionLevel(update.name)
+      newFeatures += (
+        update.name -> new FinalizedVersionRange(
+          defaultMinVersionLevel,
+          update.maxVersionLevel))
+    }
+
+    val latestFeatures = featureCache.get
+    updates.asScala.iterator.map(
+      update => {
+        if (update.name.isEmpty) {
+          // Rule #1) Check that the feature name is not empty.
+          Some(new ApiError(Errors.INVALID_REQUEST,
+                   "Can not contain empty feature name in the request."))
+        } else {
+          val cacheEntry = latestFeatures.map(lf => lf.features.get(update.name)).orNull
+
+          // We handle deletion requests separately from non-deletion requests.
+          if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+            if (!update.allowDowngrade) {
+              // Rule #2) Disallow deletion of a finalized feature without allowDowngrade flag set.
+              Some(new ApiError(Errors.INVALID_REQUEST,
+                                s"Can not delete feature: '${update.name}' without setting the" +
+                                " allowDowngrade flag to true in the request."))
+            } else if (cacheEntry == null) {
+              // Rule #3) Disallow deletion of a non-existing finalized feature.
+              Some(new ApiError(Errors.INVALID_REQUEST,
+                       s"Can not delete non-existing finalized feature: '${update.name}'"))
+            }
+          } else {
+            if (cacheEntry == null) {

Review comment:
       This case seems not to be tested yet.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -2945,6 +2948,130 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFeatures(request: RequestChannel.Request): Unit = {

Review comment:
       Could we add some unit tests in `KafkaApisTest.scala`, once the refactoring is finished?




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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


   @chia7712 No worries, thanks for the suggestions! I have opened a separate PR addressing your comments. Would you be able to please review it? https://github.com/apache/kafka/pull/9393


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1656,6 +1910,204 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns an ApiError object containing Errors.INVALID_REQUEST.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+
+    val supportedVersionRange = brokerFeatures.supportedFeatures.get(update.feature)
+    if (supportedVersionRange == null) {
+      Right(new ApiError(Errors.INVALID_REQUEST,
+                         "Could not apply finalized feature update because the provided feature" +
+                         " is not supported."))
+    } else {
+      var newVersionRange: FinalizedVersionRange = null
+      try {
+        newVersionRange = new FinalizedVersionRange(supportedVersionRange.firstActiveVersion, update.maxVersionLevel)
+      } catch {
+        case _: IllegalArgumentException => {
+          // This exception means the provided maxVersionLevel is invalid. It is handled below
+          // outside of this catch clause.
+        }
+      }
+      if (newVersionRange == null) {
+        Right(new ApiError(Errors.INVALID_REQUEST,
+          "Could not apply finalized feature update because the provided" +
+          s" maxVersionLevel:${update.maxVersionLevel} is lower than the" +
+          s" first active version:${supportedVersionRange.firstActiveVersion}."))
+      } else {
+        val newFinalizedFeature =
+          Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange)))
+        val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+          BrokerFeatures.hasIncompatibleFeatures(broker.features, newFinalizedFeature)
+        })
+        if (numIncompatibleBrokers == 0) {
+          Left(newVersionRange)
+        } else {
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Could not apply finalized feature update because" +
+                             " brokers were found to have incompatible versions for the feature."))
+        }
+      }
+    }
+  }
+
+  /**
+   * Validates a feature update on an existing FinalizedVersionRange.
+   * If the validation succeeds, then, the return value contains:
+   * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature.
+   * 2. Option.empty, if the feature update was meant to delete the feature.
+   *
+   * If the validation fails, then returned value contains a suitable ApiError.
+   *
+   * @param update                 the feature update to be processed.
+   * @param existingVersionRange   the existing FinalizedVersionRange which can be empty when no
+   *                               FinalizedVersionRange exists for the associated feature
+   *
+   * @return                       the new FinalizedVersionRange to be updated into ZK or error
+   *                               as described above.
+   */
+  private def validateFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey,
+                                    existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = {
+    def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = {
+      newFinalizedVersionRangeOrIncompatibilityError(update)
+        .fold(versionRange => Left(Some(versionRange)), error => Right(error))
+    }
+
+    if (update.feature.isEmpty) {
+      // Check that the feature name is not empty.
+      Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty."))
+    } else {
+      // We handle deletion requests separately from non-deletion requests.
+      if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+        if (existingVersionRange.isEmpty) {
+          // Disallow deletion of a non-existing finalized feature.
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Can not delete non-existing finalized feature."))
+        } else {
+          Left(Option.empty)
+        }
+      } else if (update.maxVersionLevel() < 1) {
+        // Disallow deletion of a finalized feature without allowDowngrade flag set.
+        Right(new ApiError(Errors.INVALID_REQUEST,
+                           s"Can not provide maxVersionLevel: ${update.maxVersionLevel} less" +
+                           s" than 1 without setting the allowDowngrade flag to true in the request."))
+      } else {
+        existingVersionRange.map(existing =>
+          if (update.maxVersionLevel == existing.max) {
+            // Disallow a case where target maxVersionLevel matches existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not ${if (update.allowDowngrade) "downgrade" else "upgrade"}" +
+                               s" a finalized feature from existing maxVersionLevel:${existing.max}" +
+                               " to the same value."))
+          } else if (update.maxVersionLevel < existing.max && !update.allowDowngrade) {
+            // Disallow downgrade of a finalized feature without the allowDowngrade flag set.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature from existing" +
+                               s" maxVersionLevel:${existing.max} to provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} without setting the" +
+                               " allowDowngrade flag in the request."))
+          } else if (update.allowDowngrade && update.maxVersionLevel > existing.max) {
+            // Disallow a request that sets allowDowngrade flag without specifying a
+            // maxVersionLevel that's lower than the existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"When the allowDowngrade flag set in the request, the provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} can not be greater than" +
+                               s" existing maxVersionLevel:${existing.max}."))
+          } else if (update.maxVersionLevel < existing.min) {
+            // Disallow downgrade of a finalized feature below the existing finalized
+            // minVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature to maxVersionLevel:${update.maxVersionLevel}" +
+                               s" because it's lower than the existing minVersionLevel:${existing.min}."))
+          } else {
+            newVersionRangeOrError(update)
+          }
+        ).getOrElse(newVersionRangeOrError(update))
+      }
+    }
+  }
+
+  private def processFeatureUpdates(request: UpdateFeaturesRequest,
+                                    callback: UpdateFeaturesCallback): Unit = {
+    if (isActive) {
+      processFeatureUpdatesWithActiveController(request, callback)
+    } else {
+      callback(Left(new ApiError(Errors.NOT_CONTROLLER)))
+    }
+  }
+
+  private def processFeatureUpdatesWithActiveController(request: UpdateFeaturesRequest,
+                                                        callback: UpdateFeaturesCallback): Unit = {
+    val updates = request.data.featureUpdates
+    val existingFeatures = featureCache.get
+      .map(featuresAndEpoch => featuresAndEpoch.features.features().asScala)
+      .getOrElse(Map[String, FinalizedVersionRange]())
+    // A map with key being feature name and value being FinalizedVersionRange.
+    // This contains the target features to be eventually written to FeatureZNode.
+    val targetFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]() ++ existingFeatures
+    // A map with key being feature name and value being error encountered when the FeatureUpdate
+    // was applied.
+    val errors = scala.collection.mutable.Map[String, ApiError]()
+
+    // Below we process each FeatureUpdate using the following logic:
+    //  - If a FeatureUpdate is found to be valid, then:
+    //    - The corresponding entry in errors map would be updated to contain Errors.NONE.
+    //    - If the FeatureUpdate is an add or update request, then the targetFeatures map is updated
+    //      to contain the new FinalizedVersionRange for the feature.
+    //    - Otherwise if the FeatureUpdate is a delete request, then the feature is removed from the
+    //      targetFeatures map.
+    //  - Otherwise if a FeatureUpdate is found to be invalid, then:
+    //    - The corresponding entry in errors map would be updated with the appropriate ApiError.
+    //    - The entry in targetFeatures map is left untouched.
+    updates.asScala.iterator.foreach { update =>
+      validateFeatureUpdate(update, existingFeatures.get(update.feature())) match {
+        case Left(newVersionRangeOrNone) =>
+          newVersionRangeOrNone match {
+            case Some(newVersionRange) => targetFeatures += (update.feature() -> newVersionRange)
+            case None => targetFeatures -= update.feature()
+          }
+          errors += (update.feature() -> new ApiError(Errors.NONE))
+        case Right(featureUpdateFailureReason) =>
+          errors += (update.feature() -> featureUpdateFailureReason)
+      }
+    }
+
+    // If the existing and target features are the same, then, we skip the update to the
+    // FeatureZNode as no changes to the node are required. Otherwise, we replace the contents
+    // of the FeatureZNode with the new features. This may result in partial or full modification
+    // of the existing finalized features in ZK.
+    try {
+      if (!existingFeatures.equals(targetFeatures)) {
+        val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, Features.finalizedFeatures(targetFeatures.asJava))
+        val newVersion = zkClient.updateFeatureZNode(newNode)
+        featureCache.waitUntilEpochOrThrow(newVersion, request.data().timeoutMs().min(config.zkConnectionTimeoutMs))
+      }
+    } catch {
+      // For all features that correspond to valid FeatureUpdate (i.e. error is Errors.NONE),
+      // we set the error as Errors.FEATURE_UPDATE_FAILED since the FeatureZNode update has failed
+      // for these. For the rest, the existing error is left untouched.
+      case e: Exception =>
+        warn(s"Processing of feature updates: $request failed due to error: $e")
+        errors.foreach { case (feature, apiError) =>
+          if (apiError.error() == Errors.NONE) {
+            errors(feature) = new ApiError(Errors.FEATURE_UPDATE_FAILED)

Review comment:
       The default error message reads: `Unable to update finalized features due to an unexpected server error.` .
   Would that suffice?




----------------------------------------------------------------
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] chia7712 commented on a change in pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1306,6 +1307,73 @@ default AlterUserScramCredentialsResult alterUserScramCredentials(List<UserScram
     AlterUserScramCredentialsResult alterUserScramCredentials(List<UserScramCredentialAlteration> alterations,
                                                               AlterUserScramCredentialsOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker. It can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter. This is particularly useful if the user requires strongly consistent reads of
+     * finalized features.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the {@link DescribeFeaturesResult} containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);

Review comment:
       Should we add an empty-parameter variety for ```describeFeatures```? that is similar to other methods, like ```DescribeUserScramCredentialsResult``` and ```describeDelegationToken```. 
   
   




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FinalizedVersionRange.java
##########
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+
+/**
+ * Represents a range of version levels supported by every broker in a cluster for some feature.
+ */
+public class FinalizedVersionRange {

Review comment:
       Yes, but can I do it in a follow-up PR? The reason is if I were to refactor it now, this PR will bloat up.




----------------------------------------------------------------
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 edited a comment on pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #9001:
URL: https://github.com/apache/kafka/pull/9001#issuecomment-705073059


   @junrao Thanks for the links! I had a look at the links. I found similar stats in both links, with exactly 27 test failures in both links. I compared the individual test failures and found that they have all failed on the same tests. Would that mean we are OK to merge this PR (since it doesn't seem to introduce a new failure)?


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala
##########
@@ -715,7 +747,58 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness {
     doAnswer((_: InvocationOnMock) => {
       latch.countDown()
     }).doCallRealMethod().when(spyThread).awaitShutdown()
-    controller.shutdown() 
+    controller.shutdown()
+  }
+
+  private def testControllerFeatureZNodeSetup(initialZNode: Option[FeatureZNode],
+                                              interBrokerProtocolVersion: ApiVersion): Unit = {
+    val versionBeforeOpt = initialZNode match {
+      case Some(node) =>
+        zkClient.createFeatureZNode(node)
+        Some(zkClient.getDataAndVersion(FeatureZNode.path)._2)
+      case None =>
+        Option.empty
+    }
+    servers = makeServers(1, interBrokerProtocolVersion = Some(interBrokerProtocolVersion))
+    TestUtils.waitUntilControllerElected(zkClient)

Review comment:
       Done. Please take a look at the fix. I've added logic to wait for processing on a dummy event just after waiting for controller election. I'm hoping this will make sure the controller failover logic is completed before the test proceeds further to make assertions.




----------------------------------------------------------------
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 pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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


   retest this


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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


   > Just looked through the change for firstActiveVersion, which makes sense. The only question I have is that if I'm doing a bold release by removing the deprecated code completely, such that previously I have version range [1, 6] and now the code only supports [4, 6] for a pair [min, max], would my deployment be successful automatically?
   
   @abbccdda The deployment will fail if the feature was finalized at say [1, 6] previously, but the new broker only supports [4, 6]. This is where `firstActiveVersion` becomes useful. By bumping it up during a release (instead of the supported feature's `minVersion`), we are able to get past this situation. When `firstActiveVersion` is advanced in the code, and the cluster is deployed, the controller (and all brokers) know that the advancement acts a request to the controller to act upon the feature deprecation (by writing the advanced value to `FeatureZNode`).


----------------------------------------------------------------
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 edited a comment on pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #9001:
URL: https://github.com/apache/kafka/pull/9001#issuecomment-704786154


   The latest test failures do not seem related to this PR:
    * JDK 8: `org.apache.kafka.connect.integration.ExampleConnectIntegrationTest.testSourceConnector`
    * JDK 11: `org.apache.kafka.streams.integration.EosBetaUpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosBeta`
   
   The test that failed previously under JDK 15 has passed in the latest CI run: `MirrorConnectorsIntegrationTest.testReplication`.


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1656,6 +1910,204 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns an ApiError object containing Errors.INVALID_REQUEST.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+
+    val supportedVersionRange = brokerFeatures.supportedFeatures.get(update.feature)
+    if (supportedVersionRange == null) {
+      Right(new ApiError(Errors.INVALID_REQUEST,
+                         "Could not apply finalized feature update because the provided feature" +
+                         " is not supported."))
+    } else {
+      var newVersionRange: FinalizedVersionRange = null
+      try {
+        newVersionRange = new FinalizedVersionRange(supportedVersionRange.firstActiveVersion, update.maxVersionLevel)
+      } catch {
+        case _: IllegalArgumentException => {
+          // This exception means the provided maxVersionLevel is invalid. It is handled below
+          // outside of this catch clause.
+        }
+      }
+      if (newVersionRange == null) {
+        Right(new ApiError(Errors.INVALID_REQUEST,
+          "Could not apply finalized feature update because the provided" +
+          s" maxVersionLevel:${update.maxVersionLevel} is lower than the" +
+          s" first active version:${supportedVersionRange.firstActiveVersion}."))
+      } else {
+        val newFinalizedFeature =
+          Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange)))
+        val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+          BrokerFeatures.hasIncompatibleFeatures(broker.features, newFinalizedFeature)
+        })
+        if (numIncompatibleBrokers == 0) {
+          Left(newVersionRange)
+        } else {
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Could not apply finalized feature update because" +
+                             " brokers were found to have incompatible versions for the feature."))
+        }
+      }
+    }
+  }
+
+  /**
+   * Validates a feature update on an existing FinalizedVersionRange.
+   * If the validation succeeds, then, the return value contains:
+   * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature.
+   * 2. Option.empty, if the feature update was meant to delete the feature.
+   *
+   * If the validation fails, then returned value contains a suitable ApiError.
+   *
+   * @param update                 the feature update to be processed.
+   * @param existingVersionRange   the existing FinalizedVersionRange which can be empty when no
+   *                               FinalizedVersionRange exists for the associated feature
+   *
+   * @return                       the new FinalizedVersionRange to be updated into ZK or error
+   *                               as described above.
+   */
+  private def validateFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey,
+                                    existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = {
+    def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = {
+      newFinalizedVersionRangeOrIncompatibilityError(update)
+        .fold(versionRange => Left(Some(versionRange)), error => Right(error))
+    }
+
+    if (update.feature.isEmpty) {
+      // Check that the feature name is not empty.
+      Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty."))
+    } else {
+      // We handle deletion requests separately from non-deletion requests.
+      if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+        if (existingVersionRange.isEmpty) {
+          // Disallow deletion of a non-existing finalized feature.
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Can not delete non-existing finalized feature."))
+        } else {
+          Left(Option.empty)
+        }
+      } else if (update.maxVersionLevel() < 1) {
+        // Disallow deletion of a finalized feature without allowDowngrade flag set.
+        Right(new ApiError(Errors.INVALID_REQUEST,
+                           s"Can not provide maxVersionLevel: ${update.maxVersionLevel} less" +
+                           s" than 1 without setting the allowDowngrade flag to true in the request."))
+      } else {
+        existingVersionRange.map(existing =>
+          if (update.maxVersionLevel == existing.max) {
+            // Disallow a case where target maxVersionLevel matches existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not ${if (update.allowDowngrade) "downgrade" else "upgrade"}" +
+                               s" a finalized feature from existing maxVersionLevel:${existing.max}" +
+                               " to the same value."))
+          } else if (update.maxVersionLevel < existing.max && !update.allowDowngrade) {
+            // Disallow downgrade of a finalized feature without the allowDowngrade flag set.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature from existing" +
+                               s" maxVersionLevel:${existing.max} to provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} without setting the" +
+                               " allowDowngrade flag in the request."))
+          } else if (update.allowDowngrade && update.maxVersionLevel > existing.max) {
+            // Disallow a request that sets allowDowngrade flag without specifying a
+            // maxVersionLevel that's lower than the existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"When the allowDowngrade flag set in the request, the provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} can not be greater than" +
+                               s" existing maxVersionLevel:${existing.max}."))
+          } else if (update.maxVersionLevel < existing.min) {
+            // Disallow downgrade of a finalized feature below the existing finalized
+            // minVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature to maxVersionLevel:${update.maxVersionLevel}" +
+                               s" because it's lower than the existing minVersionLevel:${existing.min}."))
+          } else {
+            newVersionRangeOrError(update)
+          }
+        ).getOrElse(newVersionRangeOrError(update))
+      }
+    }
+  }
+
+  private def processFeatureUpdates(request: UpdateFeaturesRequest,
+                                    callback: UpdateFeaturesCallback): Unit = {
+    if (isActive) {
+      processFeatureUpdatesWithActiveController(request, callback)
+    } else {
+      callback(Left(new ApiError(Errors.NOT_CONTROLLER)))
+    }
+  }
+
+  private def processFeatureUpdatesWithActiveController(request: UpdateFeaturesRequest,
+                                                        callback: UpdateFeaturesCallback): Unit = {
+    val updates = request.data.featureUpdates
+    val existingFeatures = featureCache.get
+      .map(featuresAndEpoch => featuresAndEpoch.features.features().asScala)
+      .getOrElse(Map[String, FinalizedVersionRange]())
+    // A map with key being feature name and value being FinalizedVersionRange.
+    // This contains the target features to be eventually written to FeatureZNode.
+    val targetFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]() ++ existingFeatures
+    // A map with key being feature name and value being error encountered when the FeatureUpdate
+    // was applied.
+    val errors = scala.collection.mutable.Map[String, ApiError]()
+
+    // Below we process each FeatureUpdate using the following logic:
+    //  - If a FeatureUpdate is found to be valid, then:
+    //    - The corresponding entry in errors map would be updated to contain Errors.NONE.
+    //    - If the FeatureUpdate is an add or update request, then the targetFeatures map is updated
+    //      to contain the new FinalizedVersionRange for the feature.
+    //    - Otherwise if the FeatureUpdate is a delete request, then the feature is removed from the
+    //      targetFeatures map.
+    //  - Otherwise if a FeatureUpdate is found to be invalid, then:
+    //    - The corresponding entry in errors map would be updated with the appropriate ApiError.
+    //    - The entry in targetFeatures map is left untouched.
+    updates.asScala.iterator.foreach { update =>
+      validateFeatureUpdate(update, existingFeatures.get(update.feature())) match {
+        case Left(newVersionRangeOrNone) =>
+          newVersionRangeOrNone match {
+            case Some(newVersionRange) => targetFeatures += (update.feature() -> newVersionRange)
+            case None => targetFeatures -= update.feature()
+          }
+          errors += (update.feature() -> new ApiError(Errors.NONE))
+        case Right(featureUpdateFailureReason) =>
+          errors += (update.feature() -> featureUpdateFailureReason)
+      }
+    }
+
+    // If the existing and target features are the same, then, we skip the update to the
+    // FeatureZNode as no changes to the node are required. Otherwise, we replace the contents
+    // of the FeatureZNode with the new features. This may result in partial or full modification
+    // of the existing finalized features in ZK.
+    try {
+      if (!existingFeatures.equals(targetFeatures)) {
+        val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, Features.finalizedFeatures(targetFeatures.asJava))
+        val newVersion = zkClient.updateFeatureZNode(newNode)
+        featureCache.waitUntilEpochOrThrow(newVersion, request.data().timeoutMs().min(config.zkConnectionTimeoutMs))

Review comment:
       Done. Good point, removed the min now.




----------------------------------------------------------------
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] chia7712 commented on pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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


   @kowshik sorry for bringing trivial comments after this is merged. I just noticed those nits in testing new APIs in 2.7.0.


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -272,6 +281,199 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (string) and a range of versions (defined by a
+   * SupportedVersionRange). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of its own supported features in its
+   * own BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (string) and a range of version levels (defined
+   * by a FinalizedVersionRange). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. This process ensures we do not enable all the possible features immediately after
+   *    an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, it will
+   *        react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, its existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.defaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)

Review comment:
       I agree. But note that in this method, we do not process an `UpdateFeaturesRequest`. This method is called during controller election to setup feature versioning. So, I have incorporated your suggestion at the point where we process the request, look for `def processFeatureUpdatesWithActiveController` in this file where I set the ZK write timeout to be `min(timeoutMs, config.zkConnectionTimeoutMs)`.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/resources/common/message/UpdateFeaturesRequest.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "request",
+  "name": "UpdateFeaturesRequest",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "FeatureUpdates", "type": "[]FeatureUpdateKey", "versions": "0+",
+      "about": "The list of updates to finalized features.", "fields": [
+      {"name": "Feature", "type": "string", "versions": "0+", "mapKey": true,
+        "about": "The name of the finalized feature to be updated."},
+      {"name": "MaxVersionLevel", "type": "int16", "versions": "0+",
+        "about": "The new maximum version level for the finalized feature. A value >= 1 is valid. A value < 1, is special, and can be used to request the deletion of the finalized feature."},
+      {"name": "AllowDowngrade", "type": "bool", "versions": "0+",

Review comment:
       The KIP wiki has AllowDowngrade at the topic level. Could we update that?

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,199 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of its own supported features in its
+   * own BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. This process ensures we do not enable all the possible features immediately after
+   *    an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, it will
+   *        react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, its existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) =>
+            val brokerDefaultVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (brokerDefaultVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (brokerDefaultVersionRange.max() >= existingVersionRange.max() &&
+                       brokerDefaultVersionRange.min() <= existingVersionRange.max()) {
+              // Using the change below, we deprecate all version levels in the range:
+              // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1].
+              //
+              // NOTE: if existingVersionRange.min() equals brokerDefaultVersionRange.min(), then
+              // we do not deprecate any version levels (since there is none to be deprecated).
+              //
+              // Examples:
+              // 1. brokerDefaultVersionRange = [4, 7] and existingVersionRange = [1, 5].
+              //    In this case, we deprecate all version levels in the range: [1, 3].
+              // 2. brokerDefaultVersionRange = [4, 7] and existingVersionRange = [4, 5].
+              //    In this case, we do not deprecate any version levels since
+              //    brokerDefaultVersionRange.min() equals existingVersionRange.min().
+              (featureName, new FinalizedVersionRange(brokerDefaultVersionRange.min(), existingVersionRange.max()))

Review comment:
       When we roll the cluster to bump up IBP, it seems that it's possible for the min of finalized version to flip repeatedly? This can be a bit weird.
   
   Also, it seems that we should set min version based on the largest min version across all brokers?

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -82,18 +110,54 @@ object FinalizedFeatureCache extends Logging {
         " The existing cache contents are %s").format(latest, oldFeatureAndEpoch)
       throw new FeatureCacheUpdateException(errorMsg)
     } else {
-      val incompatibleFeatures = SupportedFeatures.incompatibleFeatures(latest.features)
+      val incompatibleFeatures = brokerFeatures.incompatibleFeatures(latest.features)
       if (!incompatibleFeatures.empty) {
         val errorMsg = ("FinalizedFeatureCache update failed since feature compatibility" +
           " checks failed! Supported %s has incompatibilities with the latest %s."
-          ).format(SupportedFeatures.get, latest)
+          ).format(brokerFeatures.supportedFeatures, latest)

Review comment:
       If the broker discovers that it's incompatible, should it just shut itself down?

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,199 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of its own supported features in its
+   * own BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.

Review comment:
       When we roll the cluster to bump up IBP, it seems that it's possible for status to be enabled and then disabled repeatedly? This can be a bit weird.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -977,14 +1179,30 @@ class KafkaController(val config: KafkaConfig,
 
   /**
    * Send the leader information for selected partitions to selected brokers so that they can correctly respond to
-   * metadata requests
+   * metadata requests. Particularly, when feature versioning is enabled, we filter out brokers with incompatible
+   * features from receiving the metadata requests. This is because we do not want to activate incompatible brokers,
+   * as these may have harmful consequences to the cluster.

Review comment:
       Hmm, do we need to do this? If there is an incompatible feature, the broker will realize that and can just shut itself down.

##########
File path: clients/src/main/resources/common/message/UpdateFeaturesResponse.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "response",
+  "name": "UpdateFeaturesResponse",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "Results", "type": "[]UpdatableFeatureResult", "versions": "0+",
+      "about": "Results for each feature update.", "fields": [
+      {"name": "Feature", "type": "string", "versions": "0+", "mapKey": true,

Review comment:
       The KIP wiki doesn't include this field.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1865,192 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns an ApiError object containing Errors#INVALID_REQUEST.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+
+    val incompatibilityError = "Could not apply finalized feature update because" +
+      " brokers were found to have incompatible versions for the feature."
+
+    if (brokerFeatures.supportedFeatures.get(update.feature()) == null) {
+      Right(new ApiError(Errors.INVALID_REQUEST, incompatibilityError))
+    } else {
+      val defaultMinVersionLevel = brokerFeatures.defaultMinVersionLevel(update.feature)
+      val newVersionRange = new FinalizedVersionRange(defaultMinVersionLevel, update.maxVersionLevel)

Review comment:
       If update.maxVersionLevel < defaultMinVersionLevel, we throw an IllegalStateException. Should we catch it and convert it to an error code?

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,192 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature

Review comment:
       Could you explain how the default min version is different from the min in supportedFeatures?

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1865,192 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns an ApiError object containing Errors#INVALID_REQUEST.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+
+    val incompatibilityError = "Could not apply finalized feature update because" +
+      " brokers were found to have incompatible versions for the feature."
+
+    if (brokerFeatures.supportedFeatures.get(update.feature()) == null) {

Review comment:
       Since we are doing the compatibility check for every broker, do we need to special case here just for the broker feature on the controller?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java
##########
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+import java.util.Optional;
+import org.apache.kafka.common.feature.Features;
+import org.apache.kafka.common.feature.FinalizedVersionRange;
+import org.apache.kafka.common.feature.SupportedVersionRange;
+
+/**
+ * Encapsulates details about finalized as well as supported features. This is particularly useful
+ * to hold the result returned by the {@link Admin#describeFeatures(DescribeFeaturesOptions)} API.
+ */
+public class FeatureMetadata {
+
+    private final Features<FinalizedVersionRange> finalizedFeatures;
+
+    private final Optional<Integer> finalizedFeaturesEpoch;
+
+    private final Features<SupportedVersionRange> supportedFeatures;
+
+    public FeatureMetadata(final Features<FinalizedVersionRange> finalizedFeatures,
+                           final int finalizedFeaturesEpoch,
+                           final Features<SupportedVersionRange> supportedFeatures) {
+        Objects.requireNonNull(finalizedFeatures, "Provided finalizedFeatures can not be null.");
+        Objects.requireNonNull(supportedFeatures, "Provided supportedFeatures can not be null.");
+        this.finalizedFeatures = finalizedFeatures;
+        if (finalizedFeaturesEpoch >= 0) {
+            this.finalizedFeaturesEpoch = Optional.of(finalizedFeaturesEpoch);
+        } else {
+            this.finalizedFeaturesEpoch = Optional.empty();
+        }
+        this.supportedFeatures = supportedFeatures;
+    }
+
+    /**
+     * A map of finalized feature versions, with key being finalized feature name and value
+     * containing the min/max version levels for the finalized feature.
+     */
+    public Features<FinalizedVersionRange> finalizedFeatures() {

Review comment:
       The return type is different from the KIP. Which one is correct? Since this is a public interface, in general, we don't want to expose anything other than truly necessary. This PR seems to expose a lot more public methods to the user.
   
   FinalizedVersionRange is in org.apache.kafka.common.feature. Currently, all public interfaces are specified under javadoc in build.gradle. So, we need to either include that package in javadoc or move it to a public package.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java
##########
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+import java.util.Optional;
+import org.apache.kafka.common.feature.Features;
+import org.apache.kafka.common.feature.FinalizedVersionRange;
+import org.apache.kafka.common.feature.SupportedVersionRange;
+
+/**
+ * Encapsulates details about finalized as well as supported features. This is particularly useful
+ * to hold the result returned by the {@link Admin#describeFeatures(DescribeFeaturesOptions)} API.
+ */
+public class FeatureMetadata {
+
+    private final Features<FinalizedVersionRange> finalizedFeatures;
+
+    private final Optional<Integer> finalizedFeaturesEpoch;
+
+    private final Features<SupportedVersionRange> supportedFeatures;
+
+    public FeatureMetadata(final Features<FinalizedVersionRange> finalizedFeatures,
+                           final int finalizedFeaturesEpoch,
+                           final Features<SupportedVersionRange> supportedFeatures) {
+        Objects.requireNonNull(finalizedFeatures, "Provided finalizedFeatures can not be null.");
+        Objects.requireNonNull(supportedFeatures, "Provided supportedFeatures can not be null.");
+        this.finalizedFeatures = finalizedFeatures;
+        if (finalizedFeaturesEpoch >= 0) {
+            this.finalizedFeaturesEpoch = Optional.of(finalizedFeaturesEpoch);
+        } else {
+            this.finalizedFeaturesEpoch = Optional.empty();
+        }
+        this.supportedFeatures = supportedFeatures;
+    }
+
+    /**
+     * A map of finalized feature versions, with key being finalized feature name and value
+     * containing the min/max version levels for the finalized feature.
+     */
+    public Features<FinalizedVersionRange> finalizedFeatures() {
+        return finalizedFeatures;
+    }
+
+    /**
+     * The epoch for the finalized features.
+     * If the returned value is empty, it means the finalized features are absent/unavailable.
+     */
+    public Optional<Integer> finalizedFeaturesEpoch() {

Review comment:
       The return type is different from the KIP. Which one is correct?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java
##########
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+import java.util.Optional;
+import org.apache.kafka.common.feature.Features;
+import org.apache.kafka.common.feature.FinalizedVersionRange;
+import org.apache.kafka.common.feature.SupportedVersionRange;
+
+/**
+ * Encapsulates details about finalized as well as supported features. This is particularly useful
+ * to hold the result returned by the {@link Admin#describeFeatures(DescribeFeaturesOptions)} API.

Review comment:
       The KIP also exposes host() and port(). Are they still needed?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1216,71 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker. It can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter. This is particularly useful if the user requires strongly consistent reads of
+     * finalized features.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the {@link DescribeFeaturesResult} containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);
+
+    /**
+     * Applies specified updates to finalized features. This operation is not transactional so it
+     * may succeed for some features while fail for others.
+     * <p>
+     * The API takes in a map of finalized feature names to {@link FeatureUpdate} that needs to be
+     * applied. Each entry in the map specifies the finalized feature to be added or updated or
+     * deleted, along with the new max feature version level value. This request is issued only to
+     * the controller since the API is only served by the controller. The return value contains an
+     * error code for each supplied {@link FeatureUpdate}, and the code indicates if the update
+     * succeeded or failed in the controller.
+     * <ul>
+     * <li>Downgrade of feature version level is not a regular operation/intent. It is only allowed
+     * in the controller if the {@link FeatureUpdate} has the allowDowngrade flag set - setting this
+     * flag conveys user intent to attempt downgrade of a feature max version level. Note that
+     * despite the allowDowngrade flag being set, certain downgrades may be rejected by the
+     * controller if it is deemed impossible.</li>
+     * <li>Deletion of a finalized feature version is not a regular operation/intent. It could be
+     * done by setting the allowDowngrade flag to true in the {@link FeatureUpdate}, and, setting
+     * the max version level to be less than 1.</li>
+     * </ul>
+     *<p>
+     * The following exceptions can be anticipated when calling {@code get()} on the futures
+     * obtained from the returned {@link UpdateFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.ClusterAuthorizationException}
+     *   If the authenticated user didn't have alter access to the cluster.</li>
+     *   <li>{@link org.apache.kafka.common.errors.InvalidRequestException}
+     *   If the request details are invalid. e.g., a non-existing finalized feature is attempted
+     *   to be deleted or downgraded.</li>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the updates could finish. It cannot be guaranteed whether
+     *   the updates succeeded or not.</li>
+     *   <li>{@link FeatureUpdateFailedException}
+     *   If the updates could not be applied on the controller, despite the request being valid.
+     *   This may be a temporary problem.</li>
+     * </ul>
+     * <p>
+     * This operation is supported by brokers with version 2.7.0 or higher.
+
+     * @param featureUpdates   the map of finalized feature name to {@link FeatureUpdate}
+     * @param options          the options to use
+     *
+     * @return                 the {@link UpdateFeaturesResult} containing the result
+     */
+    UpdateFeaturesResult updateFeatures(Map<String, FeatureUpdate> featureUpdates, UpdateFeaturesOptions options);

Review comment:
       Again, this method has a different signature from the KIP.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1216,71 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker. It can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter. This is particularly useful if the user requires strongly consistent reads of
+     * finalized features.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the {@link DescribeFeaturesResult} containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);

Review comment:
       The KIP doesn't have DescribeFeaturesOptions. If we are changing the KIP, could we summarize the list of the things that are changed?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4071,6 +4078,113 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        final NodeProvider provider =
+            options.sendRequestToController() ? new ControllerNodeProvider() : new LeastLoadedNodeProvider();
+
+        Call call = new Call(
+            "describeFeatures", calcDeadlineMs(now, options.timeoutMs()), provider) {
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(
+                        new FeatureMetadata(
+                            apiVersionsResponse.finalizedFeatures(),
+                            apiVersionsResponse.finalizedFeaturesEpoch(),
+                            apiVersionsResponse.supportedFeatures()));
+                } else if (options.sendRequestToController() && apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) {
+                    handleNotControllerError(Errors.NOT_CONTROLLER);
+                } else {
+                    future.completeExceptionally(
+                        Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        runnable.call(call, now);
+        return new DescribeFeaturesResult(future);
+    }
+
+    @Override
+    public UpdateFeaturesResult updateFeatures(
+        final Map<String, FeatureUpdate> featureUpdates, final UpdateFeaturesOptions options) {
+        if (featureUpdates == null || featureUpdates.isEmpty()) {
+            throw new IllegalArgumentException("Feature updates can not be null or empty.");
+        }
+        Objects.requireNonNull(options, "UpdateFeaturesOptions can not be null");
+
+        final UpdateFeaturesRequestData request = UpdateFeaturesRequest.create(featureUpdates);
+        final Map<String, KafkaFutureImpl<Void>> updateFutures = new HashMap<>();
+        for (Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+            updateFutures.put(entry.getKey(), new KafkaFutureImpl<>());
+        }
+        final long now = time.milliseconds();
+        final Call call = new Call("updateFeatures", calcDeadlineMs(now, options.timeoutMs()),
+            new ControllerNodeProvider()) {
+
+            @Override
+            UpdateFeaturesRequest.Builder createRequest(int timeoutMs) {
+                return new UpdateFeaturesRequest.Builder(request);
+            }
+
+            @Override
+            void handleResponse(AbstractResponse abstractResponse) {
+                final UpdateFeaturesResponse response =
+                    (UpdateFeaturesResponse) abstractResponse;
+
+                // Check for controller change.
+                for (UpdatableFeatureResult result : response.data().results()) {
+                    final Errors error = Errors.forCode(result.errorCode());
+                    if (error == Errors.NOT_CONTROLLER) {
+                        handleNotControllerError(error);
+                        throw error.exception();

Review comment:
       handleNotControllerError() already throws an exception.
   
   Should other errors like CLUSTER_AUTHORIZATION_FAILED be treated in the same way?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesResult.java
##########
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Map;
+import org.apache.kafka.common.KafkaFuture;
+
+/**
+ * The result of the {@link Admin#updateFeatures(Map, UpdateFeaturesOptions)} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+public class UpdateFeaturesResult {
+    private final Map<String, KafkaFuture<Void>> futures;
+
+    /**
+     * @param futures   a map from feature names to future, which can be used to check the status of
+     *                  individual feature updates.
+     */
+    public UpdateFeaturesResult(final Map<String, KafkaFuture<Void>> futures) {
+        this.futures = futures;
+    }
+
+    public Map<String, KafkaFuture<Void>> values() {

Review comment:
       The KIP doesn't have this method.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4335,6 +4343,150 @@ void handleFailure(Throwable throwable) {
                 .hi(password, salt, iterations);
     }
 
+    public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        final NodeProvider provider =
+            options.sendRequestToController() ? new ControllerNodeProvider() : new LeastLoadedNodeProvider();
+
+        final Call call = new Call(
+            "describeFeatures", calcDeadlineMs(now, options.timeoutMs()), provider) {
+
+            private FeatureMetadata createFeatureMetadata(final ApiVersionsResponse response) {
+                final Map<String, FinalizedVersionRange> finalizedFeatures = new HashMap<>();
+                for (final FinalizedFeatureKey key : response.data().finalizedFeatures().valuesSet()) {
+                    finalizedFeatures.put(key.name(), new FinalizedVersionRange(key.minVersionLevel(), key.maxVersionLevel()));
+                }
+
+                Optional<Long> finalizedFeaturesEpoch;
+                if (response.data().finalizedFeaturesEpoch() >= 0L) {
+                    finalizedFeaturesEpoch = Optional.of(response.data().finalizedFeaturesEpoch());
+                } else {
+                    finalizedFeaturesEpoch = Optional.empty();
+                }
+
+                final Map<String, SupportedVersionRange> supportedFeatures = new HashMap<>();
+                for (final SupportedFeatureKey key : response.data().supportedFeatures().valuesSet()) {
+                    supportedFeatures.put(key.name(), new SupportedVersionRange(key.minVersion(), key.maxVersion()));
+                }
+
+                return new FeatureMetadata(finalizedFeatures, finalizedFeaturesEpoch, supportedFeatures);
+            }
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(createFeatureMetadata(apiVersionsResponse));
+                } else if (options.sendRequestToController() &&
+                           apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) {
+                    handleNotControllerError(Errors.NOT_CONTROLLER);
+                } else {
+                    future.completeExceptionally(Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        runnable.call(call, now);
+        return new DescribeFeaturesResult(future);
+    }
+
+    @Override
+    public UpdateFeaturesResult updateFeatures(final Map<String, FeatureUpdate> featureUpdates,
+                                               final UpdateFeaturesOptions options) {
+        if (featureUpdates.isEmpty()) {
+            throw new IllegalArgumentException("Feature updates can not be null or empty.");
+        }
+
+        final Map<String, KafkaFutureImpl<Void>> updateFutures = new HashMap<>();
+        for (final Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+            updateFutures.put(entry.getKey(), new KafkaFutureImpl<>());
+        }
+
+        final long now = time.milliseconds();
+        final Call call = new Call("updateFeatures", calcDeadlineMs(now, options.timeoutMs()),
+            new ControllerNodeProvider()) {
+
+            @Override
+            UpdateFeaturesRequest.Builder createRequest(int timeoutMs) {
+                final UpdateFeaturesRequestData.FeatureUpdateKeyCollection featureUpdatesRequestData
+                    = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection();
+                for (Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+                    final String feature = entry.getKey();
+                    final FeatureUpdate update = entry.getValue();
+                    if (feature.trim().isEmpty()) {

Review comment:
       Done. Addressed in #9393.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1306,6 +1307,73 @@ default AlterUserScramCredentialsResult alterUserScramCredentials(List<UserScram
     AlterUserScramCredentialsResult alterUserScramCredentials(List<UserScramCredentialAlteration> alterations,
                                                               AlterUserScramCredentialsOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker. It can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter. This is particularly useful if the user requires strongly consistent reads of
+     * finalized features.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the {@link DescribeFeaturesResult} containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);

Review comment:
       Done. Addressed in #9393.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4335,6 +4343,150 @@ void handleFailure(Throwable throwable) {
                 .hi(password, salt, iterations);
     }
 
+    public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        final NodeProvider provider =
+            options.sendRequestToController() ? new ControllerNodeProvider() : new LeastLoadedNodeProvider();
+
+        final Call call = new Call(
+            "describeFeatures", calcDeadlineMs(now, options.timeoutMs()), provider) {
+
+            private FeatureMetadata createFeatureMetadata(final ApiVersionsResponse response) {
+                final Map<String, FinalizedVersionRange> finalizedFeatures = new HashMap<>();
+                for (final FinalizedFeatureKey key : response.data().finalizedFeatures().valuesSet()) {
+                    finalizedFeatures.put(key.name(), new FinalizedVersionRange(key.minVersionLevel(), key.maxVersionLevel()));
+                }
+
+                Optional<Long> finalizedFeaturesEpoch;
+                if (response.data().finalizedFeaturesEpoch() >= 0L) {
+                    finalizedFeaturesEpoch = Optional.of(response.data().finalizedFeaturesEpoch());
+                } else {
+                    finalizedFeaturesEpoch = Optional.empty();
+                }
+
+                final Map<String, SupportedVersionRange> supportedFeatures = new HashMap<>();
+                for (final SupportedFeatureKey key : response.data().supportedFeatures().valuesSet()) {
+                    supportedFeatures.put(key.name(), new SupportedVersionRange(key.minVersion(), key.maxVersion()));
+                }
+
+                return new FeatureMetadata(finalizedFeatures, finalizedFeaturesEpoch, supportedFeatures);
+            }
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(createFeatureMetadata(apiVersionsResponse));
+                } else if (options.sendRequestToController() &&
+                           apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) {
+                    handleNotControllerError(Errors.NOT_CONTROLLER);
+                } else {
+                    future.completeExceptionally(Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        runnable.call(call, now);
+        return new DescribeFeaturesResult(future);
+    }
+
+    @Override
+    public UpdateFeaturesResult updateFeatures(final Map<String, FeatureUpdate> featureUpdates,
+                                               final UpdateFeaturesOptions options) {
+        if (featureUpdates.isEmpty()) {
+            throw new IllegalArgumentException("Feature updates can not be null or empty.");
+        }
+
+        final Map<String, KafkaFutureImpl<Void>> updateFutures = new HashMap<>();
+        for (final Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+            updateFutures.put(entry.getKey(), new KafkaFutureImpl<>());
+        }
+
+        final long now = time.milliseconds();
+        final Call call = new Call("updateFeatures", calcDeadlineMs(now, options.timeoutMs()),
+            new ControllerNodeProvider()) {
+
+            @Override
+            UpdateFeaturesRequest.Builder createRequest(int timeoutMs) {
+                final UpdateFeaturesRequestData.FeatureUpdateKeyCollection featureUpdatesRequestData
+                    = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection();
+                for (Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+                    final String feature = entry.getKey();
+                    final FeatureUpdate update = entry.getValue();
+                    if (feature.trim().isEmpty()) {
+                        throw new IllegalArgumentException("Provided feature can not be null or empty.");
+                    }
+
+                    final UpdateFeaturesRequestData.FeatureUpdateKey requestItem =
+                        new UpdateFeaturesRequestData.FeatureUpdateKey();
+                    requestItem.setFeature(feature);
+                    requestItem.setMaxVersionLevel(update.maxVersionLevel());
+                    requestItem.setAllowDowngrade(update.allowDowngrade());
+                    featureUpdatesRequestData.add(requestItem);
+                }
+                return new UpdateFeaturesRequest.Builder(
+                    new UpdateFeaturesRequestData()
+                        .setTimeoutMs(timeoutMs)
+                        .setFeatureUpdates(featureUpdatesRequestData));
+            }
+
+            @Override
+            void handleResponse(AbstractResponse abstractResponse) {
+                final UpdateFeaturesResponse response =
+                    (UpdateFeaturesResponse) abstractResponse;
+
+                Errors topLevelError = Errors.forCode(response.data().errorCode());
+                switch (topLevelError) {
+                    case NONE:
+                        for (final UpdatableFeatureResult result : response.data().results()) {
+                            final KafkaFutureImpl<Void> future = updateFutures.get(result.feature());
+                            if (future == null) {
+                                log.warn("Server response mentioned unknown feature {}", result.feature());
+                            } else {
+                                final Errors error = Errors.forCode(result.errorCode());
+                                if (error == Errors.NONE) {
+                                    future.complete(null);
+                                } else {
+                                    future.completeExceptionally(error.exception(result.errorMessage()));
+                                }
+                            }
+                        }
+                        // The server should send back a response for every feature, but we do a sanity check anyway.
+                        completeUnrealizedFutures(updateFutures.entrySet().stream(),
+                            feature -> "The controller response did not contain a result for feature " + feature);
+                        break;
+                    case NOT_CONTROLLER:
+                        handleNotControllerError(topLevelError);
+                        break;
+                    default:
+                        for (final Map.Entry<String, KafkaFutureImpl<Void>> entry : updateFutures.entrySet()) {
+                            entry.getValue().completeExceptionally(topLevelError.exception());

Review comment:
       Done. Addressed in #9393.




----------------------------------------------------------------
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 pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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






----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/resources/common/message/UpdateFeaturesResponse.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "response",
+  "name": "UpdateFeaturesResponse",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [

Review comment:
       I don't see that we consistently use a top level error code across other Kafka apis, so I will leave it as it is. It feels OK for this api to not use it, as it does not make a significant difference.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -272,6 +281,199 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (string) and a range of versions (defined by a
+   * SupportedVersionRange). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of its own supported features in its
+   * own BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (string) and a range of version levels (defined
+   * by a FinalizedVersionRange). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. This process ensures we do not enable all the possible features immediately after
+   *    an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, it will
+   *        react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, its existing minimum version level is updated to the default

Review comment:
       Done. I reworded a bit and I'm now no longer using "default finalized features" and "default minimum version" in the wordings.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -1733,8 +1734,8 @@ class KafkaApis(val requestChannel: RequestChannel,
       else if (!apiVersionRequest.isValid)
         apiVersionRequest.getErrorResponse(requestThrottleMs, Errors.INVALID_REQUEST.exception)
       else {
-        val supportedFeatures = SupportedFeatures.get
-        val finalizedFeatures = FinalizedFeatureCache.get
+        val supportedFeatures = brokerFeatures.supportedFeatures
+        val finalizedFeatures = featureCache.get

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 pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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


   retest this please


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1844,185 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns a suitable error.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+    // NOTE: Below we set the finalized min version level to be the default minimum version
+    // level. If the finalized feature already exists, then, this can cause deprecation of all
+    // version levels in the closed range:
+    // [existingVersionRange.min(), defaultMinVersionLevel - 1].
+    val defaultMinVersionLevel = brokerFeatures.defaultMinVersionLevel(update.feature)
+    val newVersionRange = new FinalizedVersionRange(defaultMinVersionLevel, update.maxVersionLevel)
+    val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+      val singleFinalizedFeature =
+        Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange)))
+      BrokerFeatures.hasIncompatibleFeatures(broker.features, singleFinalizedFeature)
+    })
+    if (numIncompatibleBrokers == 0) {
+      Left(newVersionRange)
+    } else {
+      Right(
+        new ApiError(Errors.INVALID_REQUEST,
+                     s"Could not apply finalized feature update because $numIncompatibleBrokers" +
+                     " brokers were found to have incompatible features."))
+    }
+  }
+
+  /**
+   * Validate and process a finalized feature update on an existing FinalizedVersionRange for the
+   * feature.
+   *
+   * If the processing is successful, then, the return value contains:
+   * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature.
+   * 2. Option.empty, if the feature update was meant to delete the feature.
+   *
+   * If the processing failed, then returned value contains a suitable ApiError.
+   *
+   * @param update                 the feature update to be processed.
+   * @param existingVersionRange   the existing FinalizedVersionRange which can be empty when no
+   *                               FinalizedVersionRange exists for the associated feature
+   *
+   * @return                       the new FinalizedVersionRange or error, as described above.
+   */
+  private def processFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey,
+                                   existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = {
+    def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = {
+      newFinalizedVersionRangeOrIncompatibilityError(update)
+        .fold(versionRange => Left(Some(versionRange)), error => Right(error))
+    }
+
+    if (update.feature.isEmpty) {
+      // Check that the feature name is not empty.
+      Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty."))
+    } else {
+      // We handle deletion requests separately from non-deletion requests.
+      if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+        if (existingVersionRange.isEmpty) {
+          // Disallow deletion of a non-existing finalized feature.
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             s"Can not delete non-existing finalized feature: '${update.feature}'"))
+        } else {
+          Left(Option.empty)
+        }
+      } else if (update.maxVersionLevel() < 1) {

Review comment:
       A value < 1 is indicative of a deletion request (not purely a downgrade request).
   It is for convenience of generating a special error message, that we handle the case here explicitly: `...less than 1 for feature...`.
   




----------------------------------------------------------------
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 edited a comment on pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #9001:
URL: https://github.com/apache/kafka/pull/9001#issuecomment-705073059


   @junrao Thanks for the links! I had a look at the links. I found 27 test failures in both links. I compared the individual test failures and found that they have all failed on the same tests. Would that mean we are OK to merge this PR (since it doesn't seem to introduce a new failure)?


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1214,70 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker, but it can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the DescribeFeaturesResult containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);
+
+    /**
+     * Applies specified updates to finalized features. The API is atomic, meaning that if a single
+     * feature update in the request can't succeed on the controller, then none of the feature
+     * updates are carried out. This request is issued only to the controller since the API is
+     * only served by the controller.
+     * <p>
+     * The API takes as input a set of FinalizedFeatureUpdate that need to be applied. Each such
+     * update specifies the finalized feature to be added or updated or deleted, along with the new
+     * max feature version level value.
+     * <ul>
+     * <li>Downgrade of feature version level is not a regular operation/intent. It is only allowed
+     * in the controller if the feature update has the allowDowngrade flag set - setting this flag
+     * conveys user intent to attempt downgrade of a feature max version level. Note that despite
+     * the allowDowngrade flag being set, certain downgrades may be rejected by the controller if it
+     * is deemed impossible.</li>
+     * <li>Deletion of a finalized feature version is not a regular operation/intent. It is allowed
+     * only if the allowDowngrade flag is set in the feature update, and, if the max version level
+     * is set to a value less than 1.</li>
+     * </ul>
+     *<p>
+     * The following exceptions can be anticipated when calling {@code get()} on the futures
+     * obtained from the returned {@link UpdateFinalizedFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.ClusterAuthorizationException}
+     *   If the authenticated user didn't have alter access to the cluster.</li>
+     *   <li>{@link org.apache.kafka.common.errors.InvalidRequestException}
+     *   If the request details are invalid. e.g., a non-existing finalized feature is attempted
+     *   to be deleted or downgraded.</li>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the updates could finish. It cannot be guaranteed whether
+     *   the updates succeeded or not.</li>
+     *   <li>{@link org.apache.kafka.common.errors.FinalizedFeatureUpdateFailedException}
+     *   If the updates could not be applied on the controller, despite the request being valid.
+     *   This may be a temporary problem.</li>
+     * </ul>
+     * <p>
+     * This operation is supported by brokers with version 2.7.0 or higher.
+
+     * @param featureUpdates   the set of finalized feature updates
+     * @param options          the options to use
+     *
+     * @return                 the UpdateFinalizedFeaturesResult containing the result

Review comment:
       nit: get a ` {@link UpdateFinalizedFeaturesResult}` as well

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1214,70 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker, but it can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the DescribeFeaturesResult containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);
+
+    /**
+     * Applies specified updates to finalized features. The API is atomic, meaning that if a single
+     * feature update in the request can't succeed on the controller, then none of the feature
+     * updates are carried out. This request is issued only to the controller since the API is
+     * only served by the controller.
+     * <p>
+     * The API takes as input a set of FinalizedFeatureUpdate that need to be applied. Each such

Review comment:
       s/`as input a set of FinalizedFeatureUpdate`/`in a set of feature updates`

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FinalizedFeatureUpdate.java
##########
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+import java.util.Set;
+import org.apache.kafka.common.message.UpdateFinalizedFeaturesRequestData;
+
+/**
+ * Encapsulates details about an update to a finalized feature. This is particularly useful to
+ * define each feature update in the
+ * {@link Admin#updateFinalizedFeatures(Set, UpdateFinalizedFeaturesOptions)} API request.
+ */
+public class FinalizedFeatureUpdate {
+    private final String featureName;
+    private final short maxVersionLevel;
+    private final boolean allowDowngrade;
+
+    /**
+     * @param featureName       the name of the finalized feature to be updated.
+     * @param maxVersionLevel   the new maximum version level for the finalized feature.
+     *                          a value < 1 is special and indicates that the update is intended to
+     *                          delete the finalized feature, and should be accompanied by setting
+     *                          the allowDowngrade flag to true.
+     * @param allowDowngrade    - true, if this feature update was meant to downgrade the existing
+     *                            maximum version level of the finalized feature.
+     *                          - false, otherwise.
+     */
+    public FinalizedFeatureUpdate(
+        final String featureName, final short maxVersionLevel, final boolean allowDowngrade) {
+        Objects.requireNonNull(featureName, "Provided feature name can not be null.");
+        if (maxVersionLevel < 1 && !allowDowngrade) {
+            throw new IllegalArgumentException(
+                String.format(
+                    "For featureName: %s, the allowDowngrade flag is not set when the" +
+                    " provided maxVersionLevel:%d is < 1.", featureName, maxVersionLevel));
+        }
+        this.featureName = featureName;
+        this.maxVersionLevel = maxVersionLevel;
+        this.allowDowngrade = allowDowngrade;
+    }
+
+    /**
+     * @return   the name of the finalized feature to be updated.
+     */
+    public String featureName() {
+        return featureName;
+    }
+
+    /**
+     * @return   the new maximum version level for the finalized feature.
+     */
+    public short maxVersionLevel() {
+        return maxVersionLevel;
+    }
+
+    /**
+     * @return   - true, if this feature update was meant to downgrade the maximum version level of
+     *             the finalized feature.
+     *           - false, otherwise.

Review comment:
       `false otherwise` doesn't provide too much useful info.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FinalizedFeatureUpdate.java
##########
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+import java.util.Set;
+import org.apache.kafka.common.message.UpdateFinalizedFeaturesRequestData;
+
+/**
+ * Encapsulates details about an update to a finalized feature. This is particularly useful to
+ * define each feature update in the
+ * {@link Admin#updateFinalizedFeatures(Set, UpdateFinalizedFeaturesOptions)} API request.
+ */
+public class FinalizedFeatureUpdate {
+    private final String featureName;
+    private final short maxVersionLevel;
+    private final boolean allowDowngrade;
+
+    /**
+     * @param featureName       the name of the finalized feature to be updated.
+     * @param maxVersionLevel   the new maximum version level for the finalized feature.
+     *                          a value < 1 is special and indicates that the update is intended to
+     *                          delete the finalized feature, and should be accompanied by setting
+     *                          the allowDowngrade flag to true.
+     * @param allowDowngrade    - true, if this feature update was meant to downgrade the existing
+     *                            maximum version level of the finalized feature.
+     *                          - false, otherwise.
+     */
+    public FinalizedFeatureUpdate(
+        final String featureName, final short maxVersionLevel, final boolean allowDowngrade) {
+        Objects.requireNonNull(featureName, "Provided feature name can not be null.");
+        if (maxVersionLevel < 1 && !allowDowngrade) {
+            throw new IllegalArgumentException(
+                String.format(
+                    "For featureName: %s, the allowDowngrade flag is not set when the" +
+                    " provided maxVersionLevel:%d is < 1.", featureName, maxVersionLevel));
+        }
+        this.featureName = featureName;
+        this.maxVersionLevel = maxVersionLevel;
+        this.allowDowngrade = allowDowngrade;
+    }
+
+    /**
+     * @return   the name of the finalized feature to be updated.

Review comment:
       nit: seems not necessary

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature
+ *    version level deprecation. This is how it works: in order to deprecate feature version levels,
+ *    in this map the default minimum version level of a feature can be set to a new value that's
+ *    higher than 1 (let's call this new_min_version_level). In doing so, the feature version levels
+ *    in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic
+ *    that applies this map to persistent finalized feature state in ZK (this mutation happens
+ *    during controller election and during finalized feature updates via the
+ *    APIKeys.UPDATE_FINALIZED_FEATURES api).
+ *
+ * This class also provides APIs to check for incompatibilities between the features supported by
+ * the Broker and finalized features.
+ *
+ * NOTE: the update*() and clear*() APIs of this class should be used only for testing purposes.
+ */
+class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange],
+                              @volatile var defaultFeatureMinVersionLevels: Map[String, Short]) {
+  require(BrokerFeatures.areFeatureMinVersionLevelsCompatible(
+    supportedFeatures, defaultFeatureMinVersionLevels))
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(newFeatures, defaultFeatureMinVersionLevels))
+    supportedFeatures = newFeatures
+  }
+
+  /**
+   * Returns the default minimum version level for a specific feature.
+   *
+   * @param feature   the name of the feature
+   *
+   * @return          the default minimum version level for the feature if its defined.
+   *                  otherwise, returns 1.
+   */
+  def defaultMinVersionLevel(feature: String): Short = {
+    defaultFeatureMinVersionLevels.getOrElse(feature, 1)
+  }
+
+  // For testing only.
+  def setDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(supportedFeatures, newMinVersionLevels))
+    defaultFeatureMinVersionLevels = newMinVersionLevels
+  }
+
+  /**
+   * Returns the default finalized features that a new Kafka cluster with IBP config >= KAFKA_2_7_IV0
+   * needs to be bootstrapped with.
+   */
+  def getDefaultFinalizedFeatures: Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(
+      supportedFeatures.features.asScala.map {
+        case(name, versionRange) => (
+          name, new FinalizedVersionRange(defaultMinVersionLevel(name), versionRange.max))
+      }.asJava)
+  }
+
+  /**
+   * Returns the set of feature names found to be 'incompatible'.
+   * A feature incompatibility is a version mismatch between the latest feature supported by the
+   * Broker, and the provided finalized feature. This can happen because a provided finalized
+   * feature:
+   *  1) Does not exist in the Broker (i.e. it is unknown to the Broker).
+   *           [OR]
+   *  2) Exists but the FinalizedVersionRange does not match with the
+   *     supported feature's SupportedVersionRange.
+   *
+   * @param finalized   The finalized features against which incompatibilities need to be checked for.
+   *
+   * @return            The subset of input features which are incompatible. If the returned object
+   *                    is empty, it means there were no feature incompatibilities found.
+   */
+  def incompatibleFeatures(finalized: Features[FinalizedVersionRange]): Features[FinalizedVersionRange] = {
+    BrokerFeatures.incompatibleFeatures(supportedFeatures, finalized, true)

Review comment:
       nit: mark the parameter as `logIncompatibilities = true)`

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FinalizedFeatureUpdate.java
##########
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+import java.util.Set;
+import org.apache.kafka.common.message.UpdateFinalizedFeaturesRequestData;
+
+/**
+ * Encapsulates details about an update to a finalized feature. This is particularly useful to
+ * define each feature update in the
+ * {@link Admin#updateFinalizedFeatures(Set, UpdateFinalizedFeaturesOptions)} API request.
+ */
+public class FinalizedFeatureUpdate {
+    private final String featureName;
+    private final short maxVersionLevel;
+    private final boolean allowDowngrade;
+
+    /**
+     * @param featureName       the name of the finalized feature to be updated.
+     * @param maxVersionLevel   the new maximum version level for the finalized feature.
+     *                          a value < 1 is special and indicates that the update is intended to
+     *                          delete the finalized feature, and should be accompanied by setting
+     *                          the allowDowngrade flag to true.
+     * @param allowDowngrade    - true, if this feature update was meant to downgrade the existing
+     *                            maximum version level of the finalized feature.
+     *                          - false, otherwise.
+     */
+    public FinalizedFeatureUpdate(
+        final String featureName, final short maxVersionLevel, final boolean allowDowngrade) {
+        Objects.requireNonNull(featureName, "Provided feature name can not be null.");
+        if (maxVersionLevel < 1 && !allowDowngrade) {
+            throw new IllegalArgumentException(
+                String.format(
+                    "For featureName: %s, the allowDowngrade flag is not set when the" +
+                    " provided maxVersionLevel:%d is < 1.", featureName, maxVersionLevel));
+        }
+        this.featureName = featureName;
+        this.maxVersionLevel = maxVersionLevel;
+        this.allowDowngrade = allowDowngrade;
+    }
+
+    /**
+     * @return   the name of the finalized feature to be updated.
+     */
+    public String featureName() {
+        return featureName;
+    }
+
+    /**
+     * @return   the new maximum version level for the finalized feature.
+     */
+    public short maxVersionLevel() {
+        return maxVersionLevel;
+    }
+
+    /**
+     * @return   - true, if this feature update was meant to downgrade the maximum version level of
+     *             the finalized feature.
+     *           - false, otherwise.
+     */
+    public boolean allowDowngrade() {
+        return allowDowngrade;
+    }
+
+    /**
+     * Helper function that creates {@link UpdateFinalizedFeaturesRequestData} from a set of
+     * {@link FinalizedFeatureUpdate}.
+     *
+     * @param updates   the set of {@link FinalizedFeatureUpdate}
+     *
+     * @return          a newly constructed UpdateFinalizedFeaturesRequestData object
+     */
+    public static UpdateFinalizedFeaturesRequestData createRequest(Set<FinalizedFeatureUpdate> updates) {
+        final UpdateFinalizedFeaturesRequestData.FinalizedFeatureUpdateKeyCollection items
+            = new UpdateFinalizedFeaturesRequestData.FinalizedFeatureUpdateKeyCollection();
+        for (FinalizedFeatureUpdate update : updates) {
+            final UpdateFinalizedFeaturesRequestData.FinalizedFeatureUpdateKey item =
+                new UpdateFinalizedFeaturesRequestData.FinalizedFeatureUpdateKey();
+            item.setName(update.featureName());
+            item.setMaxVersionLevel(update.maxVersionLevel());
+            item.setAllowDowngrade(update.allowDowngrade());
+            items.add(item);
+        }
+        final UpdateFinalizedFeaturesRequestData data = new UpdateFinalizedFeaturesRequestData();
+        data.setFinalizedFeatureUpdates(items);

Review comment:
       nit: we could just `return new UpdateFinalizedFeaturesRequestData().setFinalizedFeatureUpdates(items)`

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature
+ *    version level deprecation. This is how it works: in order to deprecate feature version levels,
+ *    in this map the default minimum version level of a feature can be set to a new value that's
+ *    higher than 1 (let's call this new_min_version_level). In doing so, the feature version levels
+ *    in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic
+ *    that applies this map to persistent finalized feature state in ZK (this mutation happens
+ *    during controller election and during finalized feature updates via the
+ *    APIKeys.UPDATE_FINALIZED_FEATURES api).

Review comment:
       `ApiKeys.UPDATE_FINALIZED_FEATURES API`

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature
+ *    version level deprecation. This is how it works: in order to deprecate feature version levels,
+ *    in this map the default minimum version level of a feature can be set to a new value that's
+ *    higher than 1 (let's call this new_min_version_level). In doing so, the feature version levels
+ *    in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic
+ *    that applies this map to persistent finalized feature state in ZK (this mutation happens
+ *    during controller election and during finalized feature updates via the
+ *    APIKeys.UPDATE_FINALIZED_FEATURES api).
+ *
+ * This class also provides APIs to check for incompatibilities between the features supported by
+ * the Broker and finalized features.
+ *
+ * NOTE: the update*() and clear*() APIs of this class should be used only for testing purposes.
+ */
+class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange],

Review comment:
       Maybe a newbie question here: since the `supportedFeatures` could be mutated, why couldn't we just assume its min level marks the `defaultFeatureMinVersionLevels`? Trying to understand the necessity for secondary bookkeeping. Might be good to also put reasonings in the meta comment as well to clear confusion.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java
##########
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+import org.apache.kafka.common.feature.Features;
+import org.apache.kafka.common.feature.FinalizedVersionRange;
+import org.apache.kafka.common.feature.SupportedVersionRange;
+
+/**
+ * Encapsulates details about finalized as well as supported features. This is particularly useful
+ * to hold the result returned by the {@link Admin#describeFeatures(DescribeFeaturesOptions)} API.
+ */
+public class FeatureMetadata {
+
+    private final Features<FinalizedVersionRange> finalizedFeatures;
+
+    private final int finalizedFeaturesEpoch;
+
+    private final Features<SupportedVersionRange> supportedFeatures;
+
+    public FeatureMetadata(
+        final Features<FinalizedVersionRange> finalizedFeatures,
+        final int finalizedFeaturesEpoch,
+        final Features<SupportedVersionRange> supportedFeatures
+    ) {
+        Objects.requireNonNull(finalizedFeatures, "Provided finalizedFeatures can not be null.");
+        Objects.requireNonNull(supportedFeatures, "Provided supportedFeatures can not be null.");
+        this.finalizedFeatures = finalizedFeatures;
+        this.finalizedFeaturesEpoch = finalizedFeaturesEpoch;
+        this.supportedFeatures = supportedFeatures;
+    }
+
+    /**
+     * A map of finalized feature versions, with key being finalized feature name and value
+     * containing the min/max version levels for the finalized feature.
+     */
+    public Features<FinalizedVersionRange> finalizedFeatures() {
+        return finalizedFeatures;
+    }
+
+    /**
+     * The epoch for the finalized features.
+     * Valid values are >= 0. A value < 0 means the finalized features are absent/unavailable.

Review comment:
       We should consider using Optional for `finalizedFeaturesEpoch` to indicate absence.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -983,8 +1144,25 @@ class KafkaController(val config: KafkaConfig,
    */
   private[controller] def sendUpdateMetadataRequest(brokers: Seq[Int], partitions: Set[TopicPartition]): Unit = {
     try {
+      val filteredBrokers = scala.collection.mutable.Set[Int]() ++ brokers
+      if (config.isFeatureVersioningEnabled) {
+        def hasIncompatibleFeatures(broker: Broker): Boolean = {
+          val latestFinalizedFeatures = featureCache.get
+          if (latestFinalizedFeatures.isDefined) {
+            BrokerFeatures.hasIncompatibleFeatures(broker.features, latestFinalizedFeatures.get.features)
+          } else {
+            false
+          }
+        }
+        controllerContext.liveOrShuttingDownBrokers.foreach(broker => {
+          if (filteredBrokers.contains(broker.id) && hasIncompatibleFeatures(broker)) {

Review comment:
       Could the receiving broker analyze the request and decide to shut down itself? What's the gain we have by avoiding sending update metadata to incompatible brokers?

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,158 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * Enables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with enabled status. This status means the feature versioning system
+   * (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode are active. This
+   * status should be written by the controller to the FeatureZNode only when the broker IBP config
+   * is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    For a new Kafka cluster (i.e. it is deployed first time), we would like to start the cluster
+   *    with all the possible supported features finalized immediately. The new cluster will almost
+   *    never be started with an old IBP config that’s less than KAFKA_2_7_IV0. Assuming this is the
+   *    case, then here is how we it: the controller will start up and notice that the FeatureZNode
+   *    is absent in the new cluster, it will then create a FeatureZNode (with enabled status)
+   *    containing the entire list of default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. The reason to do this is that enabling all the possible features immediately after
+   *    an upgrade could be harmful to the cluster.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, then it
+   *        will react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, it's existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) => {
+            val updatedVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (updatedVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (existingVersionRange.max() >= updatedVersionRange.min()) {
+              // Through this change, we deprecate all version levels in the closed range:
+              // [existingVersionRange.min(), updatedVersionRange.min() - 1]
+              (featureName, new FinalizedVersionRange(updatedVersionRange.min(), existingVersionRange.max()))
+            } else {
+              // This is a special case: If the existing version levels fall completely outside the
+              // range of the default finalized version levels (i.e. no intersection), then, this
+              // case is not eligible for deprecation. This requires that the max version level be
+              // upgraded first to a value that's equal to the the default minimum version level.
+              info(s"Can not update minimum version level in finalized feature: $featureName,"
+              + s" since the existing $existingVersionRange does not intersect with the default"
+              + s" $updatedVersionRange.")
+              (featureName, existingVersionRange)
+            }
+          }
+        }.asJava)
+      }
+      val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures)
+      if (!newFeatureZNode.equals(existingFeatureZNode)) {
+        val newVersion = updateFeatureZNode(newFeatureZNode)
+        featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+      }
+    }
+  }
+
+  /**
+   * Disables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with disabled status. This status means the feature versioning system
+   * (KIP-584) is disabled, and, the finalized features stored in the FeatureZNode are not relevant.
+   * This status should be written by the controller to the FeatureZNode only when the broker
+   * IBP config is less than KAFKA_2_7_IV0.
+   *
+   * NOTE:
+   * 1. When this method returns, existing finalized features (if any) will be cleared from the
+   *    FeatureZNode.
+   * 2. This method, unlike enableFeatureVersioning() need not wait for the FinalizedFeatureCache
+   *    to be updated, because, such updates to the caceh (via FinalizedFeatureChangeListener)
+   *    are disabled when IBP config is < than KAFKA_2_7_IV0.
+   */
+  private def disableFeatureVersioning(): Unit = {

Review comment:
       Is it ok for us to always do `updateFeatureZNode`, since this call is idempotent?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1214,70 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker, but it can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the DescribeFeaturesResult containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);
+
+    /**
+     * Applies specified updates to finalized features. The API is atomic, meaning that if a single
+     * feature update in the request can't succeed on the controller, then none of the feature
+     * updates are carried out. This request is issued only to the controller since the API is
+     * only served by the controller.
+     * <p>
+     * The API takes as input a set of FinalizedFeatureUpdate that need to be applied. Each such
+     * update specifies the finalized feature to be added or updated or deleted, along with the new
+     * max feature version level value.
+     * <ul>
+     * <li>Downgrade of feature version level is not a regular operation/intent. It is only allowed
+     * in the controller if the feature update has the allowDowngrade flag set - setting this flag
+     * conveys user intent to attempt downgrade of a feature max version level. Note that despite
+     * the allowDowngrade flag being set, certain downgrades may be rejected by the controller if it
+     * is deemed impossible.</li>
+     * <li>Deletion of a finalized feature version is not a regular operation/intent. It is allowed
+     * only if the allowDowngrade flag is set in the feature update, and, if the max version level
+     * is set to a value less than 1.</li>
+     * </ul>
+     *<p>
+     * The following exceptions can be anticipated when calling {@code get()} on the futures
+     * obtained from the returned {@link UpdateFinalizedFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.ClusterAuthorizationException}
+     *   If the authenticated user didn't have alter access to the cluster.</li>
+     *   <li>{@link org.apache.kafka.common.errors.InvalidRequestException}
+     *   If the request details are invalid. e.g., a non-existing finalized feature is attempted

Review comment:
       Looking at `UpdateFinalizedFeaturesResult`, we don't have a per feature based error code returned. If this is the case, how could we know which feature is missing?

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature
+ *    version level deprecation. This is how it works: in order to deprecate feature version levels,
+ *    in this map the default minimum version level of a feature can be set to a new value that's
+ *    higher than 1 (let's call this new_min_version_level). In doing so, the feature version levels
+ *    in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic
+ *    that applies this map to persistent finalized feature state in ZK (this mutation happens
+ *    during controller election and during finalized feature updates via the
+ *    APIKeys.UPDATE_FINALIZED_FEATURES api).
+ *
+ * This class also provides APIs to check for incompatibilities between the features supported by
+ * the Broker and finalized features.
+ *
+ * NOTE: the update*() and clear*() APIs of this class should be used only for testing purposes.
+ */
+class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange],
+                              @volatile var defaultFeatureMinVersionLevels: Map[String, Short]) {
+  require(BrokerFeatures.areFeatureMinVersionLevelsCompatible(
+    supportedFeatures, defaultFeatureMinVersionLevels))
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(newFeatures, defaultFeatureMinVersionLevels))
+    supportedFeatures = newFeatures
+  }
+
+  /**
+   * Returns the default minimum version level for a specific feature.
+   *
+   * @param feature   the name of the feature
+   *
+   * @return          the default minimum version level for the feature if its defined.
+   *                  otherwise, returns 1.
+   */
+  def defaultMinVersionLevel(feature: String): Short = {
+    defaultFeatureMinVersionLevels.getOrElse(feature, 1)
+  }
+
+  // For testing only.
+  def setDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(supportedFeatures, newMinVersionLevels))
+    defaultFeatureMinVersionLevels = newMinVersionLevels
+  }
+
+  /**
+   * Returns the default finalized features that a new Kafka cluster with IBP config >= KAFKA_2_7_IV0
+   * needs to be bootstrapped with.
+   */
+  def getDefaultFinalizedFeatures: Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(
+      supportedFeatures.features.asScala.map {
+        case(name, versionRange) => (
+          name, new FinalizedVersionRange(defaultMinVersionLevel(name), versionRange.max))
+      }.asJava)
+  }
+
+  /**
+   * Returns the set of feature names found to be 'incompatible'.
+   * A feature incompatibility is a version mismatch between the latest feature supported by the
+   * Broker, and the provided finalized feature. This can happen because a provided finalized
+   * feature:
+   *  1) Does not exist in the Broker (i.e. it is unknown to the Broker).
+   *           [OR]
+   *  2) Exists but the FinalizedVersionRange does not match with the
+   *     supported feature's SupportedVersionRange.
+   *
+   * @param finalized   The finalized features against which incompatibilities need to be checked for.
+   *
+   * @return            The subset of input features which are incompatible. If the returned object
+   *                    is empty, it means there were no feature incompatibilities found.
+   */
+  def incompatibleFeatures(finalized: Features[FinalizedVersionRange]): Features[FinalizedVersionRange] = {
+    BrokerFeatures.incompatibleFeatures(supportedFeatures, finalized, true)
+  }
+}
+
+object BrokerFeatures extends Logging {
+
+  def createDefault(): BrokerFeatures = {
+    // The arguments are currently empty, but, in the future as we define features we should
+    // populate the required values here.
+    new BrokerFeatures(emptySupportedFeatures, Map[String, Short]())
+  }
+
+  /**
+   * Returns true if any of the provided finalized features are incompatible with the provided
+   * supported features.
+   *
+   * @param supportedFeatures   The supported features to be compared
+   * @param finalizedFeatures   The finalized features to be compared
+   *
+   * @return                    - True if there are any feature incompatibilities found.
+   *                            - False otherwise.
+   */
+  def hasIncompatibleFeatures(supportedFeatures: Features[SupportedVersionRange],
+                              finalizedFeatures: Features[FinalizedVersionRange]): Boolean = {
+    !incompatibleFeatures(supportedFeatures, finalizedFeatures, false).empty
+  }
+
+  private def incompatibleFeatures(supportedFeatures: Features[SupportedVersionRange],
+                                   finalizedFeatures: Features[FinalizedVersionRange],
+                                   logIncompatibilities: Boolean): Features[FinalizedVersionRange] = {
+    val incompatibilities = finalizedFeatures.features.asScala.map {
+      case (feature, versionLevels) => {
+        val supportedVersions = supportedFeatures.get(feature)
+        if (supportedVersions == null) {
+          (feature, versionLevels, "{feature=%s, reason='Unsupported feature'}".format(feature))
+        } else if (versionLevels.isIncompatibleWith(supportedVersions)) {
+          (feature, versionLevels, "{feature=%s, reason='%s is incompatible with %s'}".format(
+            feature, versionLevels, supportedVersions))
+        } else {
+          (feature, versionLevels, null)
+        }
+      }
+    }.filter{ case(_, _, errorReason) => errorReason != null}.toList
+
+    if (logIncompatibilities && incompatibilities.nonEmpty) {
+      warn(
+        "Feature incompatibilities seen: " + incompatibilities.map{
+          case(_, _, errorReason) => errorReason })
+    }
+    Features.finalizedFeatures(incompatibilities.map{
+      case(feature, versionLevels, _) => (feature, versionLevels) }.toMap.asJava)
+  }
+
+  /**
+   * A check that ensures each feature defined with min version level is a supported feature, and
+   * the min version level value is valid (i.e. it is compatible with the supported version range).
+   *
+   * @param supportedFeatures         the supported features
+   * @param featureMinVersionLevels   the feature minimum version levels
+   *
+   * @return                          - true, if the above described check passes.
+   *                                  - false, otherwise.
+   */
+  private def areFeatureMinVersionLevelsCompatible(
+    supportedFeatures: Features[SupportedVersionRange],
+    featureMinVersionLevels: Map[String, Short]
+  ): Boolean = {
+    featureMinVersionLevels.forall {
+      case(featureName, minVersionLevel) => {

Review comment:
       redundant {}

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FinalizedFeatureUpdate.java
##########
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+import java.util.Set;
+import org.apache.kafka.common.message.UpdateFinalizedFeaturesRequestData;
+
+/**
+ * Encapsulates details about an update to a finalized feature. This is particularly useful to
+ * define each feature update in the
+ * {@link Admin#updateFinalizedFeatures(Set, UpdateFinalizedFeaturesOptions)} API request.
+ */
+public class FinalizedFeatureUpdate {
+    private final String featureName;
+    private final short maxVersionLevel;
+    private final boolean allowDowngrade;
+
+    /**
+     * @param featureName       the name of the finalized feature to be updated.
+     * @param maxVersionLevel   the new maximum version level for the finalized feature.
+     *                          a value < 1 is special and indicates that the update is intended to
+     *                          delete the finalized feature, and should be accompanied by setting
+     *                          the allowDowngrade flag to true.
+     * @param allowDowngrade    - true, if this feature update was meant to downgrade the existing
+     *                            maximum version level of the finalized feature.
+     *                          - false, otherwise.
+     */
+    public FinalizedFeatureUpdate(

Review comment:
       nit: one parameter each line, with the first parameter on the same line as constructor name.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1214,70 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker, but it can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the DescribeFeaturesResult containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);
+
+    /**
+     * Applies specified updates to finalized features. The API is atomic, meaning that if a single
+     * feature update in the request can't succeed on the controller, then none of the feature
+     * updates are carried out. This request is issued only to the controller since the API is
+     * only served by the controller.
+     * <p>
+     * The API takes as input a set of FinalizedFeatureUpdate that need to be applied. Each such
+     * update specifies the finalized feature to be added or updated or deleted, along with the new
+     * max feature version level value.
+     * <ul>
+     * <li>Downgrade of feature version level is not a regular operation/intent. It is only allowed
+     * in the controller if the feature update has the allowDowngrade flag set - setting this flag
+     * conveys user intent to attempt downgrade of a feature max version level. Note that despite
+     * the allowDowngrade flag being set, certain downgrades may be rejected by the controller if it
+     * is deemed impossible.</li>
+     * <li>Deletion of a finalized feature version is not a regular operation/intent. It is allowed

Review comment:
       For the entire sentence, I assume you want to say something like
   ```
   It could be done by turning on the allowDowngrade flag and setting the max version level to be less than 1
   ```

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,158 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * Enables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with enabled status. This status means the feature versioning system
+   * (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode are active. This
+   * status should be written by the controller to the FeatureZNode only when the broker IBP config
+   * is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    For a new Kafka cluster (i.e. it is deployed first time), we would like to start the cluster
+   *    with all the possible supported features finalized immediately. The new cluster will almost
+   *    never be started with an old IBP config that’s less than KAFKA_2_7_IV0. Assuming this is the
+   *    case, then here is how we it: the controller will start up and notice that the FeatureZNode

Review comment:
       `then here is how we it` could be removed:
   `Assuming this is the case, then the controller...`

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -2945,6 +2950,137 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFinalizedFeatures(request: RequestChannel.Request): Unit = {
+    val updateFinalizedFeaturesRequest = request.body[UpdateFinalizedFeaturesRequest]
+    def sendResponseCallback(error: Errors, msgOverride: Option[String]): Unit = {
+      sendResponseExemptThrottle(request, new UpdateFinalizedFeaturesResponse(
+        new UpdateFinalizedFeaturesResponseData()
+          .setErrorCode(error.code())
+          .setErrorMessage(msgOverride.getOrElse(error.message()))))
+    }
+
+    if (!authorize(request.context, ALTER, CLUSTER, CLUSTER_NAME)) {
+      sendResponseCallback(Errors.CLUSTER_AUTHORIZATION_FAILED, Option.empty)
+    } else if (!controller.isActive) {
+      sendResponseCallback(Errors.NOT_CONTROLLER, Option.empty)
+    } else if (!config.isFeatureVersioningEnabled) {
+      sendResponseCallback(Errors.INVALID_REQUEST, Some("Feature versioning system is disabled."))
+    } else {
+      val targetFeaturesOrError = getTargetFinalizedFeaturesOrError(updateFinalizedFeaturesRequest)
+      targetFeaturesOrError match {
+        case Left(targetFeatures) =>
+          controller.updateFinalizedFeatures(targetFeatures, sendResponseCallback)
+        case Right(error) =>
+          sendResponseCallback(error.error, Some(error.message))
+      }
+    }
+  }
+
+  /**
+   * Validates the provided UpdateFinalizedFeaturesRequest, checking for various error cases.
+   * If the validation is successful, returns the target finalized features constructed from the
+   * request.
+   *
+   * @param request   the request to be validated
+   *
+   * @return          - the target finalized features, if request validation is successful
+   *                  - an ApiError if request validation fails
+   */
+  def getTargetFinalizedFeaturesOrError(request: UpdateFinalizedFeaturesRequest):
+    Either[Features[FinalizedVersionRange], ApiError] = {
+    val updates = request.data.finalizedFeatureUpdates
+    if (updates.isEmpty) {
+      return Right(new ApiError(Errors.INVALID_REQUEST,
+        "Can not provide empty FinalizedFeatureUpdates in the request."))
+    }
+
+    val latestFeatures = featureCache.get
+    val newFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]()
+    updates.asScala.foreach(
+      update => {
+        // Rule #1) Check that the feature name is not empty.
+        if (update.name.isEmpty) {

Review comment:
       Commonly in scala we try to avoid using return, consider using `if-else` instead.

##########
File path: clients/src/main/resources/common/message/UpdateFinalizedFeaturesResponse.json
##########
@@ -0,0 +1,28 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "response",
+  "name": "UpdateFinalizedFeaturesResponse",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "ErrorCode", "type": "int16", "versions": "0+",
+      "about": "The error code or `0` if there was no error." },
+    { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+",

Review comment:
       This is a bit unique, since we should commonly rely on the error code to propagate information instead of a message which has unbounded size. Could you explain why we couldn't simply re-invent a new error code if existing ones are not sufficient?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1214,70 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any

Review comment:
       We should suggest in what circumstances a user may require sending the request directly to the controller, to me if there is a case where user wants stronger consistency.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -2945,6 +2950,137 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFinalizedFeatures(request: RequestChannel.Request): Unit = {
+    val updateFinalizedFeaturesRequest = request.body[UpdateFinalizedFeaturesRequest]
+    def sendResponseCallback(error: Errors, msgOverride: Option[String]): Unit = {
+      sendResponseExemptThrottle(request, new UpdateFinalizedFeaturesResponse(
+        new UpdateFinalizedFeaturesResponseData()
+          .setErrorCode(error.code())
+          .setErrorMessage(msgOverride.getOrElse(error.message()))))

Review comment:
       We don't need to include the same error information twice, as the client side will recognize anyway.

##########
File path: core/src/test/scala/unit/kafka/server/UpdateFinalizedFeaturesTest.scala
##########
@@ -0,0 +1,450 @@
+package kafka.server

Review comment:
       missing header

##########
File path: clients/src/main/resources/common/message/UpdateFinalizedFeaturesRequest.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "request",
+  "name": "UpdateFinalizedFeaturesRequest",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "FinalizedFeatureUpdates", "type": "[]FinalizedFeatureUpdateKey", "versions": "0+",
+      "about": "The list of updates to finalized features.", "fields": [
+      {"name": "Name", "type":  "string", "versions":  "0+", "mapKey": true,
+        "about": "The name of the finalized feature to be updated."},
+      {"name":  "MaxVersionLevel", "type":  "int16", "versions":  "0+",

Review comment:
       space

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -82,18 +108,55 @@ object FinalizedFeatureCache extends Logging {
         " The existing cache contents are %s").format(latest, oldFeatureAndEpoch)
       throw new FeatureCacheUpdateException(errorMsg)
     } else {
-      val incompatibleFeatures = SupportedFeatures.incompatibleFeatures(latest.features)
+      val incompatibleFeatures = brokerFeatures.incompatibleFeatures(latest.features)
       if (!incompatibleFeatures.empty) {
         val errorMsg = ("FinalizedFeatureCache update failed since feature compatibility" +
           " checks failed! Supported %s has incompatibilities with the latest %s."
-          ).format(SupportedFeatures.get, latest)
+          ).format(brokerFeatures.supportedFeatures, latest)
         throw new FeatureCacheUpdateException(errorMsg)
       } else {
-        val logMsg = "Updated cache from existing finalized %s to latest finalized %s".format(
+        val logMsg = "Updated cache from existing %s to latest %s".format(
           oldFeatureAndEpoch, latest)
-        featuresAndEpoch = Some(latest)
+        synchronized {
+          featuresAndEpoch = Some(latest)
+          notifyAll()
+        }
         info(logMsg)
       }
     }
   }
+
+  /**
+   * Causes the current thread to wait no more than timeoutMs for the specified condition to be met.
+   * It is guaranteed that the provided condition will always be invoked only from within a
+   * synchronized block.
+   *
+   * @param waitCondition   the condition to be waited upon:
+   *                         - if the condition returns true, then, the wait will stop.
+   *                         - if the condition returns false, it means the wait must continue until
+   *                           timeout.
+   *
+   * @param timeoutMs       the timeout (in milli seconds)
+   *
+   * @throws                TimeoutException if the condition is not met within timeoutMs.
+   */
+  private def waitUntilConditionOrThrow(waitCondition: () => Boolean, timeoutMs: Long): Unit = {
+    if(timeoutMs < 0L) {
+      throw new IllegalArgumentException(s"Expected timeoutMs >= 0, but $timeoutMs was provided.")
+    }
+    synchronized {
+      var sleptTimeMs = 0L
+      while (!waitCondition()) {
+        val timeoutLeftMs = timeoutMs - sleptTimeMs
+        if (timeoutLeftMs <= 0) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${timeoutMs}ms for required condition to be met." +
+              s" Current epoch: ${featuresAndEpoch.map(fe => fe.epoch).getOrElse("<none>")}.")
+        }
+        val timeBeforeNanos = System.nanoTime

Review comment:
       Do we need this precision of exact wait time? Could we just track the function start time and compare with current system time for expiration?

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -177,15 +214,33 @@ public static ApiVersionsResponse createApiVersionsResponse(
             }
         }
 
+        return new ApiVersionsResponse(
+            createApiVersionsResponseData(
+                throttleTimeMs,
+                Errors.NONE,
+                apiKeys,
+                latestSupportedFeatures,
+                finalizedFeatures,
+                finalizedFeaturesEpoch));
+    }
+
+    public static ApiVersionsResponseData createApiVersionsResponseData(

Review comment:
       Seems not necessary to have this helper as it doesn't reduce the code length.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,158 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * Enables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with enabled status. This status means the feature versioning system
+   * (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode are active. This
+   * status should be written by the controller to the FeatureZNode only when the broker IBP config
+   * is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    For a new Kafka cluster (i.e. it is deployed first time), we would like to start the cluster
+   *    with all the possible supported features finalized immediately. The new cluster will almost
+   *    never be started with an old IBP config that’s less than KAFKA_2_7_IV0. Assuming this is the
+   *    case, then here is how we it: the controller will start up and notice that the FeatureZNode
+   *    is absent in the new cluster, it will then create a FeatureZNode (with enabled status)
+   *    containing the entire list of default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. The reason to do this is that enabling all the possible features immediately after
+   *    an upgrade could be harmful to the cluster.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, then it
+   *        will react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, it's existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) => {
+            val updatedVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (updatedVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (existingVersionRange.max() >= updatedVersionRange.min()) {

Review comment:
       What about the case where `existingVersionRange.min() > updatedVersionRange.max()` is true? For example:
   ```
   existing: [4, 5]
   updated: [1, 2]
   ```
   Are we enabling version 3 as well?

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -2945,6 +2950,137 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFinalizedFeatures(request: RequestChannel.Request): Unit = {
+    val updateFinalizedFeaturesRequest = request.body[UpdateFinalizedFeaturesRequest]
+    def sendResponseCallback(error: Errors, msgOverride: Option[String]): Unit = {
+      sendResponseExemptThrottle(request, new UpdateFinalizedFeaturesResponse(
+        new UpdateFinalizedFeaturesResponseData()
+          .setErrorCode(error.code())
+          .setErrorMessage(msgOverride.getOrElse(error.message()))))
+    }
+
+    if (!authorize(request.context, ALTER, CLUSTER, CLUSTER_NAME)) {
+      sendResponseCallback(Errors.CLUSTER_AUTHORIZATION_FAILED, Option.empty)
+    } else if (!controller.isActive) {
+      sendResponseCallback(Errors.NOT_CONTROLLER, Option.empty)
+    } else if (!config.isFeatureVersioningEnabled) {
+      sendResponseCallback(Errors.INVALID_REQUEST, Some("Feature versioning system is disabled."))
+    } else {
+      val targetFeaturesOrError = getTargetFinalizedFeaturesOrError(updateFinalizedFeaturesRequest)
+      targetFeaturesOrError match {
+        case Left(targetFeatures) =>
+          controller.updateFinalizedFeatures(targetFeatures, sendResponseCallback)
+        case Right(error) =>
+          sendResponseCallback(error.error, Some(error.message))
+      }
+    }
+  }
+
+  /**
+   * Validates the provided UpdateFinalizedFeaturesRequest, checking for various error cases.
+   * If the validation is successful, returns the target finalized features constructed from the
+   * request.
+   *
+   * @param request   the request to be validated
+   *
+   * @return          - the target finalized features, if request validation is successful
+   *                  - an ApiError if request validation fails
+   */
+  def getTargetFinalizedFeaturesOrError(request: UpdateFinalizedFeaturesRequest):
+    Either[Features[FinalizedVersionRange], ApiError] = {
+    val updates = request.data.finalizedFeatureUpdates
+    if (updates.isEmpty) {
+      return Right(new ApiError(Errors.INVALID_REQUEST,
+        "Can not provide empty FinalizedFeatureUpdates in the request."))
+    }
+
+    val latestFeatures = featureCache.get
+    val newFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]()
+    updates.asScala.foreach(
+      update => {
+        // Rule #1) Check that the feature name is not empty.
+        if (update.name.isEmpty) {
+          return Right(
+            new ApiError(Errors.INVALID_REQUEST,
+              "Can not contain empty feature name in the request."))
+        }
+
+        val cacheEntry = latestFeatures.map(lf => lf.features.get(update.name)).orNull
+
+        // We handle deletion requests separately from non-deletion requests.
+        if (UpdateFinalizedFeaturesRequest.isDeleteRequest(update)) { // Deletion request
+          // Rule #2) Disallow deletion of a finalized feature without allowDowngrade flag set.
+          if (!update.allowDowngrade) {
+            return Right(
+              new ApiError(Errors.INVALID_REQUEST,
+                s"Can not delete feature: '${update.name}' without setting the" +
+                  " allowDowngrade flag in the request."))

Review comment:
       `setting the allowDowngrade flag to true in the request`

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature
+ *    version level deprecation. This is how it works: in order to deprecate feature version levels,
+ *    in this map the default minimum version level of a feature can be set to a new value that's
+ *    higher than 1 (let's call this new_min_version_level). In doing so, the feature version levels
+ *    in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic
+ *    that applies this map to persistent finalized feature state in ZK (this mutation happens
+ *    during controller election and during finalized feature updates via the
+ *    APIKeys.UPDATE_FINALIZED_FEATURES api).
+ *
+ * This class also provides APIs to check for incompatibilities between the features supported by
+ * the Broker and finalized features.
+ *
+ * NOTE: the update*() and clear*() APIs of this class should be used only for testing purposes.
+ */
+class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange],
+                              @volatile var defaultFeatureMinVersionLevels: Map[String, Short]) {
+  require(BrokerFeatures.areFeatureMinVersionLevelsCompatible(
+    supportedFeatures, defaultFeatureMinVersionLevels))
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(newFeatures, defaultFeatureMinVersionLevels))
+    supportedFeatures = newFeatures
+  }
+
+  /**
+   * Returns the default minimum version level for a specific feature.
+   *
+   * @param feature   the name of the feature
+   *
+   * @return          the default minimum version level for the feature if its defined.
+   *                  otherwise, returns 1.
+   */
+  def defaultMinVersionLevel(feature: String): Short = {
+    defaultFeatureMinVersionLevels.getOrElse(feature, 1)
+  }
+
+  // For testing only.
+  def setDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(supportedFeatures, newMinVersionLevels))
+    defaultFeatureMinVersionLevels = newMinVersionLevels
+  }
+
+  /**
+   * Returns the default finalized features that a new Kafka cluster with IBP config >= KAFKA_2_7_IV0
+   * needs to be bootstrapped with.
+   */
+  def getDefaultFinalizedFeatures: Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(
+      supportedFeatures.features.asScala.map {
+        case(name, versionRange) => (
+          name, new FinalizedVersionRange(defaultMinVersionLevel(name), versionRange.max))
+      }.asJava)
+  }
+
+  /**
+   * Returns the set of feature names found to be 'incompatible'.
+   * A feature incompatibility is a version mismatch between the latest feature supported by the
+   * Broker, and the provided finalized feature. This can happen because a provided finalized
+   * feature:
+   *  1) Does not exist in the Broker (i.e. it is unknown to the Broker).
+   *           [OR]
+   *  2) Exists but the FinalizedVersionRange does not match with the
+   *     supported feature's SupportedVersionRange.
+   *
+   * @param finalized   The finalized features against which incompatibilities need to be checked for.
+   *
+   * @return            The subset of input features which are incompatible. If the returned object
+   *                    is empty, it means there were no feature incompatibilities found.
+   */
+  def incompatibleFeatures(finalized: Features[FinalizedVersionRange]): Features[FinalizedVersionRange] = {
+    BrokerFeatures.incompatibleFeatures(supportedFeatures, finalized, true)
+  }
+}
+
+object BrokerFeatures extends Logging {
+
+  def createDefault(): BrokerFeatures = {
+    // The arguments are currently empty, but, in the future as we define features we should
+    // populate the required values here.
+    new BrokerFeatures(emptySupportedFeatures, Map[String, Short]())
+  }
+
+  /**
+   * Returns true if any of the provided finalized features are incompatible with the provided
+   * supported features.
+   *
+   * @param supportedFeatures   The supported features to be compared
+   * @param finalizedFeatures   The finalized features to be compared
+   *
+   * @return                    - True if there are any feature incompatibilities found.
+   *                            - False otherwise.
+   */
+  def hasIncompatibleFeatures(supportedFeatures: Features[SupportedVersionRange],
+                              finalizedFeatures: Features[FinalizedVersionRange]): Boolean = {
+    !incompatibleFeatures(supportedFeatures, finalizedFeatures, false).empty
+  }
+
+  private def incompatibleFeatures(supportedFeatures: Features[SupportedVersionRange],
+                                   finalizedFeatures: Features[FinalizedVersionRange],
+                                   logIncompatibilities: Boolean): Features[FinalizedVersionRange] = {
+    val incompatibilities = finalizedFeatures.features.asScala.map {
+      case (feature, versionLevels) => {

Review comment:
       redundant {}

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -2945,6 +2950,137 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFinalizedFeatures(request: RequestChannel.Request): Unit = {
+    val updateFinalizedFeaturesRequest = request.body[UpdateFinalizedFeaturesRequest]
+    def sendResponseCallback(error: Errors, msgOverride: Option[String]): Unit = {
+      sendResponseExemptThrottle(request, new UpdateFinalizedFeaturesResponse(
+        new UpdateFinalizedFeaturesResponseData()
+          .setErrorCode(error.code())
+          .setErrorMessage(msgOverride.getOrElse(error.message()))))
+    }
+
+    if (!authorize(request.context, ALTER, CLUSTER, CLUSTER_NAME)) {
+      sendResponseCallback(Errors.CLUSTER_AUTHORIZATION_FAILED, Option.empty)
+    } else if (!controller.isActive) {
+      sendResponseCallback(Errors.NOT_CONTROLLER, Option.empty)
+    } else if (!config.isFeatureVersioningEnabled) {
+      sendResponseCallback(Errors.INVALID_REQUEST, Some("Feature versioning system is disabled."))
+    } else {
+      val targetFeaturesOrError = getTargetFinalizedFeaturesOrError(updateFinalizedFeaturesRequest)
+      targetFeaturesOrError match {
+        case Left(targetFeatures) =>
+          controller.updateFinalizedFeatures(targetFeatures, sendResponseCallback)
+        case Right(error) =>
+          sendResponseCallback(error.error, Some(error.message))
+      }
+    }
+  }
+
+  /**
+   * Validates the provided UpdateFinalizedFeaturesRequest, checking for various error cases.
+   * If the validation is successful, returns the target finalized features constructed from the
+   * request.
+   *
+   * @param request   the request to be validated
+   *
+   * @return          - the target finalized features, if request validation is successful
+   *                  - an ApiError if request validation fails
+   */
+  def getTargetFinalizedFeaturesOrError(request: UpdateFinalizedFeaturesRequest):
+    Either[Features[FinalizedVersionRange], ApiError] = {
+    val updates = request.data.finalizedFeatureUpdates
+    if (updates.isEmpty) {
+      return Right(new ApiError(Errors.INVALID_REQUEST,
+        "Can not provide empty FinalizedFeatureUpdates in the request."))
+    }
+
+    val latestFeatures = featureCache.get
+    val newFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]()
+    updates.asScala.foreach(
+      update => {
+        // Rule #1) Check that the feature name is not empty.
+        if (update.name.isEmpty) {
+          return Right(
+            new ApiError(Errors.INVALID_REQUEST,
+              "Can not contain empty feature name in the request."))
+        }
+
+        val cacheEntry = latestFeatures.map(lf => lf.features.get(update.name)).orNull
+
+        // We handle deletion requests separately from non-deletion requests.
+        if (UpdateFinalizedFeaturesRequest.isDeleteRequest(update)) { // Deletion request

Review comment:
       nit: the comment seems unnecessary on L3011

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1214,70 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker, but it can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the DescribeFeaturesResult containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);
+
+    /**
+     * Applies specified updates to finalized features. The API is atomic, meaning that if a single
+     * feature update in the request can't succeed on the controller, then none of the feature
+     * updates are carried out. This request is issued only to the controller since the API is
+     * only served by the controller.
+     * <p>
+     * The API takes as input a set of FinalizedFeatureUpdate that need to be applied. Each such
+     * update specifies the finalized feature to be added or updated or deleted, along with the new
+     * max feature version level value.
+     * <ul>
+     * <li>Downgrade of feature version level is not a regular operation/intent. It is only allowed
+     * in the controller if the feature update has the allowDowngrade flag set - setting this flag
+     * conveys user intent to attempt downgrade of a feature max version level. Note that despite
+     * the allowDowngrade flag being set, certain downgrades may be rejected by the controller if it
+     * is deemed impossible.</li>
+     * <li>Deletion of a finalized feature version is not a regular operation/intent. It is allowed
+     * only if the allowDowngrade flag is set in the feature update, and, if the max version level
+     * is set to a value less than 1.</li>
+     * </ul>
+     *<p>
+     * The following exceptions can be anticipated when calling {@code get()} on the futures
+     * obtained from the returned {@link UpdateFinalizedFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.ClusterAuthorizationException}
+     *   If the authenticated user didn't have alter access to the cluster.</li>
+     *   <li>{@link org.apache.kafka.common.errors.InvalidRequestException}
+     *   If the request details are invalid. e.g., a non-existing finalized feature is attempted
+     *   to be deleted or downgraded.</li>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the updates could finish. It cannot be guaranteed whether
+     *   the updates succeeded or not.</li>
+     *   <li>{@link org.apache.kafka.common.errors.FinalizedFeatureUpdateFailedException}
+     *   If the updates could not be applied on the controller, despite the request being valid.
+     *   This may be a temporary problem.</li>
+     * </ul>
+     * <p>
+     * This operation is supported by brokers with version 2.7.0 or higher.
+
+     * @param featureUpdates   the set of finalized feature updates
+     * @param options          the options to use
+     *
+     * @return                 the UpdateFinalizedFeaturesResult containing the result
+     */
+    UpdateFinalizedFeaturesResult updateFinalizedFeatures(

Review comment:
       The definition seems not aligned with the KIP which states `updateFeatures`, do you think it's necessary to mention `finalized` in all the function signatures?

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -2945,6 +2950,137 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFinalizedFeatures(request: RequestChannel.Request): Unit = {
+    val updateFinalizedFeaturesRequest = request.body[UpdateFinalizedFeaturesRequest]
+    def sendResponseCallback(error: Errors, msgOverride: Option[String]): Unit = {
+      sendResponseExemptThrottle(request, new UpdateFinalizedFeaturesResponse(
+        new UpdateFinalizedFeaturesResponseData()
+          .setErrorCode(error.code())
+          .setErrorMessage(msgOverride.getOrElse(error.message()))))
+    }
+
+    if (!authorize(request.context, ALTER, CLUSTER, CLUSTER_NAME)) {
+      sendResponseCallback(Errors.CLUSTER_AUTHORIZATION_FAILED, Option.empty)
+    } else if (!controller.isActive) {
+      sendResponseCallback(Errors.NOT_CONTROLLER, Option.empty)
+    } else if (!config.isFeatureVersioningEnabled) {
+      sendResponseCallback(Errors.INVALID_REQUEST, Some("Feature versioning system is disabled."))
+    } else {
+      val targetFeaturesOrError = getTargetFinalizedFeaturesOrError(updateFinalizedFeaturesRequest)
+      targetFeaturesOrError match {
+        case Left(targetFeatures) =>
+          controller.updateFinalizedFeatures(targetFeatures, sendResponseCallback)
+        case Right(error) =>
+          sendResponseCallback(error.error, Some(error.message))
+      }
+    }
+  }
+
+  /**
+   * Validates the provided UpdateFinalizedFeaturesRequest, checking for various error cases.
+   * If the validation is successful, returns the target finalized features constructed from the
+   * request.
+   *
+   * @param request   the request to be validated
+   *
+   * @return          - the target finalized features, if request validation is successful
+   *                  - an ApiError if request validation fails
+   */
+  def getTargetFinalizedFeaturesOrError(request: UpdateFinalizedFeaturesRequest):

Review comment:
       access should be private

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1214,70 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker, but it can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the DescribeFeaturesResult containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);
+
+    /**
+     * Applies specified updates to finalized features. The API is atomic, meaning that if a single
+     * feature update in the request can't succeed on the controller, then none of the feature
+     * updates are carried out. This request is issued only to the controller since the API is
+     * only served by the controller.
+     * <p>
+     * The API takes as input a set of FinalizedFeatureUpdate that need to be applied. Each such
+     * update specifies the finalized feature to be added or updated or deleted, along with the new
+     * max feature version level value.
+     * <ul>
+     * <li>Downgrade of feature version level is not a regular operation/intent. It is only allowed
+     * in the controller if the feature update has the allowDowngrade flag set - setting this flag
+     * conveys user intent to attempt downgrade of a feature max version level. Note that despite
+     * the allowDowngrade flag being set, certain downgrades may be rejected by the controller if it
+     * is deemed impossible.</li>
+     * <li>Deletion of a finalized feature version is not a regular operation/intent. It is allowed
+     * only if the allowDowngrade flag is set in the feature update, and, if the max version level
+     * is set to a value less than 1.</li>
+     * </ul>
+     *<p>
+     * The following exceptions can be anticipated when calling {@code get()} on the futures
+     * obtained from the returned {@link UpdateFinalizedFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.ClusterAuthorizationException}
+     *   If the authenticated user didn't have alter access to the cluster.</li>
+     *   <li>{@link org.apache.kafka.common.errors.InvalidRequestException}
+     *   If the request details are invalid. e.g., a non-existing finalized feature is attempted
+     *   to be deleted or downgraded.</li>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the updates could finish. It cannot be guaranteed whether
+     *   the updates succeeded or not.</li>
+     *   <li>{@link org.apache.kafka.common.errors.FinalizedFeatureUpdateFailedException}
+     *   If the updates could not be applied on the controller, despite the request being valid.
+     *   This may be a temporary problem.</li>
+     * </ul>
+     * <p>
+     * This operation is supported by brokers with version 2.7.0 or higher.
+
+     * @param featureUpdates   the set of finalized feature updates
+     * @param options          the options to use
+     *
+     * @return                 the UpdateFinalizedFeaturesResult containing the result
+     */
+    UpdateFinalizedFeaturesResult updateFinalizedFeatures(

Review comment:
       nit: put first parameter on this line.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/UpdateFinalizedFeaturesResponse.java
##########
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+import org.apache.kafka.common.message.UpdateFinalizedFeaturesResponseData;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.Struct;
+
+public class UpdateFinalizedFeaturesResponse extends AbstractResponse {

Review comment:
       Would be good to redundantly copy over the expected error codes from `Admin.java` definition, similar to other response class such as `OffsetCommitResponse`

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1825,36 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def processUpdateFinalizedFeatures(newFeatures: Features[FinalizedVersionRange],
+                                             callback: UpdateFinalizedFeaturesCallback): Unit = {
+    if (isActive) {
+      val incompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.filter(broker => {
+        BrokerFeatures.hasIncompatibleFeatures(broker.features, newFeatures)
+      })
+      if (incompatibleBrokers.size > 0) {
+        callback(
+          Errors.INVALID_REQUEST,
+          Some(
+            s"Could not apply finalized feature updates because ${incompatibleBrokers.size} brokers"
+            + s" were found to have incompatible features. newFeatures: $newFeatures"
+           + s", incompatibleBrokers: $incompatibleBrokers."))
+      } else {
+        try {
+          val newVersion = zkClient.updateFeatureZNode(
+            new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures))
+          featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)

Review comment:
       Could we avoid blocking controller processing here, by putting the callback into a delayed queue or sth?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -3984,6 +3988,108 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        Call callViaLeastLoadedNode = new Call("describeFeatures", calcDeadlineMs(now, options.timeoutMs()),
+            new LeastLoadedNodeProvider()) {
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(
+                        new FeatureMetadata(
+                            apiVersionsResponse.finalizedFeatures(),
+                            apiVersionsResponse.finalizedFeaturesEpoch(),
+                            apiVersionsResponse.supportedFeatures()));
+                } else {
+                    future.completeExceptionally(
+                        Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        Call call = callViaLeastLoadedNode;
+        if (options.sendRequestToController()) {
+            call = new Call("describeFeatures", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {
+
+                @Override
+                ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                    return (ApiVersionsRequest.Builder) callViaLeastLoadedNode.createRequest(timeoutMs);
+                }
+
+                @Override
+                void handleResponse(AbstractResponse response) {
+                    final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                    if (apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) {
+                        handleNotControllerError(Errors.NOT_CONTROLLER);
+                    } else {
+                        callViaLeastLoadedNode.handleResponse(response);

Review comment:
       It looks weird to complete `callViaLeastLoadedNode` in a controller response handler. I'm inclined to increase a bit on the code duplication, based on `if (options.sendRequestToController())` to have two separate request traces like:
   ```
   if (options.sendRequestToController()) {
               ...
               runnable.call(callControllerNode, now);
           } else {
               ...
               runnable.call(callViaLeastLoadedNode, now);
           }
   ```
   and try to complete the same future.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,158 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * Enables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with enabled status. This status means the feature versioning system
+   * (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode are active. This
+   * status should be written by the controller to the FeatureZNode only when the broker IBP config
+   * is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    For a new Kafka cluster (i.e. it is deployed first time), we would like to start the cluster
+   *    with all the possible supported features finalized immediately. The new cluster will almost
+   *    never be started with an old IBP config that’s less than KAFKA_2_7_IV0. Assuming this is the
+   *    case, then here is how we it: the controller will start up and notice that the FeatureZNode
+   *    is absent in the new cluster, it will then create a FeatureZNode (with enabled status)
+   *    containing the entire list of default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. The reason to do this is that enabling all the possible features immediately after
+   *    an upgrade could be harmful to the cluster.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, then it
+   *        will react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, it's existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) => {
+            val updatedVersionRange = defaultFinalizedFeatures.get(featureName)

Review comment:
       If this is broker required feature set, I feel we could name it something like `brokerRequiredVersionRange`. `Updated` sounds a bit blur for reader, as it couldn't infer the subject.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1825,36 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def processUpdateFinalizedFeatures(newFeatures: Features[FinalizedVersionRange],
+                                             callback: UpdateFinalizedFeaturesCallback): Unit = {
+    if (isActive) {
+      val incompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.filter(broker => {
+        BrokerFeatures.hasIncompatibleFeatures(broker.features, newFeatures)
+      })
+      if (incompatibleBrokers.size > 0) {

Review comment:
       Replace with `nonEmpty`

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature
+ *    version level deprecation. This is how it works: in order to deprecate feature version levels,
+ *    in this map the default minimum version level of a feature can be set to a new value that's
+ *    higher than 1 (let's call this new_min_version_level). In doing so, the feature version levels
+ *    in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic
+ *    that applies this map to persistent finalized feature state in ZK (this mutation happens
+ *    during controller election and during finalized feature updates via the
+ *    APIKeys.UPDATE_FINALIZED_FEATURES api).
+ *
+ * This class also provides APIs to check for incompatibilities between the features supported by
+ * the Broker and finalized features.
+ *
+ * NOTE: the update*() and clear*() APIs of this class should be used only for testing purposes.

Review comment:
       We only need to mark testing only comment on the functions 

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,158 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * Enables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with enabled status. This status means the feature versioning system
+   * (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode are active. This
+   * status should be written by the controller to the FeatureZNode only when the broker IBP config
+   * is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    For a new Kafka cluster (i.e. it is deployed first time), we would like to start the cluster
+   *    with all the possible supported features finalized immediately. The new cluster will almost

Review comment:
       `For a new Kafka cluster (i.e. it is deployed first time), we would like to start the cluster with all the possible supported features finalized immediately.`
   I think this comment is hard to understand if reader has zero context on the feature versioning. It would be good to include a short explanation on what does a `supported feature` mean, and what it means to be `finalized`.
   `The new cluster will almost never be started with an old IBP config that’s less than KAFKA_2_7_IV0.`
   This sentence is positioned awkwardly. I would suggest we just propose `As a new cluster starting with IBP setting equal to or greater than KAFKA_2_7_IV0`

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature
+ *    version level deprecation. This is how it works: in order to deprecate feature version levels,
+ *    in this map the default minimum version level of a feature can be set to a new value that's
+ *    higher than 1 (let's call this new_min_version_level). In doing so, the feature version levels
+ *    in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic
+ *    that applies this map to persistent finalized feature state in ZK (this mutation happens
+ *    during controller election and during finalized feature updates via the
+ *    APIKeys.UPDATE_FINALIZED_FEATURES api).
+ *
+ * This class also provides APIs to check for incompatibilities between the features supported by
+ * the Broker and finalized features.
+ *
+ * NOTE: the update*() and clear*() APIs of this class should be used only for testing purposes.
+ */
+class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange],
+                              @volatile var defaultFeatureMinVersionLevels: Map[String, Short]) {
+  require(BrokerFeatures.areFeatureMinVersionLevelsCompatible(
+    supportedFeatures, defaultFeatureMinVersionLevels))
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(newFeatures, defaultFeatureMinVersionLevels))
+    supportedFeatures = newFeatures
+  }
+
+  /**
+   * Returns the default minimum version level for a specific feature.
+   *
+   * @param feature   the name of the feature
+   *
+   * @return          the default minimum version level for the feature if its defined.
+   *                  otherwise, returns 1.
+   */
+  def defaultMinVersionLevel(feature: String): Short = {
+    defaultFeatureMinVersionLevels.getOrElse(feature, 1)
+  }
+
+  // For testing only.
+  def setDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(supportedFeatures, newMinVersionLevels))
+    defaultFeatureMinVersionLevels = newMinVersionLevels
+  }
+
+  /**
+   * Returns the default finalized features that a new Kafka cluster with IBP config >= KAFKA_2_7_IV0
+   * needs to be bootstrapped with.
+   */
+  def getDefaultFinalizedFeatures: Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(
+      supportedFeatures.features.asScala.map {
+        case(name, versionRange) => (
+          name, new FinalizedVersionRange(defaultMinVersionLevel(name), versionRange.max))
+      }.asJava)
+  }
+
+  /**
+   * Returns the set of feature names found to be 'incompatible'.
+   * A feature incompatibility is a version mismatch between the latest feature supported by the
+   * Broker, and the provided finalized feature. This can happen because a provided finalized
+   * feature:
+   *  1) Does not exist in the Broker (i.e. it is unknown to the Broker).
+   *           [OR]
+   *  2) Exists but the FinalizedVersionRange does not match with the
+   *     supported feature's SupportedVersionRange.
+   *
+   * @param finalized   The finalized features against which incompatibilities need to be checked for.
+   *
+   * @return            The subset of input features which are incompatible. If the returned object
+   *                    is empty, it means there were no feature incompatibilities found.
+   */
+  def incompatibleFeatures(finalized: Features[FinalizedVersionRange]): Features[FinalizedVersionRange] = {
+    BrokerFeatures.incompatibleFeatures(supportedFeatures, finalized, true)
+  }
+}
+
+object BrokerFeatures extends Logging {
+
+  def createDefault(): BrokerFeatures = {
+    // The arguments are currently empty, but, in the future as we define features we should
+    // populate the required values here.
+    new BrokerFeatures(emptySupportedFeatures, Map[String, Short]())
+  }
+
+  /**
+   * Returns true if any of the provided finalized features are incompatible with the provided
+   * supported features.
+   *
+   * @param supportedFeatures   The supported features to be compared
+   * @param finalizedFeatures   The finalized features to be compared
+   *
+   * @return                    - True if there are any feature incompatibilities found.
+   *                            - False otherwise.
+   */
+  def hasIncompatibleFeatures(supportedFeatures: Features[SupportedVersionRange],
+                              finalizedFeatures: Features[FinalizedVersionRange]): Boolean = {
+    !incompatibleFeatures(supportedFeatures, finalizedFeatures, false).empty
+  }
+
+  private def incompatibleFeatures(supportedFeatures: Features[SupportedVersionRange],
+                                   finalizedFeatures: Features[FinalizedVersionRange],
+                                   logIncompatibilities: Boolean): Features[FinalizedVersionRange] = {
+    val incompatibilities = finalizedFeatures.features.asScala.map {

Review comment:
       `incompatibleFeatures`?

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature
+ *    version level deprecation. This is how it works: in order to deprecate feature version levels,
+ *    in this map the default minimum version level of a feature can be set to a new value that's
+ *    higher than 1 (let's call this new_min_version_level). In doing so, the feature version levels
+ *    in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic
+ *    that applies this map to persistent finalized feature state in ZK (this mutation happens
+ *    during controller election and during finalized feature updates via the
+ *    APIKeys.UPDATE_FINALIZED_FEATURES api).
+ *
+ * This class also provides APIs to check for incompatibilities between the features supported by
+ * the Broker and finalized features.
+ *
+ * NOTE: the update*() and clear*() APIs of this class should be used only for testing purposes.
+ */
+class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange],
+                              @volatile var defaultFeatureMinVersionLevels: Map[String, Short]) {
+  require(BrokerFeatures.areFeatureMinVersionLevelsCompatible(
+    supportedFeatures, defaultFeatureMinVersionLevels))
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(newFeatures, defaultFeatureMinVersionLevels))
+    supportedFeatures = newFeatures
+  }
+
+  /**
+   * Returns the default minimum version level for a specific feature.
+   *
+   * @param feature   the name of the feature
+   *
+   * @return          the default minimum version level for the feature if its defined.
+   *                  otherwise, returns 1.
+   */
+  def defaultMinVersionLevel(feature: String): Short = {
+    defaultFeatureMinVersionLevels.getOrElse(feature, 1)
+  }
+
+  // For testing only.
+  def setDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(supportedFeatures, newMinVersionLevels))
+    defaultFeatureMinVersionLevels = newMinVersionLevels
+  }
+
+  /**
+   * Returns the default finalized features that a new Kafka cluster with IBP config >= KAFKA_2_7_IV0
+   * needs to be bootstrapped with.
+   */
+  def getDefaultFinalizedFeatures: Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(
+      supportedFeatures.features.asScala.map {
+        case(name, versionRange) => (
+          name, new FinalizedVersionRange(defaultMinVersionLevel(name), versionRange.max))
+      }.asJava)
+  }
+
+  /**
+   * Returns the set of feature names found to be 'incompatible'.
+   * A feature incompatibility is a version mismatch between the latest feature supported by the
+   * Broker, and the provided finalized feature. This can happen because a provided finalized
+   * feature:
+   *  1) Does not exist in the Broker (i.e. it is unknown to the Broker).
+   *           [OR]
+   *  2) Exists but the FinalizedVersionRange does not match with the
+   *     supported feature's SupportedVersionRange.
+   *
+   * @param finalized   The finalized features against which incompatibilities need to be checked for.
+   *
+   * @return            The subset of input features which are incompatible. If the returned object
+   *                    is empty, it means there were no feature incompatibilities found.
+   */
+  def incompatibleFeatures(finalized: Features[FinalizedVersionRange]): Features[FinalizedVersionRange] = {
+    BrokerFeatures.incompatibleFeatures(supportedFeatures, finalized, true)
+  }
+}
+
+object BrokerFeatures extends Logging {
+
+  def createDefault(): BrokerFeatures = {
+    // The arguments are currently empty, but, in the future as we define features we should
+    // populate the required values here.
+    new BrokerFeatures(emptySupportedFeatures, Map[String, Short]())
+  }
+
+  /**
+   * Returns true if any of the provided finalized features are incompatible with the provided
+   * supported features.
+   *
+   * @param supportedFeatures   The supported features to be compared
+   * @param finalizedFeatures   The finalized features to be compared
+   *
+   * @return                    - True if there are any feature incompatibilities found.
+   *                            - False otherwise.
+   */
+  def hasIncompatibleFeatures(supportedFeatures: Features[SupportedVersionRange],
+                              finalizedFeatures: Features[FinalizedVersionRange]): Boolean = {
+    !incompatibleFeatures(supportedFeatures, finalizedFeatures, false).empty
+  }
+
+  private def incompatibleFeatures(supportedFeatures: Features[SupportedVersionRange],
+                                   finalizedFeatures: Features[FinalizedVersionRange],
+                                   logIncompatibilities: Boolean): Features[FinalizedVersionRange] = {
+    val incompatibilities = finalizedFeatures.features.asScala.map {
+      case (feature, versionLevels) => {
+        val supportedVersions = supportedFeatures.get(feature)
+        if (supportedVersions == null) {
+          (feature, versionLevels, "{feature=%s, reason='Unsupported feature'}".format(feature))
+        } else if (versionLevels.isIncompatibleWith(supportedVersions)) {
+          (feature, versionLevels, "{feature=%s, reason='%s is incompatible with %s'}".format(
+            feature, versionLevels, supportedVersions))
+        } else {
+          (feature, versionLevels, null)
+        }
+      }
+    }.filter{ case(_, _, errorReason) => errorReason != null}.toList
+
+    if (logIncompatibilities && incompatibilities.nonEmpty) {
+      warn(
+        "Feature incompatibilities seen: " + incompatibilities.map{

Review comment:
       nit: I have seen that we use both `map{` and `map {`, could we try using only one format consistently within the current file?




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -272,6 +281,147 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (string) and a range of versions (defined by a
+   * SupportedVersionRange). It refers to a feature that a particular broker advertises support for.
+   * Each broker advertises the version ranges of its own supported features in its own
+   * BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (string) and a range of version levels (defined
+   * by a FinalizedVersionRange). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there was an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has now been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). But the IBP config is still set to lower than KAFKA_2_7_IV0, and may be
+   *    set to a higher value later. In this case, we want to start with no finalized features and
+   *    allow the user to finalize them whenever they are ready i.e. in the future whenever the
+   *    user sets IBP config to be greater than or equal to KAFKA_2_7_IV0, then the user could start
+   *    finalizing the features. This process ensures we do not enable all the possible features
+   *    immediately after an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent.
+   *        - If the node is absent, it will react by creating a FeatureZNode with disabled status
+   *          and empty finalized features.
+   *        - Otherwise, if a node already exists in enabled status then the controller will just
+   *          flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled.
+   *         - If the node is in disabled status, the controller won’t upgrade all features immediately.
+   *           Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *           user finalize the features later.
+   *         - Otherwise, if a node already exists in enabled status then the controller will leave
+   *           the node umodified.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine there was an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker
+   *    binary has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and
+   *    higher). The controller will start up and find that a FeatureZNode is already present with
+   *    enabled status and existing finalized features. In such a case, the controller leaves the node
+   *    unmodified.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled,
+                                          brokerFeatures.defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      if (!existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {

Review comment:
       This test may not be enough. The issue is that when a controller fails over, it's possible that new brokers have joined the cluster during the failover. So, if existingFeatureZNode is enabled, it may not be reflecting the state in those newly joined brokers. So, it seems that we need to do the validation for every broker during controller failover in that case.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1656,6 +1910,204 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns an ApiError object containing Errors.INVALID_REQUEST.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+
+    val supportedVersionRange = brokerFeatures.supportedFeatures.get(update.feature)
+    if (supportedVersionRange == null) {
+      Right(new ApiError(Errors.INVALID_REQUEST,
+                         "Could not apply finalized feature update because the provided feature" +
+                         " is not supported."))
+    } else {
+      var newVersionRange: FinalizedVersionRange = null
+      try {
+        newVersionRange = new FinalizedVersionRange(supportedVersionRange.firstActiveVersion, update.maxVersionLevel)
+      } catch {
+        case _: IllegalArgumentException => {
+          // This exception means the provided maxVersionLevel is invalid. It is handled below
+          // outside of this catch clause.
+        }
+      }
+      if (newVersionRange == null) {
+        Right(new ApiError(Errors.INVALID_REQUEST,
+          "Could not apply finalized feature update because the provided" +
+          s" maxVersionLevel:${update.maxVersionLevel} is lower than the" +
+          s" first active version:${supportedVersionRange.firstActiveVersion}."))
+      } else {
+        val newFinalizedFeature =
+          Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange)))
+        val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+          BrokerFeatures.hasIncompatibleFeatures(broker.features, newFinalizedFeature)
+        })
+        if (numIncompatibleBrokers == 0) {
+          Left(newVersionRange)
+        } else {
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Could not apply finalized feature update because" +
+                             " brokers were found to have incompatible versions for the feature."))
+        }
+      }
+    }
+  }
+
+  /**
+   * Validates a feature update on an existing FinalizedVersionRange.
+   * If the validation succeeds, then, the return value contains:
+   * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature.
+   * 2. Option.empty, if the feature update was meant to delete the feature.
+   *
+   * If the validation fails, then returned value contains a suitable ApiError.
+   *
+   * @param update                 the feature update to be processed.
+   * @param existingVersionRange   the existing FinalizedVersionRange which can be empty when no
+   *                               FinalizedVersionRange exists for the associated feature
+   *
+   * @return                       the new FinalizedVersionRange to be updated into ZK or error
+   *                               as described above.
+   */
+  private def validateFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey,
+                                    existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = {
+    def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = {
+      newFinalizedVersionRangeOrIncompatibilityError(update)
+        .fold(versionRange => Left(Some(versionRange)), error => Right(error))
+    }
+
+    if (update.feature.isEmpty) {
+      // Check that the feature name is not empty.
+      Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty."))
+    } else {
+      // We handle deletion requests separately from non-deletion requests.
+      if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+        if (existingVersionRange.isEmpty) {
+          // Disallow deletion of a non-existing finalized feature.
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Can not delete non-existing finalized feature."))
+        } else {
+          Left(Option.empty)
+        }
+      } else if (update.maxVersionLevel() < 1) {
+        // Disallow deletion of a finalized feature without allowDowngrade flag set.
+        Right(new ApiError(Errors.INVALID_REQUEST,
+                           s"Can not provide maxVersionLevel: ${update.maxVersionLevel} less" +
+                           s" than 1 without setting the allowDowngrade flag to true in the request."))
+      } else {
+        existingVersionRange.map(existing =>
+          if (update.maxVersionLevel == existing.max) {
+            // Disallow a case where target maxVersionLevel matches existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not ${if (update.allowDowngrade) "downgrade" else "upgrade"}" +
+                               s" a finalized feature from existing maxVersionLevel:${existing.max}" +
+                               " to the same value."))
+          } else if (update.maxVersionLevel < existing.max && !update.allowDowngrade) {
+            // Disallow downgrade of a finalized feature without the allowDowngrade flag set.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature from existing" +
+                               s" maxVersionLevel:${existing.max} to provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} without setting the" +
+                               " allowDowngrade flag in the request."))
+          } else if (update.allowDowngrade && update.maxVersionLevel > existing.max) {
+            // Disallow a request that sets allowDowngrade flag without specifying a
+            // maxVersionLevel that's lower than the existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"When the allowDowngrade flag set in the request, the provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} can not be greater than" +
+                               s" existing maxVersionLevel:${existing.max}."))
+          } else if (update.maxVersionLevel < existing.min) {
+            // Disallow downgrade of a finalized feature below the existing finalized
+            // minVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature to maxVersionLevel:${update.maxVersionLevel}" +
+                               s" because it's lower than the existing minVersionLevel:${existing.min}."))
+          } else {
+            newVersionRangeOrError(update)
+          }
+        ).getOrElse(newVersionRangeOrError(update))
+      }
+    }
+  }
+
+  private def processFeatureUpdates(request: UpdateFeaturesRequest,
+                                    callback: UpdateFeaturesCallback): Unit = {
+    if (isActive) {
+      processFeatureUpdatesWithActiveController(request, callback)
+    } else {
+      callback(Left(new ApiError(Errors.NOT_CONTROLLER)))
+    }
+  }
+
+  private def processFeatureUpdatesWithActiveController(request: UpdateFeaturesRequest,
+                                                        callback: UpdateFeaturesCallback): Unit = {
+    val updates = request.data.featureUpdates
+    val existingFeatures = featureCache.get
+      .map(featuresAndEpoch => featuresAndEpoch.features.features().asScala)
+      .getOrElse(Map[String, FinalizedVersionRange]())
+    // A map with key being feature name and value being FinalizedVersionRange.
+    // This contains the target features to be eventually written to FeatureZNode.
+    val targetFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]() ++ existingFeatures
+    // A map with key being feature name and value being error encountered when the FeatureUpdate
+    // was applied.
+    val errors = scala.collection.mutable.Map[String, ApiError]()
+
+    // Below we process each FeatureUpdate using the following logic:
+    //  - If a FeatureUpdate is found to be valid, then:
+    //    - The corresponding entry in errors map would be updated to contain Errors.NONE.
+    //    - If the FeatureUpdate is an add or update request, then the targetFeatures map is updated
+    //      to contain the new FinalizedVersionRange for the feature.
+    //    - Otherwise if the FeatureUpdate is a delete request, then the feature is removed from the
+    //      targetFeatures map.
+    //  - Otherwise if a FeatureUpdate is found to be invalid, then:
+    //    - The corresponding entry in errors map would be updated with the appropriate ApiError.
+    //    - The entry in targetFeatures map is left untouched.
+    updates.asScala.iterator.foreach { update =>
+      validateFeatureUpdate(update, existingFeatures.get(update.feature())) match {
+        case Left(newVersionRangeOrNone) =>
+          newVersionRangeOrNone match {
+            case Some(newVersionRange) => targetFeatures += (update.feature() -> newVersionRange)
+            case None => targetFeatures -= update.feature()
+          }
+          errors += (update.feature() -> new ApiError(Errors.NONE))
+        case Right(featureUpdateFailureReason) =>
+          errors += (update.feature() -> featureUpdateFailureReason)
+      }
+    }
+
+    // If the existing and target features are the same, then, we skip the update to the
+    // FeatureZNode as no changes to the node are required. Otherwise, we replace the contents
+    // of the FeatureZNode with the new features. This may result in partial or full modification
+    // of the existing finalized features in ZK.
+    try {
+      if (!existingFeatures.equals(targetFeatures)) {
+        val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, Features.finalizedFeatures(targetFeatures.asJava))
+        val newVersion = zkClient.updateFeatureZNode(newNode)
+        featureCache.waitUntilEpochOrThrow(newVersion, request.data().timeoutMs().min(config.zkConnectionTimeoutMs))
+      }
+    } catch {
+      // For all features that correspond to valid FeatureUpdate (i.e. error is Errors.NONE),
+      // we set the error as Errors.FEATURE_UPDATE_FAILED since the FeatureZNode update has failed
+      // for these. For the rest, the existing error is left untouched.
+      case e: Exception =>
+        warn(s"Processing of feature updates: $request failed due to error: $e")
+        errors.foreach { case (feature, apiError) =>
+          if (apiError.error() == Errors.NONE) {
+            errors(feature) = new ApiError(Errors.FEATURE_UPDATE_FAILED)

Review comment:
       Would the default error message suffice?: `Unable to update finalized features due to an unexpected server 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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesResult.java
##########
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Map;
+import org.apache.kafka.common.KafkaFuture;
+
+/**
+ * The result of the {@link Admin#updateFeatures(Map, UpdateFeaturesOptions)} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+public class UpdateFeaturesResult {
+    private final Map<String, KafkaFuture<Void>> futures;
+
+    /**
+     * @param futures   a map from feature names to future, which can be used to check the status of
+     *                  individual feature updates.
+     */
+    public UpdateFeaturesResult(final Map<String, KafkaFuture<Void>> futures) {
+        this.futures = futures;
+    }
+
+    public Map<String, KafkaFuture<Void>> values() {

Review comment:
       Done. The KIP has been updated to have this method now.




----------------------------------------------------------------
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 pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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


   @kowshik : Thanks for following up. I will merged this PR as it is since the system test failures are not new. 
   
   Also, in Scala, we prefer sealed traits over Enumeration since the former gives you exhaustiveness checking. With Scala enums, you don't get a warning if you add a new value that is not handled in a given pattern match. Maybe you can address that in your followup PR.


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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


   @chia7712 No worries, thanks for the suggestions! I have opened a separate PR addressing your comments. Would you be able to please review it? https://github.com/apache/kafka/pull/9393


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/feature/FinalizedVersionRange.java
##########
@@ -40,14 +40,16 @@ public static FinalizedVersionRange fromMap(Map<String, Short> versionRangeMap)
 
     /**
      * Checks if the [min, max] version level range of this object does *NOT* fall within the
-     * [min, max] version range of the provided SupportedVersionRange parameter.
+     * [min, first_active_version, max] range of the provided SupportedVersionRange parameter.

Review comment:
       We need to keep the existing validation. Here is a case where `minVersionLevel < firstActiveVersion` is true, but still there are no incompatibilities:
   ```
   SupportedVersionRange={minVersion=1, firstActiveVersion=4, maxVersion=7}
   FinalizedVersionRange={minVersionLevel=2, maxVersionLevel=6}
   ```
   
   For example, the above can happen during step 1 of feature verison level deprecation. Imagine the following:
    * A supported feature exists with `SupportedVersionRange={minVersion=1, firstActiveVersion=4, maxVersion=7}`
    * The above feature is finalized at `{minVersionLevel=2, maxVersionLevel=6}` in ZK already.
   
   Then imagine a new Kafka release is deployed that raises `firstActiveVersion` for the supported feature from 1 -> 4 (in order to deprecate versions: 1,2,3). In such a case, during Kafka server startup (where we check for feature incompatibilities), we would run into the comparison cited above between the new `SupportedVersionRange` and existing `FinalizedVersionRange`. But it is not considered to be a case of incompatibility.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -82,18 +110,54 @@ object FinalizedFeatureCache extends Logging {
         " The existing cache contents are %s").format(latest, oldFeatureAndEpoch)
       throw new FeatureCacheUpdateException(errorMsg)
     } else {
-      val incompatibleFeatures = SupportedFeatures.incompatibleFeatures(latest.features)
+      val incompatibleFeatures = brokerFeatures.incompatibleFeatures(latest.features)
       if (!incompatibleFeatures.empty) {
         val errorMsg = ("FinalizedFeatureCache update failed since feature compatibility" +
           " checks failed! Supported %s has incompatibilities with the latest %s."
-          ).format(SupportedFeatures.get, latest)
+          ).format(brokerFeatures.supportedFeatures, latest)

Review comment:
       Good question. The existing behavior is that it shuts itself down, as triggered by this LOC. The reason to do it is that an incompatible broker can potentially do harmful things to a cluster (because max version level upgrades are used for breaking changes): https://github.com/apache/kafka/blob/89a3ba69e03acbe9635ee1039abb567bf0c6631b/core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala#L154-L156. 




----------------------------------------------------------------
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 pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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


   @abbccdda : Any more comments from you?


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/resources/common/message/ApiVersionsResponse.json
##########
@@ -55,8 +55,8 @@
           "about": "The maximum supported version for the feature." }
       ]
     },
-    {"name": "FinalizedFeaturesEpoch", "type": "int32", "versions": "3+",
-      "tag": 1, "taggedVersions": "3+", "default": "-1",
+    {"name": "FinalizedFeaturesEpoch", "type": "int64", "versions": "3+",

Review comment:
       Space before "name".

##########
File path: clients/src/main/resources/common/message/UpdateFeaturesRequest.json
##########
@@ -0,0 +1,35 @@
+// 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.
+
+{
+  "apiKey": 57,
+  "type": "request",
+  "name": "UpdateFeaturesRequest",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "timeoutMs", "type": "int32", "versions": "0+", "default": "60000",

Review comment:
       This is not included in the KIP. Should we update the KIP?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FinalizedVersionRange.java
##########
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+
+/**
+ * Represents a range of version levels supported by every broker in a cluster for some feature.
+ */
+public class FinalizedVersionRange {
+    private final short minVersionLevel;
+
+    private final short maxVersionLevel;
+
+    /**
+     * Raises an exception unless the following condition is met:
+     * minVersionLevel >= 1 and maxVersionLevel >= 1 and maxVersionLevel >= minVersionLevel.
+     *
+     * @param minVersionLevel   The minimum version level value.
+     * @param maxVersionLevel   The maximum version level value.
+     *
+     * @throws IllegalArgumentException   Raised when the condition described above is not met.
+     */
+    public FinalizedVersionRange(final short minVersionLevel, final short maxVersionLevel) {

Review comment:
       Since the user is not expected to instantiate this, should we make the constructor non-public?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java
##########
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import static java.util.stream.Collectors.joining;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * Encapsulates details about finalized as well as supported features. This is particularly useful
+ * to hold the result returned by the {@link Admin#describeFeatures(DescribeFeaturesOptions)} API.
+ */
+public class FeatureMetadata {
+
+    private final Map<String, FinalizedVersionRange> finalizedFeatures;
+
+    private final Optional<Long> finalizedFeaturesEpoch;
+
+    private final Map<String, SupportedVersionRange> supportedFeatures;
+
+    public FeatureMetadata(final Map<String, FinalizedVersionRange> finalizedFeatures,

Review comment:
       Since the user is not expected to instantiate this, should we make the constructor non-public?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java
##########
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+/**
+ * Options for {@link AdminClient#describeFeatures(DescribeFeaturesOptions)}.
+ *
+ * The API of this class is evolving. See {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeFeaturesOptions extends AbstractOptions<DescribeFeaturesOptions> {

Review comment:
       Since this is public facing, could we include the description in the KIP?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesOptions.java
##########
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Map;
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+/**
+ * Options for {@link AdminClient#updateFeatures(Map, UpdateFeaturesOptions)}.
+ *
+ * The API of this class is evolving. See {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class UpdateFeaturesOptions extends AbstractOptions<UpdateFeaturesOptions> {

Review comment:
       Are we adding the timeout option based on the KIP discussion?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FinalizedVersionRange.java
##########
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+
+/**
+ * Represents a range of version levels supported by every broker in a cluster for some feature.
+ */
+public class FinalizedVersionRange {

Review comment:
       This seems identical to SupportedVersionRange. Should we just have one, sth like VersionRange?

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,179 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the latest features supported by the Broker and also provides APIs to
+ * check for incompatibilities between the features supported by the Broker and finalized features.
+ * The class also enables feature version level deprecation, as explained below. This class is
+ * immutable in production. It provides few APIs to mutate state only for the purpose of testing.
+ *
+ * Feature version level deprecation:
+ * ==================================
+ *
+ * Deprecation of certain version levels of a feature is a process to stop supporting the
+ * functionality offered by the feature at a those version levels, across the entire Kafka cluster.
+ * Feature version deprecation is a simple 2-step process explained below. In each step below, an
+ * example is provided to help understand the process better:
+ *
+ * STEP 1:

Review comment:
       I guess after the first step, deprecated finalized versions are no longer advertised to the client, but they can still be used by existing connections?

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -1733,8 +1734,8 @@ class KafkaApis(val requestChannel: RequestChannel,
       else if (!apiVersionRequest.isValid)
         apiVersionRequest.getErrorResponse(requestThrottleMs, Errors.INVALID_REQUEST.exception)
       else {
-        val supportedFeatures = SupportedFeatures.get
-        val finalizedFeatures = FinalizedFeatureCache.get
+        val supportedFeatures = brokerFeatures.supportedFeatures
+        val finalizedFeatures = featureCache.get

Review comment:
       Perhaps it's better for the following code to use match instead if/else.

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,179 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the latest features supported by the Broker and also provides APIs to
+ * check for incompatibilities between the features supported by the Broker and finalized features.
+ * The class also enables feature version level deprecation, as explained below. This class is
+ * immutable in production. It provides few APIs to mutate state only for the purpose of testing.
+ *
+ * Feature version level deprecation:
+ * ==================================
+ *
+ * Deprecation of certain version levels of a feature is a process to stop supporting the
+ * functionality offered by the feature at a those version levels, across the entire Kafka cluster.

Review comment:
       "at a those " typo?

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -272,6 +281,199 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (string) and a range of versions (defined by a
+   * SupportedVersionRange). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of its own supported features in its
+   * own BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (string) and a range of version levels (defined
+   * by a FinalizedVersionRange). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. This process ensures we do not enable all the possible features immediately after
+   *    an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, it will
+   *        react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, its existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.defaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) =>
+            val brokerDefaultVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (brokerDefaultVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (brokerDefaultVersionRange.max() >= existingVersionRange.max() &&
+                       brokerDefaultVersionRange.min() <= existingVersionRange.max()) {
+              // Using the change below, we deprecate all version levels in the range:
+              // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1].
+              //
+              // NOTE: if existingVersionRange.min() equals brokerDefaultVersionRange.min(), then
+              // we do not deprecate any version levels (since there is none to be deprecated).
+              //
+              // Examples:
+              // 1. brokerDefaultVersionRange = [4, 7] and existingVersionRange = [1, 5].
+              //    In this case, we deprecate all version levels in the range: [1, 3].
+              // 2. brokerDefaultVersionRange = [4, 7] and existingVersionRange = [4, 5].
+              //    In this case, we do not deprecate any version levels since
+              //    brokerDefaultVersionRange.min() equals existingVersionRange.min().
+              (featureName, new FinalizedVersionRange(brokerDefaultVersionRange.min(), existingVersionRange.max()))
+            } else {
+              // This is a serious error. We should never be reaching here, since we already
+              // verify once during KafkaServer startup that existing finalized feature versions in
+              // the FeatureZNode contained no incompatibilities. If we are here, it means that one
+              // of the following is true:
+              // 1. The existing version levels fall completely outside the range of the default
+              // finalized version levels (i.e. no intersection), or
+              // 2. The existing version levels are incompatible with default finalized version
+              // levels.
+              //
+              // Examples of invalid cases that can cause this exception to be triggered:
+              // 1. No intersection      : brokerDefaultVersionRange = [4, 7] and existingVersionRange = [2, 3].
+              // 2. No intersection      : brokerDefaultVersionRange = [2, 3] and existingVersionRange = [4, 7].
+              // 3. Incompatible versions: brokerDefaultVersionRange = [2, 3] and existingVersionRange = [1, 7].
+              throw new IllegalStateException(
+                s"Can not update minimum version level in finalized feature: $featureName,"
+                + s" since the existing $existingVersionRange is not eligible for a change"
+                + s" based on the default $brokerDefaultVersionRange. This should never happen"
+                + s" since feature version incompatibilities are already checked during"
+                + s" Kafka server startup.")
+            }
+        }.asJava)
+      }
+      val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures)
+      if (!newFeatureZNode.equals(existingFeatureZNode)) {
+        val newVersion = updateFeatureZNode(newFeatureZNode)
+        featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+      }
+    }
+  }
+
+  /**
+   * Disables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with disabled status. This status means the feature versioning system
+   * (KIP-584) is disabled, and, the finalized features stored in the FeatureZNode are not relevant.
+   * This status should be written by the controller to the FeatureZNode only when the broker
+   * IBP config is less than KAFKA_2_7_IV0.
+   *
+   * NOTE:
+   * 1. When this method returns, existing finalized features (if any) will be cleared from the
+   *    FeatureZNode.
+   * 2. This method, unlike enableFeatureVersioning() need not wait for the FinalizedFeatureCache
+   *    to be updated, because, such updates to the cache (via FinalizedFeatureChangeListener)
+   *    are disabled when IBP config is < than KAFKA_2_7_IV0.
+   */
+  private def disableFeatureVersioning(): Unit = {
+    val newNode = FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures())
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      createFeatureZNode(newNode)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      if (!existingFeatureZNode.status.equals(FeatureZNodeStatus.Disabled)) {
+        updateFeatureZNode(newNode)
+      }
+    }
+  }
+
+  private def setupFeatureVersioning(): Unit = {

Review comment:
       setupFeatureVersioning => mayBeSetupFeatureVersioning ?

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -3109,6 +3110,37 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFeatures(request: RequestChannel.Request): Unit = {
+    val updateFeaturesRequest = request.body[UpdateFeaturesRequest]
+
+    def sendResponseCallback(errors: Either[ApiError, Map[String, ApiError]]): Unit = {
+      def createResponse(throttleTimeMs: Int): UpdateFeaturesResponse = {
+        errors match {
+          case Left(topLevelError) => {
+            val featureUpdateNoErrors = updateFeaturesRequest
+              .data().featureUpdates().asScala

Review comment:
       I think the convention is that if there is a top level error, the second level will just be empty since there is not need to process them individually.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FeatureUpdate.java
##########
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;

Review comment:
       This package is not part of the javadoc and thus is not part of the public interface.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -43,7 +43,7 @@
  */
 public class ApiVersionsResponse extends AbstractResponse {
 
-    public static final int UNKNOWN_FINALIZED_FEATURES_EPOCH = -1;
+    public static final long UNKNOWN_FINALIZED_FEATURES_EPOCH = -1;

Review comment:
       -1 =>  -1L?

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -272,6 +281,199 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (string) and a range of versions (defined by a
+   * SupportedVersionRange). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of its own supported features in its
+   * own BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (string) and a range of version levels (defined
+   * by a FinalizedVersionRange). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. This process ensures we do not enable all the possible features immediately after
+   *    an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, it will
+   *        react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, its existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.defaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) =>
+            val brokerDefaultVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (brokerDefaultVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (brokerDefaultVersionRange.max() >= existingVersionRange.max() &&
+                       brokerDefaultVersionRange.min() <= existingVersionRange.max()) {
+              // Using the change below, we deprecate all version levels in the range:
+              // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1].
+              //
+              // NOTE: if existingVersionRange.min() equals brokerDefaultVersionRange.min(), then
+              // we do not deprecate any version levels (since there is none to be deprecated).
+              //
+              // Examples:
+              // 1. brokerDefaultVersionRange = [4, 7] and existingVersionRange = [1, 5].
+              //    In this case, we deprecate all version levels in the range: [1, 3].
+              // 2. brokerDefaultVersionRange = [4, 7] and existingVersionRange = [4, 5].
+              //    In this case, we do not deprecate any version levels since
+              //    brokerDefaultVersionRange.min() equals existingVersionRange.min().
+              (featureName, new FinalizedVersionRange(brokerDefaultVersionRange.min(), existingVersionRange.max()))
+            } else {
+              // This is a serious error. We should never be reaching here, since we already
+              // verify once during KafkaServer startup that existing finalized feature versions in
+              // the FeatureZNode contained no incompatibilities. If we are here, it means that one
+              // of the following is true:
+              // 1. The existing version levels fall completely outside the range of the default
+              // finalized version levels (i.e. no intersection), or
+              // 2. The existing version levels are incompatible with default finalized version
+              // levels.
+              //
+              // Examples of invalid cases that can cause this exception to be triggered:
+              // 1. No intersection      : brokerDefaultVersionRange = [4, 7] and existingVersionRange = [2, 3].
+              // 2. No intersection      : brokerDefaultVersionRange = [2, 3] and existingVersionRange = [4, 7].
+              // 3. Incompatible versions: brokerDefaultVersionRange = [2, 3] and existingVersionRange = [1, 7].
+              throw new IllegalStateException(
+                s"Can not update minimum version level in finalized feature: $featureName,"
+                + s" since the existing $existingVersionRange is not eligible for a change"
+                + s" based on the default $brokerDefaultVersionRange. This should never happen"
+                + s" since feature version incompatibilities are already checked during"
+                + s" Kafka server startup.")
+            }
+        }.asJava)
+      }
+      val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures)
+      if (!newFeatureZNode.equals(existingFeatureZNode)) {
+        val newVersion = updateFeatureZNode(newFeatureZNode)
+        featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+      }
+    }
+  }
+
+  /**
+   * Disables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with disabled status. This status means the feature versioning system
+   * (KIP-584) is disabled, and, the finalized features stored in the FeatureZNode are not relevant.
+   * This status should be written by the controller to the FeatureZNode only when the broker
+   * IBP config is less than KAFKA_2_7_IV0.
+   *
+   * NOTE:
+   * 1. When this method returns, existing finalized features (if any) will be cleared from the
+   *    FeatureZNode.
+   * 2. This method, unlike enableFeatureVersioning() need not wait for the FinalizedFeatureCache
+   *    to be updated, because, such updates to the cache (via FinalizedFeatureChangeListener)
+   *    are disabled when IBP config is < than KAFKA_2_7_IV0.
+   */
+  private def disableFeatureVersioning(): Unit = {
+    val newNode = FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures())
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      createFeatureZNode(newNode)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      if (!existingFeatureZNode.status.equals(FeatureZNodeStatus.Disabled)) {
+        updateFeatureZNode(newNode)
+      }
+    }
+  }
+
+  private def setupFeatureVersioning(): Unit = {
+    if (config.isFeatureVersioningEnabled) {

Review comment:
       Perhaps isFeatureVersioningSupported is a better name?

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -272,6 +281,199 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (string) and a range of versions (defined by a
+   * SupportedVersionRange). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of its own supported features in its
+   * own BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (string) and a range of version levels (defined
+   * by a FinalizedVersionRange). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. This process ensures we do not enable all the possible features immediately after
+   *    an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, it will
+   *        react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, its existing minimum version level is updated to the default

Review comment:
       Could you define the default finalized features? Also, default minimum version seems outdated now.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/FinalizedVersionRange.java
##########
@@ -40,14 +40,16 @@ public static FinalizedVersionRange fromMap(Map<String, Short> versionRangeMap)
 
     /**
      * Checks if the [min, max] version level range of this object does *NOT* fall within the
-     * [min, max] version range of the provided SupportedVersionRange parameter.
+     * [min, first_active_version, max] range of the provided SupportedVersionRange parameter.

Review comment:
       Should we just verify the range [first_active_version, max]?

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,179 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the latest features supported by the Broker and also provides APIs to
+ * check for incompatibilities between the features supported by the Broker and finalized features.
+ * The class also enables feature version level deprecation, as explained below. This class is
+ * immutable in production. It provides few APIs to mutate state only for the purpose of testing.
+ *
+ * Feature version level deprecation:
+ * ==================================
+ *
+ * Deprecation of certain version levels of a feature is a process to stop supporting the
+ * functionality offered by the feature at a those version levels, across the entire Kafka cluster.
+ * Feature version deprecation is a simple 2-step process explained below. In each step below, an
+ * example is provided to help understand the process better:
+ *
+ * STEP 1:
+ * In the first step, a major Kafka release is made with a Broker code change (explained later
+ * below) that establishes the intent to deprecate certain versions of one or more features
+ * cluster-wide. When this new Kafka release is deployed to the cluster, the feature versioning
+ * system (via the controller) will automatically persist the new minVersionLevel for the feature in
+ * Zk to propagate the deprecation of certain versions. After this happens, any external client that
+ * queries the Broker to learn the feature versions will at some point start to see the new value
+ * for the finalized minVersionLevel for the feature. This makes the version deprecation permanent.
+ *
+ * Here is how the above code change needs to be done:
+ * In order to deprecate feature version levels, in the supportedFeatures map you need to supply a
+ * specific firstActiveVersion value that's higher than the minVersion for the feature. The
+ * value for firstActiveVersion should be 1 beyond the highest version that you intend to deprecate
+ * for that feature. When features are finalized via the ApiKeys.UPDATE_FEATURES api, the feature
+ * version levels in the closed range: [minVersion, firstActiveVersion - 1] are automatically
+ * deprecated in ZK by the controller logic.
+ * Example:
+ * - Let us assume the existing finalized feature in ZK:
+ *   {
+ *      "feature_1" -> FinalizedVersionRange(minVersionLevel=1, maxVersionLevel=5)
+ *   }
+ *   Now, supposing you would like to deprecate feature version levels: [1, 2].
+ *   Then, in the supportedFeatures map you should supply the following:
+ *   supportedFeatures = {
+ *     "feature1" -> SupportedVersionRange(minVersion=1, firstActiveVersion=3, maxVersion=5)
+ *   }
+ * - If you do NOT want to deprecate a version level for a feature, then in the supportedFeatures
+ *   map you should supply the firstActiveVersion to be the same as the minVersion supplied for that
+ *   feature.
+ *   Example:
+ *   supportedFeatures = {
+ *     "feature1" -> SupportedVersionRange(minVersion=1, firstActiveVersion=1, maxVersion=5)
+ *   }
+ *   This indicates no intent to deprecate any version levels for the feature.
+ *
+ * STEP 2:
+ * After the first step is over, you may (at some point) want to permanently remove the code/logic
+ * for the functionality offered by the deprecated feature versions. This is the second step. Here a
+ * subsequent major Kafka release is made with another Broker code change that removes the code for
+ * the functionality offered by the deprecated feature versions. This would completely drop support
+ * for the deprecated versions. Such a code change needs to be supplemented by supplying a
+ * suitable higher minVersion value for the feature in the supportedFeatures map.
+ * Example:
+ * - In the example above in step 1, we showed how to deprecate version levels [1, 2] for
+ *   "feature_1". Now let us assume the following finalized feature in ZK (after the deprecation
+ *   has been carried out):
+ *   {
+ *     "feature_1" -> FinalizedVersionRange(minVersionLevel=3, maxVersionLevel=5)
+ *   }
+ *   Now, supposing you would like to permanently remove support for feature versions: [1, 2].
+ *   Then, in the supportedFeatures map you should now supply the following:
+ *   supportedFeatures = {
+ *     "feature1" -> SupportedVersionRange(minVersion=3, firstActiveVersion=3, maxVersion=5)

Review comment:
       Is it useful to expose firstActiveVersion to the client?

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1656,6 +1893,203 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns an ApiError object containing Errors.INVALID_REQUEST.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+
+    val supportedVersionRange = brokerFeatures.supportedFeatures.get(update.feature)
+    if (supportedVersionRange == null) {
+      Right(new ApiError(Errors.INVALID_REQUEST,
+                         "Could not apply finalized feature update because the provided feature" +
+                         " is not supported."))
+    } else {
+      var newVersionRange: FinalizedVersionRange = null
+      try {
+        newVersionRange = new FinalizedVersionRange(supportedVersionRange.firstActiveVersion, update.maxVersionLevel)
+      } catch {
+        case _: IllegalArgumentException => {
+          // This exception means the provided maxVersionLevel is invalid. It is handled below
+          // outside of this catch clause.
+        }
+      }
+      if (newVersionRange == null) {
+        Right(new ApiError(Errors.INVALID_REQUEST,
+          "Could not apply finalized feature update because the provided" +
+          s" maxVersionLevel:${update.maxVersionLevel} is lower than the" +
+          s" first active version:${supportedVersionRange.firstActiveVersion}."))
+      } else {
+        val newFinalizedFeature =
+          Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange)))
+        val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+          BrokerFeatures.hasIncompatibleFeatures(broker.features, newFinalizedFeature)
+        })
+        if (numIncompatibleBrokers == 0) {
+          Left(newVersionRange)
+        } else {
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Could not apply finalized feature update because" +
+                             " brokers were found to have incompatible versions for the feature."))
+        }
+      }
+    }
+  }
+
+  /**
+   * Validates a feature update on an existing FinalizedVersionRange.
+   * If the validation succeeds, then, the return value contains:
+   * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature.
+   * 2. Option.empty, if the feature update was meant to delete the feature.
+   *
+   * If the validation fails, then returned value contains a suitable ApiError.
+   *
+   * @param update                 the feature update to be processed.
+   * @param existingVersionRange   the existing FinalizedVersionRange which can be empty when no
+   *                               FinalizedVersionRange exists for the associated feature
+   *
+   * @return                       the new FinalizedVersionRange to be updated into ZK or error
+   *                               as described above.
+   */
+  private def validateFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey,
+                                   existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = {

Review comment:
       indentation

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -112,8 +112,9 @@ class KafkaApis(val requestChannel: RequestChannel,
                 brokerTopicStats: BrokerTopicStats,
                 val clusterId: String,
                 time: Time,
-                val tokenManager: DelegationTokenManager)
-  extends ApiRequestHandler with Logging {
+                val tokenManager: DelegationTokenManager,
+                val brokerFeatures: BrokerFeatures,
+                val featureCache: FinalizedFeatureCache) extends ApiRequestHandler with Logging {

Review comment:
       featureCache => finalizedFeatureCache ?

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -34,7 +34,7 @@ import scala.concurrent.TimeoutException
  *
  * @param zkClient     the Zookeeper client
  */
-class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+class FinalizedFeatureChangeListener(private val featureCache: FinalizedFeatureCache, private val zkClient: KafkaZkClient) extends Logging {

Review comment:
       featureCache => finalizedFeatureCache?

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -272,6 +281,199 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (string) and a range of versions (defined by a
+   * SupportedVersionRange). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of its own supported features in its
+   * own BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (string) and a range of version levels (defined
+   * by a FinalizedVersionRange). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. This process ensures we do not enable all the possible features immediately after
+   *    an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, it will
+   *        react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, its existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.defaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)

Review comment:
       Since we are including the timeout in the UpdateFeature request, perhaps we could just use that timeout here.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1656,6 +1893,203 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns an ApiError object containing Errors.INVALID_REQUEST.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+
+    val supportedVersionRange = brokerFeatures.supportedFeatures.get(update.feature)
+    if (supportedVersionRange == null) {
+      Right(new ApiError(Errors.INVALID_REQUEST,
+                         "Could not apply finalized feature update because the provided feature" +
+                         " is not supported."))
+    } else {
+      var newVersionRange: FinalizedVersionRange = null
+      try {
+        newVersionRange = new FinalizedVersionRange(supportedVersionRange.firstActiveVersion, update.maxVersionLevel)
+      } catch {
+        case _: IllegalArgumentException => {
+          // This exception means the provided maxVersionLevel is invalid. It is handled below
+          // outside of this catch clause.
+        }
+      }
+      if (newVersionRange == null) {
+        Right(new ApiError(Errors.INVALID_REQUEST,
+          "Could not apply finalized feature update because the provided" +
+          s" maxVersionLevel:${update.maxVersionLevel} is lower than the" +
+          s" first active version:${supportedVersionRange.firstActiveVersion}."))
+      } else {
+        val newFinalizedFeature =
+          Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange)))
+        val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+          BrokerFeatures.hasIncompatibleFeatures(broker.features, newFinalizedFeature)
+        })
+        if (numIncompatibleBrokers == 0) {
+          Left(newVersionRange)
+        } else {
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Could not apply finalized feature update because" +
+                             " brokers were found to have incompatible versions for the feature."))
+        }
+      }
+    }
+  }
+
+  /**
+   * Validates a feature update on an existing FinalizedVersionRange.
+   * If the validation succeeds, then, the return value contains:
+   * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature.
+   * 2. Option.empty, if the feature update was meant to delete the feature.
+   *
+   * If the validation fails, then returned value contains a suitable ApiError.
+   *
+   * @param update                 the feature update to be processed.
+   * @param existingVersionRange   the existing FinalizedVersionRange which can be empty when no
+   *                               FinalizedVersionRange exists for the associated feature
+   *
+   * @return                       the new FinalizedVersionRange to be updated into ZK or error
+   *                               as described above.
+   */
+  private def validateFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey,
+                                   existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = {
+    def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = {
+      newFinalizedVersionRangeOrIncompatibilityError(update)
+        .fold(versionRange => Left(Some(versionRange)), error => Right(error))
+    }
+
+    if (update.feature.isEmpty) {
+      // Check that the feature name is not empty.
+      Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty."))
+    } else {
+      // We handle deletion requests separately from non-deletion requests.
+      if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+        if (existingVersionRange.isEmpty) {
+          // Disallow deletion of a non-existing finalized feature.
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Can not delete non-existing finalized feature."))
+        } else {
+          Left(Option.empty)
+        }
+      } else if (update.maxVersionLevel() < 1) {
+        // Disallow deletion of a finalized feature without allowDowngrade flag set.
+        Right(new ApiError(Errors.INVALID_REQUEST,
+                           s"Can not provide maxVersionLevel: ${update.maxVersionLevel} less" +
+                           s" than 1 without setting the allowDowngrade flag to true in the request."))
+      } else {
+        existingVersionRange.map(existing =>
+          if (update.maxVersionLevel == existing.max) {
+            // Disallow a case where target maxVersionLevel matches existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not ${if (update.allowDowngrade) "downgrade" else "upgrade"}" +
+                               s" a finalized feature from existing maxVersionLevel:${existing.max}" +
+                               " to the same value."))
+          } else if (update.maxVersionLevel < existing.max && !update.allowDowngrade) {
+            // Disallow downgrade of a finalized feature without the allowDowngrade flag set.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature from existing" +
+                               s" maxVersionLevel:${existing.max} to provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} without setting the" +
+                               " allowDowngrade flag in the request."))
+          } else if (update.allowDowngrade && update.maxVersionLevel > existing.max) {
+            // Disallow a request that sets allowDowngrade flag without specifying a
+            // maxVersionLevel that's lower than the existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"When the allowDowngrade flag set in the request, the provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} can not be greater than" +
+                               s" existing maxVersionLevel:${existing.max}."))
+          } else if (update.maxVersionLevel < existing.min) {
+            // Disallow downgrade of a finalized feature below the existing finalized
+            // minVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature to maxVersionLevel:${update.maxVersionLevel}" +
+                               s" because it's lower than the existing minVersionLevel:${existing.min}."))
+          } else {
+            newVersionRangeOrError(update)
+          }
+        ).getOrElse(newVersionRangeOrError(update))
+      }
+    }
+  }
+
+  private def processFeatureUpdates(request: UpdateFeaturesRequest,
+                                    callback: UpdateFeaturesCallback): Unit = {
+    if (isActive) {
+      processFeatureUpdatesWithActiveController(request, callback)
+    } else {
+      callback(Left(new ApiError(Errors.NOT_CONTROLLER)))
+    }
+  }
+
+  private def processFeatureUpdatesWithActiveController(request: UpdateFeaturesRequest,
+                                                        callback: UpdateFeaturesCallback): Unit = {
+    val updates = request.data.featureUpdates
+    val existingFeatures = featureCache.get
+      .map(featuresAndEpoch => featuresAndEpoch.features.features().asScala)
+      .getOrElse(Map[String, FinalizedVersionRange]())
+    // A map with key being feature name and value being FinalizedVersionRange.
+    // This contains the target features to be eventually written to FeatureZNode.
+    val targetFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]() ++ existingFeatures
+    // A map with key being feature name and value being error encountered when the FeatureUpdate
+    // was applied.
+    val errors = scala.collection.mutable.Map[String, ApiError]()
+
+    // Below we process each FeatureUpdate using the following logic:
+    //  - If a FeatureUpdate is found to be valid, then:
+    //    - The corresponding entry in errors map would be updated to contain ApiError(Errors.NONE).
+    //    - If the FeatureUpdate is an add or update request, then the targetFeatures map is updated
+    //      to contain the new FinalizedVersionRange for the feature.
+    //    - Otherwise if the FeatureUpdate is a delete request, then the feature is removed from the
+    //      targetFeatures map.
+    //  - Otherwise if a FeatureUpdate is found to be invalid, then:
+    //    - The corresponding entry in errors map would be updated with the appropriate ApiError.
+    //    - The entry in targetFeatures map is left untouched.
+    updates.asScala.iterator.foreach { update =>
+      validateFeatureUpdate(update, existingFeatures.get(update.feature())) match {
+        case Left(newVersionRangeOrNone) =>
+          newVersionRangeOrNone
+            .map(newVersionRange => targetFeatures += (update.feature() -> newVersionRange))
+            .getOrElse(targetFeatures -= update.feature())
+          errors += (update.feature() -> new ApiError(Errors.NONE))
+        case Right(featureUpdateFailureReason) =>
+          errors += (update.feature() -> featureUpdateFailureReason)
+      }
+    }
+
+    // If the existing and target features are the same, then, we skip the update to the
+    // FeatureZNode as no changes to the node are required. Otherwise, we replace the contents
+    // of the FeatureZNode with the new features. This may result in partial or full modification
+    // of the existing finalized features in ZK.
+    try {
+      if (!existingFeatures.equals(targetFeatures)) {
+        val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, Features.finalizedFeatures(targetFeatures.asJava))
+        val newVersion = zkClient.updateFeatureZNode(newNode)
+        featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+      }
+    } catch {
+      // For all features that correspond to valid FeatureUpdate (i.e. error is Errors.NONE),
+      // we set the error as Errors.FEATURE_UPDATE_FAILED since the FeatureZNode update has failed
+      // for these. For the rest, the existing error is left untouched.
+      case e: Exception =>
+        errors.foreach { case (feature, apiError) =>
+          if (apiError.error() == Errors.NONE) {
+            errors(feature) = new ApiError(Errors.FEATURE_UPDATE_FAILED,
+                                           Errors.FEATURE_UPDATE_FAILED.message() + " Error: " + e)

Review comment:
       Do we need to return the stacktrace to the caller? Since this is unexpected, perhaps we can log a warn?

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1656,6 +1893,203 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns an ApiError object containing Errors.INVALID_REQUEST.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+
+    val supportedVersionRange = brokerFeatures.supportedFeatures.get(update.feature)
+    if (supportedVersionRange == null) {
+      Right(new ApiError(Errors.INVALID_REQUEST,
+                         "Could not apply finalized feature update because the provided feature" +
+                         " is not supported."))
+    } else {
+      var newVersionRange: FinalizedVersionRange = null
+      try {
+        newVersionRange = new FinalizedVersionRange(supportedVersionRange.firstActiveVersion, update.maxVersionLevel)
+      } catch {
+        case _: IllegalArgumentException => {
+          // This exception means the provided maxVersionLevel is invalid. It is handled below
+          // outside of this catch clause.
+        }
+      }
+      if (newVersionRange == null) {
+        Right(new ApiError(Errors.INVALID_REQUEST,
+          "Could not apply finalized feature update because the provided" +
+          s" maxVersionLevel:${update.maxVersionLevel} is lower than the" +
+          s" first active version:${supportedVersionRange.firstActiveVersion}."))
+      } else {
+        val newFinalizedFeature =
+          Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange)))
+        val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+          BrokerFeatures.hasIncompatibleFeatures(broker.features, newFinalizedFeature)
+        })
+        if (numIncompatibleBrokers == 0) {
+          Left(newVersionRange)
+        } else {
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Could not apply finalized feature update because" +
+                             " brokers were found to have incompatible versions for the feature."))
+        }
+      }
+    }
+  }
+
+  /**
+   * Validates a feature update on an existing FinalizedVersionRange.
+   * If the validation succeeds, then, the return value contains:
+   * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature.
+   * 2. Option.empty, if the feature update was meant to delete the feature.
+   *
+   * If the validation fails, then returned value contains a suitable ApiError.
+   *
+   * @param update                 the feature update to be processed.
+   * @param existingVersionRange   the existing FinalizedVersionRange which can be empty when no
+   *                               FinalizedVersionRange exists for the associated feature
+   *
+   * @return                       the new FinalizedVersionRange to be updated into ZK or error
+   *                               as described above.
+   */
+  private def validateFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey,
+                                   existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = {
+    def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = {
+      newFinalizedVersionRangeOrIncompatibilityError(update)
+        .fold(versionRange => Left(Some(versionRange)), error => Right(error))
+    }
+
+    if (update.feature.isEmpty) {
+      // Check that the feature name is not empty.
+      Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty."))
+    } else {
+      // We handle deletion requests separately from non-deletion requests.
+      if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+        if (existingVersionRange.isEmpty) {
+          // Disallow deletion of a non-existing finalized feature.
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Can not delete non-existing finalized feature."))
+        } else {
+          Left(Option.empty)
+        }
+      } else if (update.maxVersionLevel() < 1) {
+        // Disallow deletion of a finalized feature without allowDowngrade flag set.
+        Right(new ApiError(Errors.INVALID_REQUEST,
+                           s"Can not provide maxVersionLevel: ${update.maxVersionLevel} less" +
+                           s" than 1 without setting the allowDowngrade flag to true in the request."))
+      } else {
+        existingVersionRange.map(existing =>
+          if (update.maxVersionLevel == existing.max) {
+            // Disallow a case where target maxVersionLevel matches existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not ${if (update.allowDowngrade) "downgrade" else "upgrade"}" +
+                               s" a finalized feature from existing maxVersionLevel:${existing.max}" +
+                               " to the same value."))
+          } else if (update.maxVersionLevel < existing.max && !update.allowDowngrade) {
+            // Disallow downgrade of a finalized feature without the allowDowngrade flag set.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature from existing" +
+                               s" maxVersionLevel:${existing.max} to provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} without setting the" +
+                               " allowDowngrade flag in the request."))
+          } else if (update.allowDowngrade && update.maxVersionLevel > existing.max) {
+            // Disallow a request that sets allowDowngrade flag without specifying a
+            // maxVersionLevel that's lower than the existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"When the allowDowngrade flag set in the request, the provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} can not be greater than" +
+                               s" existing maxVersionLevel:${existing.max}."))
+          } else if (update.maxVersionLevel < existing.min) {
+            // Disallow downgrade of a finalized feature below the existing finalized
+            // minVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature to maxVersionLevel:${update.maxVersionLevel}" +
+                               s" because it's lower than the existing minVersionLevel:${existing.min}."))
+          } else {
+            newVersionRangeOrError(update)
+          }
+        ).getOrElse(newVersionRangeOrError(update))
+      }
+    }
+  }
+
+  private def processFeatureUpdates(request: UpdateFeaturesRequest,
+                                    callback: UpdateFeaturesCallback): Unit = {
+    if (isActive) {
+      processFeatureUpdatesWithActiveController(request, callback)
+    } else {
+      callback(Left(new ApiError(Errors.NOT_CONTROLLER)))
+    }
+  }
+
+  private def processFeatureUpdatesWithActiveController(request: UpdateFeaturesRequest,
+                                                        callback: UpdateFeaturesCallback): Unit = {
+    val updates = request.data.featureUpdates
+    val existingFeatures = featureCache.get
+      .map(featuresAndEpoch => featuresAndEpoch.features.features().asScala)
+      .getOrElse(Map[String, FinalizedVersionRange]())
+    // A map with key being feature name and value being FinalizedVersionRange.
+    // This contains the target features to be eventually written to FeatureZNode.
+    val targetFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]() ++ existingFeatures
+    // A map with key being feature name and value being error encountered when the FeatureUpdate
+    // was applied.
+    val errors = scala.collection.mutable.Map[String, ApiError]()
+
+    // Below we process each FeatureUpdate using the following logic:
+    //  - If a FeatureUpdate is found to be valid, then:
+    //    - The corresponding entry in errors map would be updated to contain ApiError(Errors.NONE).
+    //    - If the FeatureUpdate is an add or update request, then the targetFeatures map is updated
+    //      to contain the new FinalizedVersionRange for the feature.
+    //    - Otherwise if the FeatureUpdate is a delete request, then the feature is removed from the
+    //      targetFeatures map.
+    //  - Otherwise if a FeatureUpdate is found to be invalid, then:
+    //    - The corresponding entry in errors map would be updated with the appropriate ApiError.
+    //    - The entry in targetFeatures map is left untouched.
+    updates.asScala.iterator.foreach { update =>
+      validateFeatureUpdate(update, existingFeatures.get(update.feature())) match {
+        case Left(newVersionRangeOrNone) =>
+          newVersionRangeOrNone
+            .map(newVersionRange => targetFeatures += (update.feature() -> newVersionRange))

Review comment:
       map() is supposed to be used with no side effect. Perhaps we could use match here.




----------------------------------------------------------------
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 edited a comment on pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #9001:
URL: https://github.com/apache/kafka/pull/9001#issuecomment-702621705


   @junrao Thanks for the review comments! I have done the change proposed in https://github.com/apache/kafka/pull/9001#discussion_r498574911 in the most recent commit: 4218f95904989028a469930d0c266362bf173ece . Please have a look.


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -219,6 +226,8 @@ class KafkaController(val config: KafkaConfig,
    * This ensures another controller election will be triggered and there will always be an actively serving controller
    */
   private def onControllerFailover(): Unit = {
+    maybeSetupFeatureVersioning()

Review comment:
       Done. Good point. It looks appropriate to me that we exit the broker in this case. I've captured the exception and added a call to `Exit.exit(1)`, is there a better way to do it?




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FinalizedVersionRange.java
##########
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+
+/**
+ * Represents a range of version levels supported by every broker in a cluster for some feature.
+ */
+public class FinalizedVersionRange {
+    private final short minVersionLevel;
+
+    private final short maxVersionLevel;
+
+    /**
+     * Raises an exception unless the following condition is met:
+     * minVersionLevel >= 1 and maxVersionLevel >= 1 and maxVersionLevel >= minVersionLevel.
+     *
+     * @param minVersionLevel   The minimum version level value.
+     * @param maxVersionLevel   The maximum version level value.
+     *
+     * @throws IllegalArgumentException   Raised when the condition described above is not met.
+     */
+    public FinalizedVersionRange(final short minVersionLevel, final short maxVersionLevel) {

Review comment:
       It is instantiated from `kafka.server.UpdateFeaturesTest`, so have to keep the c'tor public.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java
##########
@@ -17,9 +17,16 @@
 package org.apache.kafka.common.feature;
 
 import java.util.Map;
+import java.util.Objects;
+import org.apache.kafka.common.utils.Utils;
 
 /**
- * An extended {@link BaseVersionRange} representing the min/max versions for supported features.
+ * An extended {@link BaseVersionRange} representing the min, max and first active versions for a
+ * supported feature:
+ *  - minVersion: This is the minimum supported version for the feature.
+ *  - maxVersion: This the maximum supported version for the feature.
+ *  - firstActiveVersion: This is the first active version for the feature. Versions in the range

Review comment:
       @junrao Thinking about it again, I see a way forward here. The key thing seems to be that you feel it is rare to deprecate feature versions in AK. I agree with the same. So, I propose we just do not have to solve that deprecation problem in this PR, until we find a clear route that the AK community agrees with.
   
   In this PR I propose that I revert the `firstActiveVersion` change, leaving the rest of the things the way they are. In the future, we can develop a concrete solution for version deprecation i.e. advancing `minVersion` of supported feature, potentially using `firstActiveVersion` or other ways (it is up for discussion, maybe in a separate KIP). What do you feel?
   
   Going back to your point:
   > I was thinking what if we relax the current check by just making sure that maxVersion of finalized is within the supported range. Basically in your example, if supported minVersion goes to 2, it's still allowed since it's less than maxVersion of finalized. However, if supported minVersion goes to 7, this fails the broker since it's more than maxVersion of finalized.
   
   There is a consequence to relaxing the current check:
   The controller can not effectively finalize `minVersionLevel` for the feature, because, with a relaxed check we do not know whether all brokers in the cluster support a particular `minVersion` when the controller finalizes the `minVersionLevel` at a particular value. It seems useful to keep the concept of `minVersionLevel` like the way it is now (i.e. it is the lowest version guaranteed to be supported by any broker in the cluster for a feature). And as I said above, in the future, we can decide on ways to mutate it safely (maybe through `firstActiveVersion` or other means).
   
   




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -272,6 +281,161 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (string) and a range of versions (defined by a
+   * SupportedVersionRange). It refers to a feature that a particular broker advertises support for.
+   * Each broker advertises the version ranges of its own supported features in its own
+   * BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (string) and a range of version levels (defined
+   * by a FinalizedVersionRange). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there was an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has now been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). But the IBP config is still set to lower than KAFKA_2_7_IV0, and may be
+   *    set to a higher value later. In this case, we want to start with no finalized features and
+   *    allow the user to finalize them whenever they are ready i.e. in the future whenever the
+   *    user sets IBP config to be greater than or equal to KAFKA_2_7_IV0, then the user could start
+   *    finalizing the features. This process ensures we do not enable all the possible features
+   *    immediately after an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent.
+   *        - If the node is absent, it will react by creating a FeatureZNode with disabled status
+   *          and empty finalized features.
+   *        - Otherwise, if a node already exists in enabled status then the controller will just
+   *          flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled.
+   *         - If the node is in disabled status, the controller won’t upgrade all features immediately.
+   *           Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *           user finalize the features later.
+   *         - Otherwise, if a node already exists in enabled status then the controller will leave
+   *           the node umodified.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine there was an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker
+   *    binary has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and
+   *    higher). The controller will start up and find that a FeatureZNode is already present with
+   *    enabled status and existing finalized features. In such a case, the controller leaves the node
+   *    unmodified.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled,
+                                          brokerFeatures.defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      val newFeatures = existingFeatureZNode.status match {
+        case FeatureZNodeStatus.Enabled => existingFeatureZNode.features
+        case FeatureZNodeStatus.Disabled =>
+          if (!existingFeatureZNode.features.empty()) {
+            warn(s"FeatureZNode at path: ${FeatureZNode.path} with disabled status" +
+              " contains non-empty features.")
+          }
+          Features.emptyFinalizedFeatures
+      }
+      val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures)

Review comment:
       Done. I have improved it now introducing a type definition called `FeatureZNodeStatus` that points to `Value`.
   IIUC you were referring to this LOC, correct?  https://github.com/apache/kafka/blob/4f96c5b424956355339dd3216c426c1c0388fe9e/core/src/main/scala/kafka/zk/ZkData.scala#L851
   Here the enum: `FeatureZNodeStatus` is defined and used in the same file.  I thought I'd add an `import` to fix it like the below, but it was a little unusual to add an `import` statement right above the class definition:
   
   ```
   import FeatureZNodeStatus._
   case class FeatureZNode(status: FeatureZNodeStatus, features: Features[FinalizedVersionRange]) {
   }
   ```
   
   With my recent change, in the future it should be possible to `import FeatureZNodeStatus._` within other files when referring to the enum value.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1215,71 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker. It can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter. This is particularly useful if the user requires strongly consistent reads of
+     * finalized features.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the {@link DescribeFeaturesResult} containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);

Review comment:
       Sorry, I do not understand why should describeFeatures (in post KIP-500) be handled only by controller?
   




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java
##########
@@ -17,9 +17,16 @@
 package org.apache.kafka.common.feature;
 
 import java.util.Map;
+import java.util.Objects;
+import org.apache.kafka.common.utils.Utils;
 
 /**
- * An extended {@link BaseVersionRange} representing the min/max versions for supported features.
+ * An extended {@link BaseVersionRange} representing the min, max and first active versions for a
+ * supported feature:
+ *  - minVersion: This is the minimum supported version for the feature.
+ *  - maxVersion: This the maximum supported version for the feature.
+ *  - firstActiveVersion: This is the first active version for the feature. Versions in the range

Review comment:
       @junrao Does the below feel right to you?
   
   The key thing seems to be that you feel it is rare to deprecate feature versions in AK. I agree with the same. So, I propose we just do not have to solve the deprecation problem in this PR, until we find a clear route that the AK community agrees with. In this PR I propose to revert the `firstActiveVersion` change, leaving the rest of the things the way they are. In the future, we can develop a concrete solution for version deprecation i.e. the part on how to advance `minVersion` of supported feature, may be (or may not be) using `firstActiveVersion` or other ways (it is up for discussion, maybe in a separate KIP).
   
   Going back to your point:
   > I was thinking what if we relax the current check by just making sure that maxVersion of finalized is within the supported range. Basically in your example, if supported minVersion goes to 2, it's still allowed since it's less than maxVersion of finalized. However, if supported minVersion goes to 7, this fails the broker since it's more than maxVersion of finalized.
   
   There is a consequence to relaxing the current check:
   The controller can not effectively finalize `minVersionLevel` for the feature, because, with a relaxed check we do not know whether all brokers in the cluster support a particular `minVersion` when the controller finalizes the `minVersionLevel` at a particular value. It seems useful to keep the concept of `minVersionLevel` like the way it is now (i.e. it is the lowest version guaranteed to be supported by any broker in the cluster for a feature). And as I said above, in the future, we can decide on ways to mutate it safely (maybe through `firstActiveVersion` or other means).
   
   




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java
##########
@@ -17,9 +17,16 @@
 package org.apache.kafka.common.feature;
 
 import java.util.Map;
+import java.util.Objects;
+import org.apache.kafka.common.utils.Utils;
 
 /**
- * An extended {@link BaseVersionRange} representing the min/max versions for supported features.
+ * An extended {@link BaseVersionRange} representing the min, max and first active versions for a
+ * supported feature:
+ *  - minVersion: This is the minimum supported version for the feature.
+ *  - maxVersion: This the maximum supported version for the feature.
+ *  - firstActiveVersion: This is the first active version for the feature. Versions in the range

Review comment:
       @kowshik : I was thinking what if we relax the current check by just making sure that maxVersion of finalized is within the supported range. Basically in your example, if supported minVersion goes to 2, it's still allowed since it's less than maxVersion of finalized. However, if supported minVersion goes to 7, this fails the broker since it's more than maxVersion of finalized.
   
   Your concern for the relaxed check seems to be around deploying a wrong version of the broker by mistake. I am not sure if that's a big concern. If the wrong broker affects maxVersion of finalized, the broker won't start. If the wrong broker affects minVersion of finalized, if we deprecated slowly, it won't impact the existing clients.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -112,8 +112,9 @@ class KafkaApis(val requestChannel: RequestChannel,
                 brokerTopicStats: BrokerTopicStats,
                 val clusterId: String,
                 time: Time,
-                val tokenManager: DelegationTokenManager)
-  extends ApiRequestHandler with Logging {
+                val tokenManager: DelegationTokenManager,
+                val brokerFeatures: BrokerFeatures,
+                val featureCache: FinalizedFeatureCache) extends ApiRequestHandler with Logging {

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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/errors/FeatureUpdateFailedException.java
##########
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.errors;
+
+public class FeatureUpdateFailedException extends ApiException {

Review comment:
       This exception corresponds to `Errors.FEATURE_UPDATE_FAILED`. The client can receive this exception during a call to `updateFeatures`, whenever a feature update can not be written to ZK. So this has to be a public 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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java
##########
@@ -17,9 +17,16 @@
 package org.apache.kafka.common.feature;
 
 import java.util.Map;
+import java.util.Objects;
+import org.apache.kafka.common.utils.Utils;
 
 /**
- * An extended {@link BaseVersionRange} representing the min/max versions for supported features.
+ * An extended {@link BaseVersionRange} representing the min, max and first active versions for a
+ * supported feature:
+ *  - minVersion: This is the minimum supported version for the feature.
+ *  - maxVersion: This the maximum supported version for the feature.
+ *  - firstActiveVersion: This is the first active version for the feature. Versions in the range

Review comment:
       @junrao Does the below feel right to you?
   
   The key thing seems to be that you feel it is rare to deprecate feature versions in AK. I agree with the same. So, I propose we just do not have to solve the deprecation problem in this PR, until we find a clear route that the AK community agrees with. In this PR I propose to revert the `firstActiveVersion` change, leaving the rest of the things the way they are. In the future, we can develop a concrete solution for version deprecation i.e. advancing `minVersion` of supported feature, potentially using `firstActiveVersion` or other ways (it is up for discussion, maybe in a separate KIP). What do you feel?
   
   Going back to your point:
   > I was thinking what if we relax the current check by just making sure that maxVersion of finalized is within the supported range. Basically in your example, if supported minVersion goes to 2, it's still allowed since it's less than maxVersion of finalized. However, if supported minVersion goes to 7, this fails the broker since it's more than maxVersion of finalized.
   
   There is a consequence to relaxing the current check:
   The controller can not effectively finalize `minVersionLevel` for the feature, because, with a relaxed check we do not know whether all brokers in the cluster support a particular `minVersion` when the controller finalizes the `minVersionLevel` at a particular value. It seems useful to keep the concept of `minVersionLevel` like the way it is now (i.e. it is the lowest version guaranteed to be supported by any broker in the cluster for a feature). And as I said above, in the future, we can decide on ways to mutate it safely (maybe through `firstActiveVersion` or other means).
   
   




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -2945,6 +2948,130 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFeatures(request: RequestChannel.Request): Unit = {

Review comment:
       This does not seem to be required, since it is already achieved via `UpdateFeaturesTest`. Infact there we test using admin client, which is even better as it tests e2e client to server functionality.
   
   What do we gain by adding the additional tests in `KafkaApisTest` ?




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java
##########
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+import java.util.Optional;
+import org.apache.kafka.common.feature.Features;
+import org.apache.kafka.common.feature.FinalizedVersionRange;
+import org.apache.kafka.common.feature.SupportedVersionRange;
+
+/**
+ * Encapsulates details about finalized as well as supported features. This is particularly useful
+ * to hold the result returned by the {@link Admin#describeFeatures(DescribeFeaturesOptions)} API.

Review comment:
       Done. I've removed those methods from the KIP.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.kafka.common.message.UpdateFeaturesResponseData;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.Struct;
+
+
+/**
+ * Possible error codes:
+ *
+ *   - {@link Errors#CLUSTER_AUTHORIZATION_FAILED}
+ *   - {@link Errors#NOT_CONTROLLER}
+ *   - {@link Errors#INVALID_REQUEST}
+ *   - {@link Errors#FEATURE_UPDATE_FAILED}
+ */
+public class UpdateFeaturesResponse extends AbstractResponse {
+
+    private final UpdateFeaturesResponseData data;
+
+    public UpdateFeaturesResponse(UpdateFeaturesResponseData data) {
+        this.data = data;
+    }
+
+    public UpdateFeaturesResponse(Struct struct) {
+        final short latestVersion = (short) (UpdateFeaturesResponseData.SCHEMAS.length - 1);
+        this.data = new UpdateFeaturesResponseData(struct, latestVersion);
+    }
+
+    public UpdateFeaturesResponse(Struct struct, short version) {
+        this.data = new UpdateFeaturesResponseData(struct, version);
+    }
+
+    public Map<String, ApiError> errors() {
+        return data.results().valuesSet().stream().collect(
+            Collectors.toMap(
+                result -> result.feature(),

Review comment:
       Like how? I don't understand. Isn't that what I'm doing currently?




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/resources/common/message/UpdateFeaturesRequest.json
##########
@@ -0,0 +1,35 @@
+// 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.
+
+{
+  "apiKey": 57,
+  "type": "request",
+  "name": "UpdateFeaturesRequest",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "timeoutMs", "type": "int32", "versions": "0+", "default": "60000",

Review comment:
       Done. Updated the KIP. Please refer to [this](https://cwiki.apache.org/confluence/display/KAFKA/KIP-584%3A+Versioning+scheme+for+features#KIP584:Versioningschemeforfeatures-UpdateFeaturesRequestschema) section.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,199 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of its own supported features in its
+   * own BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. This process ensures we do not enable all the possible features immediately after
+   *    an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, it will
+   *        react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, its existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) =>
+            val brokerDefaultVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (brokerDefaultVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (brokerDefaultVersionRange.max() >= existingVersionRange.max() &&
+                       brokerDefaultVersionRange.min() <= existingVersionRange.max()) {
+              // Using the change below, we deprecate all version levels in the range:
+              // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1].
+              //
+              // NOTE: if existingVersionRange.min() equals brokerDefaultVersionRange.min(), then
+              // we do not deprecate any version levels (since there is none to be deprecated).
+              //
+              // Examples:
+              // 1. brokerDefaultVersionRange = [4, 7] and existingVersionRange = [1, 5].
+              //    In this case, we deprecate all version levels in the range: [1, 3].
+              // 2. brokerDefaultVersionRange = [4, 7] and existingVersionRange = [4, 5].
+              //    In this case, we do not deprecate any version levels since
+              //    brokerDefaultVersionRange.min() equals existingVersionRange.min().
+              (featureName, new FinalizedVersionRange(brokerDefaultVersionRange.min(), existingVersionRange.max()))

Review comment:
       > When we roll the cluster to bump up IBP, it seems that it's possible for the min of finalized version to flip repeatedly? This can be a bit weird.
   
   True, this is possible. Good point. To be sure I understood, are you referring broadly to any future IBP bump? Or specifically are you referring to the IBP bump from a value less than KAFKA_2_7_IV0 to a value greater than or equal to KAFKA_2_7_IV0? (since KAFKA_2_7_IV0 is the IBP where the feature versioning system gets activated)
   
   To answer your question, I'm not sure how to avoid the flip. It is to be noted that min version level changes are used only for feature version deprecation. Due to the flipping values, it merely means some version levels would go a few times from deprecated -> available -> deprecated -> available...., until the IBP bump has been completed cluster-wide. I can't (yet) think of a case where the flip is dangerous, since:
   1. We have this check: https://github.com/apache/kafka/blob/89a3ba69e03acbe9635ee1039abb567bf0c6631b/core/src/main/scala/kafka/server/BrokerFeatures.scala#L47-L48  and
   2. As best practice, we can recommend to not change a) minVersion of SupportedFeature as well as b) default minVersionLevel within the same release. The reason being that we typically first deprecate a feature version level before we remove the code to drop support for it i.e. (b) usually has to happen before (a).




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,178 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature
+ *    version level deprecation. This is how it works: in order to deprecate feature version levels,
+ *    in this map the default minimum version level of a feature can be set to a new value that's
+ *    higher than 1 (let's call this latest_min_version_level). In doing so, the feature version levels
+ *    in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic
+ *    that applies this map to persistent finalized feature state in ZK (this mutation happens
+ *    during controller election and during finalized feature updates via the
+ *    ApiKeys.UPDATE_FINALIZED_FEATURES api). This will automatically mean external clients of Kafka
+ *    would need to stop using the finalized min version levels that have been deprecated.
+ *
+ * This class also provides APIs to check for incompatibilities between the features supported by
+ * the Broker and finalized features. This class is immutable in production. It provides few APIs to
+ * mutate state only for the purpose of testing.
+ */
+class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange],
+                              @volatile var defaultFeatureMinVersionLevels: Map[String, Short]) {
+  require(BrokerFeatures.areFeatureMinVersionLevelsCompatible(
+    supportedFeatures, defaultFeatureMinVersionLevels))
+
+  // For testing only.
+  def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(newFeatures, defaultFeatureMinVersionLevels))
+    supportedFeatures = newFeatures
+  }
+
+  /**
+   * Returns the default minimum version level for a specific feature.
+   *
+   * @param feature   the name of the feature
+   *
+   * @return          the default minimum version level for the feature if its defined.
+   *                  otherwise, returns 1.
+   */
+  def defaultMinVersionLevel(feature: String): Short = {
+    defaultFeatureMinVersionLevels.getOrElse(feature, 1)
+  }
+
+  // For testing only.
+  def setDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = {
+    require(
+      BrokerFeatures.areFeatureMinVersionLevelsCompatible(supportedFeatures, newMinVersionLevels))
+    defaultFeatureMinVersionLevels = newMinVersionLevels
+  }
+
+  /**
+   * Returns the default finalized features that a new Kafka cluster with IBP config >= KAFKA_2_7_IV0
+   * needs to be bootstrapped with.
+   */
+  def getDefaultFinalizedFeatures: Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(
+      supportedFeatures.features.asScala.map {
+        case(name, versionRange) => (
+          name, new FinalizedVersionRange(defaultMinVersionLevel(name), versionRange.max))
+      }.asJava)
+  }
+
+  /**
+   * Returns the set of feature names found to be incompatible.
+   * A feature incompatibility is a version mismatch between the latest feature supported by the
+   * Broker, and the provided finalized feature. This can happen because a provided finalized
+   * feature:
+   *  1) Does not exist in the Broker (i.e. it is unknown to the Broker).
+   *           [OR]
+   *  2) Exists but the FinalizedVersionRange does not match with the
+   *     supported feature's SupportedVersionRange.
+   *
+   * @param finalized   The finalized features against which incompatibilities need to be checked for.
+   *
+   * @return            The subset of input features which are incompatible. If the returned object
+   *                    is empty, it means there were no feature incompatibilities found.
+   */
+  def incompatibleFeatures(finalized: Features[FinalizedVersionRange]): Features[FinalizedVersionRange] = {
+    BrokerFeatures.incompatibleFeatures(supportedFeatures, finalized, logIncompatibilities = true)
+  }
+}
+
+object BrokerFeatures extends Logging {
+
+  def createDefault(): BrokerFeatures = {
+    // The arguments are currently empty, but, in the future as we define features we should
+    // populate the required values here.
+    new BrokerFeatures(emptySupportedFeatures, Map[String, Short]())
+  }
+
+  /**
+   * Returns true if any of the provided finalized features are incompatible with the provided
+   * supported features.
+   *
+   * @param supportedFeatures   The supported features to be compared
+   * @param finalizedFeatures   The finalized features to be compared
+   *
+   * @return                    - True if there are any feature incompatibilities found.
+   *                            - False otherwise.
+   */
+  def hasIncompatibleFeatures(supportedFeatures: Features[SupportedVersionRange],
+                              finalizedFeatures: Features[FinalizedVersionRange]): Boolean = {
+    !incompatibleFeatures(supportedFeatures, finalizedFeatures, false).empty
+  }
+
+  private def incompatibleFeatures(supportedFeatures: Features[SupportedVersionRange],
+                                   finalizedFeatures: Features[FinalizedVersionRange],
+                                   logIncompatibilities: Boolean): Features[FinalizedVersionRange] = {
+    val incompatibleFeaturesInfo = finalizedFeatures.features.asScala.map {
+      case (feature, versionLevels) =>
+        val supportedVersions = supportedFeatures.get(feature)
+        if (supportedVersions == null) {
+          (feature, versionLevels, "{feature=%s, reason='Unsupported feature'}".format(feature))
+        } else if (versionLevels.isIncompatibleWith(supportedVersions)) {
+          (feature, versionLevels, "{feature=%s, reason='%s is incompatible with %s'}".format(
+            feature, versionLevels, supportedVersions))
+        } else {
+          (feature, versionLevels, null)
+        }
+    }.filter{ case(_, _, errorReason) => errorReason != null}.toList
+
+    if (logIncompatibilities && incompatibleFeaturesInfo.nonEmpty) {
+      warn(
+        "Feature incompatibilities seen: " + incompatibleFeaturesInfo.map {
+          case(_, _, errorReason) => errorReason })
+    }
+    Features.finalizedFeatures(incompatibleFeaturesInfo.map {
+      case(feature, versionLevels, _) => (feature, versionLevels) }.toMap.asJava)
+  }
+
+  /**
+   * A check that ensures each feature defined with min version level is a supported feature, and
+   * the min version level value is valid (i.e. it is compatible with the supported version range).
+   *
+   * @param supportedFeatures         the supported features
+   * @param featureMinVersionLevels   the feature minimum version levels
+   *
+   * @return                          - true, if the above described check passes.
+   *                                  - false, otherwise.
+   */
+  private def areFeatureMinVersionLevelsCompatible(
+    supportedFeatures: Features[SupportedVersionRange],
+    featureMinVersionLevels: Map[String, Short]
+  ): Boolean = {
+    featureMinVersionLevels.forall {
+      case(featureName, minVersionLevel) =>
+        val supportedFeature = supportedFeatures.get(featureName)
+        (supportedFeature != null) &&
+          !new FinalizedVersionRange(minVersionLevel, supportedFeature.max())

Review comment:
       Existing approach is equally readable too. I'd rather leave it this way.




----------------------------------------------------------------
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] dajac commented on pull request #9001: KAFKA-10028: Implement KIP-584 write path

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


   @kowshik I just noticed that you haven't updated the code which creates the `ApiVersionsResponse` in `SaslServerAuthenticator`. Is it intentional or something left to be 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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesOptions.java
##########
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Map;
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+/**
+ * Options for {@link AdminClient#updateFeatures(Map, UpdateFeaturesOptions)}.
+ *
+ * The API of this class is evolving. See {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class UpdateFeaturesOptions extends AbstractOptions<UpdateFeaturesOptions> {

Review comment:
       Yes, it is already added. The base class: `AbstractOptions` contains a `timeoutMs` attribute and the value is set in the `UpdateFeaturesRequest`.




----------------------------------------------------------------
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 edited a comment on pull request #9001: KAFKA-10028: Implement KIP-584 write path

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #9001:
URL: https://github.com/apache/kafka/pull/9001#issuecomment-656902905


   @dajac Thank you for taking a look! IIUC you are referring to these lines:
   
   https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java#L550-L553
   
   My requirement is that under the hood of the newly added API: `org.apache.kafka.clients.Admin#describeFeatures`, the `ApiVersionsResponse` returned to the `AdminClient` needs to contain the features information. Note that this new API issues an explicit `ApiVersionsRequest` under the hood. In such a case do you think I should populate the features information in the above lines in `SaslServerAuthenticator` too? I'm trying to understand where would this come into play (sorry I know little to nothing about `SaslServerAuthenticator`).


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,179 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of it’s own supported features in its
+   * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the one and only entity modifying
+   * the information about finalized features and their version levels.
+   *
+   * This method sets up the FeatureZNode with enabled status. This status means the feature
+   * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode
+   * are active. This status should be written by the controller to the FeatureZNode only when the
+   * broker IBP config is greater than or equal to KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    Broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the Broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. The reason to do this is that enabling all the possible features immediately after
+   *    an upgrade could be harmful to the cluster.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, then it
+   *        will react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, it's existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) => {
+            val brokerDefaultVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (brokerDefaultVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (existingVersionRange.max() >= brokerDefaultVersionRange.min() &&
+                       brokerDefaultVersionRange.max() >= existingVersionRange.max()) {
+              // Through this change, we deprecate all version levels in the closed range:
+              // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1]
+              (featureName, new FinalizedVersionRange(brokerDefaultVersionRange.min(), existingVersionRange.max()))
+            } else {
+              // If the existing version levels fall completely outside the
+              // range of the default finalized version levels (i.e. no intersection), or, if the
+              // existing version levels are ineligible for a modification since they are
+              // incompatible with default finalized version levels, then we skip the update.
+              warn(s"Can not update minimum version level in finalized feature: $featureName,"
+                + s" since the existing $existingVersionRange is not eligible for a change"
+                + s" based on the default $brokerDefaultVersionRange.")
+              (featureName, existingVersionRange)
+            }
+          }
+        }.asJava)
+      }
+      val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures)
+      if (!newFeatureZNode.equals(existingFeatureZNode)) {

Review comment:
       Yes. If the existing node is `(disabled, {})` then we would like to change it to `(enabled, features)`. Therefore, we have to check the features as well as the `FeatureZNodeStatus`.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FinalizedVersionRange.java
##########
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+
+/**
+ * Represents a range of version levels supported by every broker in a cluster for some feature.
+ */
+public class FinalizedVersionRange {

Review comment:
       Do we want to have a different name from `org.apache.kafka.common.feature.FinalizedVersionRange`, such as `FinalizedVersionLevels`? Same case for `SupportedVersionRange`, personally I feel the same class name makes the navigation harder.

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,187 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the latest features supported by the Broker and also provides APIs to
+ * check for incompatibilities between the features supported by the Broker and finalized features.
+ * The class also enables feature version level deprecation, as explained below. This class is
+ * immutable in production. It provides few APIs to mutate state only for the purpose of testing.
+ *
+ * Feature version level deprecation:
+ * ==================================
+ *
+ * Deprecation of certain version levels of a feature is a process to stop supporting the
+ * functionality offered by the feature at those version levels, across the entire Kafka cluster.
+ * Feature version deprecation is a simple 2-step process explained below. In each step below, an
+ * example is provided to help understand the process better:
+ *
+ * STEP 1:
+ * =======
+ *
+ * In the first step, a major Kafka release is made with a Broker code change (explained later
+ * below) that establishes the intent to deprecate certain versions of one or more features
+ * cluster-wide. When this new Kafka release is deployed to the cluster, deprecated finalized
+ * feature versions are no longer advertised to the client, but they can still be used by existing
+ * connections. The way it works is that the feature versioning system (via the controller) will
+ * automatically persist the new minVersionLevel for the feature in ZK to propagate the deprecation
+ * of certain versions. After this happens, any external client that queries the Broker to learn the
+ * feature versions will at some point start to see the new value for the finalized minVersionLevel
+ * for the feature. The external clients are expected to stop using the deprecated versions at least
+ * by the time that they learn about it.
+ *
+ * Here is how the above code change needs to be done:
+ * In order to deprecate feature version levels, in the supportedFeatures map you need to supply a
+ * specific firstActiveVersion value that's higher than the minVersion for the feature. The
+ * value for firstActiveVersion should be 1 beyond the highest version that you intend to deprecate
+ * for that feature. Whenever the controller is elected or the features are finalized via the
+ * ApiKeys.UPDATE_FEATURES api, the feature version levels in the closed range:
+ * [minVersion, firstActiveVersion - 1] are automatically deprecated in ZK by the controller logic.
+ *
+ * Example:
+ * - Let us assume the existing finalized feature in ZK:
+ *   {
+ *      "feature" -> FinalizedVersionRange(minVersionLevel=1, maxVersionLevel=5)
+ *   }
+ *   Now, supposing you would like to deprecate feature version levels: [1, 2].
+ *   Then, in the supportedFeatures map you should supply the following:
+ *   supportedFeatures = {
+ *     "feature" -> SupportedVersionRange(minVersion=1, firstActiveVersion=3, maxVersion=5)
+ *   }
+ * - If you do NOT want to deprecate a version level for a feature, then, in the supportedFeatures

Review comment:
       I think we could just make the `firstActiveVersion = minVersion` by default, to avoid the requirement for configuring firstActiveVersion

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -20,26 +20,31 @@ package kafka.server
 import kafka.utils.Logging
 import org.apache.kafka.common.feature.{Features, FinalizedVersionRange}
 
+import scala.concurrent.TimeoutException
+import scala.math.max
+
 // Raised whenever there was an error in updating the FinalizedFeatureCache with features.
 class FeatureCacheUpdateException(message: String) extends RuntimeException(message) {
 }
 
 // Helper class that represents finalized features along with an epoch value.
-case class FinalizedFeaturesAndEpoch(features: Features[FinalizedVersionRange], epoch: Int) {
+case class FinalizedFeaturesAndEpoch(features: Features[FinalizedVersionRange], epoch: Long) {

Review comment:
       So we are saving the ZK epoch in a long, which was supposed to be an int field?

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,187 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the latest features supported by the Broker and also provides APIs to
+ * check for incompatibilities between the features supported by the Broker and finalized features.
+ * The class also enables feature version level deprecation, as explained below. This class is
+ * immutable in production. It provides few APIs to mutate state only for the purpose of testing.
+ *
+ * Feature version level deprecation:
+ * ==================================
+ *
+ * Deprecation of certain version levels of a feature is a process to stop supporting the
+ * functionality offered by the feature at those version levels, across the entire Kafka cluster.
+ * Feature version deprecation is a simple 2-step process explained below. In each step below, an
+ * example is provided to help understand the process better:
+ *
+ * STEP 1:
+ * =======
+ *
+ * In the first step, a major Kafka release is made with a Broker code change (explained later
+ * below) that establishes the intent to deprecate certain versions of one or more features
+ * cluster-wide. When this new Kafka release is deployed to the cluster, deprecated finalized
+ * feature versions are no longer advertised to the client, but they can still be used by existing
+ * connections. The way it works is that the feature versioning system (via the controller) will
+ * automatically persist the new minVersionLevel for the feature in ZK to propagate the deprecation
+ * of certain versions. After this happens, any external client that queries the Broker to learn the
+ * feature versions will at some point start to see the new value for the finalized minVersionLevel
+ * for the feature. The external clients are expected to stop using the deprecated versions at least
+ * by the time that they learn about it.
+ *
+ * Here is how the above code change needs to be done:
+ * In order to deprecate feature version levels, in the supportedFeatures map you need to supply a
+ * specific firstActiveVersion value that's higher than the minVersion for the feature. The
+ * value for firstActiveVersion should be 1 beyond the highest version that you intend to deprecate
+ * for that feature. Whenever the controller is elected or the features are finalized via the
+ * ApiKeys.UPDATE_FEATURES api, the feature version levels in the closed range:
+ * [minVersion, firstActiveVersion - 1] are automatically deprecated in ZK by the controller logic.
+ *
+ * Example:
+ * - Let us assume the existing finalized feature in ZK:
+ *   {
+ *      "feature" -> FinalizedVersionRange(minVersionLevel=1, maxVersionLevel=5)
+ *   }
+ *   Now, supposing you would like to deprecate feature version levels: [1, 2].
+ *   Then, in the supportedFeatures map you should supply the following:
+ *   supportedFeatures = {
+ *     "feature" -> SupportedVersionRange(minVersion=1, firstActiveVersion=3, maxVersion=5)
+ *   }
+ * - If you do NOT want to deprecate a version level for a feature, then, in the supportedFeatures
+ *   map you should supply the firstActiveVersion to be the same as the minVersion supplied for that
+ *   feature.
+ *   Example:
+ *   supportedFeatures = {
+ *     "feature" -> SupportedVersionRange(minVersion=1, firstActiveVersion=1, maxVersion=5)
+ *   }
+ *   The above indicates no intent to deprecate any version levels for the feature.
+ *
+ * STEP 2:
+ * =======
+ *
+ * After the first step is over, you may (at some point) want to permanently remove the code/logic
+ * for the functionality offered by the deprecated feature versions. This is the second step. Here a
+ * subsequent major Kafka release is made with another Broker code change that removes the code for
+ * the functionality offered by the deprecated feature versions. This would completely drop support
+ * for the deprecated versions. Such a code change needs to be supplemented by supplying a
+ * suitable higher minVersion value for the feature in the supportedFeatures map.
+ * Example:
+ * - In the example above in step 1, we showed how to deprecate version levels [1, 2] for
+ *   "feature". Now let us assume the following finalized feature in ZK (after the deprecation
+ *   has been carried out):
+ *   {
+ *     "feature" -> FinalizedVersionRange(minVersionLevel=3, maxVersionLevel=5)
+ *   }
+ *   Now, supposing you would like to permanently remove support for feature versions: [1, 2].
+ *   Then, in the supportedFeatures map you should now supply the following:

Review comment:
       Similar here to make `firstActiveVersion = minVersion` as default.




----------------------------------------------------------------
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] chia7712 commented on a change in pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4335,6 +4343,150 @@ void handleFailure(Throwable throwable) {
                 .hi(password, salt, iterations);
     }
 
+    public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        final NodeProvider provider =
+            options.sendRequestToController() ? new ControllerNodeProvider() : new LeastLoadedNodeProvider();
+
+        final Call call = new Call(
+            "describeFeatures", calcDeadlineMs(now, options.timeoutMs()), provider) {
+
+            private FeatureMetadata createFeatureMetadata(final ApiVersionsResponse response) {
+                final Map<String, FinalizedVersionRange> finalizedFeatures = new HashMap<>();
+                for (final FinalizedFeatureKey key : response.data().finalizedFeatures().valuesSet()) {
+                    finalizedFeatures.put(key.name(), new FinalizedVersionRange(key.minVersionLevel(), key.maxVersionLevel()));
+                }
+
+                Optional<Long> finalizedFeaturesEpoch;
+                if (response.data().finalizedFeaturesEpoch() >= 0L) {
+                    finalizedFeaturesEpoch = Optional.of(response.data().finalizedFeaturesEpoch());
+                } else {
+                    finalizedFeaturesEpoch = Optional.empty();
+                }
+
+                final Map<String, SupportedVersionRange> supportedFeatures = new HashMap<>();
+                for (final SupportedFeatureKey key : response.data().supportedFeatures().valuesSet()) {
+                    supportedFeatures.put(key.name(), new SupportedVersionRange(key.minVersion(), key.maxVersion()));
+                }
+
+                return new FeatureMetadata(finalizedFeatures, finalizedFeaturesEpoch, supportedFeatures);
+            }
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(createFeatureMetadata(apiVersionsResponse));
+                } else if (options.sendRequestToController() &&
+                           apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) {
+                    handleNotControllerError(Errors.NOT_CONTROLLER);
+                } else {
+                    future.completeExceptionally(Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        runnable.call(call, now);
+        return new DescribeFeaturesResult(future);
+    }
+
+    @Override
+    public UpdateFeaturesResult updateFeatures(final Map<String, FeatureUpdate> featureUpdates,
+                                               final UpdateFeaturesOptions options) {
+        if (featureUpdates.isEmpty()) {
+            throw new IllegalArgumentException("Feature updates can not be null or empty.");
+        }
+
+        final Map<String, KafkaFutureImpl<Void>> updateFutures = new HashMap<>();
+        for (final Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+            updateFutures.put(entry.getKey(), new KafkaFutureImpl<>());
+        }
+
+        final long now = time.milliseconds();
+        final Call call = new Call("updateFeatures", calcDeadlineMs(now, options.timeoutMs()),
+            new ControllerNodeProvider()) {
+
+            @Override
+            UpdateFeaturesRequest.Builder createRequest(int timeoutMs) {
+                final UpdateFeaturesRequestData.FeatureUpdateKeyCollection featureUpdatesRequestData
+                    = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection();
+                for (Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+                    final String feature = entry.getKey();
+                    final FeatureUpdate update = entry.getValue();
+                    if (feature.trim().isEmpty()) {
+                        throw new IllegalArgumentException("Provided feature can not be null or empty.");
+                    }
+
+                    final UpdateFeaturesRequestData.FeatureUpdateKey requestItem =
+                        new UpdateFeaturesRequestData.FeatureUpdateKey();
+                    requestItem.setFeature(feature);
+                    requestItem.setMaxVersionLevel(update.maxVersionLevel());
+                    requestItem.setAllowDowngrade(update.allowDowngrade());
+                    featureUpdatesRequestData.add(requestItem);
+                }
+                return new UpdateFeaturesRequest.Builder(
+                    new UpdateFeaturesRequestData()
+                        .setTimeoutMs(timeoutMs)
+                        .setFeatureUpdates(featureUpdatesRequestData));
+            }
+
+            @Override
+            void handleResponse(AbstractResponse abstractResponse) {
+                final UpdateFeaturesResponse response =
+                    (UpdateFeaturesResponse) abstractResponse;
+
+                Errors topLevelError = Errors.forCode(response.data().errorCode());
+                switch (topLevelError) {
+                    case NONE:
+                        for (final UpdatableFeatureResult result : response.data().results()) {
+                            final KafkaFutureImpl<Void> future = updateFutures.get(result.feature());
+                            if (future == null) {
+                                log.warn("Server response mentioned unknown feature {}", result.feature());
+                            } else {
+                                final Errors error = Errors.forCode(result.errorCode());
+                                if (error == Errors.NONE) {
+                                    future.complete(null);
+                                } else {
+                                    future.completeExceptionally(error.exception(result.errorMessage()));
+                                }
+                            }
+                        }
+                        // The server should send back a response for every feature, but we do a sanity check anyway.
+                        completeUnrealizedFutures(updateFutures.entrySet().stream(),
+                            feature -> "The controller response did not contain a result for feature " + feature);
+                        break;
+                    case NOT_CONTROLLER:
+                        handleNotControllerError(topLevelError);
+                        break;
+                    default:
+                        for (final Map.Entry<String, KafkaFutureImpl<Void>> entry : updateFutures.entrySet()) {
+                            entry.getValue().completeExceptionally(topLevelError.exception());

Review comment:
       the top-level error message is not propagated.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,178 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of its own supported features in its
+   * own BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. This process ensures we do not enable all the possible features immediately after
+   *    an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, it will
+   *        react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, its existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) =>
+            val brokerDefaultVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (brokerDefaultVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (brokerDefaultVersionRange.max() >= existingVersionRange.max() &&
+                       brokerDefaultVersionRange.min() <= existingVersionRange.max()) {
+              // Through this change, we deprecate all version levels in the closed range:
+              // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1]
+              (featureName, new FinalizedVersionRange(brokerDefaultVersionRange.min(), existingVersionRange.max()))
+            } else {
+              // If the existing version levels fall completely outside the
+              // range of the default finalized version levels (i.e. no intersection), or, if the
+              // existing version levels are ineligible for a modification since they are
+              // incompatible with default finalized version levels, then we skip the update.
+              warn(s"Can not update minimum version level in finalized feature: $featureName,"
+                + s" since the existing $existingVersionRange is not eligible for a change"
+                + s" based on the default $brokerDefaultVersionRange.")
+              (featureName, existingVersionRange)
+            }
+        }.asJava)
+      }
+      val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures)
+      if (!newFeatureZNode.equals(existingFeatureZNode)) {
+        val newVersion = updateFeatureZNode(newFeatureZNode)
+        featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+      }
+    }
+  }
+
+  /**
+   * Disables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with disabled status. This status means the feature versioning system
+   * (KIP-584) is disabled, and, the finalized features stored in the FeatureZNode are not relevant.
+   * This status should be written by the controller to the FeatureZNode only when the broker
+   * IBP config is less than KAFKA_2_7_IV0.
+   *
+   * NOTE:
+   * 1. When this method returns, existing finalized features (if any) will be cleared from the
+   *    FeatureZNode.
+   * 2. This method, unlike enableFeatureVersioning() need not wait for the FinalizedFeatureCache
+   *    to be updated, because, such updates to the cache (via FinalizedFeatureChangeListener)
+   *    are disabled when IBP config is < than KAFKA_2_7_IV0.
+   */
+  private def disableFeatureVersioning(): Unit = {
+    val newNode = FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures())
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      createFeatureZNode(newNode)

Review comment:
       No, that is not required. Please refer to the documentation above under `NOTE` for this method where I have explained why.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1844,188 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns a suitable error.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+    // NOTE: Below we set the finalized min version level to be the default minimum version
+    // level. If the finalized feature already exists, then, this can cause deprecation of all
+    // version levels in the closed range:
+    // [existingVersionRange.min(), defaultMinVersionLevel - 1].
+    val defaultMinVersionLevel = brokerFeatures.defaultMinVersionLevel(update.feature)
+    val newVersionRange = new FinalizedVersionRange(defaultMinVersionLevel, update.maxVersionLevel)
+    val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+      val singleFinalizedFeature =
+        Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange)))
+      BrokerFeatures.hasIncompatibleFeatures(broker.features, singleFinalizedFeature)
+    })
+    if (numIncompatibleBrokers == 0) {
+      Left(newVersionRange)
+    } else {
+      Right(
+        new ApiError(Errors.INVALID_REQUEST,
+                     s"Could not apply finalized feature update because $numIncompatibleBrokers" +
+                     " brokers were found to have incompatible features."))
+    }
+  }
+
+  /**
+   * Validate and process a finalized feature update.
+   *
+   * If the processing is successful, then, the return value contains:
+   * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature.
+   * 2. Option.empty, if the feature update was meant to delete the feature.
+   *
+   * If the processing failed, then returned value contains a suitable ApiError.
+   *
+   * @param update   the feature update to be processed.
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def processFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = {
+    val existingFeatures = featureCache.get
+      .map(featuresAndEpoch => featuresAndEpoch.features.features().asScala)
+      .getOrElse(Map[String, FinalizedVersionRange]())
+
+    def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = {
+      newFinalizedVersionRangeOrIncompatibilityError(update)
+        .fold(versionRange => Left(Some(versionRange)), error => Right(error))
+    }
+
+    if (update.feature.isEmpty) {
+      // Check that the feature name is not empty.
+      Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty."))
+    } else {
+      val cacheEntry = existingFeatures.get(update.feature).orNull
+
+      // We handle deletion requests separately from non-deletion requests.
+      if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+        if (cacheEntry == null) {
+          // Disallow deletion of a non-existing finalized feature.
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             s"Can not delete non-existing finalized feature: '${update.feature}'"))
+        } else {
+          Left(Option.empty)
+        }
+      } else if (update.maxVersionLevel() < 1) {
+        // Disallow deletion of a finalized feature without allowDowngrade flag set.
+        Right(new ApiError(Errors.INVALID_REQUEST,
+                           s"Can not provide maxVersionLevel: ${update.maxVersionLevel} less" +
+                           s" than 1 for feature: '${update.feature}' without setting the" +
+                           " allowDowngrade flag to true in the request."))
+      } else {
+        if (cacheEntry == null) {
+          newVersionRangeOrError(update)
+        } else {
+          if (update.maxVersionLevel == cacheEntry.max()) {
+            // Disallow a case where target maxVersionLevel matches existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not ${if (update.allowDowngrade) "downgrade" else "upgrade"}" +
+                               s" a finalized feature: '${update.feature}' from existing" +
+                               s" maxVersionLevel:${cacheEntry.max} to the same value."))
+          } else if (update.maxVersionLevel < cacheEntry.max && !update.allowDowngrade) {
+            // Disallow downgrade of a finalized feature without the allowDowngrade flag set.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature: '${update.feature}' from" +
+                               s" existing maxVersionLevel:${cacheEntry.max} to provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} without setting the" +
+                               " allowDowngrade flag in the request."))
+          } else if (update.allowDowngrade && update.maxVersionLevel > cacheEntry.max) {

Review comment:
       Updated the doc. Let's keep the check, if it happens then it's a user error. Especially because this can not happen if the user is using the tooling that we are going to provide in AK.

##########
File path: clients/src/main/resources/common/message/UpdateFeaturesRequest.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "request",
+  "name": "UpdateFeaturesRequest",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "FeatureUpdates", "type": "[]FeatureUpdateKey", "versions": "0+",
+      "about": "The list of updates to finalized features.", "fields": [
+      {"name": "Feature", "type": "string", "versions": "0+", "mapKey": true,
+        "about": "The name of the finalized feature to be updated."},
+      {"name": "MaxVersionLevel", "type": "int16", "versions": "0+",
+        "about": "The new maximum version level for the finalized feature. A value >= 1 is valid. A value < 1, is special, and can be used to request the deletion of the finalized feature."},
+      {"name": "AllowDowngrade", "type": "bool", "versions": "0+",
+        "about": "When set to true, the finalized feature version level is allowed to be downgraded/deleted."}

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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala
##########
@@ -185,7 +185,7 @@ class BrokerEndPointTest {
       "endpoints":["CLIENT://host1:9092", "REPLICATION://host1:9093"],
       "listener_security_protocol_map":{"CLIENT":"SSL", "REPLICATION":"PLAINTEXT"},
       "rack":"dc1",
-      "features": {"feature1": {"min_version": 1, "max_version": 2}, "feature2": {"min_version": 2, "max_version": 4}}
+      "features": {"feature1": {"min_version": 1, "first_active_version": 1, "max_version": 2}, "feature2": {"min_version": 2, "first_active_version": 2, "max_version": 4}}

Review comment:
       Done. Nice catch!




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java
##########
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import static java.util.stream.Collectors.joining;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * Encapsulates details about finalized as well as supported features. This is particularly useful
+ * to hold the result returned by the {@link Admin#describeFeatures(DescribeFeaturesOptions)} API.
+ */
+public class FeatureMetadata {
+
+    private final Map<String, FinalizedVersionRange> finalizedFeatures;
+
+    private final Optional<Long> finalizedFeaturesEpoch;
+
+    private final Map<String, SupportedVersionRange> supportedFeatures;
+
+    public FeatureMetadata(final Map<String, FinalizedVersionRange> finalizedFeatures,

Review comment:
       It is instantiated from `kafka.server.UpdateFeaturesTest`, so have to keep the c'tor public.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/resources/common/message/UpdateFeaturesResponse.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "response",
+  "name": "UpdateFeaturesResponse",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [

Review comment:
       Done. I have added a top-level error code now.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1215,71 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker. It can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter. This is particularly useful if the user requires strongly consistent reads of
+     * finalized features.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the {@link DescribeFeaturesResult} containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);
+
+    /**
+     * Applies specified updates to finalized features. This operation is not transactional so it
+     * may succeed for some features while fail for others.
+     * <p>
+     * The API takes in a map of finalized feature name to {@link FeatureUpdate} that needs to be
+     * applied. Each entry in the map specifies the finalized feature to be added or updated or
+     * deleted, along with the new max feature version level value. This request is issued only to
+     * the controller since the API is only served by the controller. The return value contains an
+     * error code for each supplied {@link FeatureUpdate}, and the code indicates if the update
+     * succeeded or failed in the controller.
+     * <ul>
+     * <li>Downgrade of feature version level is not a regular operation/intent. It is only allowed
+     * in the controller if the {@link FeatureUpdate} has the allowDowngrade flag set - setting this
+     * flag conveys user intent to attempt downgrade of a feature max version level. Note that
+     * despite the allowDowngrade flag being set, certain downgrades may be rejected by the
+     * controller if it is deemed impossible.</li>
+     * <li>Deletion of a finalized feature version is not a regular operation/intent. It could be
+     * done by setting the allowDowngrade flag to true in the {@link FeatureUpdate}, and, setting
+     * the max version level to be less than 1.</li>
+     * </ul>
+     *<p>
+     * The following exceptions can be anticipated when calling {@code get()} on the futures
+     * obtained from the returned {@link UpdateFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.ClusterAuthorizationException}
+     *   If the authenticated user didn't have alter access to the cluster.</li>
+     *   <li>{@link org.apache.kafka.common.errors.InvalidRequestException}

Review comment:
       Done.

##########
File path: clients/src/main/resources/common/message/UpdateFeaturesRequest.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "request",
+  "name": "UpdateFeaturesRequest",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "FeatureUpdates", "type": "[]FeatureUpdateKey", "versions": "0+",
+      "about": "The list of updates to finalized features.", "fields": [
+      {"name": "Feature", "type": "string", "versions": "0+", "mapKey": true,
+        "about": "The name of the finalized feature to be updated."},
+      {"name": "MaxVersionLevel", "type": "int16", "versions": "0+",
+        "about": "The new maximum version level for the finalized feature. A value >= 1 is valid. A value < 1, is special, and can be used to request the deletion of the finalized feature."},
+      {"name": "AllowDowngrade", "type": "bool", "versions": "0+",

Review comment:
       Done. Fixed now.

##########
File path: clients/src/main/resources/common/message/UpdateFeaturesResponse.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "response",
+  "name": "UpdateFeaturesResponse",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "Results", "type": "[]UpdatableFeatureResult", "versions": "0+",
+      "about": "Results for each feature update.", "fields": [
+      {"name": "Feature", "type": "string", "versions": "0+", "mapKey": true,

Review comment:
       Done. I've updated the KIP-584 write up, please refer to [this section](https://cwiki.apache.org/confluence/display/KAFKA/KIP-584%3A+Versioning+scheme+for+features#KIP584:Versioningschemeforfeatures-UpdateFeaturesResponseschema) in the KIP.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java
##########
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+import java.util.Optional;
+import org.apache.kafka.common.feature.Features;
+import org.apache.kafka.common.feature.FinalizedVersionRange;
+import org.apache.kafka.common.feature.SupportedVersionRange;
+
+/**
+ * Encapsulates details about finalized as well as supported features. This is particularly useful
+ * to hold the result returned by the {@link Admin#describeFeatures(DescribeFeaturesOptions)} API.
+ */
+public class FeatureMetadata {
+
+    private final Features<FinalizedVersionRange> finalizedFeatures;
+
+    private final Optional<Integer> finalizedFeaturesEpoch;
+
+    private final Features<SupportedVersionRange> supportedFeatures;
+
+    public FeatureMetadata(final Features<FinalizedVersionRange> finalizedFeatures,
+                           final int finalizedFeaturesEpoch,
+                           final Features<SupportedVersionRange> supportedFeatures) {
+        Objects.requireNonNull(finalizedFeatures, "Provided finalizedFeatures can not be null.");
+        Objects.requireNonNull(supportedFeatures, "Provided supportedFeatures can not be null.");
+        this.finalizedFeatures = finalizedFeatures;
+        if (finalizedFeaturesEpoch >= 0) {
+            this.finalizedFeaturesEpoch = Optional.of(finalizedFeaturesEpoch);
+        } else {
+            this.finalizedFeaturesEpoch = Optional.empty();
+        }
+        this.supportedFeatures = supportedFeatures;
+    }
+
+    /**
+     * A map of finalized feature versions, with key being finalized feature name and value
+     * containing the min/max version levels for the finalized feature.
+     */
+    public Features<FinalizedVersionRange> finalizedFeatures() {

Review comment:
       Done. I've fixed this now to align with the KIP.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java
##########
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+import java.util.Optional;
+import org.apache.kafka.common.feature.Features;
+import org.apache.kafka.common.feature.FinalizedVersionRange;
+import org.apache.kafka.common.feature.SupportedVersionRange;
+
+/**
+ * Encapsulates details about finalized as well as supported features. This is particularly useful
+ * to hold the result returned by the {@link Admin#describeFeatures(DescribeFeaturesOptions)} API.
+ */
+public class FeatureMetadata {
+
+    private final Features<FinalizedVersionRange> finalizedFeatures;
+
+    private final Optional<Integer> finalizedFeaturesEpoch;
+
+    private final Features<SupportedVersionRange> supportedFeatures;
+
+    public FeatureMetadata(final Features<FinalizedVersionRange> finalizedFeatures,
+                           final int finalizedFeaturesEpoch,
+                           final Features<SupportedVersionRange> supportedFeatures) {
+        Objects.requireNonNull(finalizedFeatures, "Provided finalizedFeatures can not be null.");
+        Objects.requireNonNull(supportedFeatures, "Provided supportedFeatures can not be null.");
+        this.finalizedFeatures = finalizedFeatures;
+        if (finalizedFeaturesEpoch >= 0) {
+            this.finalizedFeaturesEpoch = Optional.of(finalizedFeaturesEpoch);
+        } else {
+            this.finalizedFeaturesEpoch = Optional.empty();
+        }
+        this.supportedFeatures = supportedFeatures;
+    }
+
+    /**
+     * A map of finalized feature versions, with key being finalized feature name and value
+     * containing the min/max version levels for the finalized feature.
+     */
+    public Features<FinalizedVersionRange> finalizedFeatures() {
+        return finalizedFeatures;
+    }
+
+    /**
+     * The epoch for the finalized features.
+     * If the returned value is empty, it means the finalized features are absent/unavailable.
+     */
+    public Optional<Integer> finalizedFeaturesEpoch() {

Review comment:
       Done. I've updated the KIP to use `Optional<Integer>` as well.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java
##########
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+import java.util.Optional;
+import org.apache.kafka.common.feature.Features;
+import org.apache.kafka.common.feature.FinalizedVersionRange;
+import org.apache.kafka.common.feature.SupportedVersionRange;
+
+/**
+ * Encapsulates details about finalized as well as supported features. This is particularly useful
+ * to hold the result returned by the {@link Admin#describeFeatures(DescribeFeaturesOptions)} API.

Review comment:
       Done. I've removed those methods from the KIP.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1216,71 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker. It can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter. This is particularly useful if the user requires strongly consistent reads of
+     * finalized features.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the {@link DescribeFeaturesResult} containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);

Review comment:
       Done. I've updated the KIP to mention `DescribeFeaturesOptions`.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1216,71 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker. It can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter. This is particularly useful if the user requires strongly consistent reads of
+     * finalized features.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the {@link DescribeFeaturesResult} containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);
+
+    /**
+     * Applies specified updates to finalized features. This operation is not transactional so it
+     * may succeed for some features while fail for others.
+     * <p>
+     * The API takes in a map of finalized feature names to {@link FeatureUpdate} that needs to be
+     * applied. Each entry in the map specifies the finalized feature to be added or updated or
+     * deleted, along with the new max feature version level value. This request is issued only to
+     * the controller since the API is only served by the controller. The return value contains an
+     * error code for each supplied {@link FeatureUpdate}, and the code indicates if the update
+     * succeeded or failed in the controller.
+     * <ul>
+     * <li>Downgrade of feature version level is not a regular operation/intent. It is only allowed
+     * in the controller if the {@link FeatureUpdate} has the allowDowngrade flag set - setting this
+     * flag conveys user intent to attempt downgrade of a feature max version level. Note that
+     * despite the allowDowngrade flag being set, certain downgrades may be rejected by the
+     * controller if it is deemed impossible.</li>
+     * <li>Deletion of a finalized feature version is not a regular operation/intent. It could be
+     * done by setting the allowDowngrade flag to true in the {@link FeatureUpdate}, and, setting
+     * the max version level to be less than 1.</li>
+     * </ul>
+     *<p>
+     * The following exceptions can be anticipated when calling {@code get()} on the futures
+     * obtained from the returned {@link UpdateFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.ClusterAuthorizationException}
+     *   If the authenticated user didn't have alter access to the cluster.</li>
+     *   <li>{@link org.apache.kafka.common.errors.InvalidRequestException}
+     *   If the request details are invalid. e.g., a non-existing finalized feature is attempted
+     *   to be deleted or downgraded.</li>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the updates could finish. It cannot be guaranteed whether
+     *   the updates succeeded or not.</li>
+     *   <li>{@link FeatureUpdateFailedException}
+     *   If the updates could not be applied on the controller, despite the request being valid.
+     *   This may be a temporary problem.</li>
+     * </ul>
+     * <p>
+     * This operation is supported by brokers with version 2.7.0 or higher.
+
+     * @param featureUpdates   the map of finalized feature name to {@link FeatureUpdate}
+     * @param options          the options to use
+     *
+     * @return                 the {@link UpdateFeaturesResult} containing the result
+     */
+    UpdateFeaturesResult updateFeatures(Map<String, FeatureUpdate> featureUpdates, UpdateFeaturesOptions options);

Review comment:
       Done. I've updated the KIP to align with whats used here.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1216,71 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker. It can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter. This is particularly useful if the user requires strongly consistent reads of
+     * finalized features.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the {@link DescribeFeaturesResult} containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);
+
+    /**
+     * Applies specified updates to finalized features. This operation is not transactional so it
+     * may succeed for some features while fail for others.
+     * <p>
+     * The API takes in a map of finalized feature names to {@link FeatureUpdate} that needs to be
+     * applied. Each entry in the map specifies the finalized feature to be added or updated or
+     * deleted, along with the new max feature version level value. This request is issued only to
+     * the controller since the API is only served by the controller. The return value contains an
+     * error code for each supplied {@link FeatureUpdate}, and the code indicates if the update
+     * succeeded or failed in the controller.
+     * <ul>
+     * <li>Downgrade of feature version level is not a regular operation/intent. It is only allowed
+     * in the controller if the {@link FeatureUpdate} has the allowDowngrade flag set - setting this
+     * flag conveys user intent to attempt downgrade of a feature max version level. Note that
+     * despite the allowDowngrade flag being set, certain downgrades may be rejected by the
+     * controller if it is deemed impossible.</li>
+     * <li>Deletion of a finalized feature version is not a regular operation/intent. It could be
+     * done by setting the allowDowngrade flag to true in the {@link FeatureUpdate}, and, setting
+     * the max version level to be less than 1.</li>
+     * </ul>
+     *<p>
+     * The following exceptions can be anticipated when calling {@code get()} on the futures
+     * obtained from the returned {@link UpdateFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.ClusterAuthorizationException}
+     *   If the authenticated user didn't have alter access to the cluster.</li>
+     *   <li>{@link org.apache.kafka.common.errors.InvalidRequestException}
+     *   If the request details are invalid. e.g., a non-existing finalized feature is attempted
+     *   to be deleted or downgraded.</li>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the updates could finish. It cannot be guaranteed whether
+     *   the updates succeeded or not.</li>
+     *   <li>{@link FeatureUpdateFailedException}
+     *   If the updates could not be applied on the controller, despite the request being valid.
+     *   This may be a temporary problem.</li>
+     * </ul>
+     * <p>
+     * This operation is supported by brokers with version 2.7.0 or higher.
+
+     * @param featureUpdates   the map of finalized feature name to {@link FeatureUpdate}
+     * @param options          the options to use
+     *
+     * @return                 the {@link UpdateFeaturesResult} containing the result
+     */
+    UpdateFeaturesResult updateFeatures(Map<String, FeatureUpdate> featureUpdates, UpdateFeaturesOptions options);

Review comment:
       Done. I've updated the KIP to align with whats used here, so both are the same now.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesResult.java
##########
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Map;
+import org.apache.kafka.common.KafkaFuture;
+
+/**
+ * The result of the {@link Admin#updateFeatures(Map, UpdateFeaturesOptions)} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+public class UpdateFeaturesResult {
+    private final Map<String, KafkaFuture<Void>> futures;
+
+    /**
+     * @param futures   a map from feature names to future, which can be used to check the status of
+     *                  individual feature updates.
+     */
+    public UpdateFeaturesResult(final Map<String, KafkaFuture<Void>> futures) {
+        this.futures = futures;
+    }
+
+    public Map<String, KafkaFuture<Void>> values() {

Review comment:
       Done. The KIP has been updated to have this method now.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4071,6 +4078,113 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        final NodeProvider provider =
+            options.sendRequestToController() ? new ControllerNodeProvider() : new LeastLoadedNodeProvider();
+
+        Call call = new Call(
+            "describeFeatures", calcDeadlineMs(now, options.timeoutMs()), provider) {
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(
+                        new FeatureMetadata(
+                            apiVersionsResponse.finalizedFeatures(),
+                            apiVersionsResponse.finalizedFeaturesEpoch(),
+                            apiVersionsResponse.supportedFeatures()));
+                } else if (options.sendRequestToController() && apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) {
+                    handleNotControllerError(Errors.NOT_CONTROLLER);
+                } else {
+                    future.completeExceptionally(
+                        Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        runnable.call(call, now);
+        return new DescribeFeaturesResult(future);
+    }
+
+    @Override
+    public UpdateFeaturesResult updateFeatures(
+        final Map<String, FeatureUpdate> featureUpdates, final UpdateFeaturesOptions options) {
+        if (featureUpdates == null || featureUpdates.isEmpty()) {
+            throw new IllegalArgumentException("Feature updates can not be null or empty.");
+        }
+        Objects.requireNonNull(options, "UpdateFeaturesOptions can not be null");
+
+        final UpdateFeaturesRequestData request = UpdateFeaturesRequest.create(featureUpdates);
+        final Map<String, KafkaFutureImpl<Void>> updateFutures = new HashMap<>();
+        for (Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+            updateFutures.put(entry.getKey(), new KafkaFutureImpl<>());
+        }
+        final long now = time.milliseconds();
+        final Call call = new Call("updateFeatures", calcDeadlineMs(now, options.timeoutMs()),
+            new ControllerNodeProvider()) {
+
+            @Override
+            UpdateFeaturesRequest.Builder createRequest(int timeoutMs) {
+                return new UpdateFeaturesRequest.Builder(request);
+            }
+
+            @Override
+            void handleResponse(AbstractResponse abstractResponse) {
+                final UpdateFeaturesResponse response =
+                    (UpdateFeaturesResponse) abstractResponse;
+
+                // Check for controller change.
+                for (UpdatableFeatureResult result : response.data().results()) {
+                    final Errors error = Errors.forCode(result.errorCode());
+                    if (error == Errors.NOT_CONTROLLER) {
+                        handleNotControllerError(error);
+                        throw error.exception();

Review comment:
       1) Fixed the code to not throw exception again.
   2) For CLUSTER_AUTHORIZATION_FAILED, I'm not sure how could we treat it the same way. In the case of the NOT_CONTROLLER error, the admin client code would retry the request once again when the exception is raised. But when cluster authorization fails, would a retry help?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4071,6 +4078,113 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        final NodeProvider provider =
+            options.sendRequestToController() ? new ControllerNodeProvider() : new LeastLoadedNodeProvider();
+
+        Call call = new Call(
+            "describeFeatures", calcDeadlineMs(now, options.timeoutMs()), provider) {
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(
+                        new FeatureMetadata(
+                            apiVersionsResponse.finalizedFeatures(),
+                            apiVersionsResponse.finalizedFeaturesEpoch(),
+                            apiVersionsResponse.supportedFeatures()));
+                } else if (options.sendRequestToController() && apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) {
+                    handleNotControllerError(Errors.NOT_CONTROLLER);
+                } else {
+                    future.completeExceptionally(
+                        Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        runnable.call(call, now);
+        return new DescribeFeaturesResult(future);
+    }
+
+    @Override
+    public UpdateFeaturesResult updateFeatures(
+        final Map<String, FeatureUpdate> featureUpdates, final UpdateFeaturesOptions options) {
+        if (featureUpdates == null || featureUpdates.isEmpty()) {
+            throw new IllegalArgumentException("Feature updates can not be null or empty.");
+        }
+        Objects.requireNonNull(options, "UpdateFeaturesOptions can not be null");
+
+        final UpdateFeaturesRequestData request = UpdateFeaturesRequest.create(featureUpdates);
+        final Map<String, KafkaFutureImpl<Void>> updateFutures = new HashMap<>();
+        for (Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+            updateFutures.put(entry.getKey(), new KafkaFutureImpl<>());
+        }
+        final long now = time.milliseconds();
+        final Call call = new Call("updateFeatures", calcDeadlineMs(now, options.timeoutMs()),
+            new ControllerNodeProvider()) {
+
+            @Override
+            UpdateFeaturesRequest.Builder createRequest(int timeoutMs) {
+                return new UpdateFeaturesRequest.Builder(request);
+            }
+
+            @Override
+            void handleResponse(AbstractResponse abstractResponse) {
+                final UpdateFeaturesResponse response =
+                    (UpdateFeaturesResponse) abstractResponse;
+
+                // Check for controller change.
+                for (UpdatableFeatureResult result : response.data().results()) {
+                    final Errors error = Errors.forCode(result.errorCode());
+                    if (error == Errors.NOT_CONTROLLER) {
+                        handleNotControllerError(error);
+                        throw error.exception();

Review comment:
       > handleNotControllerError() already throws an exception.
   Done. Fixed the code to not throw exception again when handling NOT_CONTROLLER error.
   
   > Should other errors like CLUSTER_AUTHORIZATION_FAILED be treated in the same way?
   I'm not sure how could we treat it the same way. In the case of the NOT_CONTROLLER error, the admin client code would retry the request once again when the exception is raised. But when cluster authorization fails, would a retry help?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4071,6 +4078,113 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        final NodeProvider provider =
+            options.sendRequestToController() ? new ControllerNodeProvider() : new LeastLoadedNodeProvider();
+
+        Call call = new Call(
+            "describeFeatures", calcDeadlineMs(now, options.timeoutMs()), provider) {
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(
+                        new FeatureMetadata(
+                            apiVersionsResponse.finalizedFeatures(),
+                            apiVersionsResponse.finalizedFeaturesEpoch(),
+                            apiVersionsResponse.supportedFeatures()));
+                } else if (options.sendRequestToController() && apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) {
+                    handleNotControllerError(Errors.NOT_CONTROLLER);
+                } else {
+                    future.completeExceptionally(
+                        Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        runnable.call(call, now);
+        return new DescribeFeaturesResult(future);
+    }
+
+    @Override
+    public UpdateFeaturesResult updateFeatures(
+        final Map<String, FeatureUpdate> featureUpdates, final UpdateFeaturesOptions options) {
+        if (featureUpdates == null || featureUpdates.isEmpty()) {
+            throw new IllegalArgumentException("Feature updates can not be null or empty.");
+        }
+        Objects.requireNonNull(options, "UpdateFeaturesOptions can not be null");
+
+        final UpdateFeaturesRequestData request = UpdateFeaturesRequest.create(featureUpdates);
+        final Map<String, KafkaFutureImpl<Void>> updateFutures = new HashMap<>();
+        for (Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+            updateFutures.put(entry.getKey(), new KafkaFutureImpl<>());
+        }
+        final long now = time.milliseconds();
+        final Call call = new Call("updateFeatures", calcDeadlineMs(now, options.timeoutMs()),
+            new ControllerNodeProvider()) {
+
+            @Override
+            UpdateFeaturesRequest.Builder createRequest(int timeoutMs) {
+                return new UpdateFeaturesRequest.Builder(request);
+            }
+
+            @Override
+            void handleResponse(AbstractResponse abstractResponse) {
+                final UpdateFeaturesResponse response =
+                    (UpdateFeaturesResponse) abstractResponse;
+
+                // Check for controller change.
+                for (UpdatableFeatureResult result : response.data().results()) {
+                    final Errors error = Errors.forCode(result.errorCode());
+                    if (error == Errors.NOT_CONTROLLER) {
+                        handleNotControllerError(error);
+                        throw error.exception();

Review comment:
       > handleNotControllerError() already throws an exception.
   
   Done. Fixed the code to not throw exception again when handling NOT_CONTROLLER error.
   
   > Should other errors like CLUSTER_AUTHORIZATION_FAILED be treated in the same way?
   
   I'm not sure how could we treat it the same way. In the case of the NOT_CONTROLLER error, the admin client code would retry the request once again when the exception is raised. But when cluster authorization fails, would a retry help?

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -977,14 +1179,30 @@ class KafkaController(val config: KafkaConfig,
 
   /**
    * Send the leader information for selected partitions to selected brokers so that they can correctly respond to
-   * metadata requests
+   * metadata requests. Particularly, when feature versioning is enabled, we filter out brokers with incompatible
+   * features from receiving the metadata requests. This is because we do not want to activate incompatible brokers,
+   * as these may have harmful consequences to the cluster.

Review comment:
       Done. I've changed the code such that we skip the broker registration if it's detected as incompatible.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1865,192 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns an ApiError object containing Errors#INVALID_REQUEST.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+
+    val incompatibilityError = "Could not apply finalized feature update because" +
+      " brokers were found to have incompatible versions for the feature."
+
+    if (brokerFeatures.supportedFeatures.get(update.feature()) == null) {
+      Right(new ApiError(Errors.INVALID_REQUEST, incompatibilityError))
+    } else {
+      val defaultMinVersionLevel = brokerFeatures.defaultMinVersionLevel(update.feature)
+      val newVersionRange = new FinalizedVersionRange(defaultMinVersionLevel, update.maxVersionLevel)

Review comment:
       Done. This is fixed now.

##########
File path: clients/src/main/resources/common/message/UpdateFeaturesRequest.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "request",
+  "name": "UpdateFeaturesRequest",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "FeatureUpdates", "type": "[]FeatureUpdateKey", "versions": "0+",
+      "about": "The list of updates to finalized features.", "fields": [
+      {"name": "Feature", "type": "string", "versions": "0+", "mapKey": true,
+        "about": "The name of the finalized feature to be updated."},
+      {"name": "MaxVersionLevel", "type": "int16", "versions": "0+",
+        "about": "The new maximum version level for the finalized feature. A value >= 1 is valid. A value < 1, is special, and can be used to request the deletion of the finalized feature."},
+      {"name": "AllowDowngrade", "type": "bool", "versions": "0+",

Review comment:
       Done. Fixed the KIP and the code, so that they align with each other now.

##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,192 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the following:
+ *
+ * 1. The latest features supported by the Broker.
+ *
+ * 2. The default minimum version levels for specific features. This map enables feature

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,178 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of its own supported features in its
+   * own BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. This process ensures we do not enable all the possible features immediately after
+   *    an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, it will
+   *        react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, its existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) =>
+            val brokerDefaultVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (brokerDefaultVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (brokerDefaultVersionRange.max() >= existingVersionRange.max() &&
+                       brokerDefaultVersionRange.min() <= existingVersionRange.max()) {
+              // Through this change, we deprecate all version levels in the closed range:
+              // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1]
+              (featureName, new FinalizedVersionRange(brokerDefaultVersionRange.min(), existingVersionRange.max()))
+            } else {
+              // If the existing version levels fall completely outside the
+              // range of the default finalized version levels (i.e. no intersection), or, if the
+              // existing version levels are ineligible for a modification since they are
+              // incompatible with default finalized version levels, then we skip the update.
+              warn(s"Can not update minimum version level in finalized feature: $featureName,"
+                + s" since the existing $existingVersionRange is not eligible for a change"
+                + s" based on the default $brokerDefaultVersionRange.")
+              (featureName, existingVersionRange)
+            }
+        }.asJava)
+      }
+      val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures)
+      if (!newFeatureZNode.equals(existingFeatureZNode)) {
+        val newVersion = updateFeatureZNode(newFeatureZNode)
+        featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+      }
+    }
+  }
+
+  /**
+   * Disables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with disabled status. This status means the feature versioning system
+   * (KIP-584) is disabled, and, the finalized features stored in the FeatureZNode are not relevant.
+   * This status should be written by the controller to the FeatureZNode only when the broker
+   * IBP config is less than KAFKA_2_7_IV0.
+   *
+   * NOTE:
+   * 1. When this method returns, existing finalized features (if any) will be cleared from the
+   *    FeatureZNode.
+   * 2. This method, unlike enableFeatureVersioning() need not wait for the FinalizedFeatureCache
+   *    to be updated, because, such updates to the cache (via FinalizedFeatureChangeListener)
+   *    are disabled when IBP config is < than KAFKA_2_7_IV0.
+   */
+  private def disableFeatureVersioning(): Unit = {
+    val newNode = FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures())
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      createFeatureZNode(newNode)

Review comment:
       No, that is not required. Please refer to the documentation above under `NOTE` for this method where I have explained why.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1865,192 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns an ApiError object containing Errors#INVALID_REQUEST.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+
+    val incompatibilityError = "Could not apply finalized feature update because" +
+      " brokers were found to have incompatible versions for the feature."
+
+    if (brokerFeatures.supportedFeatures.get(update.feature()) == null) {

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 edited a comment on pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #9001:
URL: https://github.com/apache/kafka/pull/9001#issuecomment-703048608


   @junrao Thanks for the review! I've addressed the comments, the PR is ready for another pass. I've also fixed the compilation errors.


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement KIP-584 write path

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


   @dajac Thank you for taking a look! IIUC you are referring to these lines:
   
   https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java#L550-L553
   
   My requirement is that for the newly added API: `org.apache.kafka.clients.Admin#describeFeatures`, the `ApiVersionsResponse` returned to the `AdminClient` needs to contain the features information. It seems to me that the `SaslServerAuthenticator` does not serve the newly added API, so I can skip including the features information in the above lines.
   
   Is my observation correct?


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/resources/common/message/UpdateFeaturesResponse.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "response",
+  "name": "UpdateFeaturesResponse",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "Results", "type": "[]UpdatableFeatureResult", "versions": "0+",
+      "about": "Results for each feature update.", "fields": [
+      {"name": "Feature", "type": "string", "versions": "0+", "mapKey": true,

Review comment:
       Yes, we changed to have an error code per feature update. I'll update the KIP-584 write up.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java
##########
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+/**
+ * Options for {@link AdminClient#describeFeatures(DescribeFeaturesOptions)}.
+ *
+ * The API of this class is evolving. See {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeFeaturesOptions extends AbstractOptions<DescribeFeaturesOptions> {

Review comment:
       Done. Updated the KIP. Please refer to [this](https://cwiki.apache.org/confluence/display/KAFKA/KIP-584%3A+Versioning+scheme+for+features#KIP584:Versioningschemeforfeatures-AdminAPIchanges) section.
   
   




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/resources/common/message/UpdateFeaturesResponse.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "response",
+  "name": "UpdateFeaturesResponse",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [

Review comment:
       I don't see that we consistently use a top level error code, so I will leave it as it is.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1215,71 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker. It can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter. This is particularly useful if the user requires strongly consistent reads of
+     * finalized features.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the {@link DescribeFeaturesResult} containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);
+
+    /**
+     * Applies specified updates to finalized features. This operation is not transactional so it
+     * may succeed for some features while fail for others.
+     * <p>
+     * The API takes in a map of finalized feature name to {@link FeatureUpdate} that needs to be
+     * applied. Each entry in the map specifies the finalized feature to be added or updated or
+     * deleted, along with the new max feature version level value. This request is issued only to
+     * the controller since the API is only served by the controller. The return value contains an
+     * error code for each supplied {@link FeatureUpdate}, and the code indicates if the update
+     * succeeded or failed in the controller.
+     * <ul>
+     * <li>Downgrade of feature version level is not a regular operation/intent. It is only allowed
+     * in the controller if the {@link FeatureUpdate} has the allowDowngrade flag set - setting this
+     * flag conveys user intent to attempt downgrade of a feature max version level. Note that
+     * despite the allowDowngrade flag being set, certain downgrades may be rejected by the
+     * controller if it is deemed impossible.</li>
+     * <li>Deletion of a finalized feature version is not a regular operation/intent. It could be
+     * done by setting the allowDowngrade flag to true in the {@link FeatureUpdate}, and, setting
+     * the max version level to be less than 1.</li>
+     * </ul>
+     *<p>
+     * The following exceptions can be anticipated when calling {@code get()} on the futures
+     * obtained from the returned {@link UpdateFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.ClusterAuthorizationException}
+     *   If the authenticated user didn't have alter access to the cluster.</li>
+     *   <li>{@link org.apache.kafka.common.errors.InvalidRequestException}

Review comment:
       Answered below.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java
##########
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Objects;
+import java.util.Optional;
+import org.apache.kafka.common.feature.Features;
+import org.apache.kafka.common.feature.FinalizedVersionRange;
+import org.apache.kafka.common.feature.SupportedVersionRange;
+
+/**
+ * Encapsulates details about finalized as well as supported features. This is particularly useful
+ * to hold the result returned by the {@link Admin#describeFeatures(DescribeFeaturesOptions)} API.
+ */
+public class FeatureMetadata {
+
+    private final Features<FinalizedVersionRange> finalizedFeatures;
+
+    private final Optional<Integer> finalizedFeaturesEpoch;
+
+    private final Features<SupportedVersionRange> supportedFeatures;
+
+    public FeatureMetadata(final Features<FinalizedVersionRange> finalizedFeatures,
+                           final int finalizedFeaturesEpoch,
+                           final Features<SupportedVersionRange> supportedFeatures) {
+        Objects.requireNonNull(finalizedFeatures, "Provided finalizedFeatures can not be null.");
+        Objects.requireNonNull(supportedFeatures, "Provided supportedFeatures can not be null.");
+        this.finalizedFeatures = finalizedFeatures;
+        if (finalizedFeaturesEpoch >= 0) {
+            this.finalizedFeaturesEpoch = Optional.of(finalizedFeaturesEpoch);
+        } else {
+            this.finalizedFeaturesEpoch = Optional.empty();
+        }
+        this.supportedFeatures = supportedFeatures;
+    }
+
+    /**
+     * A map of finalized feature versions, with key being finalized feature name and value
+     * containing the min/max version levels for the finalized feature.
+     */
+    public Features<FinalizedVersionRange> finalizedFeatures() {
+        return finalizedFeatures;
+    }
+
+    /**
+     * The epoch for the finalized features.
+     * If the returned value is empty, it means the finalized features are absent/unavailable.
+     */
+    public Optional<Integer> finalizedFeaturesEpoch() {

Review comment:
       Done. I've updated the KIP to use `Optional<Integer>` 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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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


   @junrao Thanks for the review! I've addressed the comments in c31d6b5245c635e659ff0f203bd08bc015a15ffb.


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -3109,6 +3109,36 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleUpdateFeatures(request: RequestChannel.Request): Unit = {
+    val updateFeaturesRequest = request.body[UpdateFeaturesRequest]
+
+    def sendResponseCallback(errors: Either[ApiError, Map[String, ApiError]]): Unit = {
+      def createResponse(throttleTimeMs: Int): UpdateFeaturesResponse = {
+        errors match {
+          case Left(topLevelError) =>
+            UpdateFeaturesResponse.createWithErrors(
+              topLevelError,
+              new util.HashMap[String, ApiError](),

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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java
##########
@@ -17,9 +17,16 @@
 package org.apache.kafka.common.feature;
 
 import java.util.Map;
+import java.util.Objects;
+import org.apache.kafka.common.utils.Utils;
 
 /**
- * An extended {@link BaseVersionRange} representing the min/max versions for supported features.
+ * An extended {@link BaseVersionRange} representing the min, max and first active versions for a
+ * supported feature:
+ *  - minVersion: This is the minimum supported version for the feature.
+ *  - maxVersion: This the maximum supported version for the feature.
+ *  - firstActiveVersion: This is the first active version for the feature. Versions in the range

Review comment:
       @junrao Does the below feel right to you?
   
   The key thing seems to be that you feel it is rare to deprecate feature versions in AK. I agree with the same. So, I propose we just do not have to solve the deprecation problem in this PR, until we find a clear route that the AK community agrees with. In this PR I propose to revert the `firstActiveVersion` change, leaving the rest of the things the way they are. In the future, we can develop a concrete solution for version deprecation i.e. the part on how to advance `minVersion` of supported feature, may be (or may not be) using `firstActiveVersion` or other ways (it is up for discussion, maybe in a separate KIP). I have made this proposed change in the most recent commit: 4218f95904989028a469930d0c266362bf173ece.
   
   Going back to your point:
   > I was thinking what if we relax the current check by just making sure that maxVersion of finalized is within the supported range. Basically in your example, if supported minVersion goes to 2, it's still allowed since it's less than maxVersion of finalized. However, if supported minVersion goes to 7, this fails the broker since it's more than maxVersion of finalized.
   
   There is a consequence to relaxing the current check:
   The controller can not effectively finalize `minVersionLevel` for the feature, because, with a relaxed check we do not know whether all brokers in the cluster support a particular `minVersion` when the controller finalizes the `minVersionLevel` at a particular value. It seems useful to keep the concept of `minVersionLevel` like the way it is now (i.e. it is the lowest version guaranteed to be supported by any broker in the cluster for a feature). And as I said above, in the future, we can decide on ways to mutate it safely (maybe through `firstActiveVersion` or other means).
   
   




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/BrokerFeatures.scala
##########
@@ -0,0 +1,179 @@
+/**
+ * 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.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A class that encapsulates the latest features supported by the Broker and also provides APIs to
+ * check for incompatibilities between the features supported by the Broker and finalized features.
+ * The class also enables feature version level deprecation, as explained below. This class is
+ * immutable in production. It provides few APIs to mutate state only for the purpose of testing.
+ *
+ * Feature version level deprecation:
+ * ==================================
+ *
+ * Deprecation of certain version levels of a feature is a process to stop supporting the
+ * functionality offered by the feature at a those version levels, across the entire Kafka cluster.

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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,199 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of its own supported features in its
+   * own BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.

Review comment:
       To be sure we are on same page, is this because of a controller failover during an IBP bump?
   It seems to me that this can happen mainly when IBP is being bumped from a value less than KAFKA_2_7_IV0 to a value greater than or equal to KAFKA_2_7_IV0 (assuming subsequent IBP bumps will be from KAFKA_2_7_IV0 to a higher value, so the node status will remain enabled).
   
   In general, I'm not sure how to avoid this node status flip until IBP bump has been completed cluster-wide. 
   




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4052,6 +4058,128 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) {
+        final KafkaFutureImpl<FeatureMetadata> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        final NodeProvider provider =
+            options.sendRequestToController() ? new ControllerNodeProvider() : new LeastLoadedNodeProvider();
+
+        Call call = new Call(
+            "describeFeatures", calcDeadlineMs(now, options.timeoutMs()), provider) {
+
+            @Override
+            ApiVersionsRequest.Builder createRequest(int timeoutMs) {
+                return new ApiVersionsRequest.Builder();
+            }
+
+            @Override
+            void handleResponse(AbstractResponse response) {
+                final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response;
+                if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) {
+                    future.complete(
+                        new FeatureMetadata(
+                            apiVersionsResponse.finalizedFeatures(),
+                            apiVersionsResponse.finalizedFeaturesEpoch(),
+                            apiVersionsResponse.supportedFeatures()));
+                } else if (options.sendRequestToController() && apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) {
+                    handleNotControllerError(Errors.NOT_CONTROLLER);
+                } else {
+                    future.completeExceptionally(
+                        Errors.forCode(apiVersionsResponse.data.errorCode()).exception());
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(Collections.singletonList(future), throwable);
+            }
+        };
+
+        runnable.call(call, now);
+        return new DescribeFeaturesResult(future);
+    }
+
+    @Override
+    public UpdateFeaturesResult updateFeatures(
+        final Map<String, FeatureUpdate> featureUpdates, final UpdateFeaturesOptions options) {
+        if (featureUpdates == null || featureUpdates.isEmpty()) {
+            throw new IllegalArgumentException("Feature updates can not be null or empty.");
+        }
+        Objects.requireNonNull(options, "UpdateFeaturesOptions can not be null");
+
+        final Map<String, KafkaFutureImpl<Void>> updateFutures = new HashMap<>();
+        final UpdateFeaturesRequestData.FeatureUpdateKeyCollection featureUpdatesRequestData
+            = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection();
+        for (Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+            final String feature = entry.getKey();
+            final FeatureUpdate update = entry.getValue();
+            if (feature.trim().isEmpty()) {
+                throw new IllegalArgumentException("Provided feature can not be null or empty.");
+            }
+
+            updateFutures.put(feature, new KafkaFutureImpl<>());
+            final UpdateFeaturesRequestData.FeatureUpdateKey requestItem =
+                new UpdateFeaturesRequestData.FeatureUpdateKey();
+            requestItem.setFeature(feature);
+            requestItem.setMaxVersionLevel(update.maxVersionLevel());
+            requestItem.setAllowDowngrade(update.allowDowngrade());
+            featureUpdatesRequestData.add(requestItem);
+        }
+        final UpdateFeaturesRequestData request = new UpdateFeaturesRequestData().setFeatureUpdates(featureUpdatesRequestData);
+
+        final long now = time.milliseconds();
+        final Call call = new Call("updateFeatures", calcDeadlineMs(now, options.timeoutMs()),
+            new ControllerNodeProvider()) {
+
+            @Override
+            UpdateFeaturesRequest.Builder createRequest(int timeoutMs) {
+                return new UpdateFeaturesRequest.Builder(request);
+            }
+
+            @Override
+            void handleResponse(AbstractResponse abstractResponse) {
+                final UpdateFeaturesResponse response =
+                    (UpdateFeaturesResponse) abstractResponse;
+
+                // Check for controller change.
+                for (UpdatableFeatureResult result : response.data().results()) {
+                    final Errors error = Errors.forCode(result.errorCode());
+                    if (error == Errors.NOT_CONTROLLER) {
+                        handleNotControllerError(error);
+                        throw error.exception();
+                    }
+                }
+
+                for (UpdatableFeatureResult result : response.data().results()) {
+                    final KafkaFutureImpl<Void> future = updateFutures.get(result.feature());
+                    if (future == null) {

Review comment:
       It does not overlap. This checks for unexpected responses for features that we never intended to update. `completeUnrealizedFutures` is for futures that we never got a response for from the server -- we need to complete such futures exceptionally.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1656,6 +1910,204 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns an ApiError object containing Errors.INVALID_REQUEST.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+
+    val supportedVersionRange = brokerFeatures.supportedFeatures.get(update.feature)
+    if (supportedVersionRange == null) {
+      Right(new ApiError(Errors.INVALID_REQUEST,
+                         "Could not apply finalized feature update because the provided feature" +
+                         " is not supported."))
+    } else {
+      var newVersionRange: FinalizedVersionRange = null
+      try {
+        newVersionRange = new FinalizedVersionRange(supportedVersionRange.firstActiveVersion, update.maxVersionLevel)
+      } catch {
+        case _: IllegalArgumentException => {
+          // This exception means the provided maxVersionLevel is invalid. It is handled below
+          // outside of this catch clause.
+        }
+      }
+      if (newVersionRange == null) {
+        Right(new ApiError(Errors.INVALID_REQUEST,
+          "Could not apply finalized feature update because the provided" +
+          s" maxVersionLevel:${update.maxVersionLevel} is lower than the" +
+          s" first active version:${supportedVersionRange.firstActiveVersion}."))
+      } else {
+        val newFinalizedFeature =
+          Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange)))
+        val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+          BrokerFeatures.hasIncompatibleFeatures(broker.features, newFinalizedFeature)
+        })
+        if (numIncompatibleBrokers == 0) {
+          Left(newVersionRange)
+        } else {
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Could not apply finalized feature update because" +
+                             " brokers were found to have incompatible versions for the feature."))
+        }
+      }
+    }
+  }
+
+  /**
+   * Validates a feature update on an existing FinalizedVersionRange.
+   * If the validation succeeds, then, the return value contains:
+   * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature.
+   * 2. Option.empty, if the feature update was meant to delete the feature.
+   *
+   * If the validation fails, then returned value contains a suitable ApiError.
+   *
+   * @param update                 the feature update to be processed.
+   * @param existingVersionRange   the existing FinalizedVersionRange which can be empty when no
+   *                               FinalizedVersionRange exists for the associated feature
+   *
+   * @return                       the new FinalizedVersionRange to be updated into ZK or error
+   *                               as described above.
+   */
+  private def validateFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey,
+                                    existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = {
+    def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = {
+      newFinalizedVersionRangeOrIncompatibilityError(update)
+        .fold(versionRange => Left(Some(versionRange)), error => Right(error))
+    }
+
+    if (update.feature.isEmpty) {
+      // Check that the feature name is not empty.
+      Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty."))
+    } else {
+      // We handle deletion requests separately from non-deletion requests.
+      if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+        if (existingVersionRange.isEmpty) {
+          // Disallow deletion of a non-existing finalized feature.
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Can not delete non-existing finalized feature."))
+        } else {
+          Left(Option.empty)
+        }
+      } else if (update.maxVersionLevel() < 1) {
+        // Disallow deletion of a finalized feature without allowDowngrade flag set.
+        Right(new ApiError(Errors.INVALID_REQUEST,
+                           s"Can not provide maxVersionLevel: ${update.maxVersionLevel} less" +
+                           s" than 1 without setting the allowDowngrade flag to true in the request."))
+      } else {
+        existingVersionRange.map(existing =>
+          if (update.maxVersionLevel == existing.max) {
+            // Disallow a case where target maxVersionLevel matches existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not ${if (update.allowDowngrade) "downgrade" else "upgrade"}" +
+                               s" a finalized feature from existing maxVersionLevel:${existing.max}" +
+                               " to the same value."))
+          } else if (update.maxVersionLevel < existing.max && !update.allowDowngrade) {
+            // Disallow downgrade of a finalized feature without the allowDowngrade flag set.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature from existing" +
+                               s" maxVersionLevel:${existing.max} to provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} without setting the" +
+                               " allowDowngrade flag in the request."))
+          } else if (update.allowDowngrade && update.maxVersionLevel > existing.max) {
+            // Disallow a request that sets allowDowngrade flag without specifying a
+            // maxVersionLevel that's lower than the existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"When the allowDowngrade flag set in the request, the provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} can not be greater than" +
+                               s" existing maxVersionLevel:${existing.max}."))
+          } else if (update.maxVersionLevel < existing.min) {
+            // Disallow downgrade of a finalized feature below the existing finalized
+            // minVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature to maxVersionLevel:${update.maxVersionLevel}" +
+                               s" because it's lower than the existing minVersionLevel:${existing.min}."))
+          } else {
+            newVersionRangeOrError(update)
+          }
+        ).getOrElse(newVersionRangeOrError(update))
+      }
+    }
+  }
+
+  private def processFeatureUpdates(request: UpdateFeaturesRequest,
+                                    callback: UpdateFeaturesCallback): Unit = {
+    if (isActive) {
+      processFeatureUpdatesWithActiveController(request, callback)
+    } else {
+      callback(Left(new ApiError(Errors.NOT_CONTROLLER)))
+    }
+  }
+
+  private def processFeatureUpdatesWithActiveController(request: UpdateFeaturesRequest,
+                                                        callback: UpdateFeaturesCallback): Unit = {
+    val updates = request.data.featureUpdates
+    val existingFeatures = featureCache.get
+      .map(featuresAndEpoch => featuresAndEpoch.features.features().asScala)
+      .getOrElse(Map[String, FinalizedVersionRange]())
+    // A map with key being feature name and value being FinalizedVersionRange.
+    // This contains the target features to be eventually written to FeatureZNode.
+    val targetFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]() ++ existingFeatures
+    // A map with key being feature name and value being error encountered when the FeatureUpdate
+    // was applied.
+    val errors = scala.collection.mutable.Map[String, ApiError]()
+
+    // Below we process each FeatureUpdate using the following logic:
+    //  - If a FeatureUpdate is found to be valid, then:
+    //    - The corresponding entry in errors map would be updated to contain Errors.NONE.
+    //    - If the FeatureUpdate is an add or update request, then the targetFeatures map is updated
+    //      to contain the new FinalizedVersionRange for the feature.
+    //    - Otherwise if the FeatureUpdate is a delete request, then the feature is removed from the
+    //      targetFeatures map.
+    //  - Otherwise if a FeatureUpdate is found to be invalid, then:
+    //    - The corresponding entry in errors map would be updated with the appropriate ApiError.
+    //    - The entry in targetFeatures map is left untouched.
+    updates.asScala.iterator.foreach { update =>
+      validateFeatureUpdate(update, existingFeatures.get(update.feature())) match {
+        case Left(newVersionRangeOrNone) =>
+          newVersionRangeOrNone match {
+            case Some(newVersionRange) => targetFeatures += (update.feature() -> newVersionRange)
+            case None => targetFeatures -= update.feature()
+          }
+          errors += (update.feature() -> new ApiError(Errors.NONE))
+        case Right(featureUpdateFailureReason) =>
+          errors += (update.feature() -> featureUpdateFailureReason)
+      }
+    }
+
+    // If the existing and target features are the same, then, we skip the update to the
+    // FeatureZNode as no changes to the node are required. Otherwise, we replace the contents
+    // of the FeatureZNode with the new features. This may result in partial or full modification
+    // of the existing finalized features in ZK.
+    try {
+      if (!existingFeatures.equals(targetFeatures)) {
+        val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, Features.finalizedFeatures(targetFeatures.asJava))
+        val newVersion = zkClient.updateFeatureZNode(newNode)
+        featureCache.waitUntilEpochOrThrow(newVersion, request.data().timeoutMs().min(config.zkConnectionTimeoutMs))

Review comment:
       Good point, removed the min now.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -272,6 +281,199 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (string) and a range of versions (defined by a
+   * SupportedVersionRange). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of its own supported features in its
+   * own BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (string) and a range of version levels (defined
+   * by a FinalizedVersionRange). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. This process ensures we do not enable all the possible features immediately after
+   *    an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, it will
+   *        react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, its existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.defaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) =>
+            val brokerDefaultVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (brokerDefaultVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (brokerDefaultVersionRange.max() >= existingVersionRange.max() &&
+                       brokerDefaultVersionRange.min() <= existingVersionRange.max()) {
+              // Using the change below, we deprecate all version levels in the range:
+              // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1].
+              //
+              // NOTE: if existingVersionRange.min() equals brokerDefaultVersionRange.min(), then
+              // we do not deprecate any version levels (since there is none to be deprecated).
+              //
+              // Examples:
+              // 1. brokerDefaultVersionRange = [4, 7] and existingVersionRange = [1, 5].
+              //    In this case, we deprecate all version levels in the range: [1, 3].
+              // 2. brokerDefaultVersionRange = [4, 7] and existingVersionRange = [4, 5].
+              //    In this case, we do not deprecate any version levels since
+              //    brokerDefaultVersionRange.min() equals existingVersionRange.min().
+              (featureName, new FinalizedVersionRange(brokerDefaultVersionRange.min(), existingVersionRange.max()))
+            } else {
+              // This is a serious error. We should never be reaching here, since we already
+              // verify once during KafkaServer startup that existing finalized feature versions in
+              // the FeatureZNode contained no incompatibilities. If we are here, it means that one
+              // of the following is true:
+              // 1. The existing version levels fall completely outside the range of the default
+              // finalized version levels (i.e. no intersection), or
+              // 2. The existing version levels are incompatible with default finalized version
+              // levels.
+              //
+              // Examples of invalid cases that can cause this exception to be triggered:
+              // 1. No intersection      : brokerDefaultVersionRange = [4, 7] and existingVersionRange = [2, 3].
+              // 2. No intersection      : brokerDefaultVersionRange = [2, 3] and existingVersionRange = [4, 7].
+              // 3. Incompatible versions: brokerDefaultVersionRange = [2, 3] and existingVersionRange = [1, 7].
+              throw new IllegalStateException(
+                s"Can not update minimum version level in finalized feature: $featureName,"
+                + s" since the existing $existingVersionRange is not eligible for a change"
+                + s" based on the default $brokerDefaultVersionRange. This should never happen"
+                + s" since feature version incompatibilities are already checked during"
+                + s" Kafka server startup.")
+            }
+        }.asJava)
+      }
+      val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures)
+      if (!newFeatureZNode.equals(existingFeatureZNode)) {
+        val newVersion = updateFeatureZNode(newFeatureZNode)
+        featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+      }
+    }
+  }
+
+  /**
+   * Disables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with disabled status. This status means the feature versioning system
+   * (KIP-584) is disabled, and, the finalized features stored in the FeatureZNode are not relevant.
+   * This status should be written by the controller to the FeatureZNode only when the broker
+   * IBP config is less than KAFKA_2_7_IV0.
+   *
+   * NOTE:
+   * 1. When this method returns, existing finalized features (if any) will be cleared from the
+   *    FeatureZNode.
+   * 2. This method, unlike enableFeatureVersioning() need not wait for the FinalizedFeatureCache
+   *    to be updated, because, such updates to the cache (via FinalizedFeatureChangeListener)
+   *    are disabled when IBP config is < than KAFKA_2_7_IV0.
+   */
+  private def disableFeatureVersioning(): Unit = {
+    val newNode = FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures())
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      createFeatureZNode(newNode)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      if (!existingFeatureZNode.status.equals(FeatureZNodeStatus.Disabled)) {
+        updateFeatureZNode(newNode)
+      }
+    }
+  }
+
+  private def setupFeatureVersioning(): Unit = {

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1656,6 +1893,203 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns an ApiError object containing Errors.INVALID_REQUEST.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+
+    val supportedVersionRange = brokerFeatures.supportedFeatures.get(update.feature)
+    if (supportedVersionRange == null) {
+      Right(new ApiError(Errors.INVALID_REQUEST,
+                         "Could not apply finalized feature update because the provided feature" +
+                         " is not supported."))
+    } else {
+      var newVersionRange: FinalizedVersionRange = null
+      try {
+        newVersionRange = new FinalizedVersionRange(supportedVersionRange.firstActiveVersion, update.maxVersionLevel)
+      } catch {
+        case _: IllegalArgumentException => {
+          // This exception means the provided maxVersionLevel is invalid. It is handled below
+          // outside of this catch clause.
+        }
+      }
+      if (newVersionRange == null) {
+        Right(new ApiError(Errors.INVALID_REQUEST,
+          "Could not apply finalized feature update because the provided" +
+          s" maxVersionLevel:${update.maxVersionLevel} is lower than the" +
+          s" first active version:${supportedVersionRange.firstActiveVersion}."))
+      } else {
+        val newFinalizedFeature =
+          Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange)))
+        val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+          BrokerFeatures.hasIncompatibleFeatures(broker.features, newFinalizedFeature)
+        })
+        if (numIncompatibleBrokers == 0) {
+          Left(newVersionRange)
+        } else {
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Could not apply finalized feature update because" +
+                             " brokers were found to have incompatible versions for the feature."))
+        }
+      }
+    }
+  }
+
+  /**
+   * Validates a feature update on an existing FinalizedVersionRange.
+   * If the validation succeeds, then, the return value contains:
+   * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature.
+   * 2. Option.empty, if the feature update was meant to delete the feature.
+   *
+   * If the validation fails, then returned value contains a suitable ApiError.
+   *
+   * @param update                 the feature update to be processed.
+   * @param existingVersionRange   the existing FinalizedVersionRange which can be empty when no
+   *                               FinalizedVersionRange exists for the associated feature
+   *
+   * @return                       the new FinalizedVersionRange to be updated into ZK or error
+   *                               as described above.
+   */
+  private def validateFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey,
+                                   existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = {
+    def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = {
+      newFinalizedVersionRangeOrIncompatibilityError(update)
+        .fold(versionRange => Left(Some(versionRange)), error => Right(error))
+    }
+
+    if (update.feature.isEmpty) {
+      // Check that the feature name is not empty.
+      Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty."))
+    } else {
+      // We handle deletion requests separately from non-deletion requests.
+      if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+        if (existingVersionRange.isEmpty) {
+          // Disallow deletion of a non-existing finalized feature.
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Can not delete non-existing finalized feature."))
+        } else {
+          Left(Option.empty)
+        }
+      } else if (update.maxVersionLevel() < 1) {
+        // Disallow deletion of a finalized feature without allowDowngrade flag set.
+        Right(new ApiError(Errors.INVALID_REQUEST,
+                           s"Can not provide maxVersionLevel: ${update.maxVersionLevel} less" +
+                           s" than 1 without setting the allowDowngrade flag to true in the request."))
+      } else {
+        existingVersionRange.map(existing =>
+          if (update.maxVersionLevel == existing.max) {
+            // Disallow a case where target maxVersionLevel matches existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not ${if (update.allowDowngrade) "downgrade" else "upgrade"}" +
+                               s" a finalized feature from existing maxVersionLevel:${existing.max}" +
+                               " to the same value."))
+          } else if (update.maxVersionLevel < existing.max && !update.allowDowngrade) {
+            // Disallow downgrade of a finalized feature without the allowDowngrade flag set.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature from existing" +
+                               s" maxVersionLevel:${existing.max} to provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} without setting the" +
+                               " allowDowngrade flag in the request."))
+          } else if (update.allowDowngrade && update.maxVersionLevel > existing.max) {
+            // Disallow a request that sets allowDowngrade flag without specifying a
+            // maxVersionLevel that's lower than the existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"When the allowDowngrade flag set in the request, the provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} can not be greater than" +
+                               s" existing maxVersionLevel:${existing.max}."))
+          } else if (update.maxVersionLevel < existing.min) {
+            // Disallow downgrade of a finalized feature below the existing finalized
+            // minVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature to maxVersionLevel:${update.maxVersionLevel}" +
+                               s" because it's lower than the existing minVersionLevel:${existing.min}."))
+          } else {
+            newVersionRangeOrError(update)
+          }
+        ).getOrElse(newVersionRangeOrError(update))
+      }
+    }
+  }
+
+  private def processFeatureUpdates(request: UpdateFeaturesRequest,
+                                    callback: UpdateFeaturesCallback): Unit = {
+    if (isActive) {
+      processFeatureUpdatesWithActiveController(request, callback)
+    } else {
+      callback(Left(new ApiError(Errors.NOT_CONTROLLER)))
+    }
+  }
+
+  private def processFeatureUpdatesWithActiveController(request: UpdateFeaturesRequest,
+                                                        callback: UpdateFeaturesCallback): Unit = {
+    val updates = request.data.featureUpdates
+    val existingFeatures = featureCache.get
+      .map(featuresAndEpoch => featuresAndEpoch.features.features().asScala)
+      .getOrElse(Map[String, FinalizedVersionRange]())
+    // A map with key being feature name and value being FinalizedVersionRange.
+    // This contains the target features to be eventually written to FeatureZNode.
+    val targetFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]() ++ existingFeatures
+    // A map with key being feature name and value being error encountered when the FeatureUpdate
+    // was applied.
+    val errors = scala.collection.mutable.Map[String, ApiError]()
+
+    // Below we process each FeatureUpdate using the following logic:
+    //  - If a FeatureUpdate is found to be valid, then:
+    //    - The corresponding entry in errors map would be updated to contain ApiError(Errors.NONE).
+    //    - If the FeatureUpdate is an add or update request, then the targetFeatures map is updated
+    //      to contain the new FinalizedVersionRange for the feature.
+    //    - Otherwise if the FeatureUpdate is a delete request, then the feature is removed from the
+    //      targetFeatures map.
+    //  - Otherwise if a FeatureUpdate is found to be invalid, then:
+    //    - The corresponding entry in errors map would be updated with the appropriate ApiError.
+    //    - The entry in targetFeatures map is left untouched.
+    updates.asScala.iterator.foreach { update =>
+      validateFeatureUpdate(update, existingFeatures.get(update.feature())) match {
+        case Left(newVersionRangeOrNone) =>
+          newVersionRangeOrNone
+            .map(newVersionRange => targetFeatures += (update.feature() -> newVersionRange))

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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -983,8 +1144,25 @@ class KafkaController(val config: KafkaConfig,
    */
   private[controller] def sendUpdateMetadataRequest(brokers: Seq[Int], partitions: Set[TopicPartition]): Unit = {
     try {
+      val filteredBrokers = scala.collection.mutable.Set[Int]() ++ brokers
+      if (config.isFeatureVersioningEnabled) {
+        def hasIncompatibleFeatures(broker: Broker): Boolean = {
+          val latestFinalizedFeatures = featureCache.get
+          if (latestFinalizedFeatures.isDefined) {
+            BrokerFeatures.hasIncompatibleFeatures(broker.features, latestFinalizedFeatures.get.features)
+          } else {
+            false
+          }
+        }
+        controllerContext.liveOrShuttingDownBrokers.foreach(broker => {
+          if (filteredBrokers.contains(broker.id) && hasIncompatibleFeatures(broker)) {

Review comment:
       If the broker has feature incompatibilities, then it should die as soon as it has received the ZK update (it would die from within `FinalizedFeatureChangeListener`).




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -177,15 +214,33 @@ public static ApiVersionsResponse createApiVersionsResponse(
             }
         }
 
+        return new ApiVersionsResponse(
+            createApiVersionsResponseData(
+                throttleTimeMs,
+                Errors.NONE,
+                apiKeys,
+                latestSupportedFeatures,
+                finalizedFeatures,
+                finalizedFeaturesEpoch));
+    }
+
+    public static ApiVersionsResponseData createApiVersionsResponseData(

Review comment:
       It calls into couple other helper functions. Let us keep it.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -266,6 +275,178 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (String) and a range of versions (defined by a
+   * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises
+   * support for. Each broker advertises the version ranges of its own supported features in its
+   * own BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (String) and a range of version levels (defined
+   * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    default supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). This means the user is upgrading from an earlier version of the broker
+   *    binary. In this case, we want to start with no finalized features and allow the user to
+   *    finalize them whenever they are ready i.e. in the future whenever the user sets IBP config
+   *    to be greater than or equal to KAFKA_2_7_IV0, then the user could start finalizing the
+   *    features. This process ensures we do not enable all the possible features immediately after
+   *    an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent. If absent, it will
+   *        react by creating a FeatureZNode with disabled status and empty finalized features.
+   *        Otherwise, if a node already exists in enabled status then the controller will just
+   *        flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled. In such a case, it won’t upgrade all features immediately.
+   *        Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *        user finalize the features later.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary
+   *    has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and higher).
+   *    The controller will start up and find that a FeatureZNode is already present with enabled
+   *    status and existing finalized features. In such a case, the controller needs to scan the
+   *    existing finalized features and mutate them for the purpose of version level deprecation
+   *    (if needed).
+   *    This is how we handle this case: If an existing finalized feature is present in the default
+   *    finalized features, then, its existing minimum version level is updated to the default
+   *    minimum version level maintained in the BrokerFeatures object. The goal of this mutation is
+   *    to permanently deprecate one or more feature version levels. The range of feature version
+   *    levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level].
+   *    NOTE: Deprecating a feature version level is an incompatible change, which requires a major
+   *    release of Kafka. In such a release, the minimum version level maintained within the
+   *    BrokerFeatures class is updated suitably to record the deprecation of the feature.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures()
+      if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map {
+          case (featureName, existingVersionRange) =>
+            val brokerDefaultVersionRange = defaultFinalizedFeatures.get(featureName)
+            if (brokerDefaultVersionRange == null) {
+              warn(s"Existing finalized feature: $featureName with $existingVersionRange"
+                + s" is absent in default finalized $defaultFinalizedFeatures")
+              (featureName, existingVersionRange)
+            } else if (brokerDefaultVersionRange.max() >= existingVersionRange.max() &&
+                       brokerDefaultVersionRange.min() <= existingVersionRange.max()) {
+              // Through this change, we deprecate all version levels in the closed range:
+              // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1]
+              (featureName, new FinalizedVersionRange(brokerDefaultVersionRange.min(), existingVersionRange.max()))
+            } else {
+              // If the existing version levels fall completely outside the
+              // range of the default finalized version levels (i.e. no intersection), or, if the
+              // existing version levels are ineligible for a modification since they are
+              // incompatible with default finalized version levels, then we skip the update.
+              warn(s"Can not update minimum version level in finalized feature: $featureName,"
+                + s" since the existing $existingVersionRange is not eligible for a change"
+                + s" based on the default $brokerDefaultVersionRange.")
+              (featureName, existingVersionRange)
+            }
+        }.asJava)
+      }
+      val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures)
+      if (!newFeatureZNode.equals(existingFeatureZNode)) {
+        val newVersion = updateFeatureZNode(newFeatureZNode)
+        featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+      }
+    }
+  }
+
+  /**
+   * Disables the feature versioning system (KIP-584).
+   *
+   * Sets up the FeatureZNode with disabled status. This status means the feature versioning system
+   * (KIP-584) is disabled, and, the finalized features stored in the FeatureZNode are not relevant.
+   * This status should be written by the controller to the FeatureZNode only when the broker
+   * IBP config is less than KAFKA_2_7_IV0.
+   *
+   * NOTE:
+   * 1. When this method returns, existing finalized features (if any) will be cleared from the
+   *    FeatureZNode.
+   * 2. This method, unlike enableFeatureVersioning() need not wait for the FinalizedFeatureCache
+   *    to be updated, because, such updates to the cache (via FinalizedFeatureChangeListener)
+   *    are disabled when IBP config is < than KAFKA_2_7_IV0.
+   */
+  private def disableFeatureVersioning(): Unit = {
+    val newNode = FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures())
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      createFeatureZNode(newNode)

Review comment:
       No, that is not required. Please refer to the documentation under `NOTE` for this method where I have explained why.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
##########
@@ -3615,6 +3662,137 @@ public void testListOffsetsNonRetriableErrors() throws Exception {
         }
     }
 
+    @Test
+    public void testUpdateFeaturesDuringSuccess() throws Exception {
+        testUpdateFeatures(
+            makeTestFeatureUpdates(),
+            makeTestFeatureUpdateErrors(Errors.NONE));
+    }
+
+    @Test
+    public void testUpdateFeaturesInvalidRequestError() throws Exception {
+        testUpdateFeatures(
+            makeTestFeatureUpdates(),
+            makeTestFeatureUpdateErrors(Errors.INVALID_REQUEST));
+    }
+
+    @Test
+    public void testUpdateFeaturesUpdateFailedError() throws Exception {
+        testUpdateFeatures(
+            makeTestFeatureUpdates(),
+            makeTestFeatureUpdateErrors(Errors.FEATURE_UPDATE_FAILED));
+    }
+
+    @Test
+    public void testUpdateFeaturesPartialSuccess() throws Exception {
+        final Map<String, Errors> errors = makeTestFeatureUpdateErrors(Errors.NONE);
+        errors.put("test_feature_2", Errors.INVALID_REQUEST);
+        testUpdateFeatures(makeTestFeatureUpdates(), errors);
+    }
+
+    private Map<String, FeatureUpdate> makeTestFeatureUpdates() {
+        return Utils.mkMap(
+            Utils.mkEntry("test_feature_1", new FeatureUpdate((short) 2, false)),
+            Utils.mkEntry("test_feature_2", new FeatureUpdate((short) 3, true)));
+    }
+
+    private Map<String, Errors> makeTestFeatureUpdateErrors(final Errors error) {
+        final Map<String, FeatureUpdate> updates = makeTestFeatureUpdates();
+        final Map<String, Errors> errors = new HashMap<>();
+        for (Map.Entry<String, FeatureUpdate> entry : updates.entrySet()) {
+            errors.put(entry.getKey(), error);
+        }
+        return errors;
+    }
+
+    private void testUpdateFeatures(Map<String, FeatureUpdate> featureUpdates,
+                                    Map<String, Errors> featureUpdateErrors) throws Exception {
+        try (final AdminClientUnitTestEnv env = mockClientEnv()) {
+            env.kafkaClient().prepareResponse(
+                body -> body instanceof UpdateFeaturesRequest,
+                prepareUpdateFeaturesResponse(featureUpdateErrors));
+            final Map<String, KafkaFuture<Void>> futures = env.adminClient().updateFeatures(
+                featureUpdates,
+                new UpdateFeaturesOptions().timeoutMs(10000)).values();
+            for (Map.Entry<String, KafkaFuture<Void>> entry : futures.entrySet()) {
+                final KafkaFuture<Void> future = entry.getValue();
+                final Errors error = featureUpdateErrors.get(entry.getKey());
+                if (error == Errors.NONE) {
+                    future.get();
+                } else {
+                    final ExecutionException e = assertThrows(ExecutionException.class,
+                        () -> future.get());

Review comment:
       Isn't that what I'm using currently?




----------------------------------------------------------------
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 pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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


   retest this please


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/resources/common/message/UpdateFeaturesRequest.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "request",
+  "name": "UpdateFeaturesRequest",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "FeatureUpdates", "type": "[]FeatureUpdateKey", "versions": "0+",
+      "about": "The list of updates to finalized features.", "fields": [
+      {"name": "Feature", "type": "string", "versions": "0+", "mapKey": true,
+        "about": "The name of the finalized feature to be updated."},
+      {"name": "MaxVersionLevel", "type": "int16", "versions": "0+",
+        "about": "The new maximum version level for the finalized feature. A value >= 1 is valid. A value < 1, is special, and can be used to request the deletion of the finalized feature."},
+      {"name": "AllowDowngrade", "type": "bool", "versions": "0+",

Review comment:
       OK. There are a couple of places that this PR is inconsistent with the KIP.
   
   1. The KIP has 2 levels of arrays: []FeatureUpdateKey and []FeatureKey. This PR only has one array.
   2. The KIP has a timeoutMs field and this PR doesn'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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1844,185 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns a suitable error.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+    // NOTE: Below we set the finalized min version level to be the default minimum version
+    // level. If the finalized feature already exists, then, this can cause deprecation of all
+    // version levels in the closed range:
+    // [existingVersionRange.min(), defaultMinVersionLevel - 1].
+    val defaultMinVersionLevel = brokerFeatures.defaultMinVersionLevel(update.feature)
+    val newVersionRange = new FinalizedVersionRange(defaultMinVersionLevel, update.maxVersionLevel)
+    val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+      val singleFinalizedFeature =
+        Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange)))
+      BrokerFeatures.hasIncompatibleFeatures(broker.features, singleFinalizedFeature)
+    })
+    if (numIncompatibleBrokers == 0) {
+      Left(newVersionRange)
+    } else {
+      Right(
+        new ApiError(Errors.INVALID_REQUEST,
+                     s"Could not apply finalized feature update because $numIncompatibleBrokers" +
+                     " brokers were found to have incompatible features."))
+    }
+  }
+
+  /**
+   * Validate and process a finalized feature update on an existing FinalizedVersionRange for the
+   * feature.
+   *
+   * If the processing is successful, then, the return value contains:
+   * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature.
+   * 2. Option.empty, if the feature update was meant to delete the feature.
+   *
+   * If the processing failed, then returned value contains a suitable ApiError.
+   *
+   * @param update                 the feature update to be processed.
+   * @param existingVersionRange   the existing FinalizedVersionRange which can be empty when no
+   *                               FinalizedVersionRange exists for the associated feature
+   *
+   * @return                       the new FinalizedVersionRange or error, as described above.
+   */
+  private def processFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey,
+                                   existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = {
+    def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = {
+      newFinalizedVersionRangeOrIncompatibilityError(update)
+        .fold(versionRange => Left(Some(versionRange)), error => Right(error))
+    }
+
+    if (update.feature.isEmpty) {
+      // Check that the feature name is not empty.
+      Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty."))
+    } else {
+      // We handle deletion requests separately from non-deletion requests.
+      if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+        if (existingVersionRange.isEmpty) {
+          // Disallow deletion of a non-existing finalized feature.
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             s"Can not delete non-existing finalized feature: '${update.feature}'"))
+        } else {
+          Left(Option.empty)
+        }
+      } else if (update.maxVersionLevel() < 1) {

Review comment:
       A value < 1 is indicative of a deletion request (a kind of downgrade request).
   It is for convenience of generating a special error message, that we handle the case here explicitly: `...less than 1 for feature...`.
   




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/feature/FinalizedVersionRange.java
##########
@@ -40,14 +40,16 @@ public static FinalizedVersionRange fromMap(Map<String, Short> versionRangeMap)
 
     /**
      * Checks if the [min, max] version level range of this object does *NOT* fall within the
-     * [min, max] version range of the provided SupportedVersionRange parameter.
+     * [min, first_active_version, max] range of the provided SupportedVersionRange parameter.

Review comment:
       We need to keep the existing validation. Here is a case where `minVersionLevel < firstActiveVersion` is true, but still there are no incompatibilities:
   ```
   SupportedVersionRange={minVersion=1, firstActiveVersion=4, maxVersion=7}
   FinalizedVersionRange={minVersionLevel=2, maxVersionLevel=6}
   ```
   
   For example, the above can happen during step 1 of feature verison level deprecation. Imagine a feature exists in ZK already and is finalized at `[2, 6]`. Then imagine a new Kafka release is deployed that raises `firstActiveVersion` for the supported feature from 1 -> 4 (in order to deprecate versions: 1,2,3). In such a case, during Kafka server startup (where we check for feature incompatibilities), we would run into the comparison cited above between the new `SupportedVersionRange` and existing `FinalizedVersionRange`. But it is not considered to be a case of incompatibility.




----------------------------------------------------------------
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 edited a comment on pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #9001:
URL: https://github.com/apache/kafka/pull/9001#issuecomment-703873486


   @junrao Thanks for the review! I've addressed the latest comments in e55358fd1a00f12ef98fc4d2d649a297ddf146da . The PR is ready for another pass.


----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/resources/common/message/UpdateFeaturesRequest.json
##########
@@ -0,0 +1,33 @@
+// 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.
+
+{
+  "apiKey": 50,
+  "type": "request",
+  "name": "UpdateFeaturesRequest",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "FeatureUpdates", "type": "[]FeatureUpdateKey", "versions": "0+",
+      "about": "The list of updates to finalized features.", "fields": [
+      {"name": "Feature", "type": "string", "versions": "0+", "mapKey": true,
+        "about": "The name of the finalized feature to be updated."},
+      {"name": "MaxVersionLevel", "type": "int16", "versions": "0+",
+        "about": "The new maximum version level for the finalized feature. A value >= 1 is valid. A value < 1, is special, and can be used to request the deletion of the finalized feature."},
+      {"name": "AllowDowngrade", "type": "bool", "versions": "0+",

Review comment:
       I'm missing something. Which lines on the KIP-584 were you referring to? I didn't find any mention of the flag being at the topic level.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesResult.java
##########
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.common.KafkaFuture;
+
+/**
+ * The result of the {@link Admin#describeFeatures(DescribeFeaturesOptions)} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+public class DescribeFeaturesResult {
+
+    private final KafkaFuture<FeatureMetadata> future;
+
+    public DescribeFeaturesResult(KafkaFuture<FeatureMetadata> future) {

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesResult.java
##########
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.Map;
+import org.apache.kafka.common.KafkaFuture;
+
+/**
+ * The result of the {@link Admin#updateFeatures(Map, UpdateFeaturesOptions)} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+public class UpdateFeaturesResult {
+    private final Map<String, KafkaFuture<Void>> futures;
+
+    /**
+     * @param futures   a map from feature name to future, which can be used to check the status of
+     *                  individual feature updates.
+     */
+    public UpdateFeaturesResult(final Map<String, KafkaFuture<Void>> futures) {

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -272,6 +281,147 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (string) and a range of versions (defined by a
+   * SupportedVersionRange). It refers to a feature that a particular broker advertises support for.
+   * Each broker advertises the version ranges of its own supported features in its own
+   * BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (string) and a range of version levels (defined
+   * by a FinalizedVersionRange). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there was an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has now been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). But the IBP config is still set to lower than KAFKA_2_7_IV0, and may be
+   *    set to a higher value later. In this case, we want to start with no finalized features and
+   *    allow the user to finalize them whenever they are ready i.e. in the future whenever the
+   *    user sets IBP config to be greater than or equal to KAFKA_2_7_IV0, then the user could start
+   *    finalizing the features. This process ensures we do not enable all the possible features
+   *    immediately after an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent.
+   *        - If the node is absent, it will react by creating a FeatureZNode with disabled status
+   *          and empty finalized features.
+   *        - Otherwise, if a node already exists in enabled status then the controller will just
+   *          flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled.
+   *         - If the node is in disabled status, the controller won’t upgrade all features immediately.
+   *           Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *           user finalize the features later.
+   *         - Otherwise, if a node already exists in enabled status then the controller will leave
+   *           the node umodified.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine there was an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker
+   *    binary has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and
+   *    higher). The controller will start up and find that a FeatureZNode is already present with
+   *    enabled status and existing finalized features. In such a case, the controller leaves the node
+   *    unmodified.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled,
+                                          brokerFeatures.defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      if (!existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {
+        val newVersion = updateFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled,

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1656,6 +1840,204 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns an ApiError object containing Errors.INVALID_REQUEST.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+
+    val supportedVersionRange = brokerFeatures.supportedFeatures.get(update.feature)
+    if (supportedVersionRange == null) {
+      Right(new ApiError(Errors.INVALID_REQUEST,
+                         "Could not apply finalized feature update because the provided feature" +
+                         " is not supported."))
+    } else {
+      var newVersionRange: FinalizedVersionRange = null
+      try {
+        newVersionRange = new FinalizedVersionRange(supportedVersionRange.min, update.maxVersionLevel)
+      } catch {
+        case _: IllegalArgumentException => {
+          // This exception means the provided maxVersionLevel is invalid. It is handled below
+          // outside of this catch clause.
+        }
+      }
+      if (newVersionRange == null) {
+        Right(new ApiError(Errors.INVALID_REQUEST,
+          "Could not apply finalized feature update because the provided" +
+          s" maxVersionLevel:${update.maxVersionLevel} is lower than the" +
+          s" supported minVersion:${supportedVersionRange.min}."))
+      } else {
+        val newFinalizedFeature =
+          Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange)))
+        val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => {
+          BrokerFeatures.hasIncompatibleFeatures(broker.features, newFinalizedFeature)
+        })
+        if (numIncompatibleBrokers == 0) {
+          Left(newVersionRange)
+        } else {
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Could not apply finalized feature update because" +
+                             " brokers were found to have incompatible versions for the feature."))
+        }
+      }
+    }
+  }
+
+  /**
+   * Validates a feature update on an existing FinalizedVersionRange.
+   * If the validation succeeds, then, the return value contains:
+   * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature.
+   * 2. Option.empty, if the feature update was meant to delete the feature.
+   *
+   * If the validation fails, then returned value contains a suitable ApiError.
+   *
+   * @param update                 the feature update to be processed.
+   * @param existingVersionRange   the existing FinalizedVersionRange which can be empty when no
+   *                               FinalizedVersionRange exists for the associated feature
+   *
+   * @return                       the new FinalizedVersionRange to be updated into ZK or error
+   *                               as described above.
+   */
+  private def validateFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey,
+                                    existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = {
+    def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = {
+      newFinalizedVersionRangeOrIncompatibilityError(update)
+        .fold(versionRange => Left(Some(versionRange)), error => Right(error))
+    }
+
+    if (update.feature.isEmpty) {
+      // Check that the feature name is not empty.
+      Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty."))
+    } else {
+      // We handle deletion requests separately from non-deletion requests.
+      if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+        if (existingVersionRange.isEmpty) {
+          // Disallow deletion of a non-existing finalized feature.
+          Right(new ApiError(Errors.INVALID_REQUEST,
+                             "Can not delete non-existing finalized feature."))
+        } else {
+          Left(Option.empty)
+        }
+      } else if (update.maxVersionLevel() < 1) {
+        // Disallow deletion of a finalized feature without allowDowngrade flag set.
+        Right(new ApiError(Errors.INVALID_REQUEST,
+                           s"Can not provide maxVersionLevel: ${update.maxVersionLevel} less" +
+                           s" than 1 without setting the allowDowngrade flag to true in the request."))
+      } else {
+        existingVersionRange.map(existing =>
+          if (update.maxVersionLevel == existing.max) {
+            // Disallow a case where target maxVersionLevel matches existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not ${if (update.allowDowngrade) "downgrade" else "upgrade"}" +
+                               s" a finalized feature from existing maxVersionLevel:${existing.max}" +
+                               " to the same value."))
+          } else if (update.maxVersionLevel < existing.max && !update.allowDowngrade) {
+            // Disallow downgrade of a finalized feature without the allowDowngrade flag set.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature from existing" +
+                               s" maxVersionLevel:${existing.max} to provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} without setting the" +
+                               " allowDowngrade flag in the request."))
+          } else if (update.allowDowngrade && update.maxVersionLevel > existing.max) {
+            // Disallow a request that sets allowDowngrade flag without specifying a
+            // maxVersionLevel that's lower than the existing maxVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"When the allowDowngrade flag set in the request, the provided" +
+                               s" maxVersionLevel:${update.maxVersionLevel} can not be greater than" +
+                               s" existing maxVersionLevel:${existing.max}."))
+          } else if (update.maxVersionLevel < existing.min) {
+            // Disallow downgrade of a finalized feature below the existing finalized
+            // minVersionLevel.
+            Right(new ApiError(Errors.INVALID_REQUEST,
+                               s"Can not downgrade finalized feature to maxVersionLevel:${update.maxVersionLevel}" +
+                               s" because it's lower than the existing minVersionLevel:${existing.min}."))
+          } else {
+            newVersionRangeOrError(update)
+          }
+        ).getOrElse(newVersionRangeOrError(update))
+      }
+    }
+  }
+
+  private def processFeatureUpdates(request: UpdateFeaturesRequest,
+                                    callback: UpdateFeaturesCallback): Unit = {
+    if (isActive) {
+      processFeatureUpdatesWithActiveController(request, callback)
+    } else {
+      callback(Left(new ApiError(Errors.NOT_CONTROLLER)))
+    }
+  }
+
+  private def processFeatureUpdatesWithActiveController(request: UpdateFeaturesRequest,
+                                                        callback: UpdateFeaturesCallback): Unit = {
+    val updates = request.data.featureUpdates
+    val existingFeatures = featureCache.get
+      .map(featuresAndEpoch => featuresAndEpoch.features.features().asScala)
+      .getOrElse(Map[String, FinalizedVersionRange]())
+    // A map with key being feature name and value being FinalizedVersionRange.
+    // This contains the target features to be eventually written to FeatureZNode.
+    val targetFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]() ++ existingFeatures
+    // A map with key being feature name and value being error encountered when the FeatureUpdate
+    // was applied.
+    val errors = scala.collection.mutable.Map[String, ApiError]()
+
+    // Below we process each FeatureUpdate using the following logic:
+    //  - If a FeatureUpdate is found to be valid, then:
+    //    - The corresponding entry in errors map would be updated to contain Errors.NONE.
+    //    - If the FeatureUpdate is an add or update request, then the targetFeatures map is updated
+    //      to contain the new FinalizedVersionRange for the feature.
+    //    - Otherwise if the FeatureUpdate is a delete request, then the feature is removed from the
+    //      targetFeatures map.
+    //  - Otherwise if a FeatureUpdate is found to be invalid, then:
+    //    - The corresponding entry in errors map would be updated with the appropriate ApiError.
+    //    - The entry in targetFeatures map is left untouched.
+    updates.asScala.iterator.foreach { update =>
+      validateFeatureUpdate(update, existingFeatures.get(update.feature())) match {
+        case Left(newVersionRangeOrNone) =>
+          newVersionRangeOrNone match {
+            case Some(newVersionRange) => targetFeatures += (update.feature() -> newVersionRange)
+            case None => targetFeatures -= update.feature()
+          }
+          errors += (update.feature() -> new ApiError(Errors.NONE))
+        case Right(featureUpdateFailureReason) =>
+          errors += (update.feature() -> featureUpdateFailureReason)
+      }
+    }
+
+    // If the existing and target features are the same, then, we skip the update to the
+    // FeatureZNode as no changes to the node are required. Otherwise, we replace the contents
+    // of the FeatureZNode with the new features. This may result in partial or full modification
+    // of the existing finalized features in ZK.
+    try {
+      if (!existingFeatures.equals(targetFeatures)) {
+        val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, Features.finalizedFeatures(targetFeatures.asJava))
+        val newVersion = zkClient.updateFeatureZNode(newNode)

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -272,6 +281,147 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  private def createFeatureZNode(newNode: FeatureZNode): Int = {
+    info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode")
+    zkClient.createFeatureZNode(newNode)
+    val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    newVersion
+  }
+
+  private def updateFeatureZNode(updatedNode: FeatureZNode): Int = {
+    info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode")
+    zkClient.updateFeatureZNode(updatedNode)
+  }
+
+  /**
+   * This method enables the feature versioning system (KIP-584).
+   *
+   * Development in Kafka (from a high level) is organized into features. Each feature is tracked by
+   * a name and a range of version numbers. A feature can be of two types:
+   *
+   * 1. Supported feature:
+   * A supported feature is represented by a name (string) and a range of versions (defined by a
+   * SupportedVersionRange). It refers to a feature that a particular broker advertises support for.
+   * Each broker advertises the version ranges of its own supported features in its own
+   * BrokerIdZNode. The contents of the advertisement are specific to the particular broker and
+   * do not represent any guarantee of a cluster-wide availability of the feature for any particular
+   * range of versions.
+   *
+   * 2. Finalized feature:
+   * A finalized feature is represented by a name (string) and a range of version levels (defined
+   * by a FinalizedVersionRange). Whenever the feature versioning system (KIP-584) is
+   * enabled, the finalized features are stored in the cluster-wide common FeatureZNode.
+   * In comparison to a supported feature, the key difference is that a finalized feature exists
+   * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a
+   * specified range of version levels. Also, the controller is the only entity modifying the
+   * information about finalized features.
+   *
+   * This method sets up the FeatureZNode with enabled status, which means that the finalized
+   * features stored in the FeatureZNode are active. The enabled status should be written by the
+   * controller to the FeatureZNode only when the broker IBP config is greater than or equal to
+   * KAFKA_2_7_IV0.
+   *
+   * There are multiple cases handled here:
+   *
+   * 1. New cluster bootstrap:
+   *    A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
+   *    setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
+   *    the possible supported features finalized immediately. Assuming this is the case, the
+   *    controller will start up and notice that the FeatureZNode is absent in the new cluster,
+   *    it will then create a FeatureZNode (with enabled status) containing the entire list of
+   *    supported features as its finalized features.
+   *
+   * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
+   *    Imagine there was an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
+   *    broker binary has now been upgraded to a newer version that supports the feature versioning
+   *    system (KIP-584). But the IBP config is still set to lower than KAFKA_2_7_IV0, and may be
+   *    set to a higher value later. In this case, we want to start with no finalized features and
+   *    allow the user to finalize them whenever they are ready i.e. in the future whenever the
+   *    user sets IBP config to be greater than or equal to KAFKA_2_7_IV0, then the user could start
+   *    finalizing the features. This process ensures we do not enable all the possible features
+   *    immediately after an upgrade, which could be harmful to Kafka.
+   *    This is how we handle such a case:
+   *      - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
+   *        controller will start up and check if the FeatureZNode is absent.
+   *        - If the node is absent, it will react by creating a FeatureZNode with disabled status
+   *          and empty finalized features.
+   *        - Otherwise, if a node already exists in enabled status then the controller will just
+   *          flip the status to disabled and clear the finalized features.
+   *      - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
+   *        KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
+   *        and whether it is disabled.
+   *         - If the node is in disabled status, the controller won’t upgrade all features immediately.
+   *           Instead it will just switch the FeatureZNode status to enabled status. This lets the
+   *           user finalize the features later.
+   *         - Otherwise, if a node already exists in enabled status then the controller will leave
+   *           the node umodified.
+   *
+   * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
+   *    Imagine there was an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker
+   *    binary has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and
+   *    higher). The controller will start up and find that a FeatureZNode is already present with
+   *    enabled status and existing finalized features. In such a case, the controller leaves the node
+   *    unmodified.
+   *
+   * 4. Broker downgrade:
+   *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+   *    KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
+   *    value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
+   *    system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
+   *    will switch the FeatureZNode status to disabled with empty features.
+   */
+  private def enableFeatureVersioning(): Unit = {
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    if (version == ZkVersion.UnknownVersion) {
+      val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled,
+                                          brokerFeatures.defaultFinalizedFeatures))
+      featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs)
+    } else {
+      val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+      if (!existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) {

Review comment:
       Done. Excellent catch.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala
##########
@@ -0,0 +1,550 @@
+/**
+ * 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.server
+
+import java.util
+import java.util.Properties
+import java.util.concurrent.ExecutionException
+
+import kafka.api.KAFKA_2_7_IV0
+import kafka.utils.TestUtils
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion}
+import kafka.utils.TestUtils.waitUntilTrue
+import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions, UpdateFeaturesResult}
+import org.apache.kafka.common.errors.InvalidRequestException
+import org.apache.kafka.common.feature.FinalizedVersionRange
+import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
+import org.apache.kafka.common.message.UpdateFeaturesRequestData
+import org.apache.kafka.common.message.UpdateFeaturesRequestData.FeatureUpdateKeyCollection
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{UpdateFeaturesRequest, UpdateFeaturesResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.Test
+import org.junit.Assert.{assertEquals, assertFalse, assertNotEquals, assertNotNull, assertTrue}
+import org.scalatest.Assertions.{assertThrows, intercept}
+
+import scala.jdk.CollectionConverters._
+import scala.reflect.ClassTag
+import scala.util.matching.Regex
+
+class UpdateFeaturesTest extends BaseRequestTest {
+
+  override def brokerCount = 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))))
+  }
+
+  private def defaultFinalizedFeatures(): Features[FinalizedVersionRange] = {
+    Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2))))
+  }
+
+  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)
+  }
+
+  private def updateDefaultMinVersionLevelsInAllBrokers(newMinVersionLevels: Map[String, Short]): Unit = {
+    servers.foreach(s => {
+      s.brokerFeatures.setDefaultMinVersionLevels(newMinVersionLevels)
+    })
+  }
+
+  private def updateFeatureZNode(features: Features[FinalizedVersionRange]): Int = {
+    val server = serverForId(0).get
+    val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, features)
+    val newVersion = server.zkClient.updateFeatureZNode(newNode)
+    servers.foreach(s => {
+      s.featureCache.waitUntilEpochOrThrow(newVersion, s.config.zkConnectionTimeoutMs)
+    })
+    newVersion
+  }
+
+  private def getFeatureZNode(): FeatureZNode = {
+    val (mayBeFeatureZNodeBytes, version) = serverForId(0).get.zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(version, ZkVersion.UnknownVersion)
+    FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+  }
+
+  private def checkFeatures(client: Admin, expectedNode: FeatureZNode, expectedMetadata: FeatureMetadata): Unit = {
+    assertEquals(expectedNode, getFeatureZNode())
+    val featureMetadata = client.describeFeatures(
+      new DescribeFeaturesOptions().sendRequestToController(true)).featureMetadata().get()
+    assertEquals(expectedMetadata, featureMetadata)
+  }
+
+  private def checkException[ExceptionType <: Throwable](result: UpdateFeaturesResult,
+                                                         featureExceptionMsgPatterns: Map[String, Regex])
+                                                        (implicit tag: ClassTag[ExceptionType]): Unit = {
+    featureExceptionMsgPatterns.foreach {
+      case (feature, exceptionMsgPattern) =>
+        val exception = intercept[ExecutionException] {
+          result.values().get(feature).get()
+        }
+        val cause = exception.getCause
+        assertNotNull(cause)
+        assertEquals(cause.getClass, tag.runtimeClass)
+        assertTrue(cause.getMessage, exceptionMsgPattern.findFirstIn(cause.getMessage).isDefined)
+    }
+
+  }
+
+  /**
+   * Tests whether an invalid feature update does not get processed on the server as expected,
+   * and raises the ExceptionType on the client side as expected.
+   *
+   * @param invalidUpdate         the invalid feature update to be sent in the
+   *                              updateFeatures request to the server
+   * @param exceptionMsgPattern   a pattern for the expected exception message
+   */
+  private def testWithInvalidFeatureUpdate[ExceptionType <: Throwable](feature: String,
+                                                                       invalidUpdate: FeatureUpdate,
+                                                                       exceptionMsgPattern: Regex)
+                                                                      (implicit tag: ClassTag[ExceptionType]): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+    val adminClient = createAdminClient()
+    val nodeBefore = getFeatureZNode()
+
+    val result = adminClient.updateFeatures(Utils.mkMap(Utils.mkEntry(feature, invalidUpdate)), new UpdateFeaturesOptions())
+
+    checkException[ExceptionType](result, Map(feature -> exceptionMsgPattern))
+    checkFeatures(
+      adminClient,
+      nodeBefore,
+      new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures()))
+  }
+
+  @Test
+  def testShouldFailRequestIfNotController(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val nodeBefore = getFeatureZNode()
+    val updates = new FeatureUpdateKeyCollection()
+    val update = new UpdateFeaturesRequestData.FeatureUpdateKey();
+    update.setFeature("feature_1");
+    update.setMaxVersionLevel(defaultSupportedFeatures().get("feature_1").max())
+    update.setAllowDowngrade(false)
+    updates.add(update)
+
+    val response = connectAndReceive[UpdateFeaturesResponse](
+      new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData().setFeatureUpdates(updates)).build(),
+      notControllerSocketServer)
+
+    assertEquals(1, response.data.results.size)
+    val result = response.data.results.asScala.head
+    assertEquals("feature_1", result.feature)
+    assertEquals(Errors.NOT_CONTROLLER, Errors.forCode(result.errorCode))
+    assertNotNull(result.errorMessage)
+    assertFalse(result.errorMessage.isEmpty)
+    checkFeatures(
+      createAdminClient(),
+      nodeBefore,
+      new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures()))
+  }
+
+  @Test
+  def testShouldFailRequestForEmptyUpdates(): Unit = {
+    val nullMap: util.Map[String, FeatureUpdate] = null
+    val emptyMap: util.Map[String, FeatureUpdate] = Utils.mkMap()
+    Set(nullMap, emptyMap).foreach { updates =>
+      val client = createAdminClient()
+      val exception = intercept[IllegalArgumentException] {
+        client.updateFeatures(updates, new UpdateFeaturesOptions())
+      }
+      assertNotNull(exception)
+      assertEquals("Feature updates can not be null or empty.", exception.getMessage)
+    }
+  }
+
+  @Test
+  def testShouldFailRequestForNullUpdateFeaturesOptions(): Unit = {
+    val client = createAdminClient()
+    val update = new FeatureUpdate(defaultSupportedFeatures().get("feature_1").max(), false)
+    val exception = intercept[NullPointerException] {
+      client.updateFeatures(Utils.mkMap(Utils.mkEntry("feature_1", update)), null)
+    }
+    assertNotNull(exception)
+    assertEquals("UpdateFeaturesOptions can not be null", exception.getMessage)
+  }
+
+  @Test
+  def testShouldFailRequestForInvalidFeatureName(): Unit = {
+    val client = createAdminClient()
+    val update = new FeatureUpdate(defaultSupportedFeatures().get("feature_1").max(), false)
+    val exception = intercept[IllegalArgumentException] {
+      client.updateFeatures(Utils.mkMap(Utils.mkEntry("", update)), new UpdateFeaturesOptions())
+    }
+    assertNotNull(exception)
+    assertTrue((".*Provided feature can not be null or empty.*"r).findFirstIn(exception.getMessage).isDefined)
+  }
+
+  @Test
+  def testShouldFailRequestWhenDowngradeFlagIsNotSetDuringDowngrade(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      "feature_1",
+      new FeatureUpdate((defaultFinalizedFeatures().get("feature_1").max() - 1).asInstanceOf[Short],false),
+      ".*Can not downgrade finalized feature: 'feature_1'.*allowDowngrade.*".r)
+  }
+
+  @Test
+  def testShouldFailRequestWhenDowngradeToHigherVersionLevelIsAttempted(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      "feature_1",
+      new FeatureUpdate(defaultSupportedFeatures().get("feature_1").max(), true),
+      ".*finalized feature: 'feature_1'.*allowDowngrade.* provided maxVersionLevel:3.*existing maxVersionLevel:2.*".r)
+  }
+
+  @Test
+  def testShouldFailRequestInClientWhenDowngradeFlagIsNotSetDuringDeletion(): Unit = {
+    assertThrows[IllegalArgumentException] {
+      new FeatureUpdate(0, false)
+    }
+  }
+
+  @Test
+  def testShouldFailRequestInServerWhenDowngradeFlagIsNotSetDuringDeletion(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val adminClient = createAdminClient()
+    val nodeBefore = getFeatureZNode()
+
+    val updates
+      = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection();
+    val update = new UpdateFeaturesRequestData.FeatureUpdateKey();
+    update.setFeature("feature_1")
+    update.setMaxVersionLevel(0)
+    update.setAllowDowngrade(false)
+    updates.add(update);
+    val requestData = new UpdateFeaturesRequestData()
+    requestData.setFeatureUpdates(updates);
+
+    val response = connectAndReceive[UpdateFeaturesResponse](
+      new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData().setFeatureUpdates(updates)).build(),
+      controllerSocketServer)
+
+    assertEquals(1, response.data().results().size())
+    val result = response.data.results.asScala.head
+    assertEquals("feature_1", result.feature)
+    assertEquals(Errors.INVALID_REQUEST, Errors.forCode(result.errorCode))
+    assertNotNull(result.errorMessage)
+    assertFalse(result.errorMessage.isEmpty)
+    val exceptionMsgPattern = ".*Can not provide maxVersionLevel: 0 less than 1 for feature: 'feature_1'.*allowDowngrade.*".r
+    assertTrue(exceptionMsgPattern.findFirstIn(result.errorMessage).isDefined)
+    checkFeatures(
+      adminClient,
+      nodeBefore,
+      new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures()))
+  }
+
+  @Test
+  def testShouldFailRequestDuringDeletionOfNonExistingFeature(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      "feature_non_existing",
+      new FeatureUpdate(0, true),
+      ".*Can not delete non-existing finalized feature: 'feature_non_existing'.*".r)
+  }
+
+  @Test
+  def testShouldFailRequestWhenUpgradingToSameVersionLevel(): Unit = {
+    testWithInvalidFeatureUpdate[InvalidRequestException](
+      "feature_1",
+      new FeatureUpdate(defaultFinalizedFeatures().get("feature_1").max(), false),
+      ".*Can not upgrade a finalized feature: 'feature_1'.*to the same value.*".r)
+  }
+
+  @Test
+  def testShouldFailRequestWhenDowngradingBelowMinVersionLevel(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures())
+    val minVersionLevel = 2.asInstanceOf[Short]
+    updateDefaultMinVersionLevelsInAllBrokers(Map[String, Short]("feature_1" -> minVersionLevel))
+    val initialFinalizedFeatures = Features.finalizedFeatures(
+      Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(minVersionLevel, 2))))
+    val versionBefore = updateFeatureZNode(initialFinalizedFeatures)
+
+    val update = new FeatureUpdate((minVersionLevel - 1).asInstanceOf[Short], true)
+    val adminClient = createAdminClient()
+    val nodeBefore = getFeatureZNode()
+
+    val result = adminClient.updateFeatures(
+      Utils.mkMap(Utils.mkEntry("feature_1", update)), new UpdateFeaturesOptions())
+
+    checkException[InvalidRequestException](
+      result,
+      Map("feature_1" -> ".*Can not downgrade finalized feature: 'feature_1' to maxVersionLevel:1.*existing minVersionLevel:2.*".r))
+    checkFeatures(
+      adminClient,
+      nodeBefore,
+      new FeatureMetadata(initialFinalizedFeatures, versionBefore, defaultSupportedFeatures()))
+  }
+
+  @Test
+  def testShouldFailRequestDuringBrokerMaxVersionLevelIncompatibility(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    val controller = servers.filter { server => server.kafkaController.isActive}.head
+    val nonControllerServers = servers.filter { server => !server.kafkaController.isActive}
+    val unsupportedBrokers = Set[KafkaServer](nonControllerServers.head)
+    val supportedBrokers = Set[KafkaServer](nonControllerServers(1), controller)
+
+    updateSupportedFeatures(defaultSupportedFeatures(), supportedBrokers)
+
+    val validMinVersion = defaultSupportedFeatures().get("feature_1").min()
+    val unsupportedMaxVersion =
+      (defaultSupportedFeatures().get("feature_1").max() - 1).asInstanceOf[Short]
+    val badSupportedFeatures = Features.supportedFeatures(
+      Utils.mkMap(
+        Utils.mkEntry("feature_1",
+          new SupportedVersionRange(
+            validMinVersion,
+            unsupportedMaxVersion))))
+    updateSupportedFeatures(badSupportedFeatures, unsupportedBrokers)
+
+    val versionBefore = updateFeatureZNode(defaultFinalizedFeatures())
+
+    val invalidUpdate = new FeatureUpdate(defaultSupportedFeatures().get("feature_1").max(), false)
+    val nodeBefore = getFeatureZNode()
+    val adminClient = createAdminClient()
+    val result = adminClient.updateFeatures(
+      Utils.mkMap(Utils.mkEntry("feature_1", invalidUpdate)),
+      new UpdateFeaturesOptions())
+
+    checkException[InvalidRequestException](result, Map("feature_1" -> ".*1 broker.*incompatible.*".r))
+    checkFeatures(
+      adminClient,
+      nodeBefore,
+      new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures()))
+  }
+
+  @Test
+  def testSuccessfulFeatureUpgradeAndWithNoExistingFinalizedFeatures(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeaturesInAllBrokers(
+      Features.supportedFeatures(
+        Utils.mkMap(
+          Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)),
+          Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5)))))
+    updateDefaultMinVersionLevelsInAllBrokers(Map[String, Short]("feature_1" -> 1, "feature_2" -> 2))
+    val versionBefore = updateFeatureZNode(Features.emptyFinalizedFeatures())
+
+    val targetFinalizedFeatures = Features.finalizedFeatures(
+      Utils.mkMap(
+        Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 3)),
+        Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 3))))
+    val update1 = new FeatureUpdate(targetFinalizedFeatures.get("feature_1").max(), false)
+    val update2 = new FeatureUpdate(targetFinalizedFeatures.get("feature_2").max(), false)
+
+    val expected = new FeatureMetadata(
+      targetFinalizedFeatures,
+      versionBefore + 1,
+      Features.supportedFeatures(
+        Utils.mkMap(
+          Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)),
+          Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5)))))
+
+    val adminClient = createAdminClient()
+    adminClient.updateFeatures(
+      Utils.mkMap(Utils.mkEntry("feature_1", update1), Utils.mkEntry("feature_2", update2)),
+      new UpdateFeaturesOptions()
+    ).all().get()
+
+    checkFeatures(
+      adminClient,
+      new FeatureZNode(FeatureZNodeStatus.Enabled, targetFinalizedFeatures),
+      expected)
+  }
+
+  @Test
+  def testSuccessfulFeatureUpgradeAndDowngrade(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    updateSupportedFeaturesInAllBrokers(
+      Features.supportedFeatures(
+        Utils.mkMap(
+          Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)),
+          Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5)))))
+    updateDefaultMinVersionLevelsInAllBrokers(Map[String, Short]("feature_1" -> 1, "feature_2" -> 2))
+    val versionBefore = updateFeatureZNode(
+      Features.finalizedFeatures(
+        Utils.mkMap(
+          Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2)),
+          Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 4)))))
+
+    val targetFinalizedFeatures = Features.finalizedFeatures(
+      Utils.mkMap(
+        Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 3)),
+        Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 3))))
+    val update1 = new FeatureUpdate(targetFinalizedFeatures.get("feature_1").max(), false)
+    val update2 = new FeatureUpdate(targetFinalizedFeatures.get("feature_2").max(), true)
+
+    val expected = new FeatureMetadata(
+      targetFinalizedFeatures,
+      versionBefore + 1,
+      Features.supportedFeatures(
+        Utils.mkMap(
+          Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)),
+          Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5)))))
+
+    val adminClient = createAdminClient()
+    adminClient.updateFeatures(
+      Utils.mkMap(Utils.mkEntry("feature_1", update1), Utils.mkEntry("feature_2", update2)),
+      new UpdateFeaturesOptions()
+    ).all().get()
+
+    checkFeatures(
+      adminClient,
+      new FeatureZNode(FeatureZNodeStatus.Enabled, targetFinalizedFeatures),
+      expected)
+  }
+
+  @Test
+  def testPartialSuccessDuringValidFeatureUpgradeAndInvalidDowngrade(): Unit = {
+    TestUtils.waitUntilControllerElected(zkClient)
+
+    val initialSupportedFeatures = Features.supportedFeatures(

Review comment:
       Just 2 occurrences (one in this test and other in the next test). I'd leave it the way it is as the test is readable with values inlined in the test body.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1647,6 +1865,192 @@ class KafkaController(val config: KafkaConfig,
     }
   }
 
+  /**
+   * Returns the new FinalizedVersionRange for the feature, if there are no feature
+   * incompatibilities seen with all known brokers for the provided feature update.
+   * Otherwise returns an ApiError object containing Errors#INVALID_REQUEST.
+   *
+   * @param update   the feature update to be processed (this can not be meant to delete the feature)
+   *
+   * @return         the new FinalizedVersionRange or error, as described above.
+   */
+  private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = {
+    if (UpdateFeaturesRequest.isDeleteRequest(update)) {
+      throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update")
+    }
+
+    val incompatibilityError = "Could not apply finalized feature update because" +
+      " brokers were found to have incompatible versions for the feature."
+
+    if (brokerFeatures.supportedFeatures.get(update.feature()) == null) {

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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1216,71 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker. It can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter. This is particularly useful if the user requires strongly consistent reads of
+     * finalized features.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the {@link DescribeFeaturesResult} containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);

Review comment:
       Done. I've updated the KIP to mention `DescribeFeaturesOptions`.




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1215,71 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * Describes finalized as well as supported features. By default, the request is issued to any
+     * broker. It can be optionally directed only to the controller via DescribeFeaturesOptions
+     * parameter. This is particularly useful if the user requires strongly consistent reads of
+     * finalized features.
+     * <p>
+     * The following exceptions can be anticipated when calling {@code get()} on the future from the
+     * returned {@link DescribeFeaturesResult}:
+     * <ul>
+     *   <li>{@link org.apache.kafka.common.errors.TimeoutException}
+     *   If the request timed out before the describe operation could finish.</li>
+     * </ul>
+     * <p>
+     * @param options   the options to use
+     *
+     * @return          the {@link DescribeFeaturesResult} containing the result
+     */
+    DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options);

Review comment:
       Yea, you are right, I think this comment belongs to updateFeatures




----------------------------------------------------------------
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 #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -82,18 +108,54 @@ object FinalizedFeatureCache extends Logging {
         " The existing cache contents are %s").format(latest, oldFeatureAndEpoch)
       throw new FeatureCacheUpdateException(errorMsg)
     } else {
-      val incompatibleFeatures = SupportedFeatures.incompatibleFeatures(latest.features)
+      val incompatibleFeatures = brokerFeatures.incompatibleFeatures(latest.features)
       if (!incompatibleFeatures.empty) {
         val errorMsg = ("FinalizedFeatureCache update failed since feature compatibility" +
           " checks failed! Supported %s has incompatibilities with the latest %s."
-          ).format(SupportedFeatures.get, latest)
+          ).format(brokerFeatures.supportedFeatures, latest)
         throw new FeatureCacheUpdateException(errorMsg)
       } else {
-        val logMsg = "Updated cache from existing finalized %s to latest finalized %s".format(
+        val logMsg = "Updated cache from existing %s to latest %s".format(
           oldFeatureAndEpoch, latest)
-        featuresAndEpoch = Some(latest)
+        synchronized {
+          featuresAndEpoch = Some(latest)
+          notifyAll()
+        }
         info(logMsg)
       }
     }
   }
+
+  /**
+   * Causes the current thread to wait no more than timeoutMs for the specified condition to be met.
+   * It is guaranteed that the provided condition will always be invoked only from within a
+   * synchronized block.
+   *
+   * @param waitCondition   the condition to be waited upon:
+   *                         - if the condition returns true, then, the wait will stop.
+   *                         - if the condition returns false, it means the wait must continue until
+   *                           timeout.
+   *
+   * @param timeoutMs       the timeout (in milli seconds)
+   *
+   * @throws                TimeoutException if the condition is not met within timeoutMs.
+   */
+  private def waitUntilConditionOrThrow(waitCondition: () => Boolean, timeoutMs: Long): Unit = {
+    if(timeoutMs < 0L) {
+      throw new IllegalArgumentException(s"Expected timeoutMs >= 0, but $timeoutMs was provided.")
+    }
+    val waitEndTimeNanos = System.nanoTime() + (timeoutMs * 1_000_000)

Review comment:
       Since the app depends on monotonically increasing elapsed time values, `System.nanoTime()` is preferred. 
   `System.currentTimeMillis()` can change due to daylight saving time, users changing the time settings, leap seconds, and internet time sync etc.




----------------------------------------------------------------
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 edited a comment on pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #9001:
URL: https://github.com/apache/kafka/pull/9001#issuecomment-700585584


   @junrao Thanks a lot for the review! I've addressed the comments in the recent commit: 06d8b47131f168db88e4f7d5bda3dd025ba9a2a2. I've provided a response to all of your comments. There are few I couldn't address, and 1-2 comments I'll address in the near future (needs a little discussion).


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