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/05/17 09:01:13 UTC

[GitHub] [kafka] kowshik opened a new pull request #8680: KAFKA-9755: Implement read path for feature versioning system (KIP-584)

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


   **TL;DR:**
   In this PR, I have implemented various classes and integration for the read path of the feature versioning system ([KIP-584](https://cwiki.apache.org/confluence/display/KAFKA/KIP-584%3A+Versioning+scheme+for+features)). The finalized features information is going to be stored in ZK under the node `/feature`. The read path implemented in this PR comprises broadly of the following:
   
   - A facility is provided in the broker to listen to and propagate finalized feature changes from ZK.
   - When new finalized features are read, feature incompatibilities are detected by comparing against supported features in the broker.
   - `ApiVersionsResponse` is served containing supported and finalized feature information (using the newly added tagged fields).
   
   **New IBP config value:**
   The feature versioning system is implemented such that it is activated only if the IBP >= `KAFKA_2_6_IV1` (newly defined value in this PR). That is because, we would like to keep the feature version system disabled (almost as if it is non-existent) until the user upgrades the cluster to or beyond IBP `KAFKA_2_6_IV1`.
   
   **New common libraries (Java):**
   
   A set of common libraries abstracting features have been introduced. Features are a map with key being `String` (feature name) and value being a  `VersionRangeType` (the range of supported or finalized versions for the feature). The libraries are defined as 3 new classes in: `org.apache.kafka.common.feature.{Features, VersionRange, VersionLevelRange}`.
   
   The reason why it is kept in this common package is that future client and tooling development could reuse some/all of these abstractions.
   
   **New broker libraries (Scala):**
   
   The above common libraries are used within the Broker code to implement the read path for feature information, as explained below.
      1. Implemented a cache of cluster-wide finalized features (see class: `FinalizedFeatureCache`). This cache stores the latest finalized features and epoch read from the `/feature` node in ZK. Currently the main reader of this cache is the read path that serves an `ApiVersionsRequest` returning the features information in the response. In the future, as we start using the versioning system more, the cache could be read by future read paths intending to learn the finalized feature information.
      2. Implemented a feature ZK node change listener, that, listens to changes in the `/feature` node in ZK, and, invalidates the cache defined in `FinalizedFeatureCache` (see above point). See class: `FinalizedFeatureChangeListener`.  An instance of this class is maintained in `KafkaServer`, and initialized only if the IBP is set to `KAFKA_2_6_IV1` or higher.
      3. Implemented a facility to define supported features within the Broker (these are specific to a broker binary). See companion object: `SupportedFeatures`. Currently the list of supported features is empty, because, we have not identified features (yet). In the future this class can be populated as we begin to define supported features. Note that the public interface of this class provides a way to compare supported vs finalized features to detect incompatibilities.
   
   **New Zookeeper node: `/feature`:**
   The finalized feature information is going to be stored in ZK under the node `/feature`.
   1. Implemented a class `FeatureZNode` providing attributes and encode/decode APIs that abstract the structure of the new feature ZK node.
   2. Implemented new APIs in `KafkaZkClient.scala` to read/update/delete the `/feature` ZK node.
     
   **Updated ApiVersionsResponse**:
   
   Defined new tagged fields in existing `ApiVersionsResponse` schema. In the request serving path (in `KafkaApis.scala`), these fields are populated in the response with the latest supported features (from `SupportedFeatures`) and latest finalized features (from `FinalizedFeatureCache`).
   
   **Tests:**
   Added test suites for all of the new classes, and changes made in 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] junrao commented on a change in pull request #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -113,14 +127,44 @@ public static ApiVersionsResponse fromStruct(Struct struct, short version) {
         }
     }
 
-    public static ApiVersionsResponse apiVersionsResponse(int throttleTimeMs, byte maxMagic) {
+    public static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures) {
+        return apiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, Optional.empty(), Optional.empty());
+    }
+
+    public static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures,
+        Features<FinalizedVersionRange> finalizedFeatures,
+        int finalizedFeaturesEpoch) {
+        return apiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, Optional.of(finalizedFeatures), Optional.of(finalizedFeaturesEpoch));
+    }
+
+    private static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures,
+        Optional<Features<FinalizedVersionRange>> finalizedFeatures,
+        Optional<Integer> finalizedFeaturesEpoch) {

Review comment:
       Could we pass in `Optional<FinalizedFeaturesAndEpoch>` instead of two separate Optional?

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

Review comment:
       missing license header

##########
File path: core/src/main/scala/kafka/cluster/Broker.scala
##########
@@ -34,14 +36,21 @@ object Broker {
                                          brokerId: Int,
                                          endpoints: util.List[Endpoint],
                                          interBrokerEndpoint: Endpoint) extends AuthorizerServerInfo
+
+  def apply(id: Int, endPoints: Seq[EndPoint], rack: Option[String]): Broker = {
+    new Broker(id, endPoints, rack, emptySupportedFeatures)
+  }
 }
 
 /**
  * A Kafka broker.
- * A broker has an id, a collection of end-points, an optional rack and a listener to security protocol map.
- * Each end-point is (host, port, listenerName).
+ *
+ * @param id          a broker id
+ * @param endPoints   a collection of EndPoint. Each end-point is (host, port, listener name, security protocol).
+ * @param rack        an optional rack
+ * @param features    optional supported features

Review comment:
       The comment can be a bit misleading since features is not Optional.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,231 @@
+/**
+ * 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.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+import scala.util.control.Exception.ignoring
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked exactly
+     * once successfully. A subsequent invocation will raise an exception.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     * @throws   FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     * @throws   RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      debug(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        featureZNode.status match {
+          case FeatureZNodeStatus.Disabled => {
+            info(s"Feature ZK node at path: $featureZkNodePath is in disabled status.")
+            FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+          }
+          case FeatureZNodeStatus.Enabled => {
+            FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+          }
+          case _ => throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully. This method returns
+     * immediately if an updateLatestOrThrow call had already completed successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             TimeoutException if the wait can not be completed in waitTimeMs
+     *                     milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (!notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        ignoring(classOf[InterruptedException]) {
+          queue.take.updateLatestOrThrow()
+        }
+      } catch {
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will eventually exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDataChange(): Unit = {
+      info(s"Feature ZK node updated at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDeletion(): Unit = {
+      warn(s"Feature ZK node deleted at path: $path")
+      // This event may happen, rarely (ex: ZK corruption or operational error).
+      // In such a case, we prefer to just log a warning and treat the case as if the node is absent,
+      // and populate the FinalizedFeatureCache with empty finalized features.
+      queue.add(new FeatureCacheUpdater(path))
+    }
+  }
+
+  private val queue = new LinkedBlockingQueue[FeatureCacheUpdater]
+
+  private val thread = new ChangeNotificationProcessorThread("feature-zk-node-event-process-thread")
+
+  /**
+   * This method initializes the feature ZK node change listener. Optionally, it also ensures to
+   * update the FinalizedFeatureCache once with the latest contents of the feature ZK node
+   * (if the node exists). This step helps ensure that feature incompatibilities (if any) in brokers
+   * are conveniently detected before the initOrThrow() method returns to the caller. If feature
+   * incompatibilities are detected, this method will throw an Exception to the caller, and the Broker
+   * will exit eventually.
+   *
+   * @param waitOnceForCacheUpdateMs   # of milli seconds to wait for feature cache to be updated once.
+   *                                   If this parameter <= 0, no wait operation happens.
+   *
+   * @throws Exception if feature incompatibility check could not be finished in a timely manner
+   */
+  def initOrThrow(waitOnceForCacheUpdateMs: Long): Unit = {
+    if (waitOnceForCacheUpdateMs <= 0) {
+      throw new IllegalArgumentException(
+        s"Expected waitOnceForCacheUpdateMs > 0, but provided: $waitOnceForCacheUpdateMs")
+    }
+
+    thread.start()

Review comment:
       To handle ZK session expiration, we need to register a StateChangeHandler. That way, we can read the /features path from ZK when the new session is established since the feature could have changed btw the old and the new ZK sessions. See object ZkStateChangeHandler as an example.

##########
File path: core/src/main/scala/kafka/server/KafkaServer.scala
##########
@@ -660,6 +674,10 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP
         if (zkClient != null)
           CoreUtils.swallow(zkClient.close(), this)
 
+        if (featureChangeListener != null) {
+          CoreUtils.swallow(featureChangeListener.close(), this)

Review comment:
       It's probably better to close this before zkClient since the close call unregister from zkClient. 

##########
File path: core/src/test/scala/kafka/zk/FeatureZNodeTest.scala
##########
@@ -0,0 +1,111 @@
+package kafka.zk

Review comment:
       missing license header

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -135,7 +179,42 @@ public static ApiVersionsResponse createApiVersionsResponse(int throttleTimeMs,
         data.setThrottleTimeMs(throttleTimeMs);
         data.setErrorCode(Errors.NONE.code());
         data.setApiKeys(apiKeys);
+        data.setSupportedFeatures(createSupportedFeatureKeys(latestSupportedFeatures));
+        if (finalizedFeatures.isPresent()) {
+            data.setFinalizedFeatures(createFinalizedFeatureKeys(finalizedFeatures.get()));
+        }
+        if (finalizedFeaturesEpoch.isPresent()) {
+            data.setFinalizedFeaturesEpoch(finalizedFeaturesEpoch.get());

Review comment:
       If finalizedFeaturesEpoch is not present, we probably want to set the field to sth like -1 instead of leaving it as the default value of 0.

##########
File path: clients/src/test/java/org/apache/kafka/common/feature/FeaturesTest.java
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThrows;
+
+public class FeaturesTest {
+
+    @Test
+    public void testEmptyFeatures() {
+        Map<String, Map<String, Short>> emptyMap = new HashMap<>();
+
+        Features<FinalizedVersionRange> emptyFinalizedFeatures = Features.emptyFinalizedFeatures();
+        assertTrue(emptyFinalizedFeatures.features().isEmpty());
+        assertTrue(emptyFinalizedFeatures.toMap().isEmpty());
+        assertEquals(emptyFinalizedFeatures, Features.fromFinalizedFeaturesMap(emptyMap));
+
+        Features<SupportedVersionRange> emptySupportedFeatures = Features.emptySupportedFeatures();
+        assertTrue(emptySupportedFeatures.features().isEmpty());
+        assertTrue(emptySupportedFeatures.toMap().isEmpty());
+        assertEquals(emptySupportedFeatures, Features.fromSupportedFeaturesMap(emptyMap));
+    }
+
+    @Test
+    public void testNullFeatures() {
+        assertThrows(
+            NullPointerException.class,
+            () -> Features.finalizedFeatures(null));
+        assertThrows(
+            NullPointerException.class,
+            () -> Features.supportedFeatures(null));
+    }
+
+    @Test
+    public void testGetAllFeaturesAPI() {
+        SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2);
+        SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4);
+        Map<String, SupportedVersionRange> allFeatures =
+            mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2));
+        Features<SupportedVersionRange> features = Features.supportedFeatures(allFeatures);
+        assertEquals(allFeatures, features.features());
+    }
+
+    @Test
+    public void testGetAPI() {
+        SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2);
+        SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4);
+        Map<String, SupportedVersionRange> allFeatures = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2));
+        Features<SupportedVersionRange> features = Features.supportedFeatures(allFeatures);
+        assertEquals(v1, features.get("feature_1"));
+        assertEquals(v2, features.get("feature_2"));
+        assertNull(features.get("nonexistent_feature"));
+    }
+
+    @Test
+    public void testFromFeaturesMapToFeaturesMap() {
+        SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2);
+        SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4);
+        Map<String, SupportedVersionRange> allFeatures = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2));
+
+        Features<SupportedVersionRange> features = Features.supportedFeatures(allFeatures);
+
+        Map<String, Map<String, Short>> expected = mkMap(
+            mkEntry("feature_1", mkMap(mkEntry("min_version", (short) 1), mkEntry("max_version", (short) 2))),
+            mkEntry("feature_2", mkMap(mkEntry("min_version", (short) 3), mkEntry("max_version", (short) 4))));
+        assertEquals(expected, features.toMap());
+        assertEquals(features, Features.fromSupportedFeaturesMap(expected));
+    }
+
+    @Test
+    public void testFromToFinalizedFeaturesMap() {
+        FinalizedVersionRange v1 = new FinalizedVersionRange((short) 1, (short) 2);
+        FinalizedVersionRange v2 = new FinalizedVersionRange((short) 3, (short) 4);
+        Map<String, FinalizedVersionRange> allFeatures = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2));
+
+        Features<FinalizedVersionRange> features = Features.finalizedFeatures(allFeatures);
+
+        Map<String, Map<String, Short>> expected = mkMap(
+            mkEntry("feature_1", mkMap(mkEntry("min_version_level", (short) 1), mkEntry("max_version_level", (short) 2))),
+            mkEntry("feature_2", mkMap(mkEntry("min_version_level", (short) 3), mkEntry("max_version_level", (short) 4))));
+        assertEquals(expected, features.toMap());
+        assertEquals(features, Features.fromFinalizedFeaturesMap(expected));
+    }
+
+    @Test
+    public void testToStringFinalizedFeatures() {
+        FinalizedVersionRange v1 = new FinalizedVersionRange((short) 1, (short) 2);
+        FinalizedVersionRange v2 = new FinalizedVersionRange((short) 3, (short) 4);
+        Map<String, FinalizedVersionRange> allFeatures = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2));
+
+        Features<FinalizedVersionRange> features = Features.finalizedFeatures(allFeatures);
+
+        assertEquals(
+            "Features{(feature_1 -> FinalizedVersionRange[min_version_level:1, max_version_level:2]), (feature_2 -> FinalizedVersionRange[min_version_level:3, max_version_level:4])}",
+            features.toString());
+    }
+
+    @Test
+    public void testToStringSupportedFeatures() {
+        SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2);
+        SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4);
+        Map<String, SupportedVersionRange> allFeatures
+            = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2));
+
+        Features<SupportedVersionRange> features = Features.supportedFeatures(allFeatures);
+
+        assertEquals(
+            "Features{(feature_1 -> SupportedVersionRange[min_version:1, max_version:2]), (feature_2 -> SupportedVersionRange[min_version:3, max_version:4])}",
+            features.toString());
+    }
+
+    @Test
+    public void testFromToSupportedFeaturesMap() {

Review comment:
       The name of the method probably should include failure?

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -744,6 +781,165 @@ object DelegationTokenInfoZNode {
   def decode(bytes: Array[Byte]): Option[TokenInformation] = DelegationTokenManager.fromBytes(bytes)
 }
 
+/**
+ * Represents the status of the FeatureZNode.
+ *
+ * Enabled  -> This status means the feature versioning system (KIP-584) is enabled, and, the
+ *             finalized features stored in the FeatureZNode are active. This status is written by
+ *             the controller to the FeatureZNode only when the broker IBP config is greater than
+ *             or equal to KAFKA_2_6_IV1.
+ *
+ * Disabled -> This status means the feature versioning system (KIP-584) is disabled, and, the
+ *             the finalized features stored in the FeatureZNode is not relevant. This status is
+ *             written by the controller to the FeatureZNode only when the broker IBP config
+ *             is less than KAFKA_2_6_IV1.
+ *
+ * The purpose behind the FeatureZNodeStatus is that it helps differentiates between the following
+ * cases:
+ *
+ * 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_6_IV1. In such a case, the
+ *    controller will start up and notice that the FeatureZNode is absent in the new cluster.
+ *    To handle the requirement, the controller will create a FeatureZNode (with enabled status)
+ *    containing the entire list of supported features as its finalized features.
+ *
+ * 2. Cluster upgrade:
+ *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_6_IV1, but
+ *    the Broker binary has been upgraded to a state where it 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 enable
+ *    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_6_IV1. The reason is that enabling all the possible
+ *    features immediately after an upgrade could be harmful to the cluster.
+ *    In such a case:
+ *      - Before the Broker upgrade (i.e. IBP config set to less than KAFKA_2_6_IV1), the controller
+ *        will start up and check if the FeatureZNode is absent. If true, then it will react by
+ *        creating a FeatureZNode with disabled status and empty features.
+ *      - After the Broker upgrade (i.e. IBP config set to greater than or equal to KAFKA_2_6_IV1),
+ *        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.
+ *
+ * 2. Cluster downgrade:
+ *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+ *    KAFKA_2_6_IV1. Then, the user decided to downgrade the cluster by setting IBP config to a
+ *    value less than KAFKA_2_6_IV1. 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.
+ */
+object FeatureZNodeStatus extends Enumeration {
+  val Disabled, Enabled = Value
+
+  def withNameOpt(value: Int): Option[Value] = {
+    values.find(_.id == value)
+  }
+}
+
+/**
+ * Represents the contents of the ZK node containing finalized feature information.
+ *
+ * @param status     the status of the ZK node
+ * @param features   the cluster-wide finalized features
+ */
+case class FeatureZNode(status: FeatureZNodeStatus.Value, features: Features[FinalizedVersionRange]) {
+}
+
+object FeatureZNode {
+  private val VersionKey = "version"
+  private val StatusKey = "status"
+  private val FeaturesKey = "features"
+
+  // V0 contains 'version', 'status' and 'features' keys.
+  val V0 = 0
+  val CurrentVersion = V0
+
+  def path = "/feature"
+
+  def asJavaMap(scalaMap: Map[String, Map[String, Short]]): util.Map[String, util.Map[String, java.lang.Short]] = {
+    scalaMap
+      .view.mapValues(_.view.mapValues(scalaShort => java.lang.Short.valueOf(scalaShort)).toMap.asJava)
+      .toMap
+      .asJava
+  }
+
+  /**
+   * Encodes a FeatureZNode to JSON.
+   *
+   * @param featureZNode   FeatureZNode to be encoded
+   *
+   * @return               JSON representation of the FeatureZNode, as an Array[Byte]
+   */
+  def encode(featureZNode: FeatureZNode): Array[Byte] = {
+    val jsonMap = collection.mutable.Map(
+      VersionKey -> CurrentVersion,
+      StatusKey -> featureZNode.status.id,
+      FeaturesKey -> featureZNode.features.toMap)
+    Json.encodeAsBytes(jsonMap.asJava)
+  }
+
+  /**
+   * Decodes the contents of the feature ZK node from Array[Byte] to a FeatureZNode.
+   *
+   * @param jsonBytes   the contents of the feature ZK node
+   *
+   * @return            the FeatureZNode created from jsonBytes
+   *
+   * @throws IllegalArgumentException   if the Array[Byte] can not be decoded.
+   */
+  def decode(jsonBytes: Array[Byte]): FeatureZNode = {
+    Json.tryParseBytes(jsonBytes) match {
+      case Right(js) =>
+        val featureInfo = js.asJsonObject
+        val version = featureInfo(VersionKey).to[Int]
+        if (version < V0 || version > CurrentVersion) {

Review comment:
       `version > CurrentVersion` means that we can't downgrade the broker. We will need to relax this check.

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

Review comment:
       missing license header

##########
File path: core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala
##########
@@ -0,0 +1,185 @@
+package kafka.server
+
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion, ZooKeeperTestHarness}
+import kafka.utils.{Exit, TestUtils}
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.internals.FatalExitError
+import org.junit.Assert.{assertEquals, assertFalse, assertNotEquals, assertThrows, assertTrue}
+import org.junit.{Before, Test}
+
+import scala.concurrent.TimeoutException
+import scala.jdk.CollectionConverters._
+
+class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness {
+  @Before
+  override def setUp(): Unit = {
+    super.setUp()
+    FinalizedFeatureCache.clear()
+    SupportedFeatures.clear()
+  }
+
+  private def createSupportedFeatures(): Features[SupportedVersionRange] = {
+    val supportedFeaturesMap = Map[String, SupportedVersionRange](
+      "feature_1" -> new SupportedVersionRange(1, 4),
+      "feature_2" -> new SupportedVersionRange(1, 3))
+    SupportedFeatures.update(Features.supportedFeatures(supportedFeaturesMap.asJava))
+    SupportedFeatures.get
+  }
+
+  private def createFinalizedFeatures(): FinalizedFeaturesAndEpoch = {
+    val finalizedFeaturesMap = Map[String, FinalizedVersionRange](
+      "feature_1" -> new FinalizedVersionRange(2, 3))
+    val finalizedFeatures = Features.finalizedFeatures(finalizedFeaturesMap.asJava)
+    zkClient.createFeatureZNode(FeatureZNode(FeatureZNodeStatus.Enabled, finalizedFeatures))
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(version, ZkVersion.UnknownVersion)
+    assertFalse(mayBeFeatureZNodeBytes.isEmpty)
+    FinalizedFeaturesAndEpoch(finalizedFeatures, version)
+  }
+
+  private def createListener(expectedCacheContent: Option[FinalizedFeaturesAndEpoch]): FinalizedFeatureChangeListener = {
+    val listener = new FinalizedFeatureChangeListener(zkClient)
+    assertFalse(listener.isListenerInitiated)
+    assertTrue(FinalizedFeatureCache.isEmpty)
+    listener.initOrThrow(15000)
+    assertTrue(listener.isListenerInitiated)
+    if (expectedCacheContent.isDefined) {
+      val mayBeNewCacheContent = FinalizedFeatureCache.get
+      assertFalse(mayBeNewCacheContent.isEmpty)
+      val newCacheContent = mayBeNewCacheContent.get
+      assertEquals(expectedCacheContent.get.features, newCacheContent.features)
+      assertEquals(expectedCacheContent.get.epoch, newCacheContent.epoch)
+    } else {
+      val mayBeNewCacheContent = FinalizedFeatureCache.get
+      assertTrue(mayBeNewCacheContent.isEmpty)
+    }
+    listener
+  }
+
+  /**
+   * 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).
+   */
+  @Test
+  def testInitSuccessAndNotificationSuccess(): Unit = {
+    createSupportedFeatures()
+    val initialFinalizedFeatures = createFinalizedFeatures()
+    val listener = createListener(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)
+  }
+
+  /**
+   * Tests that the listener can be initialized, and that it can process FeatureZNode deletion
+   * successfully.
+   */
+  @Test
+  def testFeatureZNodeDeleteNotificationProcessing(): Unit = {
+    createSupportedFeatures()
+    val initialFinalizedFeatures = createFinalizedFeatures()
+    val listener = createListener(Some(initialFinalizedFeatures))
+
+    zkClient.deleteFeatureZNode()
+    val (mayBeFeatureZNodeDeletedBytes, deletedVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    assertEquals(deletedVersion, ZkVersion.UnknownVersion)
+    assertTrue(mayBeFeatureZNodeDeletedBytes.isEmpty)
+    TestUtils.waitUntilTrue(() => {
+      FinalizedFeatureCache.isEmpty
+    }, "Timed out waiting for FinalizedFeatureCache to become empty")
+    assertTrue(listener.isListenerInitiated)
+  }
+
+  /**
+   * Tests that the listener can be initialized, and that it can process disabling of a FeatureZNode
+   * successfully.
+   */
+  @Test
+  def testFeatureZNodeDisablingNotificationProcessing(): Unit = {
+    createSupportedFeatures()
+    val initialFinalizedFeatures = createFinalizedFeatures()
+    val listener = createListener(Some(initialFinalizedFeatures))
+
+    val updatedFinalizedFeaturesMap = Map[String, FinalizedVersionRange]()
+    val updatedFinalizedFeatures = Features.finalizedFeatures(updatedFinalizedFeaturesMap.asJava)
+    zkClient.updateFeatureZNode(FeatureZNode(FeatureZNodeStatus.Disabled, updatedFinalizedFeatures))
+    val (mayBeFeatureZNodeNewBytes, updatedVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(updatedVersion, ZkVersion.UnknownVersion)
+    assertFalse(mayBeFeatureZNodeNewBytes.isEmpty)
+    assertTrue(updatedVersion > initialFinalizedFeatures.epoch)
+    TestUtils.waitUntilTrue(() => {
+      FinalizedFeatureCache.get.isDefined &&
+        FinalizedFeatureCache.get.get.features.equals(updatedFinalizedFeatures) &&
+        FinalizedFeatureCache.get.get.epoch == updatedVersion
+    }, "Timed out waiting for FinalizedFeatureCache to become empty")
+    assertTrue(listener.isListenerInitiated)
+  }
+
+  /**
+   * Tests that the listener initialization fails when it picks up a feature incompatibility from
+   * ZK from an "Enabled" FeatureZNode.
+   */
+  @Test
+  def testInitFailureDueToFeatureIncompatibility(): Unit = {
+    createSupportedFeatures()
+
+    val incompatibleFinalizedFeaturesMap = Map[String, FinalizedVersionRange](
+      "feature_1" -> new FinalizedVersionRange(2, 5))
+    val incompatibleFinalizedFeatures = Features.finalizedFeatures(incompatibleFinalizedFeaturesMap.asJava)
+    zkClient.createFeatureZNode(FeatureZNode(FeatureZNodeStatus.Enabled, incompatibleFinalizedFeatures))
+    val (mayBeFeatureZNodeBytes, initialVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(initialVersion, ZkVersion.UnknownVersion)
+    assertFalse(mayBeFeatureZNodeBytes.isEmpty)
+
+    Exit.setExitProcedure((status, _) => throw new FatalExitError(status))

Review comment:
       Do we want to throw an Exception here?

##########
File path: core/src/test/scala/kafka/zk/FeatureZNodeTest.scala
##########
@@ -0,0 +1,111 @@
+package kafka.zk
+
+import java.nio.charset.StandardCharsets
+
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange}
+import org.apache.kafka.common.feature.Features._
+import org.junit.Assert.{assertEquals, assertThrows}
+import org.junit.Test
+
+import scala.jdk.CollectionConverters._
+
+class FeatureZNodeTest {
+
+  @Test
+  def testEncodeDecode(): Unit = {
+    val featureZNode = FeatureZNode(
+      FeatureZNodeStatus.Enabled,
+      Features.finalizedFeatures(
+        Map[String, FinalizedVersionRange](
+          "feature1" -> new FinalizedVersionRange(1, 2),
+          "feature2" -> new FinalizedVersionRange(2, 4)).asJava))
+    val decoded = FeatureZNode.decode(FeatureZNode.encode(featureZNode))
+    assertEquals(featureZNode.status, decoded.status)
+    assertEquals(featureZNode.features, decoded.features)

Review comment:
       Could we just assertEquals(featureZNode, decoded)?

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

Review comment:
       missing license header

##########
File path: core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala
##########
@@ -0,0 +1,185 @@
+package kafka.server
+
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion, ZooKeeperTestHarness}
+import kafka.utils.{Exit, TestUtils}
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.internals.FatalExitError
+import org.junit.Assert.{assertEquals, assertFalse, assertNotEquals, assertThrows, assertTrue}
+import org.junit.{Before, Test}
+
+import scala.concurrent.TimeoutException
+import scala.jdk.CollectionConverters._
+
+class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness {
+  @Before
+  override def setUp(): Unit = {
+    super.setUp()
+    FinalizedFeatureCache.clear()
+    SupportedFeatures.clear()
+  }
+
+  private def createSupportedFeatures(): Features[SupportedVersionRange] = {
+    val supportedFeaturesMap = Map[String, SupportedVersionRange](
+      "feature_1" -> new SupportedVersionRange(1, 4),
+      "feature_2" -> new SupportedVersionRange(1, 3))
+    SupportedFeatures.update(Features.supportedFeatures(supportedFeaturesMap.asJava))
+    SupportedFeatures.get
+  }
+
+  private def createFinalizedFeatures(): FinalizedFeaturesAndEpoch = {
+    val finalizedFeaturesMap = Map[String, FinalizedVersionRange](
+      "feature_1" -> new FinalizedVersionRange(2, 3))
+    val finalizedFeatures = Features.finalizedFeatures(finalizedFeaturesMap.asJava)
+    zkClient.createFeatureZNode(FeatureZNode(FeatureZNodeStatus.Enabled, finalizedFeatures))
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(version, ZkVersion.UnknownVersion)
+    assertFalse(mayBeFeatureZNodeBytes.isEmpty)
+    FinalizedFeaturesAndEpoch(finalizedFeatures, version)
+  }
+
+  private def createListener(expectedCacheContent: Option[FinalizedFeaturesAndEpoch]): FinalizedFeatureChangeListener = {
+    val listener = new FinalizedFeatureChangeListener(zkClient)
+    assertFalse(listener.isListenerInitiated)
+    assertTrue(FinalizedFeatureCache.isEmpty)
+    listener.initOrThrow(15000)
+    assertTrue(listener.isListenerInitiated)
+    if (expectedCacheContent.isDefined) {
+      val mayBeNewCacheContent = FinalizedFeatureCache.get
+      assertFalse(mayBeNewCacheContent.isEmpty)
+      val newCacheContent = mayBeNewCacheContent.get
+      assertEquals(expectedCacheContent.get.features, newCacheContent.features)
+      assertEquals(expectedCacheContent.get.epoch, newCacheContent.epoch)
+    } else {
+      val mayBeNewCacheContent = FinalizedFeatureCache.get
+      assertTrue(mayBeNewCacheContent.isEmpty)
+    }
+    listener
+  }
+
+  /**
+   * 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).
+   */
+  @Test
+  def testInitSuccessAndNotificationSuccess(): Unit = {
+    createSupportedFeatures()
+    val initialFinalizedFeatures = createFinalizedFeatures()
+    val listener = createListener(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)
+  }
+
+  /**
+   * Tests that the listener can be initialized, and that it can process FeatureZNode deletion
+   * successfully.
+   */
+  @Test
+  def testFeatureZNodeDeleteNotificationProcessing(): Unit = {
+    createSupportedFeatures()
+    val initialFinalizedFeatures = createFinalizedFeatures()
+    val listener = createListener(Some(initialFinalizedFeatures))
+
+    zkClient.deleteFeatureZNode()
+    val (mayBeFeatureZNodeDeletedBytes, deletedVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    assertEquals(deletedVersion, ZkVersion.UnknownVersion)
+    assertTrue(mayBeFeatureZNodeDeletedBytes.isEmpty)
+    TestUtils.waitUntilTrue(() => {
+      FinalizedFeatureCache.isEmpty
+    }, "Timed out waiting for FinalizedFeatureCache to become empty")
+    assertTrue(listener.isListenerInitiated)
+  }
+
+  /**
+   * Tests that the listener can be initialized, and that it can process disabling of a FeatureZNode
+   * successfully.
+   */
+  @Test
+  def testFeatureZNodeDisablingNotificationProcessing(): Unit = {
+    createSupportedFeatures()
+    val initialFinalizedFeatures = createFinalizedFeatures()
+    val listener = createListener(Some(initialFinalizedFeatures))
+
+    val updatedFinalizedFeaturesMap = Map[String, FinalizedVersionRange]()
+    val updatedFinalizedFeatures = Features.finalizedFeatures(updatedFinalizedFeaturesMap.asJava)
+    zkClient.updateFeatureZNode(FeatureZNode(FeatureZNodeStatus.Disabled, updatedFinalizedFeatures))
+    val (mayBeFeatureZNodeNewBytes, updatedVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(updatedVersion, ZkVersion.UnknownVersion)
+    assertFalse(mayBeFeatureZNodeNewBytes.isEmpty)
+    assertTrue(updatedVersion > initialFinalizedFeatures.epoch)
+    TestUtils.waitUntilTrue(() => {
+      FinalizedFeatureCache.get.isDefined &&
+        FinalizedFeatureCache.get.get.features.equals(updatedFinalizedFeatures) &&

Review comment:
       Hmm, if the feature is disabled, it seems that updatedFinalizedFeatures shouldn't be reflected in the cache, right?




----------------------------------------------------------------
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 #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -47,7 +56,30 @@ class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
       })
 
       info(s"Reading feature ZK node at path: $featureZkNodePath")
-      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      var mayBeFeatureZNodeBytes: Option[Array[Byte]] = null
+      var version: Int = ZkVersion.UnknownVersion
+      try {
+        val result = zkClient.getDataAndVersion(featureZkNodePath)
+        mayBeFeatureZNodeBytes = result._1
+        version = result._2
+      } catch {
+        // Convert to RuntimeException, to avoid a confusion that there is no argument passed
+        // to the updateOrThrow() method.
+        case e: IllegalArgumentException => throw new RuntimeException(e)

Review comment:
       I didn't look thoroughly enough, but the only IllegalArgumentException I found is 
   ```
    case invalidVersion =>
           throw new IllegalArgumentException(s"Expected controller epoch zkVersion $invalidVersion should be non-negative or equal to ${ZkVersion.MatchAnyVersion}")
   ```
   which should never happen as we always use `MatchAnyVersion` in `retryRequestsUntilConnected`. Are we trying to catch some other exceptions here?

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -37,6 +37,15 @@ class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
      *
      * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
      * only exactly once successfully.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     *
+     *           FeatureCacheUpdateException, if there was an error in updating the

Review comment:
       We could have multiple @throws here

##########
File path: clients/src/test/java/org/apache/kafka/common/feature/FeaturesTest.java
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThrows;
+
+public class FeaturesTest {
+
+    @Test
+    public void testEmptyFeatures() {
+        Map<String, Map<String, Long>> emptyMap = new HashMap<>();
+
+        Features<FinalizedVersionRange> emptyFinalizedFeatures = Features.emptyFinalizedFeatures();
+        assertEquals(new HashMap<>(), emptyFinalizedFeatures.features());
+        assertEquals(emptyMap, emptyFinalizedFeatures.serialize());
+        assertEquals(emptyFinalizedFeatures, Features.deserializeFinalizedFeatures(emptyMap));
+
+        Features<SupportedVersionRange> emptySupportedFeatures = Features.emptySupportedFeatures();
+        assertEquals(new HashMap<>(), emptySupportedFeatures.features());

Review comment:
       nit: we could test `emptySupportedFeatures.features().isEmpty()`

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,232 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.

Review comment:
       nit: remove `only`

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -744,6 +782,90 @@ object DelegationTokenInfoZNode {
   def decode(bytes: Array[Byte]): Option[TokenInformation] = DelegationTokenManager.fromBytes(bytes)
 }
 
+object FeatureZNodeStatus extends Enumeration {
+  val Disabled, Enabled = Value
+
+  def withNameOpt(value: Int): Option[Value] = {
+    values.find(_.id == value)
+  }
+}
+
+case class FeatureZNode(status: FeatureZNodeStatus.Value, features: Features[FinalizedVersionRange]) {
+}
+
+object FeatureZNode {

Review comment:
       I think the norm exists because we don't have automated framework by then, and doing hand-written json serialization and deserialization is a bit wasting. cc @hachikuji @cmccabe as this is a major direction discussion.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -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 org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being <VersionRangeType>.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ * @see SupportedVersionRange
+ * @see FinalizedVersionRange
+ */
+public class Features<VersionRangeType extends BaseVersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        if (features == null) {

Review comment:
       nit: one liner: `this.features = Objects.requireNonNull(features, "Provided features can not be null.");`

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Represents an immutable basic version range using 2 attributes: min and max of type long.
+ * The min and max attributes are expected to be >= 1, and with max >= min.
+ *
+ * The class also provides API to serialize/deserialize the version range to/from a map.
+ * The class allows for configurable labels for the min/max attributes, which can be specialized by
+ * sub-classes (if needed).
+ */
+class BaseVersionRange {
+    private final String minKeyLabel;
+
+    private final long minValue;
+
+    private final String maxKeyLabel;
+
+    private final long maxValue;
+
+    protected BaseVersionRange(String minKey, long minValue, String maxKeyLabel, long maxValue) {

Review comment:
       nit: minKeyLabel

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,203 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             - RuntimeException if the thread was interrupted during wait
+     *                     - TimeoutException if the wait can not be completed in waitTimeMs
+     *                       milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    private def processNotification(): Unit = {
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")

Review comment:
       My feeling is that this could be on debug level, but no strong perference.

##########
File path: clients/src/main/resources/common/message/ApiVersionsResponse.json
##########
@@ -42,6 +42,33 @@
         "about": "The maximum supported version, inclusive." }
     ]},
     { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true,
-      "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }
+      "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." },

Review comment:
       I see, makes sense.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,86 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange}
+
+// 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) {
+  override def toString(): String = {
+    "FinalizedFeaturesAndEpoch(features=%s, epoch=%s)".format(features, epoch)
+  }
+}
+
+/**
+ * A mutable cache containing the latest finalized features and epoch. This cache is populated by a
+ * {@link FinalizedFeatureChangeListener}.
+ *
+ * Currently the main reader of this cache is the read path that serves an ApiVersionsRequest,
+ * returning the features information in the response.
+ */
+object FinalizedFeatureCache extends Logging {
+  @volatile private var featuresAndEpoch: Option[FinalizedFeaturesAndEpoch] = Option.empty
+
+  /**
+   * @return   the latest known FinalizedFeaturesAndEpoch. If the returned value is empty, it means
+   *           no FinalizedFeaturesAndEpoch exists in the cache at the time when this
+   *           method is invoked. This result could change in the future whenever the
+   *           updateOrThrow method is invoked.
+   */
+  def get: Option[FinalizedFeaturesAndEpoch] = {
+    featuresAndEpoch
+  }
+
+  def empty: Boolean = {
+    featuresAndEpoch.isEmpty
+  }
+
+  /**
+   * Clears all existing finalized features and epoch from the cache.
+   */
+  def clear(): Unit = {
+    featuresAndEpoch = Option.empty
+    info("Cleared cache")
+  }
+
+  /**
+   * Updates the cache to the latestFeatures, and updates the existing epoch to latestEpoch.
+   * Raises an exception when the operation is not successful.
+   *
+   * @param latestFeatures   the latest finalized features to be set in the cache
+   * @param latestEpoch      the latest epoch value to be set in the cache
+   *
+   * @throws                 FeatureCacheUpdateException if the cache update operation fails
+   *                         due to invalid parameters or incompatibilities with the broker's
+   *                         supported features. In such a case, the existing cache contents are
+   *                         not modified.
+   */
+  def updateOrThrow(latestFeatures: Features[FinalizedVersionRange], latestEpoch: Int): Unit = {
+    updateOrThrow(FinalizedFeaturesAndEpoch(latestFeatures, latestEpoch))
+  }
+
+  private def updateOrThrow(latest: FinalizedFeaturesAndEpoch): Unit = {

Review comment:
       nit: could you elaborate why this helper function and `FinalizedFeaturesAndEpoch` struct is useful in this context? Just for easier message printing?

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,232 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     *
+     *           FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     *
+     *           RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      var mayBeFeatureZNodeBytes: Option[Array[Byte]] = null
+      var version: Int = ZkVersion.UnknownVersion
+      try {
+        val result = zkClient.getDataAndVersion(featureZkNodePath)
+        mayBeFeatureZNodeBytes = result._1
+        version = result._2
+      } catch {
+        // Convert to RuntimeException, to avoid a confusion that there is no argument passed
+        // to the updateOrThrow() method.
+        case e: IllegalArgumentException => throw new RuntimeException(e)
+      }
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             RuntimeException if the thread was interrupted during wait
+     *
+     *                     TimeoutException if the wait can not be completed in waitTimeMs
+     *                     milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Change notification queue interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDataChange(): Unit = {
+      info(s"Feature ZK node updated at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDeletion(): Unit = {
+      warn(s"Feature ZK node deleted at path: $path")
+      // This event may happen, rarely (ex: operational error).

Review comment:
       This leads to a more general question: is there a way to cleanup all the ZK feature path? Reading from the KIP, I don't see we have any admin API to do so, which makes me wonder how could this case happen in reality. In terms of severity, I think crushing the entire cluster seems to be an overkill as well, maybe we should have some blocking mechanism in place for any feature extraction call here, until we see `handleCreation` gets triggered again?

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,232 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     *
+     *           FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     *
+     *           RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      var mayBeFeatureZNodeBytes: Option[Array[Byte]] = null
+      var version: Int = ZkVersion.UnknownVersion
+      try {
+        val result = zkClient.getDataAndVersion(featureZkNodePath)
+        mayBeFeatureZNodeBytes = result._1
+        version = result._2
+      } catch {
+        // Convert to RuntimeException, to avoid a confusion that there is no argument passed
+        // to the updateOrThrow() method.
+        case e: IllegalArgumentException => throw new RuntimeException(e)
+      }
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)

Review comment:
       So here we will directly throw NoSuchElementException if `mayBeFeatureZNodeBytes` is empty? Do we want to check this case and throw a customized exception instead?

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,232 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     *
+     *           FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     *
+     *           RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      var mayBeFeatureZNodeBytes: Option[Array[Byte]] = null
+      var version: Int = ZkVersion.UnknownVersion
+      try {
+        val result = zkClient.getDataAndVersion(featureZkNodePath)
+        mayBeFeatureZNodeBytes = result._1
+        version = result._2
+      } catch {
+        // Convert to RuntimeException, to avoid a confusion that there is no argument passed
+        // to the updateOrThrow() method.
+        case e: IllegalArgumentException => throw new RuntimeException(e)
+      }
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {

Review comment:
       nit: space

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,232 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     *
+     *           FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     *
+     *           RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")

Review comment:
       This comment should be frequent and the `featureZkNodePath` is staying constant, could we just make it for debugging level?

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,232 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     *
+     *           FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     *
+     *           RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      var mayBeFeatureZNodeBytes: Option[Array[Byte]] = null
+      var version: Int = ZkVersion.UnknownVersion
+      try {
+        val result = zkClient.getDataAndVersion(featureZkNodePath)
+        mayBeFeatureZNodeBytes = result._1
+        version = result._2
+      } catch {
+        // Convert to RuntimeException, to avoid a confusion that there is no argument passed
+        // to the updateOrThrow() method.
+        case e: IllegalArgumentException => throw new RuntimeException(e)
+      }
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed

Review comment:
       I don't think this note is necessary, maybe just merge with the first line as:
   ```
   Waits until exactly one updateLatestOrThrow completes successfully.
   ```

##########
File path: core/src/main/scala/kafka/server/KafkaConfig.scala
##########
@@ -442,6 +445,8 @@ object KafkaConfig {
   val ControlledShutdownMaxRetriesProp = "controlled.shutdown.max.retries"
   val ControlledShutdownRetryBackoffMsProp = "controlled.shutdown.retry.backoff.ms"
   val ControlledShutdownEnableProp = "controlled.shutdown.enable"
+  /** ********* Features configuration ***********/
+  val FeatureChangeListenerCacheUpdateWaitTimeMsProp = "feature.listener.cache.update.wait.ms"

Review comment:
       Do you think we should add this config as part of the KIP since it is public? I think it would just be a minor update, but let's wait and see others thoughts on this.

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -744,6 +782,161 @@ object DelegationTokenInfoZNode {
   def decode(bytes: Array[Byte]): Option[TokenInformation] = DelegationTokenManager.fromBytes(bytes)
 }
 
+/**
+ * Represents the status of the FeatureZNode.
+ *
+ * Enabled  -> This status means the feature versioning system (KIP-584) is enabled, and, the
+ *             finalized features stored in the FeatureZNode are active. This status is written by
+ *             the controller to the FeatureZNode only when the broker IBP config is greater than
+ *             or equal to KAFKA_2_6_IV1.
+ *
+ * Disabled -> This status means the feature versioning system (KIP-584) is disabled, and, the
+ *             the finalized features stored in the FeatureZNode is not relevant. This status is
+ *             written by the controller to the FeatureZNode only when the broker IBP config
+ *             is less than KAFKA_2_6_IV1.
+ *
+ * The purpose behind the FeatureZNodeStatus is that it helps differentiates between the following
+ * cases:
+ *
+ * 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_6_IV1. In such a case, the
+ *    controller will start up and notice that the FeatureZNode is absent in the new cluster.
+ *    To handle the requirement, the controller will create a FeatureZNode (with enabled status)
+ *    containing the entire list of supported features as it’s finalized features.

Review comment:
       s/it's/its

##########
File path: core/src/main/scala/kafka/server/SupportedFeatures.scala
##########
@@ -0,0 +1,76 @@
+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 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.
+ *
+ * NOTE: the update() and clear() APIs of this class should be used only for testing purposes.
+ */
+object SupportedFeatures extends Logging {
+
+  /**
+   * This is the latest features supported by the Broker.
+   * This is currently empty, but in the future as we define supported features, this map should be
+   * populated.
+   */
+  @volatile private var supportedFeatures = emptySupportedFeatures
+
+  /**
+   * Returns a reference to the latest features supported by the Broker.
+   */
+  def get: Features[SupportedVersionRange] = {
+    supportedFeatures
+  }
+
+  // Should be used only for testing.
+  def update(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // Should be used only for testing.
+  def clear(): Unit = {
+    supportedFeatures = emptySupportedFeatures
+  }
+
+  /**
+   * 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 cluster-wide finalized feature. This can happen because a provided
+   * cluster-wide 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 sub-set 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] = {
+    val incompatibilities = finalized.features.asScala.collect {
+      case (feature, versionLevels) => {
+        val supportedVersions = supportedFeatures.get(feature);

Review comment:
       remove semi-colon

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -744,6 +782,161 @@ object DelegationTokenInfoZNode {
   def decode(bytes: Array[Byte]): Option[TokenInformation] = DelegationTokenManager.fromBytes(bytes)
 }
 
+/**
+ * Represents the status of the FeatureZNode.
+ *
+ * Enabled  -> This status means the feature versioning system (KIP-584) is enabled, and, the
+ *             finalized features stored in the FeatureZNode are active. This status is written by
+ *             the controller to the FeatureZNode only when the broker IBP config is greater than
+ *             or equal to KAFKA_2_6_IV1.
+ *
+ * Disabled -> This status means the feature versioning system (KIP-584) is disabled, and, the
+ *             the finalized features stored in the FeatureZNode is not relevant. This status is
+ *             written by the controller to the FeatureZNode only when the broker IBP config
+ *             is less than KAFKA_2_6_IV1.
+ *
+ * The purpose behind the FeatureZNodeStatus is that it helps differentiates between the following
+ * cases:
+ *
+ * 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_6_IV1. In such a case, the
+ *    controller will start up and notice that the FeatureZNode is absent in the new cluster.
+ *    To handle the requirement, the controller will create a FeatureZNode (with enabled status)
+ *    containing the entire list of supported features as it’s finalized features.
+ *
+ * 2. Cluster upgrade:
+ *    Imagine that a Kafka cluster exists already and the IBP config is less than KAFKA_2_6_IV1, but

Review comment:
       `a Kafka cluster exists already and the IBP config is less than KAFKA_2_6_IV1` to `an existing Kafka cluster with  IBP config less than KAFKA_2_6_IV1`




----------------------------------------------------------------
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 #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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


   ok to test


----------------------------------------------------------------
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 #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,232 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     *
+     *           FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     *
+     *           RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      var mayBeFeatureZNodeBytes: Option[Array[Byte]] = null
+      var version: Int = ZkVersion.UnknownVersion
+      try {
+        val result = zkClient.getDataAndVersion(featureZkNodePath)
+        mayBeFeatureZNodeBytes = result._1
+        version = result._2
+      } catch {
+        // Convert to RuntimeException, to avoid a confusion that there is no argument passed
+        // to the updateOrThrow() method.
+        case e: IllegalArgumentException => throw new RuntimeException(e)
+      }
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             RuntimeException if the thread was interrupted during wait
+     *
+     *                     TimeoutException if the wait can not be completed in waitTimeMs
+     *                     milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Change notification queue interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDataChange(): Unit = {
+      info(s"Feature ZK node updated at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDeletion(): Unit = {
+      warn(s"Feature ZK node deleted at path: $path")
+      // This event may happen, rarely (ex: operational error).

Review comment:
       The deletion of ZNode is a rare case, it should never happen in reality unless it is ZK corruption, or rarely an operational error that deletes some ZK nodes. It's not easy to prevent damage in such a case. From a correctness standpoint, imagine what would happen if the feature ZNode gets deleted, and, afterwards a broker restarts. It will start with empty cache, so the damage is done. Therefore, it seems that even if we add a special logic here, we can not prevent damage if the source of truth is lost.
   
   Two things to note here:
   1. The client should anyway ignore older stale epoch responses, if it had seen newer epochs that are greater. In that spirit, the client can be also made to treat the absence of finalized features in an `ApiVersionsResponse` just like a stale epoch case, if, it had seen at least one valid `ApiVersionsResponse` earlier (i.e. at least one response with some valid epoch). 
   
   2. Deletion of individual finalized feature is actually supported in [KIP-584](https://cwiki.apache.org/confluence/display/KAFKA/KIP-584%3A+Versioning+scheme+for+features), but not deletion of the entire ZNode. Search for the word 'deletion' in the KIP write-up. If needed, this deletion functionality could be extended to provide the ability o delete all features too.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [kafka] kowshik commented on a change in pull request #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/cluster/Broker.scala
##########
@@ -34,14 +36,21 @@ object Broker {
                                          brokerId: Int,
                                          endpoints: util.List[Endpoint],
                                          interBrokerEndpoint: Endpoint) extends AuthorizerServerInfo
+
+  def apply(id: Int, endPoints: Seq[EndPoint], rack: Option[String]): Broker = {
+    new Broker(id, endPoints, rack, emptySupportedFeatures)
+  }
 }
 
 /**
  * A Kafka broker.
- * A broker has an id, a collection of end-points, an optional rack and a listener to security protocol map.
- * Each end-point is (host, port, listenerName).
+ *
+ * @param id          a broker id
+ * @param endPoints   a collection of EndPoint. Each end-point is (host, port, listener name, security protocol).
+ * @param rack        an optional rack
+ * @param features    optional supported features

Review comment:
       Done. 

##########
File path: core/src/main/scala/kafka/server/KafkaServer.scala
##########
@@ -660,6 +674,10 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP
         if (zkClient != null)
           CoreUtils.swallow(zkClient.close(), this)
 
+        if (featureChangeListener != null) {
+          CoreUtils.swallow(featureChangeListener.close(), this)

Review comment:
       Done.

##########
File path: core/src/test/scala/kafka/zk/FeatureZNodeTest.scala
##########
@@ -0,0 +1,111 @@
+package kafka.zk
+
+import java.nio.charset.StandardCharsets
+
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange}
+import org.apache.kafka.common.feature.Features._
+import org.junit.Assert.{assertEquals, assertThrows}
+import org.junit.Test
+
+import scala.jdk.CollectionConverters._
+
+class FeatureZNodeTest {
+
+  @Test
+  def testEncodeDecode(): Unit = {
+    val featureZNode = FeatureZNode(
+      FeatureZNodeStatus.Enabled,
+      Features.finalizedFeatures(
+        Map[String, FinalizedVersionRange](
+          "feature1" -> new FinalizedVersionRange(1, 2),
+          "feature2" -> new FinalizedVersionRange(2, 4)).asJava))
+    val decoded = FeatureZNode.decode(FeatureZNode.encode(featureZNode))
+    assertEquals(featureZNode.status, decoded.status)
+    assertEquals(featureZNode.features, decoded.features)

Review comment:
       Done.

##########
File path: core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala
##########
@@ -0,0 +1,185 @@
+package kafka.server
+
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion, ZooKeeperTestHarness}
+import kafka.utils.{Exit, TestUtils}
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.internals.FatalExitError
+import org.junit.Assert.{assertEquals, assertFalse, assertNotEquals, assertThrows, assertTrue}
+import org.junit.{Before, Test}
+
+import scala.concurrent.TimeoutException
+import scala.jdk.CollectionConverters._
+
+class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness {
+  @Before
+  override def setUp(): Unit = {
+    super.setUp()
+    FinalizedFeatureCache.clear()
+    SupportedFeatures.clear()
+  }
+
+  private def createSupportedFeatures(): Features[SupportedVersionRange] = {
+    val supportedFeaturesMap = Map[String, SupportedVersionRange](
+      "feature_1" -> new SupportedVersionRange(1, 4),
+      "feature_2" -> new SupportedVersionRange(1, 3))
+    SupportedFeatures.update(Features.supportedFeatures(supportedFeaturesMap.asJava))
+    SupportedFeatures.get
+  }
+
+  private def createFinalizedFeatures(): FinalizedFeaturesAndEpoch = {
+    val finalizedFeaturesMap = Map[String, FinalizedVersionRange](
+      "feature_1" -> new FinalizedVersionRange(2, 3))
+    val finalizedFeatures = Features.finalizedFeatures(finalizedFeaturesMap.asJava)
+    zkClient.createFeatureZNode(FeatureZNode(FeatureZNodeStatus.Enabled, finalizedFeatures))
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(version, ZkVersion.UnknownVersion)
+    assertFalse(mayBeFeatureZNodeBytes.isEmpty)
+    FinalizedFeaturesAndEpoch(finalizedFeatures, version)
+  }
+
+  private def createListener(expectedCacheContent: Option[FinalizedFeaturesAndEpoch]): FinalizedFeatureChangeListener = {
+    val listener = new FinalizedFeatureChangeListener(zkClient)
+    assertFalse(listener.isListenerInitiated)
+    assertTrue(FinalizedFeatureCache.isEmpty)
+    listener.initOrThrow(15000)
+    assertTrue(listener.isListenerInitiated)
+    if (expectedCacheContent.isDefined) {
+      val mayBeNewCacheContent = FinalizedFeatureCache.get
+      assertFalse(mayBeNewCacheContent.isEmpty)
+      val newCacheContent = mayBeNewCacheContent.get
+      assertEquals(expectedCacheContent.get.features, newCacheContent.features)
+      assertEquals(expectedCacheContent.get.epoch, newCacheContent.epoch)
+    } else {
+      val mayBeNewCacheContent = FinalizedFeatureCache.get
+      assertTrue(mayBeNewCacheContent.isEmpty)
+    }
+    listener
+  }
+
+  /**
+   * 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).
+   */
+  @Test
+  def testInitSuccessAndNotificationSuccess(): Unit = {
+    createSupportedFeatures()
+    val initialFinalizedFeatures = createFinalizedFeatures()
+    val listener = createListener(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)
+  }
+
+  /**
+   * Tests that the listener can be initialized, and that it can process FeatureZNode deletion
+   * successfully.
+   */
+  @Test
+  def testFeatureZNodeDeleteNotificationProcessing(): Unit = {
+    createSupportedFeatures()
+    val initialFinalizedFeatures = createFinalizedFeatures()
+    val listener = createListener(Some(initialFinalizedFeatures))
+
+    zkClient.deleteFeatureZNode()
+    val (mayBeFeatureZNodeDeletedBytes, deletedVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    assertEquals(deletedVersion, ZkVersion.UnknownVersion)
+    assertTrue(mayBeFeatureZNodeDeletedBytes.isEmpty)
+    TestUtils.waitUntilTrue(() => {
+      FinalizedFeatureCache.isEmpty
+    }, "Timed out waiting for FinalizedFeatureCache to become empty")
+    assertTrue(listener.isListenerInitiated)
+  }
+
+  /**
+   * Tests that the listener can be initialized, and that it can process disabling of a FeatureZNode
+   * successfully.
+   */
+  @Test
+  def testFeatureZNodeDisablingNotificationProcessing(): Unit = {
+    createSupportedFeatures()
+    val initialFinalizedFeatures = createFinalizedFeatures()
+    val listener = createListener(Some(initialFinalizedFeatures))
+
+    val updatedFinalizedFeaturesMap = Map[String, FinalizedVersionRange]()
+    val updatedFinalizedFeatures = Features.finalizedFeatures(updatedFinalizedFeaturesMap.asJava)
+    zkClient.updateFeatureZNode(FeatureZNode(FeatureZNodeStatus.Disabled, updatedFinalizedFeatures))
+    val (mayBeFeatureZNodeNewBytes, updatedVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(updatedVersion, ZkVersion.UnknownVersion)
+    assertFalse(mayBeFeatureZNodeNewBytes.isEmpty)
+    assertTrue(updatedVersion > initialFinalizedFeatures.epoch)
+    TestUtils.waitUntilTrue(() => {
+      FinalizedFeatureCache.get.isDefined &&
+        FinalizedFeatureCache.get.get.features.equals(updatedFinalizedFeatures) &&

Review comment:
       Done. I have modified the code such that `FeatureCacheUpdater.updateLatestOrThrow` will now clear the cache whenever it sees that the feature ZK node is disabled.
   
   Great point! 

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,231 @@
+/**
+ * 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.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+import scala.util.control.Exception.ignoring
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked exactly
+     * once successfully. A subsequent invocation will raise an exception.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     * @throws   FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     * @throws   RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      debug(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        featureZNode.status match {
+          case FeatureZNodeStatus.Disabled => {
+            info(s"Feature ZK node at path: $featureZkNodePath is in disabled status.")
+            FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+          }
+          case FeatureZNodeStatus.Enabled => {
+            FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+          }
+          case _ => throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully. This method returns
+     * immediately if an updateLatestOrThrow call had already completed successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             TimeoutException if the wait can not be completed in waitTimeMs
+     *                     milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (!notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        ignoring(classOf[InterruptedException]) {
+          queue.take.updateLatestOrThrow()
+        }
+      } catch {
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will eventually exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDataChange(): Unit = {
+      info(s"Feature ZK node updated at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDeletion(): Unit = {
+      warn(s"Feature ZK node deleted at path: $path")
+      // This event may happen, rarely (ex: ZK corruption or operational error).
+      // In such a case, we prefer to just log a warning and treat the case as if the node is absent,
+      // and populate the FinalizedFeatureCache with empty finalized features.
+      queue.add(new FeatureCacheUpdater(path))
+    }
+  }
+
+  private val queue = new LinkedBlockingQueue[FeatureCacheUpdater]
+
+  private val thread = new ChangeNotificationProcessorThread("feature-zk-node-event-process-thread")
+
+  /**
+   * This method initializes the feature ZK node change listener. Optionally, it also ensures to
+   * update the FinalizedFeatureCache once with the latest contents of the feature ZK node
+   * (if the node exists). This step helps ensure that feature incompatibilities (if any) in brokers
+   * are conveniently detected before the initOrThrow() method returns to the caller. If feature
+   * incompatibilities are detected, this method will throw an Exception to the caller, and the Broker
+   * will exit eventually.
+   *
+   * @param waitOnceForCacheUpdateMs   # of milli seconds to wait for feature cache to be updated once.
+   *                                   If this parameter <= 0, no wait operation happens.
+   *
+   * @throws Exception if feature incompatibility check could not be finished in a timely manner
+   */
+  def initOrThrow(waitOnceForCacheUpdateMs: Long): Unit = {
+    if (waitOnceForCacheUpdateMs <= 0) {
+      throw new IllegalArgumentException(
+        s"Expected waitOnceForCacheUpdateMs > 0, but provided: $waitOnceForCacheUpdateMs")
+    }
+
+    thread.start()

Review comment:
       Done.

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

Review comment:
       Done.

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

Review comment:
       Done.

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

Review comment:
       Done.

##########
File path: core/src/test/scala/kafka/zk/FeatureZNodeTest.scala
##########
@@ -0,0 +1,111 @@
+package kafka.zk

Review comment:
       Done.

##########
File path: core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala
##########
@@ -0,0 +1,185 @@
+package kafka.server
+
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion, ZooKeeperTestHarness}
+import kafka.utils.{Exit, TestUtils}
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.internals.FatalExitError
+import org.junit.Assert.{assertEquals, assertFalse, assertNotEquals, assertThrows, assertTrue}
+import org.junit.{Before, Test}
+
+import scala.concurrent.TimeoutException
+import scala.jdk.CollectionConverters._
+
+class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness {
+  @Before
+  override def setUp(): Unit = {
+    super.setUp()
+    FinalizedFeatureCache.clear()
+    SupportedFeatures.clear()
+  }
+
+  private def createSupportedFeatures(): Features[SupportedVersionRange] = {
+    val supportedFeaturesMap = Map[String, SupportedVersionRange](
+      "feature_1" -> new SupportedVersionRange(1, 4),
+      "feature_2" -> new SupportedVersionRange(1, 3))
+    SupportedFeatures.update(Features.supportedFeatures(supportedFeaturesMap.asJava))
+    SupportedFeatures.get
+  }
+
+  private def createFinalizedFeatures(): FinalizedFeaturesAndEpoch = {
+    val finalizedFeaturesMap = Map[String, FinalizedVersionRange](
+      "feature_1" -> new FinalizedVersionRange(2, 3))
+    val finalizedFeatures = Features.finalizedFeatures(finalizedFeaturesMap.asJava)
+    zkClient.createFeatureZNode(FeatureZNode(FeatureZNodeStatus.Enabled, finalizedFeatures))
+    val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(version, ZkVersion.UnknownVersion)
+    assertFalse(mayBeFeatureZNodeBytes.isEmpty)
+    FinalizedFeaturesAndEpoch(finalizedFeatures, version)
+  }
+
+  private def createListener(expectedCacheContent: Option[FinalizedFeaturesAndEpoch]): FinalizedFeatureChangeListener = {
+    val listener = new FinalizedFeatureChangeListener(zkClient)
+    assertFalse(listener.isListenerInitiated)
+    assertTrue(FinalizedFeatureCache.isEmpty)
+    listener.initOrThrow(15000)
+    assertTrue(listener.isListenerInitiated)
+    if (expectedCacheContent.isDefined) {
+      val mayBeNewCacheContent = FinalizedFeatureCache.get
+      assertFalse(mayBeNewCacheContent.isEmpty)
+      val newCacheContent = mayBeNewCacheContent.get
+      assertEquals(expectedCacheContent.get.features, newCacheContent.features)
+      assertEquals(expectedCacheContent.get.epoch, newCacheContent.epoch)
+    } else {
+      val mayBeNewCacheContent = FinalizedFeatureCache.get
+      assertTrue(mayBeNewCacheContent.isEmpty)
+    }
+    listener
+  }
+
+  /**
+   * 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).
+   */
+  @Test
+  def testInitSuccessAndNotificationSuccess(): Unit = {
+    createSupportedFeatures()
+    val initialFinalizedFeatures = createFinalizedFeatures()
+    val listener = createListener(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)
+  }
+
+  /**
+   * Tests that the listener can be initialized, and that it can process FeatureZNode deletion
+   * successfully.
+   */
+  @Test
+  def testFeatureZNodeDeleteNotificationProcessing(): Unit = {
+    createSupportedFeatures()
+    val initialFinalizedFeatures = createFinalizedFeatures()
+    val listener = createListener(Some(initialFinalizedFeatures))
+
+    zkClient.deleteFeatureZNode()
+    val (mayBeFeatureZNodeDeletedBytes, deletedVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    assertEquals(deletedVersion, ZkVersion.UnknownVersion)
+    assertTrue(mayBeFeatureZNodeDeletedBytes.isEmpty)
+    TestUtils.waitUntilTrue(() => {
+      FinalizedFeatureCache.isEmpty
+    }, "Timed out waiting for FinalizedFeatureCache to become empty")
+    assertTrue(listener.isListenerInitiated)
+  }
+
+  /**
+   * Tests that the listener can be initialized, and that it can process disabling of a FeatureZNode
+   * successfully.
+   */
+  @Test
+  def testFeatureZNodeDisablingNotificationProcessing(): Unit = {
+    createSupportedFeatures()
+    val initialFinalizedFeatures = createFinalizedFeatures()
+    val listener = createListener(Some(initialFinalizedFeatures))
+
+    val updatedFinalizedFeaturesMap = Map[String, FinalizedVersionRange]()
+    val updatedFinalizedFeatures = Features.finalizedFeatures(updatedFinalizedFeaturesMap.asJava)
+    zkClient.updateFeatureZNode(FeatureZNode(FeatureZNodeStatus.Disabled, updatedFinalizedFeatures))
+    val (mayBeFeatureZNodeNewBytes, updatedVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(updatedVersion, ZkVersion.UnknownVersion)
+    assertFalse(mayBeFeatureZNodeNewBytes.isEmpty)
+    assertTrue(updatedVersion > initialFinalizedFeatures.epoch)
+    TestUtils.waitUntilTrue(() => {
+      FinalizedFeatureCache.get.isDefined &&
+        FinalizedFeatureCache.get.get.features.equals(updatedFinalizedFeatures) &&
+        FinalizedFeatureCache.get.get.epoch == updatedVersion
+    }, "Timed out waiting for FinalizedFeatureCache to become empty")
+    assertTrue(listener.isListenerInitiated)
+  }
+
+  /**
+   * Tests that the listener initialization fails when it picks up a feature incompatibility from
+   * ZK from an "Enabled" FeatureZNode.
+   */
+  @Test
+  def testInitFailureDueToFeatureIncompatibility(): Unit = {
+    createSupportedFeatures()
+
+    val incompatibleFinalizedFeaturesMap = Map[String, FinalizedVersionRange](
+      "feature_1" -> new FinalizedVersionRange(2, 5))
+    val incompatibleFinalizedFeatures = Features.finalizedFeatures(incompatibleFinalizedFeaturesMap.asJava)
+    zkClient.createFeatureZNode(FeatureZNode(FeatureZNodeStatus.Enabled, incompatibleFinalizedFeatures))
+    val (mayBeFeatureZNodeBytes, initialVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(initialVersion, ZkVersion.UnknownVersion)
+    assertFalse(mayBeFeatureZNodeBytes.isEmpty)
+
+    Exit.setExitProcedure((status, _) => throw new FatalExitError(status))

Review comment:
       Done. Changed it to use a latch that gets notified when the exit procedure is called.
   Great point!

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -744,6 +781,165 @@ object DelegationTokenInfoZNode {
   def decode(bytes: Array[Byte]): Option[TokenInformation] = DelegationTokenManager.fromBytes(bytes)
 }
 
+/**
+ * Represents the status of the FeatureZNode.
+ *
+ * Enabled  -> This status means the feature versioning system (KIP-584) is enabled, and, the
+ *             finalized features stored in the FeatureZNode are active. This status is written by
+ *             the controller to the FeatureZNode only when the broker IBP config is greater than
+ *             or equal to KAFKA_2_6_IV1.
+ *
+ * Disabled -> This status means the feature versioning system (KIP-584) is disabled, and, the
+ *             the finalized features stored in the FeatureZNode is not relevant. This status is
+ *             written by the controller to the FeatureZNode only when the broker IBP config
+ *             is less than KAFKA_2_6_IV1.
+ *
+ * The purpose behind the FeatureZNodeStatus is that it helps differentiates between the following
+ * cases:
+ *
+ * 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_6_IV1. In such a case, the
+ *    controller will start up and notice that the FeatureZNode is absent in the new cluster.
+ *    To handle the requirement, the controller will create a FeatureZNode (with enabled status)
+ *    containing the entire list of supported features as its finalized features.
+ *
+ * 2. Cluster upgrade:
+ *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_6_IV1, but
+ *    the Broker binary has been upgraded to a state where it 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 enable
+ *    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_6_IV1. The reason is that enabling all the possible
+ *    features immediately after an upgrade could be harmful to the cluster.
+ *    In such a case:
+ *      - Before the Broker upgrade (i.e. IBP config set to less than KAFKA_2_6_IV1), the controller
+ *        will start up and check if the FeatureZNode is absent. If true, then it will react by
+ *        creating a FeatureZNode with disabled status and empty features.
+ *      - After the Broker upgrade (i.e. IBP config set to greater than or equal to KAFKA_2_6_IV1),
+ *        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.
+ *
+ * 2. Cluster downgrade:
+ *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+ *    KAFKA_2_6_IV1. Then, the user decided to downgrade the cluster by setting IBP config to a
+ *    value less than KAFKA_2_6_IV1. 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.
+ */
+object FeatureZNodeStatus extends Enumeration {
+  val Disabled, Enabled = Value
+
+  def withNameOpt(value: Int): Option[Value] = {
+    values.find(_.id == value)
+  }
+}
+
+/**
+ * Represents the contents of the ZK node containing finalized feature information.
+ *
+ * @param status     the status of the ZK node
+ * @param features   the cluster-wide finalized features
+ */
+case class FeatureZNode(status: FeatureZNodeStatus.Value, features: Features[FinalizedVersionRange]) {
+}
+
+object FeatureZNode {
+  private val VersionKey = "version"
+  private val StatusKey = "status"
+  private val FeaturesKey = "features"
+
+  // V0 contains 'version', 'status' and 'features' keys.
+  val V0 = 0
+  val CurrentVersion = V0
+
+  def path = "/feature"
+
+  def asJavaMap(scalaMap: Map[String, Map[String, Short]]): util.Map[String, util.Map[String, java.lang.Short]] = {
+    scalaMap
+      .view.mapValues(_.view.mapValues(scalaShort => java.lang.Short.valueOf(scalaShort)).toMap.asJava)
+      .toMap
+      .asJava
+  }
+
+  /**
+   * Encodes a FeatureZNode to JSON.
+   *
+   * @param featureZNode   FeatureZNode to be encoded
+   *
+   * @return               JSON representation of the FeatureZNode, as an Array[Byte]
+   */
+  def encode(featureZNode: FeatureZNode): Array[Byte] = {
+    val jsonMap = collection.mutable.Map(
+      VersionKey -> CurrentVersion,
+      StatusKey -> featureZNode.status.id,
+      FeaturesKey -> featureZNode.features.toMap)
+    Json.encodeAsBytes(jsonMap.asJava)
+  }
+
+  /**
+   * Decodes the contents of the feature ZK node from Array[Byte] to a FeatureZNode.
+   *
+   * @param jsonBytes   the contents of the feature ZK node
+   *
+   * @return            the FeatureZNode created from jsonBytes
+   *
+   * @throws IllegalArgumentException   if the Array[Byte] can not be decoded.
+   */
+  def decode(jsonBytes: Array[Byte]): FeatureZNode = {
+    Json.tryParseBytes(jsonBytes) match {
+      case Right(js) =>
+        val featureInfo = js.asJsonObject
+        val version = featureInfo(VersionKey).to[Int]
+        if (version < V0 || version > CurrentVersion) {

Review comment:
       Done. Great point!

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -135,7 +179,42 @@ public static ApiVersionsResponse createApiVersionsResponse(int throttleTimeMs,
         data.setThrottleTimeMs(throttleTimeMs);
         data.setErrorCode(Errors.NONE.code());
         data.setApiKeys(apiKeys);
+        data.setSupportedFeatures(createSupportedFeatureKeys(latestSupportedFeatures));
+        if (finalizedFeatures.isPresent()) {
+            data.setFinalizedFeatures(createFinalizedFeatureKeys(finalizedFeatures.get()));
+        }
+        if (finalizedFeaturesEpoch.isPresent()) {
+            data.setFinalizedFeaturesEpoch(finalizedFeaturesEpoch.get());

Review comment:
       Done.

##########
File path: clients/src/test/java/org/apache/kafka/common/feature/FeaturesTest.java
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThrows;
+
+public class FeaturesTest {
+
+    @Test
+    public void testEmptyFeatures() {
+        Map<String, Map<String, Short>> emptyMap = new HashMap<>();
+
+        Features<FinalizedVersionRange> emptyFinalizedFeatures = Features.emptyFinalizedFeatures();
+        assertTrue(emptyFinalizedFeatures.features().isEmpty());
+        assertTrue(emptyFinalizedFeatures.toMap().isEmpty());
+        assertEquals(emptyFinalizedFeatures, Features.fromFinalizedFeaturesMap(emptyMap));
+
+        Features<SupportedVersionRange> emptySupportedFeatures = Features.emptySupportedFeatures();
+        assertTrue(emptySupportedFeatures.features().isEmpty());
+        assertTrue(emptySupportedFeatures.toMap().isEmpty());
+        assertEquals(emptySupportedFeatures, Features.fromSupportedFeaturesMap(emptyMap));
+    }
+
+    @Test
+    public void testNullFeatures() {
+        assertThrows(
+            NullPointerException.class,
+            () -> Features.finalizedFeatures(null));
+        assertThrows(
+            NullPointerException.class,
+            () -> Features.supportedFeatures(null));
+    }
+
+    @Test
+    public void testGetAllFeaturesAPI() {
+        SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2);
+        SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4);
+        Map<String, SupportedVersionRange> allFeatures =
+            mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2));
+        Features<SupportedVersionRange> features = Features.supportedFeatures(allFeatures);
+        assertEquals(allFeatures, features.features());
+    }
+
+    @Test
+    public void testGetAPI() {
+        SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2);
+        SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4);
+        Map<String, SupportedVersionRange> allFeatures = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2));
+        Features<SupportedVersionRange> features = Features.supportedFeatures(allFeatures);
+        assertEquals(v1, features.get("feature_1"));
+        assertEquals(v2, features.get("feature_2"));
+        assertNull(features.get("nonexistent_feature"));
+    }
+
+    @Test
+    public void testFromFeaturesMapToFeaturesMap() {
+        SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2);
+        SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4);
+        Map<String, SupportedVersionRange> allFeatures = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2));
+
+        Features<SupportedVersionRange> features = Features.supportedFeatures(allFeatures);
+
+        Map<String, Map<String, Short>> expected = mkMap(
+            mkEntry("feature_1", mkMap(mkEntry("min_version", (short) 1), mkEntry("max_version", (short) 2))),
+            mkEntry("feature_2", mkMap(mkEntry("min_version", (short) 3), mkEntry("max_version", (short) 4))));
+        assertEquals(expected, features.toMap());
+        assertEquals(features, Features.fromSupportedFeaturesMap(expected));
+    }
+
+    @Test
+    public void testFromToFinalizedFeaturesMap() {
+        FinalizedVersionRange v1 = new FinalizedVersionRange((short) 1, (short) 2);
+        FinalizedVersionRange v2 = new FinalizedVersionRange((short) 3, (short) 4);
+        Map<String, FinalizedVersionRange> allFeatures = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2));
+
+        Features<FinalizedVersionRange> features = Features.finalizedFeatures(allFeatures);
+
+        Map<String, Map<String, Short>> expected = mkMap(
+            mkEntry("feature_1", mkMap(mkEntry("min_version_level", (short) 1), mkEntry("max_version_level", (short) 2))),
+            mkEntry("feature_2", mkMap(mkEntry("min_version_level", (short) 3), mkEntry("max_version_level", (short) 4))));
+        assertEquals(expected, features.toMap());
+        assertEquals(features, Features.fromFinalizedFeaturesMap(expected));
+    }
+
+    @Test
+    public void testToStringFinalizedFeatures() {
+        FinalizedVersionRange v1 = new FinalizedVersionRange((short) 1, (short) 2);
+        FinalizedVersionRange v2 = new FinalizedVersionRange((short) 3, (short) 4);
+        Map<String, FinalizedVersionRange> allFeatures = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2));
+
+        Features<FinalizedVersionRange> features = Features.finalizedFeatures(allFeatures);
+
+        assertEquals(
+            "Features{(feature_1 -> FinalizedVersionRange[min_version_level:1, max_version_level:2]), (feature_2 -> FinalizedVersionRange[min_version_level:3, max_version_level:4])}",
+            features.toString());
+    }
+
+    @Test
+    public void testToStringSupportedFeatures() {
+        SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2);
+        SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4);
+        Map<String, SupportedVersionRange> allFeatures
+            = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2));
+
+        Features<SupportedVersionRange> features = Features.supportedFeatures(allFeatures);
+
+        assertEquals(
+            "Features{(feature_1 -> SupportedVersionRange[min_version:1, max_version:2]), (feature_2 -> SupportedVersionRange[min_version:3, max_version:4])}",
+            features.toString());
+    }
+
+    @Test
+    public void testFromToSupportedFeaturesMap() {

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -113,14 +127,44 @@ public static ApiVersionsResponse fromStruct(Struct struct, short version) {
         }
     }
 
-    public static ApiVersionsResponse apiVersionsResponse(int throttleTimeMs, byte maxMagic) {
+    public static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures) {
+        return apiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, Optional.empty(), Optional.empty());
+    }
+
+    public static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures,
+        Features<FinalizedVersionRange> finalizedFeatures,
+        int finalizedFeaturesEpoch) {
+        return apiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, Optional.of(finalizedFeatures), Optional.of(finalizedFeaturesEpoch));
+    }
+
+    private static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures,
+        Optional<Features<FinalizedVersionRange>> finalizedFeatures,
+        Optional<Integer> finalizedFeaturesEpoch) {

Review comment:
       Done. I'm no longer passing 2 optionals, since, we decided (below) that epoch can be set as -1 whenever it is absent.




----------------------------------------------------------------
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] hachikuji commented on pull request #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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


   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 pull request #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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


   @junrao Thanks for the review! I have fixed the checkstyle issue now in 74ff66f.


----------------------------------------------------------------
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 #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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


   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 #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Represents an immutable basic version range using 2 attributes: min and max of type long.
+ * The min and max attributes are expected to be >= 1, and with max >= min.
+ *
+ * The class also provides API to serialize/deserialize the version range to/from a map.
+ * The class allows for configurable labels for the min/max attributes, which can be specialized by
+ * sub-classes (if needed).
+ */
+class BaseVersionRange {

Review comment:
       It is thoroughly tested in it's child class test suite: `SupportedVersionRangeTest`.
   Personally I feel it is good enough this way, because, anyway to test this class we need to inherit into a sub-class (since constructor is `protected`). And by testing via `SupportedVersionRangeTest`, we achieve exactly the same.
   
   I have now added top-level documentation in the test suite of `SupportedVersionRangeTest`, explaining the above.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ * @see SupportedVersionRange
+ * @see FinalizedVersionRange
+ */
+public class Features<VersionRangeType extends BaseVersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to VersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "supported" features.
+     */
+    public static Features<SupportedVersionRange> supportedFeatures(Map<String, SupportedVersionRange> features) {
+        return new Features<>(features);
+    }
+
+    /**
+     * @param features   Map of feature name to FinalizedVersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "finalized" features.
+     */
+    public static Features<FinalizedVersionRange> finalizedFeatures(Map<String, FinalizedVersionRange> features) {
+        return new Features<>(features);
+    }
+
+    // Visible for testing.
+    public static Features<FinalizedVersionRange> emptyFinalizedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public static Features<SupportedVersionRange> emptySupportedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public Map<String, VersionRangeType> features() {
+        return features;
+    }
+
+    public boolean empty() {
+        return features.isEmpty();
+    }
+
+    /**
+     * @param  feature   name of the feature
+     *
+     * @return           the VersionRangeType corresponding to the feature name, or null if absent
+     */
+    public VersionRangeType get(String feature) {
+        return features.get(feature);
+    }
+
+    public String toString() {
+        return String.format(
+            "Features{%s}",
+            features
+                .entrySet()
+                .stream()
+                .map(entry -> String.format("(%s -> %s)", entry.getKey(), entry.getValue()))
+                .collect(joining(", "))
+        );
+    }
+
+    /**
+     * @return   A map with underlying features serialized. The returned value can be deserialized
+     *           using one of the deserialize* APIs.

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/FinalizedVersionRange.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.feature;
+
+import java.util.Map;
+
+/**
+ * A specialization of {@link BaseVersionRange} representing a range of version levels.
+ * NOTE: This is the backing class used to define the min/max version levels for finalized features.
+ */
+public class FinalizedVersionRange extends BaseVersionRange {
+    // Label for the min version key, that's used only for serialization/deserialization purposes.
+    private static final String MIN_VERSION_LEVEL_KEY_LABEL = "min_version_level";
+
+    // Label for the max version key, that's used only for serialization/deserialization purposes.
+    private static final String MAX_VERSION_LEVEL_KEY_LABEL = "max_version_level";
+
+    public FinalizedVersionRange(long minVersionLevel, long maxVersionLevel) {
+        super(MIN_VERSION_LEVEL_KEY_LABEL, minVersionLevel, MAX_VERSION_LEVEL_KEY_LABEL, maxVersionLevel);
+    }
+
+    public static FinalizedVersionRange deserialize(Map<String, Long> serialized) {
+        return new FinalizedVersionRange(
+            BaseVersionRange.valueOrThrow(MIN_VERSION_LEVEL_KEY_LABEL, serialized),
+            BaseVersionRange.valueOrThrow(MAX_VERSION_LEVEL_KEY_LABEL, serialized));
+    }
+
+    private boolean isCompatibleWith(BaseVersionRange versionRange) {
+        return min() >= versionRange.min() && max() <= versionRange.max();
+    }
+
+    /**
+     * 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.
+     *
+     * @param versionRange   the SupportedVersionRange to be checked

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ */
+public class Features<VersionRangeType extends VersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;

Review comment:
       Done. I'm raising an exception now if it is `null`.
   I see your point.
   Will be good to learn what is the convention in Kafka for constructor param null checks.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/FinalizedVersionRange.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.feature;
+
+import java.util.Map;
+
+/**
+ * A specialization of {@link BaseVersionRange} representing a range of version levels.
+ * NOTE: This is the backing class used to define the min/max version levels for finalized features.
+ */
+public class FinalizedVersionRange extends BaseVersionRange {
+    // Label for the min version key, that's used only for serialization/deserialization purposes.
+    private static final String MIN_VERSION_LEVEL_KEY_LABEL = "min_version_level";
+
+    // Label for the max version key, that's used only for serialization/deserialization purposes.
+    private static final String MAX_VERSION_LEVEL_KEY_LABEL = "max_version_level";
+
+    public FinalizedVersionRange(long minVersionLevel, long maxVersionLevel) {
+        super(MIN_VERSION_LEVEL_KEY_LABEL, minVersionLevel, MAX_VERSION_LEVEL_KEY_LABEL, maxVersionLevel);
+    }
+
+    public static FinalizedVersionRange deserialize(Map<String, Long> serialized) {
+        return new FinalizedVersionRange(
+            BaseVersionRange.valueOrThrow(MIN_VERSION_LEVEL_KEY_LABEL, serialized),
+            BaseVersionRange.valueOrThrow(MAX_VERSION_LEVEL_KEY_LABEL, serialized));
+    }
+
+    private boolean isCompatibleWith(BaseVersionRange versionRange) {

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.feature;
+
+import java.util.Map;
+
+/**
+ * A specialization of VersionRange representing a range of versions.
+ * NOTE: This is the backing class used to define the min/max versions for supported features.

Review comment:
       Done. Good point!

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,203 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,203 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             - RuntimeException if the thread was interrupted during wait

Review comment:
       Done. Removed.

##########
File path: core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala
##########
@@ -0,0 +1,228 @@
+package kafka.server
+
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion, ZooKeeperTestHarness}
+import kafka.utils.{Exit, TestUtils}
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.internals.FatalExitError
+import org.junit.Assert.{assertEquals, assertFalse, assertNotEquals, assertThrows, assertTrue}
+import org.junit.{Before, Test}
+
+import scala.concurrent.TimeoutException
+import scala.jdk.CollectionConverters._
+
+class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness {

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,203 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             - RuntimeException if the thread was interrupted during wait
+     *                     - TimeoutException if the wait can not be completed in waitTimeMs
+     *                       milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    private def processNotification(): Unit = {
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")
+      processNotification()
+    }
+
+    override def handleDataChange(): Unit = {
+      info(s"Feature ZK node updated at path: $path")
+      processNotification()
+    }
+
+    override def handleDeletion(): Unit = {
+      warn(s"Feature ZK node deleted at path: $path")
+      // This event may happen, rarely (ex: operational error).
+      // In such a case, we prefer to just log a warning and treat the case as if the node is absent,
+      // and populate the FinalizedFeatureCache with empty finalized features.
+      processNotification()
+    }
+  }
+
+  private val queue = new LinkedBlockingQueue[FeatureCacheUpdater]
+
+  private val thread = new ChangeNotificationProcessorThread("feature-zk-node-event-process-thread")
+
+  /**
+   * This method initializes the feature ZK node change listener. Optionally, it also ensures to
+   * update the FinalizedFeatureCache once with the latest contents of the feature ZK node
+   * (if the node exists). This step helps ensure that feature incompatibilities (if any) in brokers
+   * are conveniently detected before the initOrThrow() method returns to the caller. If feature
+   * incompatibilities are detected, this method will throw an Exception to the caller, and the Broker
+   * would exit eventually.
+   *
+   * @param waitOnceForCacheUpdateMs   # of milli seconds to wait for feature cache to be updated once.
+   *                                   If this parameter <= 0, no wait operation happens.
+   *
+   * @throws Exception if feature incompatibility check could not be finished in a timely manner
+   */
+  def initOrThrow(waitOnceForCacheUpdateMs: Long): Unit = {
+    thread.start()
+    zkClient.registerZNodeChangeHandlerAndCheckExistence(FeatureZNodeChangeHandler)
+
+    if (waitOnceForCacheUpdateMs > 0) {
+      val barrier = new CountDownLatch(1)
+      val ensureCacheUpdateOnce = new FeatureCacheUpdater(FeatureZNodeChangeHandler.path, Some(barrier))
+      queue.add(ensureCacheUpdateOnce)
+      try {
+        ensureCacheUpdateOnce.awaitUpdateOrThrow(waitOnceForCacheUpdateMs)
+      } catch {
+        case e: Exception => {
+          close()
+          throw e
+        }
+      }
+    }
+  }
+
+  /**
+   * Closes the feature ZK node change listener by unregistering the listener from ZK client,
+   * clearing the queue and shutting down the ChangeNotificationProcessorThread.
+   */
+  def close(): Unit = {
+    zkClient.unregisterZNodeChangeHandler(FeatureZNodeChangeHandler.path)

Review comment:
       The order probably doesn't matter in this case. But logically I decided to follow the below order since I could reason about it better:
   1. Stop the inflow of new events
   2. Clear pending events
   3. Stop the processing of all events

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,203 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             - RuntimeException if the thread was interrupted during wait
+     *                     - TimeoutException if the wait can not be completed in waitTimeMs
+     *                       milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    private def processNotification(): Unit = {

Review comment:
       Done. Removed.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,203 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      if (version == ZkVersion.UnknownVersion) {

Review comment:
       I have added documentation here in this method describing all the cases.
   The empty data case should never happen and can indicate a corruption. The reason is that we always return non-empty data in `FeatureZNode.encode`, so the ZK node content should never empty.
   
   Yes, I can add some more info to KAFKA-10028 or in the write path PR summary.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,93 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionLevelRange}
+
+// 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[VersionLevelRange], epoch: Int) {
+
+  def isValid(newEpoch: Int): Boolean = {
+    newEpoch >= epoch
+  }
+
+  override def toString(): String = {
+    "FinalizedFeaturesAndEpoch(features=%s, epoch=%s)".format(features, epoch)
+  }
+}
+
+/**
+ * A mutable cache containing the latest finalized features and epoch. This cache is populated by a
+ * FinalizedFeatureChangeListener.
+ *
+ * Currently the main reader of this cache is the read path that serves an ApiVersionsRequest
+ * returning the features information in the response. In the future, as the feature versioning
+ * system in KIP-584 is used more widely, this cache could be read by other read paths trying to
+ * learn the finalized feature information.
+ */
+object FinalizedFeatureCache extends Logging {
+  @volatile private var featuresAndEpoch: Option[FinalizedFeaturesAndEpoch] = Option.empty
+
+  /**
+   * @return   the latest known FinalizedFeaturesAndEpoch. If the returned value is empty, it means
+   *           no FinalizedFeaturesAndEpoch exists in the cache at the time when this
+   *           method is invoked. This result could change in the future whenever the
+   *           updateOrThrow method is invoked.
+   */
+  def get: Option[FinalizedFeaturesAndEpoch] = {
+    featuresAndEpoch
+  }
+
+  def empty: Boolean = {
+    featuresAndEpoch.isEmpty
+  }
+
+  /**
+   * Clears all existing finalized features and epoch from the cache.
+   */
+  def clear(): Unit = {
+    featuresAndEpoch = Option.empty
+    info("Cleared cache")
+  }
+
+  /**
+   * Updates the cache to the latestFeatures, and updates the existing epoch to latestEpoch.
+   * Raises an exception when the operation is not successful.
+   *
+   * @param latestFeatures   the latest finalized features to be set in the cache
+   * @param latestEpoch      the latest epoch value to be set in the cache
+   *
+   * @throws                 FeatureCacheUpdateException if the cache update operation fails
+   *                         due to invalid parameters or incompatibilities with the broker's
+   *                         supported features. In such a case, the existing cache contents are
+   *                         not modified.
+   */
+  def updateOrThrow(latestFeatures: Features[VersionLevelRange], latestEpoch: Int): Unit = {
+    updateOrThrow(FinalizedFeaturesAndEpoch(latestFeatures, latestEpoch))
+  }
+
+  private def updateOrThrow(latest: FinalizedFeaturesAndEpoch): Unit = {
+    val existingStr = featuresAndEpoch.map(existing => existing.toString).getOrElse("<empty>")
+    if (!featuresAndEpoch.isEmpty && featuresAndEpoch.get.epoch > latest.epoch) {
+      val errorMsg = ("FinalizedFeatureCache update failed due to invalid epoch in new finalized %s." +
+        " The existing finalized is %s").format(latest, existingStr)
+      throw new FeatureCacheUpdateException(errorMsg)
+    } else {
+      val incompatibleFeatures = SupportedFeatures.incompatibleFeatures(latest.features)
+      if (incompatibleFeatures.nonEmpty) {
+        val errorMsg = ("FinalizedFeatureCache updated failed since feature compatibility" +
+          " checks failed! Supported %s has incompatibilities with the latest finalized %s." +
+          " The incompatible features are: %s.").format(
+          SupportedFeatures.get, latest, incompatibleFeatures)
+        throw new FeatureCacheUpdateException(errorMsg)
+      }
+    }
+    val logMsg = "Updated cache from existing finalized %s to latest finalized %s".format(

Review comment:
       Done. Good point!

##########
File path: core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala
##########
@@ -0,0 +1,228 @@
+package kafka.server
+
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion, ZooKeeperTestHarness}
+import kafka.utils.{Exit, TestUtils}
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.internals.FatalExitError
+import org.junit.Assert.{assertEquals, assertFalse, assertNotEquals, assertThrows, assertTrue}
+import org.junit.{Before, Test}
+
+import scala.concurrent.TimeoutException
+import scala.jdk.CollectionConverters._
+
+class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness {
+  @Before
+  override def setUp(): Unit = {
+    super.setUp()
+    FinalizedFeatureCache.clear()
+    SupportedFeatures.clear()
+  }
+
+  /**
+   * 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).
+   */
+  @Test
+  def testInitSuccessAndNotificationSuccess(): Unit = {
+    val supportedFeatures = Map[String, SupportedVersionRange](
+      "feature_1" -> new SupportedVersionRange(1, 4),
+      "feature_2" -> new SupportedVersionRange(1, 3))
+    SupportedFeatures.update(Features.supportedFeatures(supportedFeatures.asJava))
+
+    val initialFinalizedFeaturesMap = Map[String, FinalizedVersionRange](
+      "feature_1" -> new FinalizedVersionRange(2, 3))
+    val initialFinalizedFeatures = Features.finalizedFeatures(initialFinalizedFeaturesMap.asJava)
+    zkClient.createFeatureZNode(FeatureZNode(FeatureZNodeStatus.Enabled, initialFinalizedFeatures))
+    val (mayBeFeatureZNodeBytes, initialVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(initialVersion, ZkVersion.UnknownVersion)
+    assertFalse(mayBeFeatureZNodeBytes.isEmpty)
+
+    val listener = new FinalizedFeatureChangeListener(zkClient)
+    assertFalse(listener.isListenerInitiated)
+    assertTrue(FinalizedFeatureCache.empty)
+    listener.initOrThrow(15000)
+    assertTrue(listener.isListenerInitiated)
+    val mayBeNewCacheContent = FinalizedFeatureCache.get
+    assertFalse(mayBeNewCacheContent.isEmpty)
+    val newCacheContent = mayBeNewCacheContent.get
+    assertEquals(initialFinalizedFeatures, newCacheContent.features)
+    assertEquals(initialVersion, newCacheContent.epoch)
+
+    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 > initialVersion)
+    TestUtils.waitUntilTrue(() => {
+      FinalizedFeatureCache.get.get.equals(FinalizedFeaturesAndEpoch(updatedFinalizedFeatures, updatedVersion))
+    }, "Timed out waiting for FinalizedFeatureCache to be updated with new features")
+    assertTrue(listener.isListenerInitiated)
+  }
+
+  /**
+   * Tests that the listener can be initialized, and that it can process FeatureZNode deletion
+   * successfully.
+   */
+  @Test
+  def testFeatureZNodeDeleteNotificationProcessing(): Unit = {
+    val supportedFeatures = Map[String, SupportedVersionRange](
+      "feature_1" -> new SupportedVersionRange(1, 4),
+      "feature_2" -> new SupportedVersionRange(1, 3))
+    SupportedFeatures.update(Features.supportedFeatures(supportedFeatures.asJava))
+
+    val initialFinalizedFeaturesMap = Map[String, FinalizedVersionRange](
+      "feature_1" -> new FinalizedVersionRange(2, 3))
+    val initialFinalizedFeatures = Features.finalizedFeatures(initialFinalizedFeaturesMap.asJava)
+    zkClient.createFeatureZNode(FeatureZNode(FeatureZNodeStatus.Enabled, initialFinalizedFeatures))
+    val (mayBeFeatureZNodeBytes, initialVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(initialVersion, ZkVersion.UnknownVersion)
+    assertFalse(mayBeFeatureZNodeBytes.isEmpty)
+
+    val listener = new FinalizedFeatureChangeListener(zkClient)
+    assertFalse(listener.isListenerInitiated)
+    assertTrue(FinalizedFeatureCache.empty)
+    listener.initOrThrow(15000)
+    assertTrue(listener.isListenerInitiated)
+    val mayBeNewCacheContent = FinalizedFeatureCache.get
+    assertFalse(mayBeNewCacheContent.isEmpty)
+    val newCacheContent = mayBeNewCacheContent.get
+    assertEquals(initialFinalizedFeatures, newCacheContent.features)
+    assertEquals(initialVersion, newCacheContent.epoch)
+
+    zkClient.deleteFeatureZNode()

Review comment:
       No. But we want to test the behavior about what happens during a deletion (ex: operational error).

##########
File path: clients/src/test/java/org/apache/kafka/common/feature/FinalizedVersionRangeTest.java
##########
@@ -0,0 +1,135 @@
+package org.apache.kafka.common.feature;
+
+import java.util.Map;
+
+import org.junit.Test;
+
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class FinalizedVersionRangeTest {

Review comment:
       Done. I have simplified this test suite eliminating the redundant tests, and only keeping the ones specific to `FinalizedVersionRange`. Also I have added documentation to both test suites explaining their purpose.

##########
File path: clients/src/test/java/org/apache/kafka/common/feature/FinalizedVersionRangeTest.java
##########
@@ -0,0 +1,135 @@
+package org.apache.kafka.common.feature;

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/FinalizedVersionRange.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.feature;
+
+import java.util.Map;
+
+/**
+ * A specialization of {@link BaseVersionRange} representing a range of version levels.
+ * NOTE: This is the backing class used to define the min/max version levels for finalized features.
+ */
+public class FinalizedVersionRange extends BaseVersionRange {
+    // Label for the min version key, that's used only for serialization/deserialization purposes.
+    private static final String MIN_VERSION_LEVEL_KEY_LABEL = "min_version_level";
+
+    // Label for the max version key, that's used only for serialization/deserialization purposes.
+    private static final String MAX_VERSION_LEVEL_KEY_LABEL = "max_version_level";
+
+    public FinalizedVersionRange(long minVersionLevel, long maxVersionLevel) {
+        super(MIN_VERSION_LEVEL_KEY_LABEL, minVersionLevel, MAX_VERSION_LEVEL_KEY_LABEL, maxVersionLevel);
+    }
+
+    public static FinalizedVersionRange deserialize(Map<String, Long> serialized) {
+        return new FinalizedVersionRange(
+            BaseVersionRange.valueOrThrow(MIN_VERSION_LEVEL_KEY_LABEL, serialized),
+            BaseVersionRange.valueOrThrow(MAX_VERSION_LEVEL_KEY_LABEL, serialized));
+    }
+
+    private boolean isCompatibleWith(BaseVersionRange versionRange) {
+        return min() >= versionRange.min() && max() <= versionRange.max();

Review comment:
       Done. Good point!

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,88 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange}
+
+// 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) {
+  override def toString(): String = {
+    "FinalizedFeaturesAndEpoch(features=%s, epoch=%s)".format(features, epoch)
+  }
+}
+
+/**
+ * A mutable cache containing the latest finalized features and epoch. This cache is populated by a
+ * {@link FinalizedFeatureChangeListener}.
+ *
+ * Currently the main reader of this cache is the read path that serves an ApiVersionsRequest

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/SupportedFeatures.scala
##########
@@ -0,0 +1,74 @@
+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 common 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.
+ */
+object SupportedFeatures extends Logging {
+
+  /**
+   * This is the latest features supported by the Broker.
+   * This is currently empty, but in the future as we define supported features, this map should be
+   * populated.
+   */
+  @volatile private var supportedFeatures = emptySupportedFeatures
+
+  /**
+   * Returns the latest features supported by the Broker.
+   */
+  def get: Features[SupportedVersionRange] = {
+    supportedFeatures
+  }
+
+  // Should be used only for testing.
+  def update(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // Should be used only for testing.
+  def clear(): Unit = {
+    supportedFeatures = emptySupportedFeatures
+  }
+
+  /**
+   * 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 cluster-wide finalized feature. This can happen because a provided
+   * cluster-wide 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 set of incompatible feature names. If the returned set is empty, it
+   *                    means there were no feature incompatibilities found.
+   */
+  def incompatibleFeatures(finalized: Features[FinalizedVersionRange]): Set[String] = {
+    val incompatibilities = finalized.features.asScala.collect {
+      case (feature, versionLevels) => {
+        val supportedVersions = supportedFeatures.get(feature);
+        if (supportedVersions == null) {
+          (feature, "{feature=%s, reason='Unsupported feature'}".format(feature))
+        } else if (versionLevels.isIncompatibleWith(supportedVersions)) {
+          (feature, "{feature=%s, reason='Finalized %s is incompatible with supported %s'}".format(
+            feature, versionLevels, supportedVersions))
+        } else {
+          (feature, null)
+        }
+      }
+    }.filter(entry => entry._2 != null)
+
+    if (incompatibilities.nonEmpty) {

Review comment:
       Done. Removed extra logging in the caller of this method (see `FinalizedFeatureCache`).

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ * @see SupportedVersionRange
+ * @see FinalizedVersionRange
+ */
+public class Features<VersionRangeType extends BaseVersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to VersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "supported" features.
+     */
+    public static Features<SupportedVersionRange> supportedFeatures(Map<String, SupportedVersionRange> features) {
+        return new Features<>(features);
+    }
+
+    /**
+     * @param features   Map of feature name to FinalizedVersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "finalized" features.
+     */
+    public static Features<FinalizedVersionRange> finalizedFeatures(Map<String, FinalizedVersionRange> features) {
+        return new Features<>(features);
+    }
+
+    // Visible for testing.
+    public static Features<FinalizedVersionRange> emptyFinalizedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public static Features<SupportedVersionRange> emptySupportedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public Map<String, VersionRangeType> features() {
+        return features;
+    }
+
+    public boolean empty() {
+        return features.isEmpty();
+    }
+
+    /**
+     * @param  feature   name of the feature
+     *
+     * @return           the VersionRangeType corresponding to the feature name, or null if absent
+     */
+    public VersionRangeType get(String feature) {
+        return features.get(feature);
+    }
+
+    public String toString() {
+        return String.format(
+            "Features{%s}",
+            features
+                .entrySet()
+                .stream()
+                .map(entry -> String.format("(%s -> %s)", entry.getKey(), entry.getValue()))
+                .collect(joining(", "))
+        );
+    }
+
+    /**
+     * @return   A map with underlying features serialized. The returned value can be deserialized
+     *           using one of the deserialize* APIs.
+     */
+    public Map<String, Map<String, Long>> serialize() {
+        return features.entrySet().stream().collect(
+            Collectors.toMap(
+                Map.Entry::getKey,
+                entry -> entry.getValue().serialize()));
+    }
+
+    /**
+     * Deserialize a map to Features<FinalizedVersionRange>.
+     *
+     * @param serialized   the serialized representation of a Features<FinalizedVersionRange> object,
+     *                     generated using the serialize() API.
+     *
+     * @return             the deserialized Features<FinalizedVersionRange> object
+     */
+    public static Features<FinalizedVersionRange> deserializeFinalizedFeatures(
+        Map<String, Map<String, Long>> serialized) {
+        return finalizedFeatures(serialized.entrySet().stream().collect(
+            Collectors.toMap(
+                Map.Entry::getKey,
+                entry -> FinalizedVersionRange.deserialize(entry.getValue()))));
+    }
+
+    /**
+     * Deserializes a map to Features<SupportedVersionRange>.
+     *
+     * @param serialized   the serialized representation of a Features<SupportedVersionRange> object,
+     *                     generated using the serialize() API.
+     *
+     * @return             the deserialized Features<SupportedVersionRange> object
+     */
+    public static Features<SupportedVersionRange> deserializeSupportedFeatures(

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -744,6 +782,90 @@ object DelegationTokenInfoZNode {
   def decode(bytes: Array[Byte]): Option[TokenInformation] = DelegationTokenManager.fromBytes(bytes)
 }
 
+object FeatureZNodeStatus extends Enumeration {
+  val Disabled, Enabled = Value
+
+  def withNameOpt(value: Int): Option[Value] = {
+    values.find(_.id == value)
+  }
+}
+
+case class FeatureZNode(status: FeatureZNodeStatus.Value, features: Features[FinalizedVersionRange]) {
+}
+
+object FeatureZNode {
+  private val VersionKey = "version"
+  private val StatusKey = "status"
+  private val FeaturesKey = "features"
+
+  // Version0 contains 'version', 'status' and 'features' keys.
+  val Version0 = 0

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -744,6 +782,90 @@ object DelegationTokenInfoZNode {
   def decode(bytes: Array[Byte]): Option[TokenInformation] = DelegationTokenManager.fromBytes(bytes)
 }
 
+object FeatureZNodeStatus extends Enumeration {
+  val Disabled, Enabled = Value
+
+  def withNameOpt(value: Int): Option[Value] = {
+    values.find(_.id == value)
+  }
+}
+
+case class FeatureZNode(status: FeatureZNodeStatus.Value, features: Features[FinalizedVersionRange]) {
+}
+
+object FeatureZNode {
+  private val VersionKey = "version"
+  private val StatusKey = "status"
+  private val FeaturesKey = "features"
+
+  // Version0 contains 'version', 'status' and 'features' keys.
+  val Version0 = 0
+  val CurrentVersion = Version0
+
+  def path = "/feature"
+
+  def asJavaMap(scalaMap: Map[String, Map[String, Long]]): util.Map[String, util.Map[String, java.lang.Long]] = {
+    scalaMap
+      .view.mapValues(_.view.mapValues(scalaLong => java.lang.Long.valueOf(scalaLong)).toMap.asJava)
+      .toMap
+      .asJava
+  }
+
+  def encode(featureZNode: FeatureZNode): Array[Byte] = {
+    val jsonMap = collection.mutable.Map(
+      VersionKey -> CurrentVersion,
+      StatusKey -> featureZNode.status.id,
+      FeaturesKey -> featureZNode.features.serialize)
+    Json.encodeAsBytes(jsonMap.asJava)
+  }
+
+  def decode(jsonBytes: Array[Byte]): FeatureZNode = {
+    Json.tryParseBytes(jsonBytes) match {
+      case Right(js) =>
+        val featureInfo = js.asJsonObject
+        val version = featureInfo(VersionKey).to[Int]
+        if (version < Version0 || version > CurrentVersion) {
+          throw new KafkaException(s"Unsupported version: $version of feature information: " +
+            s"${new String(jsonBytes, UTF_8)}")
+        }
+
+        val featuresMap = featureInfo
+          .get(FeaturesKey)
+          .flatMap(_.to[Option[Map[String, Map[String, Long]]]])
+        if (featuresMap.isEmpty) {
+          throw new KafkaException("Features map can not be absent in: " +
+            s"${new String(jsonBytes, UTF_8)}")
+        }
+        val features = asJavaMap(featuresMap.get)
+
+        val statusInt = featureInfo
+          .get(StatusKey)
+          .flatMap(_.to[Option[Int]])
+        if (statusInt.isEmpty) {
+          throw new KafkaException("Status can not be absent in feature information: " +
+            s"${new String(jsonBytes, UTF_8)}")
+        }
+        val status = FeatureZNodeStatus.withNameOpt(statusInt.get)
+        if (status.isEmpty) {

Review comment:
       Done.
   For the other point, I don't feel strongly for it. I feel it is OK to have an API that doesn't throw and just lets the caller decide (based on the context) if an empty returned value is incorrect.

##########
File path: core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
##########
@@ -776,6 +776,9 @@ class KafkaConfigTest {
         case KafkaConfig.KafkaMetricsReporterClassesProp => // ignore
         case KafkaConfig.KafkaMetricsPollingIntervalSecondsProp => //ignore
 
+        //Feature configuration

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/zk/KafkaZkClient.scala
##########
@@ -1567,6 +1567,36 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
     createRecursive(path, data = null, throwIfPathExists = false)
   }
 
+  // Visible for testing.
+  def createFeatureZNode(nodeContents: FeatureZNode): Unit = {

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -81,17 +83,26 @@ object BrokerIdsZNode {
 object BrokerInfo {
 
   /**
-   * Create a broker info with v4 json format (which includes multiple endpoints and rack) if
-   * the apiVersion is 0.10.0.X or above. Register the broker with v2 json format otherwise.
+   * - Create a broker info with v5 json format if the apiVersion is 2.6.x or above.
+   * - Create a broker info with v4 json format (which includes multiple endpoints and rack) if
+   *   the apiVersion is 0.10.0.X or above but lesser than 2.6.x.
+   * - Register the broker with v2 json format otherwise.
    *
    * Due to KAFKA-3100, 0.9.0.0 broker and old clients will break if JSON version is above 2.
    *
-   * We include v2 to make it possible for the broker to migrate from 0.9.0.0 to 0.10.0.X or above without having to
-   * upgrade to 0.9.0.1 first (clients have to be upgraded to 0.9.0.1 in any case).
+   * We include v2 to make it possible for the broker to migrate from 0.9.0.0 to 0.10.0.X or above
+   * without having to upgrade to 0.9.0.1 first (clients have to be upgraded to 0.9.0.1 in
+   * any case).
    */
   def apply(broker: Broker, apiVersion: ApiVersion, jmxPort: Int): BrokerInfo = {
-    // see method documentation for the reason why we do this
-    val version = if (apiVersion >= KAFKA_0_10_0_IV1) 4 else 2
+    val version = {
+      if (apiVersion >= KAFKA_0_10_0_IV1)

Review comment:
       Done. Good catch!

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -744,6 +782,90 @@ object DelegationTokenInfoZNode {
   def decode(bytes: Array[Byte]): Option[TokenInformation] = DelegationTokenManager.fromBytes(bytes)
 }
 
+object FeatureZNodeStatus extends Enumeration {
+  val Disabled, Enabled = Value
+
+  def withNameOpt(value: Int): Option[Value] = {
+    values.find(_.id == value)
+  }
+}
+
+case class FeatureZNode(status: FeatureZNodeStatus.Value, features: Features[FinalizedVersionRange]) {
+}
+
+object FeatureZNode {
+  private val VersionKey = "version"
+  private val StatusKey = "status"
+  private val FeaturesKey = "features"
+
+  // Version0 contains 'version', 'status' and 'features' keys.
+  val Version0 = 0
+  val CurrentVersion = Version0
+
+  def path = "/feature"
+
+  def asJavaMap(scalaMap: Map[String, Map[String, Long]]): util.Map[String, util.Map[String, java.lang.Long]] = {
+    scalaMap
+      .view.mapValues(_.view.mapValues(scalaLong => java.lang.Long.valueOf(scalaLong)).toMap.asJava)
+      .toMap
+      .asJava
+  }
+
+  def encode(featureZNode: FeatureZNode): Array[Byte] = {
+    val jsonMap = collection.mutable.Map(
+      VersionKey -> CurrentVersion,
+      StatusKey -> featureZNode.status.id,
+      FeaturesKey -> featureZNode.features.serialize)
+    Json.encodeAsBytes(jsonMap.asJava)
+  }
+
+  def decode(jsonBytes: Array[Byte]): FeatureZNode = {
+    Json.tryParseBytes(jsonBytes) match {
+      case Right(js) =>
+        val featureInfo = js.asJsonObject
+        val version = featureInfo(VersionKey).to[Int]
+        if (version < Version0 || version > CurrentVersion) {
+          throw new KafkaException(s"Unsupported version: $version of feature information: " +
+            s"${new String(jsonBytes, UTF_8)}")
+        }
+
+        val featuresMap = featureInfo
+          .get(FeaturesKey)
+          .flatMap(_.to[Option[Map[String, Map[String, Long]]]])
+        if (featuresMap.isEmpty) {
+          throw new KafkaException("Features map can not be absent in: " +
+            s"${new String(jsonBytes, UTF_8)}")
+        }
+        val features = asJavaMap(featuresMap.get)
+
+        val statusInt = featureInfo
+          .get(StatusKey)
+          .flatMap(_.to[Option[Int]])
+        if (statusInt.isEmpty) {
+          throw new KafkaException("Status can not be absent in feature information: " +

Review comment:
       Done. Changed to `IllegalArgumentException`. Good point!

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -744,6 +782,90 @@ object DelegationTokenInfoZNode {
   def decode(bytes: Array[Byte]): Option[TokenInformation] = DelegationTokenManager.fromBytes(bytes)
 }
 
+object FeatureZNodeStatus extends Enumeration {
+  val Disabled, Enabled = Value
+
+  def withNameOpt(value: Int): Option[Value] = {
+    values.find(_.id == value)
+  }
+}
+
+case class FeatureZNode(status: FeatureZNodeStatus.Value, features: Features[FinalizedVersionRange]) {
+}
+
+object FeatureZNode {

Review comment:
       As far as I can see, no ZK node class defined in this file is defined in such a way. Every class in this file encodes/decodes JSON by itself, and manages its own attributes.
   Should we break the norm?

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -146,7 +162,14 @@ object BrokerIdZNode {
     val plaintextEndpoint = broker.endPoints.find(_.securityProtocol == SecurityProtocol.PLAINTEXT).getOrElse(
       new EndPoint(null, -1, null, null))
     encode(brokerInfo.version, plaintextEndpoint.host, plaintextEndpoint.port, broker.endPoints, brokerInfo.jmxPort,
-      broker.rack)
+      broker.rack, broker.features)
+  }
+
+  def asJavaMap(brokerInfo: JsonObject): util.Map[String, util.Map[String, java.lang.Long]] = {

Review comment:
       Done.

##########
File path: core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala
##########
@@ -811,8 +828,16 @@ class KafkaZkClientTest extends ZooKeeperTestHarness {
     assertEquals(Seq.empty, zkClient.getSortedBrokerList)
     assertEquals(None, zkClient.getBroker(0))
 
-    val brokerInfo0 = createBrokerInfo(0, "test.host0", 9998, SecurityProtocol.PLAINTEXT)
-    val brokerInfo1 = createBrokerInfo(1, "test.host1", 9999, SecurityProtocol.SSL)
+    val brokerInfo0 = createBrokerInfo(

Review comment:
       See L848 below where it is validated. The call to `zkClient. getAllBrokersInCluster` decodes each `BrokerIdZNode` content from JSON to `BrokerInfo` object. Then, we check whether the call returns exactly the same `BrokerInfo` objects defined here, and, along the way features are checked too.

##########
File path: core/src/main/scala/kafka/server/SupportedFeatures.scala
##########
@@ -0,0 +1,74 @@
+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 common 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.
+ */
+object SupportedFeatures extends Logging {
+
+  /**
+   * This is the latest features supported by the Broker.
+   * This is currently empty, but in the future as we define supported features, this map should be
+   * populated.
+   */
+  @volatile private var supportedFeatures = emptySupportedFeatures
+
+  /**
+   * Returns the latest features supported by the Broker.

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/KafkaConfig.scala
##########
@@ -812,6 +817,8 @@ object KafkaConfig {
   val ControlledShutdownMaxRetriesDoc = "Controlled shutdown can fail for multiple reasons. This determines the number of retries when such failure happens"
   val ControlledShutdownRetryBackoffMsDoc = "Before each retry, the system needs time to recover from the state that caused the previous failure (Controller fail over, replica lag etc). This config determines the amount of time to wait before retrying."
   val ControlledShutdownEnableDoc = "Enable controlled shutdown of the server"
+  /** ********* Feature configuration ***********/
+  val FeatureChangeListenerCacheUpdateWaitTimeMsDoc = "# of milli seconds to wait for feature cache to be updated once."

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,200 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             - RuntimeException if the thread was interrupted during wait
+     *                     - TimeoutException if the wait can not be completed in waitTimeMs
+     *                       milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    private def processNotification(): Unit = {
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")
+      processNotification()
+    }
+
+    override def handleDataChange(): Unit = {
+      info(s"Feature ZK node updated at path: $path")
+      processNotification()
+    }
+
+    override def handleDeletion(): Unit = {
+      warn(s"Feature ZK node deleted at path: $path")
+      processNotification()

Review comment:
       You bring up a good point.
   
   My main concern is availability. If we exit the Broker here, then, whenever the feature ZK node gets deleted (accidentally), it could crash all brokers in the fleet all at once leading to an availability problem.
   
   With regards to violating feature semantics, good point.
   
   I'm in 2 minds here, and perhaps we can also hear @hachikuji 's thoughts on this topic.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,203 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             - RuntimeException if the thread was interrupted during wait
+     *                     - TimeoutException if the wait can not be completed in waitTimeMs
+     *                       milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    private def processNotification(): Unit = {
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")

Review comment:
       I didn't understand the question. Are you saying the logging severity should be lower or higher?
   This is a rare case anyway as the feature node doesn't get created often, so, `info` logging seems fine to me.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,203 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {

Review comment:
       Done. Yes, I feel JSON deserialization should be treated as fatal. It should never happen, and, can indicate corruption.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,203 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             - RuntimeException if the thread was interrupted during wait
+     *                     - TimeoutException if the wait can not be completed in waitTimeMs
+     *                       milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    private def processNotification(): Unit = {
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")
+      processNotification()
+    }
+
+    override def handleDataChange(): Unit = {
+      info(s"Feature ZK node updated at path: $path")
+      processNotification()
+    }
+
+    override def handleDeletion(): Unit = {
+      warn(s"Feature ZK node deleted at path: $path")
+      // This event may happen, rarely (ex: operational error).
+      // In such a case, we prefer to just log a warning and treat the case as if the node is absent,
+      // and populate the FinalizedFeatureCache with empty finalized features.
+      processNotification()
+    }
+  }
+
+  private val queue = new LinkedBlockingQueue[FeatureCacheUpdater]
+
+  private val thread = new ChangeNotificationProcessorThread("feature-zk-node-event-process-thread")
+
+  /**
+   * This method initializes the feature ZK node change listener. Optionally, it also ensures to
+   * update the FinalizedFeatureCache once with the latest contents of the feature ZK node
+   * (if the node exists). This step helps ensure that feature incompatibilities (if any) in brokers
+   * are conveniently detected before the initOrThrow() method returns to the caller. If feature
+   * incompatibilities are detected, this method will throw an Exception to the caller, and the Broker
+   * would exit eventually.
+   *
+   * @param waitOnceForCacheUpdateMs   # of milli seconds to wait for feature cache to be updated once.
+   *                                   If this parameter <= 0, no wait operation happens.
+   *
+   * @throws Exception if feature incompatibility check could not be finished in a timely manner
+   */
+  def initOrThrow(waitOnceForCacheUpdateMs: Long): Unit = {
+    thread.start()
+    zkClient.registerZNodeChangeHandlerAndCheckExistence(FeatureZNodeChangeHandler)

Review comment:
       Yes. Here is the documentation explaining the same: https://zookeeper.apache.org/doc/r3.5.7/zookeeperProgrammers.html#Java+Binding.
   
   > When a ZooKeeper object is created, two threads are created as well: an IO thread and an event thread. All IO happens on the IO thread (using Java NIO). All event callbacks happen on the event thread. Session maintenance such as reconnecting to ZooKeeper servers and maintaining heartbeat is done on the IO thread. Responses for synchronous methods are also processed in the IO thread. All responses to asynchronous methods and watch events are processed on the event thread.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,203 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             - RuntimeException if the thread was interrupted during wait
+     *                     - TimeoutException if the wait can not be completed in waitTimeMs
+     *                       milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Interrupted", e)

Review comment:
       Done. But it's actually "Change notification queue interrupted".

##########
File path: core/src/main/scala/kafka/server/SupportedFeatures.scala
##########
@@ -0,0 +1,74 @@
+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 common 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.
+ */
+object SupportedFeatures extends Logging {
+
+  /**
+   * This is the latest features supported by the Broker.
+   * This is currently empty, but in the future as we define supported features, this map should be
+   * populated.
+   */
+  @volatile private var supportedFeatures = emptySupportedFeatures
+
+  /**
+   * Returns the latest features supported by the Broker.
+   */
+  def get: Features[SupportedVersionRange] = {
+    supportedFeatures
+  }
+
+  // Should be used only for testing.
+  def update(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // Should be used only for testing.
+  def clear(): Unit = {
+    supportedFeatures = emptySupportedFeatures
+  }
+
+  /**
+   * 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 cluster-wide finalized feature. This can happen because a provided
+   * cluster-wide 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 set of incompatible feature names. If the returned set is empty, it
+   *                    means there were no feature incompatibilities found.
+   */
+  def incompatibleFeatures(finalized: Features[FinalizedVersionRange]): Set[String] = {

Review comment:
       Done. Good point!

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,203 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             - RuntimeException if the thread was interrupted during wait
+     *                     - TimeoutException if the wait can not be completed in waitTimeMs
+     *                       milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    private def processNotification(): Unit = {
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")
+      processNotification()
+    }
+
+    override def handleDataChange(): Unit = {
+      info(s"Feature ZK node updated at path: $path")
+      processNotification()
+    }
+
+    override def handleDeletion(): Unit = {
+      warn(s"Feature ZK node deleted at path: $path")
+      // This event may happen, rarely (ex: operational error).
+      // In such a case, we prefer to just log a warning and treat the case as if the node is absent,
+      // and populate the FinalizedFeatureCache with empty finalized features.
+      processNotification()
+    }
+  }
+
+  private val queue = new LinkedBlockingQueue[FeatureCacheUpdater]
+
+  private val thread = new ChangeNotificationProcessorThread("feature-zk-node-event-process-thread")
+
+  /**
+   * This method initializes the feature ZK node change listener. Optionally, it also ensures to
+   * update the FinalizedFeatureCache once with the latest contents of the feature ZK node
+   * (if the node exists). This step helps ensure that feature incompatibilities (if any) in brokers
+   * are conveniently detected before the initOrThrow() method returns to the caller. If feature
+   * incompatibilities are detected, this method will throw an Exception to the caller, and the Broker
+   * would exit eventually.
+   *
+   * @param waitOnceForCacheUpdateMs   # of milli seconds to wait for feature cache to be updated once.
+   *                                   If this parameter <= 0, no wait operation happens.
+   *
+   * @throws Exception if feature incompatibility check could not be finished in a timely manner
+   */
+  def initOrThrow(waitOnceForCacheUpdateMs: Long): Unit = {
+    thread.start()
+    zkClient.registerZNodeChangeHandlerAndCheckExistence(FeatureZNodeChangeHandler)
+
+    if (waitOnceForCacheUpdateMs > 0) {
+      val barrier = new CountDownLatch(1)

Review comment:
       Done. Removed.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,203 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             - RuntimeException if the thread was interrupted during wait
+     *                     - TimeoutException if the wait can not be completed in waitTimeMs
+     *                       milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()

Review comment:
       The function blocks indefinitely - yes. But this shouldn't cause a problem or lead to deadlock/limbo situation.
   Even if this thread is waiting for an item to become available in the queue, the waiting thread can always get interrupted by the `FinalizedFeatureChangeListener.close()` call which calls `ShutdownableThread.shutdown()`.
   
   Note that the `ShutdownableThread.shutdown()` method interrupts the thread, which should unblock any waiting `queue.take()` operation and makes it raise an `InterruptedException`: 
   
   https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/utils/ShutdownableThread.scala#L32-L59
   https://docs.oracle.com/javase/7/docs/api/java/util/concurrent/LinkedBlockingQueue.html#take()

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,203 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             - RuntimeException if the thread was interrupted during wait
+     *                     - TimeoutException if the wait can not be completed in waitTimeMs
+     *                       milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    private def processNotification(): Unit = {
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")
+      processNotification()
+    }
+
+    override def handleDataChange(): Unit = {
+      info(s"Feature ZK node updated at path: $path")
+      processNotification()
+    }
+
+    override def handleDeletion(): Unit = {
+      warn(s"Feature ZK node deleted at path: $path")
+      // This event may happen, rarely (ex: operational error).
+      // In such a case, we prefer to just log a warning and treat the case as if the node is absent,
+      // and populate the FinalizedFeatureCache with empty finalized features.
+      processNotification()
+    }
+  }
+
+  private val queue = new LinkedBlockingQueue[FeatureCacheUpdater]
+
+  private val thread = new ChangeNotificationProcessorThread("feature-zk-node-event-process-thread")
+
+  /**
+   * This method initializes the feature ZK node change listener. Optionally, it also ensures to
+   * update the FinalizedFeatureCache once with the latest contents of the feature ZK node
+   * (if the node exists). This step helps ensure that feature incompatibilities (if any) in brokers
+   * are conveniently detected before the initOrThrow() method returns to the caller. If feature
+   * incompatibilities are detected, this method will throw an Exception to the caller, and the Broker
+   * would exit eventually.
+   *
+   * @param waitOnceForCacheUpdateMs   # of milli seconds to wait for feature cache to be updated once.
+   *                                   If this parameter <= 0, no wait operation happens.
+   *
+   * @throws Exception if feature incompatibility check could not be finished in a timely manner
+   */
+  def initOrThrow(waitOnceForCacheUpdateMs: Long): Unit = {
+    thread.start()
+    zkClient.registerZNodeChangeHandlerAndCheckExistence(FeatureZNodeChangeHandler)
+
+    if (waitOnceForCacheUpdateMs > 0) {
+      val barrier = new CountDownLatch(1)
+      val ensureCacheUpdateOnce = new FeatureCacheUpdater(FeatureZNodeChangeHandler.path, Some(barrier))
+      queue.add(ensureCacheUpdateOnce)
+      try {
+        ensureCacheUpdateOnce.awaitUpdateOrThrow(waitOnceForCacheUpdateMs)
+      } catch {
+        case e: Exception => {
+          close()
+          throw e
+        }
+      }
+    }
+  }
+
+  /**
+   * Closes the feature ZK node change listener by unregistering the listener from ZK client,
+   * clearing the queue and shutting down the ChangeNotificationProcessorThread.
+   */
+  def close(): Unit = {
+    zkClient.unregisterZNodeChangeHandler(FeatureZNodeChangeHandler.path)
+    queue.clear()
+    thread.shutdown()
+    thread.join()
+  }
+
+  // Useful for testing.

Review comment:
       Done.

##########
File path: clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java
##########
@@ -17,13 +17,19 @@
 
 package org.apache.kafka.common.requests;
 
+import org.apache.kafka.common.feature.Features;

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/KafkaServer.scala
##########
@@ -210,6 +215,14 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP
         /* setup zookeeper */
         initZkClient(time)
 
+        /* initialize features */
+        _featureChangeListener = new FinalizedFeatureChangeListener(_zkClient)
+        if (config.interBrokerProtocolVersion >= KAFKA_2_6_IV1) {
+          // The feature versioning system (KIP-584) is active only when:

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 #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -81,17 +83,26 @@ object BrokerIdsZNode {
 object BrokerInfo {
 
   /**
-   * Create a broker info with v4 json format (which includes multiple endpoints and rack) if
-   * the apiVersion is 0.10.0.X or above. Register the broker with v2 json format otherwise.
+   * - Create a broker info with v5 json format if the apiVersion is 2.6.x or above.

Review comment:
       2.6.x => 2.7.x

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,243 @@
+/**
+ * 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.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.{StateChangeHandler, ZNodeChangeHandler}
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+import scala.util.control.Exception.ignoring
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked exactly
+     * once successfully. A subsequent invocation will raise an exception.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     * @throws   FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     * @throws   RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      debug(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        featureZNode.status match {
+          case FeatureZNodeStatus.Disabled => {
+            info(s"Feature ZK node at path: $featureZkNodePath is in disabled status.")
+            FinalizedFeatureCache.clear()
+          }
+          case FeatureZNodeStatus.Enabled => {
+            FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+          }
+          case _ => throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully. This method returns
+     * immediately if an updateLatestOrThrow call had already completed successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             TimeoutException if the wait can not be completed in waitTimeMs
+     *                     milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (!notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        ignoring(classOf[InterruptedException]) {
+          queue.take.updateLatestOrThrow()
+        }
+      } catch {
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will eventually exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDataChange(): Unit = {
+      info(s"Feature ZK node updated at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDeletion(): Unit = {
+      warn(s"Feature ZK node deleted at path: $path")
+      // This event may happen, rarely (ex: ZK corruption or operational error).
+      // In such a case, we prefer to just log a warning and treat the case as if the node is absent,
+      // and populate the FinalizedFeatureCache with empty finalized features.
+      queue.add(new FeatureCacheUpdater(path))
+    }
+  }
+
+  object ZkStateChangeHandler extends StateChangeHandler {
+    val path: String = FeatureZNode.path
+
+    override val name: String = s"change-notification-$path"

Review comment:
       This is not really "change-notification". So, the name can just be FeatureZNode.path.

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -81,17 +83,26 @@ object BrokerIdsZNode {
 object BrokerInfo {
 
   /**
-   * Create a broker info with v4 json format (which includes multiple endpoints and rack) if
-   * the apiVersion is 0.10.0.X or above. Register the broker with v2 json format otherwise.
+   * - Create a broker info with v5 json format if the apiVersion is 2.6.x or above.
+   * - Create a broker info with v4 json format (which includes multiple endpoints and rack) if
+   *   the apiVersion is 0.10.0.X or above but lesser than 2.6.x.

Review comment:
       2.6.x => 2.7.x




----------------------------------------------------------------
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 #8680: KAFKA-9755: Implement read path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ */
+public class Features<VersionRangeType extends VersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to VersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "supported" features.
+     */
+    public static Features<VersionRange> supportedFeatures(Map<String, VersionRange> features) {
+        return new Features<VersionRange>(features);
+    }
+
+    /**
+     * @param features   Map of feature name to VersionLevelRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "finalized" features.
+     */
+    public static Features<VersionLevelRange> finalizedFeatures(Map<String, VersionLevelRange> features) {
+        return new Features<VersionLevelRange>(features);
+    }
+
+    public static Features<VersionLevelRange> emptyFinalizedFeatures() {

Review comment:
       Is this function only used in unit test?

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ */
+public class Features<VersionRangeType extends VersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to VersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "supported" features.
+     */
+    public static Features<VersionRange> supportedFeatures(Map<String, VersionRange> features) {
+        return new Features<VersionRange>(features);

Review comment:
       Could be simplified as new Features<>

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.

Review comment:
       nit: we could use {@link VersionRangeType} to reference to the classes.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ */
+public class Features<VersionRangeType extends VersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to VersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "supported" features.
+     */
+    public static Features<VersionRange> supportedFeatures(Map<String, VersionRange> features) {
+        return new Features<VersionRange>(features);
+    }
+
+    /**
+     * @param features   Map of feature name to VersionLevelRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "finalized" features.
+     */
+    public static Features<VersionLevelRange> finalizedFeatures(Map<String, VersionLevelRange> features) {
+        return new Features<VersionLevelRange>(features);
+    }
+
+    public static Features<VersionLevelRange> emptyFinalizedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public static Features<VersionRange> emptySupportedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+
+    public Map<String, VersionRangeType> all() {
+        return features;
+    }
+
+    public boolean empty() {
+        return features.isEmpty();
+    }
+
+    public VersionRangeType get(String feature) {
+        return all().get(feature);

Review comment:
       nit: just a personal preference, but getting one less internal reference to a public function `all` makes the code usage check easier, like `features.get(feature)`. 

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ */
+public class Features<VersionRangeType extends VersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;

Review comment:
       We should ensure `features` is not null

##########
File path: clients/src/test/java/org/apache/kafka/common/feature/FeaturesTest.java
##########
@@ -0,0 +1,216 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThrows;
+
+public class FeaturesTest {
+
+    @Test
+    public void testEmptyFeatures() {
+        Map<String, Map<String, Long>> emptyMap = new HashMap<>();
+
+        Features<VersionLevelRange> emptyFinalizedFeatures = Features.emptyFinalizedFeatures();
+        assertEquals(new HashMap<>(), emptyFinalizedFeatures.all());
+        assertEquals(emptyMap, emptyFinalizedFeatures.serialize());
+        assertEquals(emptyFinalizedFeatures, Features.deserializeFinalizedFeatures(emptyMap));
+
+        Features<VersionRange> emptySupportedFeatures = Features.emptySupportedFeatures();
+        assertEquals(new HashMap<>(), emptySupportedFeatures.all());
+        assertEquals(
+            new HashMap<String, HashMap<String, Long>>(),
+            emptySupportedFeatures.serialize());
+        assertEquals(emptySupportedFeatures, Features.deserializeSupportedFeatures(emptyMap));
+    }
+
+    @Test
+    public void testAllAPI() {

Review comment:
       s/AllAPI/GetAllFeatures

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ */
+public class Features<VersionRangeType extends VersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to VersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "supported" features.
+     */
+    public static Features<VersionRange> supportedFeatures(Map<String, VersionRange> features) {
+        return new Features<VersionRange>(features);
+    }
+
+    /**
+     * @param features   Map of feature name to VersionLevelRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "finalized" features.
+     */
+    public static Features<VersionLevelRange> finalizedFeatures(Map<String, VersionLevelRange> features) {
+        return new Features<VersionLevelRange>(features);
+    }
+
+    public static Features<VersionLevelRange> emptyFinalizedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public static Features<VersionRange> emptySupportedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+

Review comment:
       nit: extra line

##########
File path: core/src/main/scala/kafka/server/SupportedFeatures.scala
##########
@@ -0,0 +1,70 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionRange, VersionLevelRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A common 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.
+ */
+object SupportedFeatures extends Logging {

Review comment:
       nit: add a line

##########
File path: clients/src/test/java/org/apache/kafka/common/feature/VersionLevelRangeTest.java
##########
@@ -0,0 +1,162 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class VersionLevelRangeTest {
+
+    @Test
+    public void testCreateFailDueToInvalidParams() {
+        // min and max can't be < 1.
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionLevelRange(0, 0));
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionLevelRange(-1, -1));
+        // min can't be < 1.
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionLevelRange(0, 1));
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionLevelRange(-1, 1));
+        // max can't be < 1.
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionLevelRange(1, 0));
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionLevelRange(1, -1));
+        // min can't be > max.
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionLevelRange(2, 1));
+    }
+
+    @Test
+    public void testSerializeDeserialize() {
+        VersionLevelRange versionLevelRange = new VersionLevelRange(1, 2);
+        assertEquals(1, versionLevelRange.min());
+        assertEquals(2, versionLevelRange.max());
+
+        Map<String, Long> serialized = versionLevelRange.serialize();
+        assertEquals(
+            new HashMap<String, Long>() {
+                {
+                    put("min_version_level", versionLevelRange.min());
+                    put("max_version_level", versionLevelRange.max());
+                }
+            },
+            serialized
+        );
+
+        VersionLevelRange deserialized = VersionLevelRange.deserialize(serialized);
+        assertEquals(1, deserialized.min());
+        assertEquals(2, deserialized.max());
+        assertEquals(versionLevelRange, deserialized);
+    }
+
+    @Test
+    public void testDeserializationFailureTest() {
+        // min_version_level can't be < 1.
+        Map<String, Long> invalidWithBadMinVersion = new HashMap<String, Long>() {
+            {
+                put("min_version_level", 0L);
+                put("max_version_level", 1L);
+            }
+        };
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> VersionLevelRange.deserialize(invalidWithBadMinVersion));
+
+        // max_version_level can't be < 1.
+        Map<String, Long> invalidWithBadMaxVersion = new HashMap<String, Long>() {
+            {
+                put("min_version_level", 1L);
+                put("max_version_level", 0L);
+            }
+        };
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> VersionLevelRange.deserialize(invalidWithBadMaxVersion));
+
+        // min_version_level and max_version_level can't be < 1.
+        Map<String, Long> invalidWithBadMinMaxVersion = new HashMap<String, Long>() {
+            {
+                put("min_version_level", 0L);
+                put("max_version_level", 0L);
+            }
+        };
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> VersionLevelRange.deserialize(invalidWithBadMinMaxVersion));
+
+        // min_version_level can't be > max_version_level.
+        Map<String, Long> invalidWithLowerMaxVersion = new HashMap<String, Long>() {
+            {
+                put("min_version_level", 2L);
+                put("max_version_level", 1L);
+            }
+        };
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> VersionLevelRange.deserialize(invalidWithLowerMaxVersion));
+
+        // min_version_level key missing.
+        Map<String, Long> invalidWithMinKeyMissing = new HashMap<String, Long>() {
+            {
+                put("max_version_level", 1L);
+            }
+        };
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> VersionLevelRange.deserialize(invalidWithMinKeyMissing));
+
+        // max_version_level key missing.
+        Map<String, Long> invalidWithMaxKeyMissing = new HashMap<String, Long>() {
+            {
+                put("min_version_level", 1L);
+            }
+        };
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> VersionLevelRange.deserialize(invalidWithMaxKeyMissing));
+    }
+
+    @Test
+    public void testToString() {
+        assertEquals("VersionLevelRange[1, 1]", new VersionLevelRange(1, 1).toString());
+        assertEquals("VersionLevelRange[1, 2]", new VersionLevelRange(1, 2).toString());
+    }
+
+    @Test
+    public void testEquals() {
+        assertTrue(new VersionLevelRange(1, 1).equals(new VersionLevelRange(1, 1)));
+        assertFalse(new VersionLevelRange(1, 1).equals(new VersionLevelRange(1, 2)));
+    }
+
+    @Test
+    public void testIsCompatibleWith() {
+        assertTrue(new VersionLevelRange(1, 1).isCompatibleWith(new VersionRange(1, 1)));
+        assertTrue(new VersionLevelRange(2, 3).isCompatibleWith(new VersionRange(1, 4)));
+        assertTrue(new VersionLevelRange(1, 4).isCompatibleWith(new VersionRange(1, 4)));
+
+        assertFalse(new VersionLevelRange(1, 4).isCompatibleWith(new VersionRange(2, 3)));
+        assertFalse(new VersionLevelRange(1, 4).isCompatibleWith(new VersionRange(2, 4)));
+        assertFalse(new VersionLevelRange(2, 4).isCompatibleWith(new VersionRange(2, 3)));
+    }
+
+    @Test
+    public void testGetters() {
+        assertEquals(1, new VersionLevelRange(1, 2).min());
+        assertEquals(2, new VersionLevelRange(1, 2).max());
+    }
+}

Review comment:
       nit: new line

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ */
+public class Features<VersionRangeType extends VersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to VersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "supported" features.
+     */
+    public static Features<VersionRange> supportedFeatures(Map<String, VersionRange> features) {
+        return new Features<VersionRange>(features);
+    }
+
+    /**
+     * @param features   Map of feature name to VersionLevelRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "finalized" features.
+     */
+    public static Features<VersionLevelRange> finalizedFeatures(Map<String, VersionLevelRange> features) {
+        return new Features<VersionLevelRange>(features);
+    }
+
+    public static Features<VersionLevelRange> emptyFinalizedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public static Features<VersionRange> emptySupportedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+
+    public Map<String, VersionRangeType> all() {
+        return features;
+    }
+
+    public boolean empty() {
+        return features.isEmpty();
+    }
+
+    public VersionRangeType get(String feature) {
+        return all().get(feature);
+    }
+
+    public String toString() {
+        return String.format(
+            "Features{%s}",
+            features
+                .entrySet()
+                .stream()
+                .map(entry -> String.format("(%s -> %s)", entry.getKey(), entry.getValue()))
+                .collect(joining(", "))
+        );
+    }
+
+    /**
+     * @return   Serializes the underlying features to a map, and returns the same.
+     *           The returned value can be deserialized using one of the deserialize* APIs.
+     */
+    public Map<String, Map<String, Long>> serialize() {
+        return features.entrySet().stream().collect(
+            Collectors.toMap(
+                Map.Entry::getKey,
+                entry -> entry.getValue().serialize()));
+    }
+
+    /**
+     * Deserializes a map to Features<VersionLevelRange>.
+     *
+     * @param serialized   the serialized representation of a Features<VersionLevelRange> object,
+     *                     generated using the serialize() API.
+     *
+     * @return             the deserialized Features<VersionLevelRange> object
+     */
+    public static Features<VersionLevelRange> deserializeFinalizedFeatures(
+        Map<String, Map<String, Long>> serialized) {
+        return finalizedFeatures(serialized.entrySet().stream().collect(
+            Collectors.toMap(
+                Map.Entry::getKey,
+                entry -> VersionLevelRange.deserialize(entry.getValue()))));
+    }
+
+    /**
+     * Deserializes a map to Features<VersionRange>.
+     *
+     * @param serialized   the serialized representation of a Features<VersionRange> object,
+     *                     generated using the serialize() API.
+     *
+     * @return             the deserialized Features<VersionRange> object
+     */
+    public static Features<VersionRange> deserializeSupportedFeatures(
+        Map<String, Map<String, Long>> serialized) {
+        return supportedFeatures(serialized.entrySet().stream().collect(
+            Collectors.toMap(
+                Map.Entry::getKey,
+                entry -> VersionRange.deserialize(entry.getValue()))));
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (this == other) {

Review comment:
       We should check `null` for other.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ */
+public class Features<VersionRangeType extends VersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to VersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "supported" features.
+     */
+    public static Features<VersionRange> supportedFeatures(Map<String, VersionRange> features) {
+        return new Features<VersionRange>(features);
+    }
+
+    /**
+     * @param features   Map of feature name to VersionLevelRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "finalized" features.
+     */
+    public static Features<VersionLevelRange> finalizedFeatures(Map<String, VersionLevelRange> features) {
+        return new Features<VersionLevelRange>(features);
+    }
+
+    public static Features<VersionLevelRange> emptyFinalizedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public static Features<VersionRange> emptySupportedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+
+    public Map<String, VersionRangeType> all() {
+        return features;
+    }
+
+    public boolean empty() {
+        return features.isEmpty();
+    }
+
+    public VersionRangeType get(String feature) {
+        return all().get(feature);

Review comment:
       Also if we could potentially have a not-found feature, we should either fail with illegal state, or make the return type `Optional<VersionRangeType>`

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ */
+public class Features<VersionRangeType extends VersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to VersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "supported" features.
+     */
+    public static Features<VersionRange> supportedFeatures(Map<String, VersionRange> features) {
+        return new Features<VersionRange>(features);
+    }
+
+    /**
+     * @param features   Map of feature name to VersionLevelRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "finalized" features.
+     */
+    public static Features<VersionLevelRange> finalizedFeatures(Map<String, VersionLevelRange> features) {
+        return new Features<VersionLevelRange>(features);
+    }
+
+    public static Features<VersionLevelRange> emptyFinalizedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public static Features<VersionRange> emptySupportedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+
+    public Map<String, VersionRangeType> all() {
+        return features;
+    }
+
+    public boolean empty() {
+        return features.isEmpty();
+    }
+
+    public VersionRangeType get(String feature) {
+        return all().get(feature);
+    }
+
+    public String toString() {
+        return String.format(
+            "Features{%s}",
+            features
+                .entrySet()
+                .stream()
+                .map(entry -> String.format("(%s -> %s)", entry.getKey(), entry.getValue()))
+                .collect(joining(", "))
+        );
+    }
+
+    /**
+     * @return   Serializes the underlying features to a map, and returns the same.
+     *           The returned value can be deserialized using one of the deserialize* APIs.
+     */
+    public Map<String, Map<String, Long>> serialize() {
+        return features.entrySet().stream().collect(
+            Collectors.toMap(
+                Map.Entry::getKey,
+                entry -> entry.getValue().serialize()));
+    }
+
+    /**
+     * Deserializes a map to Features<VersionLevelRange>.

Review comment:
       s/Deserializes/Deserialize

##########
File path: core/src/main/scala/kafka/cluster/Broker.scala
##########
@@ -34,14 +36,19 @@ object Broker {
                                          brokerId: Int,
                                          endpoints: util.List[Endpoint],
                                          interBrokerEndpoint: Endpoint) extends AuthorizerServerInfo
+
+  def apply(id: Int, endPoints: Seq[EndPoint], rack: Option[String]): Broker = {

Review comment:
       This is because the write path has not been implemented?

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -113,14 +141,26 @@ public static ApiVersionsResponse fromStruct(Struct struct, short version) {
         }
     }
 
-    public static ApiVersionsResponse apiVersionsResponse(int throttleTimeMs, byte maxMagic) {
+    public static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<VersionRange> latestSupportedFeatures,
+        Optional<Features<VersionLevelRange>> finalizedFeatures,
+        Optional<Long> finalizedFeaturesEpoch) {
         if (maxMagic == RecordBatch.CURRENT_MAGIC_VALUE && throttleTimeMs == DEFAULT_THROTTLE_TIME) {
             return DEFAULT_API_VERSIONS_RESPONSE;
         }
-        return createApiVersionsResponse(throttleTimeMs, maxMagic);
+        return createApiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, finalizedFeatures, finalizedFeaturesEpoch);
     }
 
-    public static ApiVersionsResponse createApiVersionsResponse(int throttleTimeMs, final byte minMagic) {
+    public static ApiVersionsResponse createApiVersionsResponse(

Review comment:
       Looks like we have some gaps for unit testing `ApiVersionsResponse`. Could we add unit tests for this class, since the logic `createApiVersionsResponse` becomes non-trivial now?

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ */
+public class Features<VersionRangeType extends VersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to VersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "supported" features.
+     */
+    public static Features<VersionRange> supportedFeatures(Map<String, VersionRange> features) {
+        return new Features<VersionRange>(features);
+    }
+
+    /**
+     * @param features   Map of feature name to VersionLevelRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "finalized" features.
+     */
+    public static Features<VersionLevelRange> finalizedFeatures(Map<String, VersionLevelRange> features) {
+        return new Features<VersionLevelRange>(features);

Review comment:
       Same here

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,93 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionLevelRange}
+
+// 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[VersionLevelRange], epoch: Int) {
+
+  def isValid(newEpoch: Int): Boolean = {
+    newEpoch >= epoch
+  }
+
+  override def toString(): String = {
+    "FinalizedFeaturesAndEpoch(features=%s, epoch=%s)".format(features, epoch)
+  }
+}
+
+/**
+ * A mutable cache containing the latest finalized features and epoch. This cache is populated by a
+ * FinalizedFeatureChangeListener.
+ *
+ * Currently the main reader of this cache is the read path that serves an ApiVersionsRequest
+ * returning the features information in the response. In the future, as the feature versioning
+ * system in KIP-584 is used more widely, this cache could be read by other read paths trying to
+ * learn the finalized feature information.
+ */
+object FinalizedFeatureCache extends Logging {
+  @volatile private var featuresAndEpoch: Option[FinalizedFeaturesAndEpoch] = Option.empty
+
+  /**
+   * @return   the latest known FinalizedFeaturesAndEpoch. If the returned value is empty, it means
+   *           no FinalizedFeaturesAndEpoch exists in the cache at the time when this
+   *           method is invoked. This result could change in the future whenever the
+   *           updateOrThrow method is invoked.
+   */
+  def get: Option[FinalizedFeaturesAndEpoch] = {
+    featuresAndEpoch
+  }
+
+  def empty: Boolean = {
+    featuresAndEpoch.isEmpty
+  }
+
+  /**
+   * Clears all existing finalized features and epoch from the cache.
+   */
+  def clear(): Unit = {
+    featuresAndEpoch = Option.empty
+    info("Cleared cache")
+  }
+
+  /**
+   * Updates the cache to the latestFeatures, and updates the existing epoch to latestEpoch.
+   * Raises an exception when the operation is not successful.
+   *
+   * @param latestFeatures   the latest finalized features to be set in the cache
+   * @param latestEpoch      the latest epoch value to be set in the cache
+   *
+   * @throws                 FeatureCacheUpdateException if the cache update operation fails
+   *                         due to invalid parameters or incompatibilities with the broker's
+   *                         supported features. In such a case, the existing cache contents are
+   *                         not modified.
+   */
+  def updateOrThrow(latestFeatures: Features[VersionLevelRange], latestEpoch: Int): Unit = {
+    updateOrThrow(FinalizedFeaturesAndEpoch(latestFeatures, latestEpoch))
+  }
+
+  private def updateOrThrow(latest: FinalizedFeaturesAndEpoch): Unit = {
+    val existingStr = featuresAndEpoch.map(existing => existing.toString).getOrElse("<empty>")
+    if (!featuresAndEpoch.isEmpty && featuresAndEpoch.get.epoch > latest.epoch) {

Review comment:
       s/ !featuresAndEpoch.isEmpty / featuresAndEpoch.isDefined

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ */
+public class Features<VersionRangeType extends VersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to VersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "supported" features.
+     */
+    public static Features<VersionRange> supportedFeatures(Map<String, VersionRange> features) {
+        return new Features<VersionRange>(features);
+    }
+
+    /**
+     * @param features   Map of feature name to VersionLevelRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "finalized" features.
+     */
+    public static Features<VersionLevelRange> finalizedFeatures(Map<String, VersionLevelRange> features) {
+        return new Features<VersionLevelRange>(features);
+    }
+
+    public static Features<VersionLevelRange> emptyFinalizedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public static Features<VersionRange> emptySupportedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+
+    public Map<String, VersionRangeType> all() {
+        return features;
+    }
+
+    public boolean empty() {
+        return features.isEmpty();
+    }
+
+    public VersionRangeType get(String feature) {
+        return all().get(feature);
+    }
+
+    public String toString() {
+        return String.format(
+            "Features{%s}",
+            features
+                .entrySet()
+                .stream()
+                .map(entry -> String.format("(%s -> %s)", entry.getKey(), entry.getValue()))
+                .collect(joining(", "))
+        );
+    }
+
+    /**
+     * @return   Serializes the underlying features to a map, and returns the same.

Review comment:
       Maybe rephrase as `a map with the underlying features serialized`

##########
File path: clients/src/main/resources/common/message/ApiVersionsResponse.json
##########
@@ -42,6 +42,33 @@
         "about": "The maximum supported version, inclusive." }
     ]},
     { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true,
-      "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }
+      "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." },

Review comment:
       I think we need to bump the schema version to 4? Same with `ApiVersionsRequest.json`

##########
File path: clients/src/test/java/org/apache/kafka/common/feature/FeaturesTest.java
##########
@@ -0,0 +1,216 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThrows;
+
+public class FeaturesTest {
+
+    @Test
+    public void testEmptyFeatures() {
+        Map<String, Map<String, Long>> emptyMap = new HashMap<>();
+
+        Features<VersionLevelRange> emptyFinalizedFeatures = Features.emptyFinalizedFeatures();
+        assertEquals(new HashMap<>(), emptyFinalizedFeatures.all());
+        assertEquals(emptyMap, emptyFinalizedFeatures.serialize());
+        assertEquals(emptyFinalizedFeatures, Features.deserializeFinalizedFeatures(emptyMap));
+
+        Features<VersionRange> emptySupportedFeatures = Features.emptySupportedFeatures();
+        assertEquals(new HashMap<>(), emptySupportedFeatures.all());
+        assertEquals(
+            new HashMap<String, HashMap<String, Long>>(),
+            emptySupportedFeatures.serialize());
+        assertEquals(emptySupportedFeatures, Features.deserializeSupportedFeatures(emptyMap));
+    }
+
+    @Test
+    public void testAllAPI() {
+        VersionRange v1 = new VersionRange(1, 2);
+        VersionRange v2 = new VersionRange(3, 4);
+        Map<String, VersionRange> allFeatures = new HashMap<String, VersionRange>() {
+            {
+                put("feature_1", v1);
+                put("feature_2", v2);
+            }
+        };
+        Features<VersionRange> features = Features.supportedFeatures(allFeatures);
+        assertEquals(allFeatures, features.all());
+    }
+
+    @Test
+    public void testGetAPI() {
+        VersionRange v1 = new VersionRange(1, 2);
+        VersionRange v2 = new VersionRange(3, 4);
+        Map<String, VersionRange> allFeatures = new HashMap<String, VersionRange>() {
+            {
+                put("feature_1", v1);
+                put("feature_2", v2);
+            }
+        };
+        Features<VersionRange> features = Features.supportedFeatures(allFeatures);
+        assertEquals(v1, features.get("feature_1"));
+        assertEquals(v2, features.get("feature_2"));
+        assertNull(features.get("nonexistent_feature"));
+    }
+
+    @Test
+    public void testSerializeDeserializeSupportedFeatures() {
+        VersionRange v1 = new VersionRange(1, 2);
+        VersionRange v2 = new VersionRange(3, 4);
+        Map<String, VersionRange> allFeatures = new HashMap<String, VersionRange>() {

Review comment:
       Same here

##########
File path: clients/src/test/java/org/apache/kafka/common/feature/FeaturesTest.java
##########
@@ -0,0 +1,216 @@
+package org.apache.kafka.common.feature;

Review comment:
       We need the apache license title

##########
File path: core/src/main/scala/kafka/cluster/Broker.scala
##########
@@ -34,14 +36,19 @@ object Broker {
                                          brokerId: Int,
                                          endpoints: util.List[Endpoint],
                                          interBrokerEndpoint: Endpoint) extends AuthorizerServerInfo
+
+  def apply(id: Int, endPoints: Seq[EndPoint], rack: Option[String]): Broker = {
+    new Broker(id, endPoints, rack, emptySupportedFeatures)
+  }
 }
 
 /**
  * A Kafka broker.
- * A broker has an id, a collection of end-points, an optional rack and a listener to security protocol map.
+ * A broker has an id, a collection of end-points, an optional rack and a listener to security protocol map,

Review comment:
       nit: might make sense to build meta comment for parameters:
   ```
   /**
     * 
     * @param id
     * @param endPoints
     * @param rack
     * @param features
     */
   ```

##########
File path: clients/src/test/java/org/apache/kafka/common/feature/FeaturesTest.java
##########
@@ -0,0 +1,216 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThrows;
+
+public class FeaturesTest {
+
+    @Test
+    public void testEmptyFeatures() {
+        Map<String, Map<String, Long>> emptyMap = new HashMap<>();
+
+        Features<VersionLevelRange> emptyFinalizedFeatures = Features.emptyFinalizedFeatures();
+        assertEquals(new HashMap<>(), emptyFinalizedFeatures.all());
+        assertEquals(emptyMap, emptyFinalizedFeatures.serialize());
+        assertEquals(emptyFinalizedFeatures, Features.deserializeFinalizedFeatures(emptyMap));
+
+        Features<VersionRange> emptySupportedFeatures = Features.emptySupportedFeatures();
+        assertEquals(new HashMap<>(), emptySupportedFeatures.all());
+        assertEquals(
+            new HashMap<String, HashMap<String, Long>>(),
+            emptySupportedFeatures.serialize());
+        assertEquals(emptySupportedFeatures, Features.deserializeSupportedFeatures(emptyMap));
+    }
+
+    @Test
+    public void testAllAPI() {
+        VersionRange v1 = new VersionRange(1, 2);
+        VersionRange v2 = new VersionRange(3, 4);
+        Map<String, VersionRange> allFeatures = new HashMap<String, VersionRange>() {
+            {
+                put("feature_1", v1);
+                put("feature_2", v2);
+            }
+        };
+        Features<VersionRange> features = Features.supportedFeatures(allFeatures);
+        assertEquals(allFeatures, features.all());
+    }
+
+    @Test
+    public void testGetAPI() {
+        VersionRange v1 = new VersionRange(1, 2);
+        VersionRange v2 = new VersionRange(3, 4);
+        Map<String, VersionRange> allFeatures = new HashMap<String, VersionRange>() {

Review comment:
       We could use `org.apache.kafka.common.utils.Utils#mkMap` 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] hachikuji commented on pull request #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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


   ok to test


----------------------------------------------------------------
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 #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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


   The test failures seem unrelated. Merging this to trunk.


----------------------------------------------------------------
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 #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,232 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     *
+     *           FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     *
+     *           RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -37,6 +37,15 @@ class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
      *
      * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
      * only exactly once successfully.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     *
+     *           FeatureCacheUpdateException, if there was an error in updating the

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/SupportedFeatures.scala
##########
@@ -0,0 +1,76 @@
+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 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.
+ *
+ * NOTE: the update() and clear() APIs of this class should be used only for testing purposes.
+ */
+object SupportedFeatures extends Logging {
+
+  /**
+   * This is the latest features supported by the Broker.
+   * This is currently empty, but in the future as we define supported features, this map should be
+   * populated.
+   */
+  @volatile private var supportedFeatures = emptySupportedFeatures
+
+  /**
+   * Returns a reference to the latest features supported by the Broker.
+   */
+  def get: Features[SupportedVersionRange] = {
+    supportedFeatures
+  }
+
+  // Should be used only for testing.
+  def update(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // Should be used only for testing.
+  def clear(): Unit = {
+    supportedFeatures = emptySupportedFeatures
+  }
+
+  /**
+   * 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 cluster-wide finalized feature. This can happen because a provided
+   * cluster-wide 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 sub-set 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] = {
+    val incompatibilities = finalized.features.asScala.collect {
+      case (feature, versionLevels) => {
+        val supportedVersions = supportedFeatures.get(feature);

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Represents an immutable basic version range using 2 attributes: min and max of type long.
+ * The min and max attributes are expected to be >= 1, and with max >= min.
+ *
+ * The class also provides API to serialize/deserialize the version range to/from a map.
+ * The class allows for configurable labels for the min/max attributes, which can be specialized by
+ * sub-classes (if needed).
+ */
+class BaseVersionRange {
+    private final String minKeyLabel;
+
+    private final long minValue;
+
+    private final String maxKeyLabel;
+
+    private final long maxValue;
+
+    protected BaseVersionRange(String minKey, long minValue, String maxKeyLabel, long maxValue) {

Review comment:
       Done. Also added doc.

##########
File path: core/src/main/scala/kafka/server/KafkaConfig.scala
##########
@@ -442,6 +445,8 @@ object KafkaConfig {
   val ControlledShutdownMaxRetriesProp = "controlled.shutdown.max.retries"
   val ControlledShutdownRetryBackoffMsProp = "controlled.shutdown.retry.backoff.ms"
   val ControlledShutdownEnableProp = "controlled.shutdown.enable"
+  /** ********* Features configuration ***********/
+  val FeatureChangeListenerCacheUpdateWaitTimeMsProp = "feature.listener.cache.update.wait.ms"

Review comment:
       Sounds good. Yeah, it is minor and feels like an implementation detail to me. But we can wait to see what others say.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,86 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange}
+
+// 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) {
+  override def toString(): String = {
+    "FinalizedFeaturesAndEpoch(features=%s, epoch=%s)".format(features, epoch)
+  }
+}
+
+/**
+ * A mutable cache containing the latest finalized features and epoch. This cache is populated by a
+ * {@link FinalizedFeatureChangeListener}.
+ *
+ * Currently the main reader of this cache is the read path that serves an ApiVersionsRequest,
+ * returning the features information in the response.
+ */
+object FinalizedFeatureCache extends Logging {
+  @volatile private var featuresAndEpoch: Option[FinalizedFeaturesAndEpoch] = Option.empty
+
+  /**
+   * @return   the latest known FinalizedFeaturesAndEpoch. If the returned value is empty, it means
+   *           no FinalizedFeaturesAndEpoch exists in the cache at the time when this
+   *           method is invoked. This result could change in the future whenever the
+   *           updateOrThrow method is invoked.
+   */
+  def get: Option[FinalizedFeaturesAndEpoch] = {
+    featuresAndEpoch
+  }
+
+  def empty: Boolean = {
+    featuresAndEpoch.isEmpty
+  }
+
+  /**
+   * Clears all existing finalized features and epoch from the cache.
+   */
+  def clear(): Unit = {
+    featuresAndEpoch = Option.empty
+    info("Cleared cache")
+  }
+
+  /**
+   * Updates the cache to the latestFeatures, and updates the existing epoch to latestEpoch.
+   * Raises an exception when the operation is not successful.
+   *
+   * @param latestFeatures   the latest finalized features to be set in the cache
+   * @param latestEpoch      the latest epoch value to be set in the cache
+   *
+   * @throws                 FeatureCacheUpdateException if the cache update operation fails
+   *                         due to invalid parameters or incompatibilities with the broker's
+   *                         supported features. In such a case, the existing cache contents are
+   *                         not modified.
+   */
+  def updateOrThrow(latestFeatures: Features[FinalizedVersionRange], latestEpoch: Int): Unit = {
+    updateOrThrow(FinalizedFeaturesAndEpoch(latestFeatures, latestEpoch))
+  }
+
+  private def updateOrThrow(latest: FinalizedFeaturesAndEpoch): Unit = {

Review comment:
       Actually I've eliminated the helper method now, and, there is only 1 method: `updateOrThrow(...)`.

##########
File path: clients/src/test/java/org/apache/kafka/common/feature/FeaturesTest.java
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThrows;
+
+public class FeaturesTest {
+
+    @Test
+    public void testEmptyFeatures() {
+        Map<String, Map<String, Long>> emptyMap = new HashMap<>();
+
+        Features<FinalizedVersionRange> emptyFinalizedFeatures = Features.emptyFinalizedFeatures();
+        assertEquals(new HashMap<>(), emptyFinalizedFeatures.features());
+        assertEquals(emptyMap, emptyFinalizedFeatures.serialize());
+        assertEquals(emptyFinalizedFeatures, Features.deserializeFinalizedFeatures(emptyMap));
+
+        Features<SupportedVersionRange> emptySupportedFeatures = Features.emptySupportedFeatures();
+        assertEquals(new HashMap<>(), emptySupportedFeatures.features());

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,232 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     *
+     *           FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     *
+     *           RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      var mayBeFeatureZNodeBytes: Option[Array[Byte]] = null
+      var version: Int = ZkVersion.UnknownVersion
+      try {
+        val result = zkClient.getDataAndVersion(featureZkNodePath)
+        mayBeFeatureZNodeBytes = result._1
+        version = result._2
+      } catch {
+        // Convert to RuntimeException, to avoid a confusion that there is no argument passed
+        // to the updateOrThrow() method.
+        case e: IllegalArgumentException => throw new RuntimeException(e)
+      }
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,232 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -47,7 +56,30 @@ class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
       })
 
       info(s"Reading feature ZK node at path: $featureZkNodePath")
-      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      var mayBeFeatureZNodeBytes: Option[Array[Byte]] = null
+      var version: Int = ZkVersion.UnknownVersion
+      try {
+        val result = zkClient.getDataAndVersion(featureZkNodePath)
+        mayBeFeatureZNodeBytes = result._1
+        version = result._2
+      } catch {
+        // Convert to RuntimeException, to avoid a confusion that there is no argument passed
+        // to the updateOrThrow() method.
+        case e: IllegalArgumentException => throw new RuntimeException(e)

Review comment:
       Fixed now.
   Good point. Actually the code was incorrect. I meant to wrap `FeatureZNode.decode` call with the `try-catch`, since, it throws `IllegalArgumentException`. I have fixed the code now to do the same.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,232 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     *
+     *           FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     *
+     *           RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      var mayBeFeatureZNodeBytes: Option[Array[Byte]] = null
+      var version: Int = ZkVersion.UnknownVersion
+      try {
+        val result = zkClient.getDataAndVersion(featureZkNodePath)
+        mayBeFeatureZNodeBytes = result._1
+        version = result._2
+      } catch {
+        // Convert to RuntimeException, to avoid a confusion that there is no argument passed
+        // to the updateOrThrow() method.
+        case e: IllegalArgumentException => throw new RuntimeException(e)
+      }
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -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 org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being <VersionRangeType>.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ * @see SupportedVersionRange
+ * @see FinalizedVersionRange
+ */
+public class Features<VersionRangeType extends BaseVersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        if (features == null) {

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -744,6 +782,161 @@ object DelegationTokenInfoZNode {
   def decode(bytes: Array[Byte]): Option[TokenInformation] = DelegationTokenManager.fromBytes(bytes)
 }
 
+/**
+ * Represents the status of the FeatureZNode.
+ *
+ * Enabled  -> This status means the feature versioning system (KIP-584) is enabled, and, the
+ *             finalized features stored in the FeatureZNode are active. This status is written by
+ *             the controller to the FeatureZNode only when the broker IBP config is greater than
+ *             or equal to KAFKA_2_6_IV1.
+ *
+ * Disabled -> This status means the feature versioning system (KIP-584) is disabled, and, the
+ *             the finalized features stored in the FeatureZNode is not relevant. This status is
+ *             written by the controller to the FeatureZNode only when the broker IBP config
+ *             is less than KAFKA_2_6_IV1.
+ *
+ * The purpose behind the FeatureZNodeStatus is that it helps differentiates between the following
+ * cases:
+ *
+ * 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_6_IV1. In such a case, the
+ *    controller will start up and notice that the FeatureZNode is absent in the new cluster.
+ *    To handle the requirement, the controller will create a FeatureZNode (with enabled status)
+ *    containing the entire list of supported features as it’s finalized features.
+ *
+ * 2. Cluster upgrade:
+ *    Imagine that a Kafka cluster exists already and the IBP config is less than KAFKA_2_6_IV1, but

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -744,6 +782,161 @@ object DelegationTokenInfoZNode {
   def decode(bytes: Array[Byte]): Option[TokenInformation] = DelegationTokenManager.fromBytes(bytes)
 }
 
+/**
+ * Represents the status of the FeatureZNode.
+ *
+ * Enabled  -> This status means the feature versioning system (KIP-584) is enabled, and, the
+ *             finalized features stored in the FeatureZNode are active. This status is written by
+ *             the controller to the FeatureZNode only when the broker IBP config is greater than
+ *             or equal to KAFKA_2_6_IV1.
+ *
+ * Disabled -> This status means the feature versioning system (KIP-584) is disabled, and, the
+ *             the finalized features stored in the FeatureZNode is not relevant. This status is
+ *             written by the controller to the FeatureZNode only when the broker IBP config
+ *             is less than KAFKA_2_6_IV1.
+ *
+ * The purpose behind the FeatureZNodeStatus is that it helps differentiates between the following
+ * cases:
+ *
+ * 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_6_IV1. In such a case, the
+ *    controller will start up and notice that the FeatureZNode is absent in the new cluster.
+ *    To handle the requirement, the controller will create a FeatureZNode (with enabled status)
+ *    containing the entire list of supported features as it’s finalized 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] junrao commented on pull request #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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


   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] hachikuji commented on pull request #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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


   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 merged pull request #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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


   


----------------------------------------------------------------
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 #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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


   It seems there are some checkstyle failures.
   
   ```
   > Task :clients:checkstyleTest FAILED
   20:31:38 [ant:checkstyle] [ERROR] /home/jenkins/jenkins-slave/workspace/kafka-pr-jdk11-scala2.13@2/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java:19:8: Unused import - java.util.Optional. [UnusedImports]
   ```


----------------------------------------------------------------
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 #8680: KAFKA-9755: Implement read path for feature versioning system (KIP-584)

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


   Hi @junrao and @abbccdda,
   
   This PR is ready for code review.
   Please have a look and do let me know your thoughts.
   
   cc @cmccabe and @hachikuji 
   
   Thank 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] kowshik commented on a change in pull request #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -113,14 +127,44 @@ public static ApiVersionsResponse fromStruct(Struct struct, short version) {
         }
     }
 
-    public static ApiVersionsResponse apiVersionsResponse(int throttleTimeMs, byte maxMagic) {
+    public static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures) {
+        return apiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, Optional.empty(), Optional.empty());
+    }
+
+    public static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures,
+        Features<FinalizedVersionRange> finalizedFeatures,
+        long finalizedFeaturesEpoch) {
+        return apiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, Optional.of(finalizedFeatures), Optional.of(finalizedFeaturesEpoch));
+    }
+
+    private static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures,
+        Optional<Features<FinalizedVersionRange>> finalizedFeatures,

Review comment:
       It's because non-existing supported features _can_ be represented by an empty map (i.e. broker does not advertise any features). But on the other hand, non-existing finalized features can not be represented by empty map alone, as we need a suitable epoch value that indicates the absence of finalized features. To address this case, I saw 2 ways:
   1) Provide a negative epoch value indicating absence of finalized features, OR
   2) Represent using an empty `Optional` for both finalized features and epoch.
   
   I chose the latter approach. Please, let me know if you have concerns.




----------------------------------------------------------------
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 #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,93 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionLevelRange}
+
+// Raised whenever there was an error in updating the FinalizedFeatureCache with features.

Review comment:
       As we discussed offline today, this exception is already handled in `ChangeNotificationProcessorThread.doWork()` method defined in `FinalizedFeatureChangeListener.scala`. Basically, the ZK change notification processor thread exits the Broker with a fatal error (non-zero exit code) when this exception (or any exception) is caught while trying to update `FinalizedFeatureCache`.




----------------------------------------------------------------
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] hachikuji commented on pull request #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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


   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 pull request #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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


   test 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 pull request #8680: KAFKA-9755: Implement read path for feature versioning system (KIP-584)

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


   Hi @junrao, @abbccdda,
   
   This PR is ready for code review.
   Please have a look and do let me know your thoughts.
   
   cc @cmccabe and @hachikuji 
   
   Thank 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] abbccdda commented on a change in pull request #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -113,14 +128,45 @@ public static ApiVersionsResponse fromStruct(Struct struct, short version) {
         }
     }
 
-    public static ApiVersionsResponse apiVersionsResponse(int throttleTimeMs, byte maxMagic) {
+    public static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures) {
+        return apiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, Features.emptyFinalizedFeatures(), UNKNOWN_FINALIZED_FEATURES_EPOCH);
+    }
+
+    public static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures,
+        Features<FinalizedVersionRange> finalizedFeatures,
+        int finalizedFeaturesEpoch) {
         if (maxMagic == RecordBatch.CURRENT_MAGIC_VALUE && throttleTimeMs == DEFAULT_THROTTLE_TIME) {
             return DEFAULT_API_VERSIONS_RESPONSE;
         }
-        return createApiVersionsResponse(throttleTimeMs, maxMagic);
+        return createApiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, finalizedFeatures, finalizedFeaturesEpoch);
     }
 
-    public static ApiVersionsResponse createApiVersionsResponse(int throttleTimeMs, final byte minMagic) {
+    public static ApiVersionsResponse createApiVersionsResponseWithEmptyFeatures(

Review comment:
       I overlooked this case, let's maintain this static constructor without renaming it, since it is public.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being <VersionRangeType>.
+ * Also provides API to convert the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ * @see SupportedVersionRange
+ * @see FinalizedVersionRange
+ */
+public class Features<VersionRangeType extends BaseVersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to a type of VersionRange.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        Objects.requireNonNull(features,"Provided features can not be null.");
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to SupportedVersionRange.
+     *
+     * @return           Returns a new Features object representing supported features.
+     */
+    public static Features<SupportedVersionRange> supportedFeatures(Map<String, SupportedVersionRange> features) {
+        return new Features<>(features);
+    }
+
+    /**
+     * @param features   Map of feature name to FinalizedVersionRange.
+     *
+     * @return           Returns a new Features object representing finalized features.
+     */
+    public static Features<FinalizedVersionRange> finalizedFeatures(Map<String, FinalizedVersionRange> features) {
+        return new Features<>(features);
+    }
+
+    // Visible for testing.
+    public static Features<FinalizedVersionRange> emptyFinalizedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public static Features<SupportedVersionRange> emptySupportedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public Map<String, VersionRangeType> features() {
+        return features;
+    }
+
+    public boolean empty() {
+        return features.isEmpty();
+    }
+
+    /**
+     * @param  feature   name of the feature
+     *
+     * @return           the VersionRangeType corresponding to the feature name, or null if the
+     *                   feature is absent
+     */
+    public VersionRangeType get(String feature) {
+        return features.get(feature);
+    }
+
+    public String toString() {
+        return String.format(
+            "Features{%s}",
+            features
+                .entrySet()
+                .stream()
+                .map(entry -> String.format("(%s -> %s)", entry.getKey(), entry.getValue()))
+                .collect(joining(", "))
+        );
+    }
+
+    /**
+     * @return   A map representation of the underlying features. The returned value can be converted
+     *           back to Features using one of the from*FeaturesMap() APIs of this class.
+     */
+    public Map<String, Map<String, Short>> toMap() {
+        return features.entrySet().stream().collect(
+            Collectors.toMap(
+                Map.Entry::getKey,
+                entry -> entry.getValue().toMap()));
+    }
+
+    /**
+     * An interface that defines behavior to convert from a Map to an object of type BaseVersionRange.
+     */
+    private interface MapToBaseVersionRangeConverter<V extends BaseVersionRange> {
+
+        /**
+         * Convert the map representation of an object of type <V>, to an object of type <V>.
+         *
+         * @param  baseVersionRangeMap   the map representation of a BaseVersionRange object.
+         *
+         * @return                       the object of type <V>
+         */
+        V fromMap(Map<String, Short> baseVersionRangeMap);
+    }
+
+    private static <V extends BaseVersionRange> Features<V> fromFeaturesMap(
+        Map<String, Map<String, Short>> featuresMap, MapToBaseVersionRangeConverter<V> converter) {
+        return new Features<>(featuresMap.entrySet().stream().collect(
+            Collectors.toMap(
+                Map.Entry::getKey,
+                entry -> converter.fromMap(entry.getValue()))));
+    }
+
+    /**
+     * Converts from a map to Features<FinalizedVersionRange>.
+     *
+     * @param featuresMap  the map representation of a Features<FinalizedVersionRange> object,
+     *                     generated using the toMap() API.
+     *
+     * @return             the Features<FinalizedVersionRange> object
+     */
+    public static Features<FinalizedVersionRange> fromFinalizedFeaturesMap(
+        Map<String, Map<String, Short>> featuresMap) {
+        return fromFeaturesMap(featuresMap, FinalizedVersionRange::fromMap);
+    }
+
+    /**
+     * Converts from a map to Features<SupportedVersionRange>.
+     *
+     * @param featuresMap  the map representation of a Features<SupportedVersionRange> object,
+     *                     generated using the toMap() API.
+     *
+     * @return             the Features<SupportedVersionRange> object
+     */
+    public static Features<SupportedVersionRange> fromSupportedFeaturesMap(
+        Map<String, Map<String, Short>> featuresMap) {
+        return fromFeaturesMap(featuresMap, SupportedVersionRange::fromMap);
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (this == other) {
+            return true;
+        }
+        if (other == null || !(other instanceof Features)) {

Review comment:
       we don't need to check `other == null` here, the next condition check covers it.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,243 @@
+/**
+ * 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.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.{StateChangeHandler, ZNodeChangeHandler}
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+import scala.util.control.Exception.ignoring
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked exactly
+     * once successfully. A subsequent invocation will raise an exception.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     * @throws   FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     * @throws   RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      debug(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        featureZNode.status match {
+          case FeatureZNodeStatus.Disabled => {
+            info(s"Feature ZK node at path: $featureZkNodePath is in disabled status.")
+            FinalizedFeatureCache.clear()
+          }
+          case FeatureZNodeStatus.Enabled => {
+            FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)

Review comment:
       Being a bit paranoid here, would it be possible to have out-of-order updates from ZK, such that the version number is not monotonically increasing? I'm thinking even we could throw in FinalizedFeatureCache, do we really want to kill the broker, or we should just log a warning and proceed.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Represents an immutable basic version range using 2 attributes: min and max, each of type short.
+ * The min and max attributes need to satisfy 2 rules:
+ *  - they are each expected to be >= 1, as we only consider positive version values to be valid.
+ *  - max should be >= min.
+ *
+ * The class also provides API to convert the version range to a map.
+ * The class allows for configurable labels for the min/max attributes, which can be specialized by
+ * sub-classes (if needed).
+ */
+class BaseVersionRange {
+    // Non-empty label for the min version key, that's used only to convert to/from a map.
+    private final String minKeyLabel;
+
+    // The value of the minimum version.
+    private final short minValue;
+
+    // Non-empty label for the max version key, that's used only to convert to/from a map.
+    private final String maxKeyLabel;
+
+    // The value of the maximum version.
+    private final short maxValue;
+
+    /**
+     * Raises an exception unless the following condition is met:
+     * minValue >= 1 and maxValue >= 1 and maxValue >= minValue.
+     *
+     * @param minKeyLabel   Label for the min version key, that's used only to convert to/from a map.
+     * @param minValue      The minimum version value.
+     * @param maxKeyLabel   Label for the max version key, that's used only to convert to/from a map.
+     * @param maxValue      The maximum version value.
+     *
+     * @throws IllegalArgumentException   If any of the following conditions are true:
+     *                                     - (minValue < 1) OR (maxValue < 1) OR (maxValue < minValue).
+     *                                     - minKeyLabel is empty, OR, minKeyLabel is empty.
+     */
+    protected BaseVersionRange(String minKeyLabel, short minValue, String maxKeyLabel, short maxValue) {
+        if (minValue < 1 || maxValue < 1 || maxValue < minValue) {
+            throw new IllegalArgumentException(
+                String.format(
+                    "Expected minValue >= 1, maxValue >= 1 and maxValue >= minValue, but received" +
+                    " minValue: %d, maxValue: %d", minValue, maxValue));
+        }
+        if (minKeyLabel.isEmpty()) {
+            throw new IllegalArgumentException("Expected minKeyLabel to be non-empty.");
+        }
+        if (maxKeyLabel.isEmpty()) {
+            throw new IllegalArgumentException("Expected maxKeyLabel to be non-empty.");
+        }
+        this.minKeyLabel = minKeyLabel;
+        this.minValue = minValue;
+        this.maxKeyLabel = maxKeyLabel;
+        this.maxValue = maxValue;
+    }
+
+    public short min() {
+        return minValue;
+    }
+
+    public short max() {
+        return maxValue;
+    }
+
+    public String toString() {
+        return String.format("%s[%s:%d, %s:%d]", this.getClass().getSimpleName(), this.minKeyLabel, min(), this.maxKeyLabel, max());
+    }
+
+    public Map<String, Short> toMap() {
+        return new HashMap<String, Short>() {

Review comment:
       nit: we could use Utils.mkMap here

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -744,6 +781,165 @@ object DelegationTokenInfoZNode {
   def decode(bytes: Array[Byte]): Option[TokenInformation] = DelegationTokenManager.fromBytes(bytes)
 }
 
+/**
+ * Represents the status of the FeatureZNode.
+ *
+ * Enabled  -> This status means the feature versioning system (KIP-584) is enabled, and, the
+ *             finalized features stored in the FeatureZNode are active. This status is written by
+ *             the controller to the FeatureZNode only when the broker IBP config is greater than
+ *             or equal to KAFKA_2_7_IV0.
+ *
+ * Disabled -> This status means the feature versioning system (KIP-584) is disabled, and, the
+ *             the finalized features stored in the FeatureZNode is not relevant. This status is
+ *             written by the controller to the FeatureZNode only when the broker IBP config
+ *             is less than KAFKA_2_7_IV0.
+ *
+ * The purpose behind the FeatureZNodeStatus is that it helps differentiates between the following
+ * cases:
+ *
+ * 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. In such a case, the
+ *    controller will start up and notice that the FeatureZNode is absent in the new cluster.
+ *    To handle the requirement, the controller will create a FeatureZNode (with enabled status)
+ *    containing the entire list of supported features as its finalized features.
+ *
+ * 2. Cluster upgrade:
+ *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, but
+ *    the Broker binary has been upgraded to a state where it 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 enable
+ *    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. The reason is that enabling all the possible
+ *    features immediately after an upgrade could be harmful to the cluster.
+ *    In such a case:
+ *      - Before the Broker 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 true, then it will react by
+ *        creating a FeatureZNode with disabled status and empty features.
+ *      - After the Broker 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. Cluster 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.
+ */
+object FeatureZNodeStatus extends Enumeration {
+  val Disabled, Enabled = Value
+
+  def withNameOpt(value: Int): Option[Value] = {
+    values.find(_.id == value)
+  }
+}
+
+/**
+ * Represents the contents of the ZK node containing finalized feature information.
+ *
+ * @param status     the status of the ZK node
+ * @param features   the cluster-wide finalized features
+ */
+case class FeatureZNode(status: FeatureZNodeStatus.Value, features: Features[FinalizedVersionRange]) {
+}
+
+object FeatureZNode {
+  private val VersionKey = "version"
+  private val StatusKey = "status"
+  private val FeaturesKey = "features"
+
+  // V1 contains 'version', 'status' and 'features' keys.
+  val V1 = 1
+  val CurrentVersion = V1
+
+  def path = "/feature"
+
+  def asJavaMap(scalaMap: Map[String, Map[String, Short]]): util.Map[String, util.Map[String, java.lang.Short]] = {
+    scalaMap
+      .view.mapValues(_.view.mapValues(scalaShort => java.lang.Short.valueOf(scalaShort)).toMap.asJava)
+      .toMap
+      .asJava
+  }
+
+  /**
+   * Encodes a FeatureZNode to JSON.
+   *
+   * @param featureZNode   FeatureZNode to be encoded
+   *
+   * @return               JSON representation of the FeatureZNode, as an Array[Byte]
+   */
+  def encode(featureZNode: FeatureZNode): Array[Byte] = {
+    val jsonMap = collection.mutable.Map(
+      VersionKey -> CurrentVersion,
+      StatusKey -> featureZNode.status.id,
+      FeaturesKey -> featureZNode.features.toMap)
+    Json.encodeAsBytes(jsonMap.asJava)
+  }
+
+  /**
+   * Decodes the contents of the feature ZK node from Array[Byte] to a FeatureZNode.
+   *
+   * @param jsonBytes   the contents of the feature ZK node
+   *
+   * @return            the FeatureZNode created from jsonBytes
+   *
+   * @throws IllegalArgumentException   if the Array[Byte] can not be decoded.
+   */
+  def decode(jsonBytes: Array[Byte]): FeatureZNode = {
+    Json.tryParseBytes(jsonBytes) match {
+      case Right(js) =>
+        val featureInfo = js.asJsonObject
+        val version = featureInfo(VersionKey).to[Int]
+        if (version < V1) {
+          throw new IllegalArgumentException(s"Unsupported version: $version of feature information: " +
+            s"${new String(jsonBytes, UTF_8)}")
+        }
+
+        val featuresMap = featureInfo
+          .get(FeaturesKey)
+          .flatMap(_.to[Option[Map[String, Map[String, Int]]]])
+
+        if (featuresMap.isEmpty) {
+          throw new IllegalArgumentException("Features map can not be absent in: " +
+            s"${new String(jsonBytes, UTF_8)}")
+        }
+        val features = asJavaMap(
+          featuresMap
+            .map(theMap => theMap.view.mapValues(_.view.mapValues(_.asInstanceOf[Short]).toMap).toMap)
+            .getOrElse(Map[String, Map[String, Short]]()))
+
+        val statusInt = featureInfo
+          .get(StatusKey)
+          .flatMap(_.to[Option[Int]])
+        if (statusInt.isEmpty) {
+          throw new IllegalArgumentException("Status can not be absent in feature information: " +
+            s"${new String(jsonBytes, UTF_8)}")
+        }
+        val status = FeatureZNodeStatus.withNameOpt(statusInt.get)
+        if (status.isEmpty) {
+          throw new IllegalArgumentException(
+            s"Malformed status: $statusInt  found in feature information: ${new String(jsonBytes, UTF_8)}")

Review comment:
       nit: space

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,102 @@
+/**
+ * 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}
+
+// 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) {
+  override def toString(): String = {
+    "FinalizedFeaturesAndEpoch(features=%s, epoch=%s)".format(features, epoch)
+  }
+}
+
+/**
+ * A common mutable cache containing the latest finalized features and epoch. By default the contents of
+ * the cache are empty. This cache needs to be populated at least once for it's contents to become

Review comment:
       it's -> its

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -113,14 +128,45 @@ public static ApiVersionsResponse fromStruct(Struct struct, short version) {
         }
     }
 
-    public static ApiVersionsResponse apiVersionsResponse(int throttleTimeMs, byte maxMagic) {
+    public static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures) {
+        return apiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, Features.emptyFinalizedFeatures(), UNKNOWN_FINALIZED_FEATURES_EPOCH);
+    }
+
+    public static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures,
+        Features<FinalizedVersionRange> finalizedFeatures,
+        int finalizedFeaturesEpoch) {
         if (maxMagic == RecordBatch.CURRENT_MAGIC_VALUE && throttleTimeMs == DEFAULT_THROTTLE_TIME) {
             return DEFAULT_API_VERSIONS_RESPONSE;
         }
-        return createApiVersionsResponse(throttleTimeMs, maxMagic);
+        return createApiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, finalizedFeatures, finalizedFeaturesEpoch);
     }
 
-    public static ApiVersionsResponse createApiVersionsResponse(int throttleTimeMs, final byte minMagic) {
+    public static ApiVersionsResponse createApiVersionsResponseWithEmptyFeatures(
+        int throttleTimeMs,
+        final byte minMagic) {
+        return createApiVersionsResponse(
+            throttleTimeMs,
+            minMagic,
+            Features.emptySupportedFeatures(),
+            Features.emptyFinalizedFeatures(),
+            UNKNOWN_FINALIZED_FEATURES_EPOCH);
+    }
+
+    public static ApiVersionsResponse createApiVersionsResponse(
+        int throttleTimeMs,

Review comment:
       nit: should all the parameters be final here, not just minMagic?

##########
File path: core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala
##########
@@ -149,6 +153,53 @@ class BrokerEndPointTest {
     assertEquals(None, broker.rack)
   }
 
+  @Test
+  def testFromJsonV4WithNoFeatures(): Unit = {

Review comment:
       V4 doesn't have feature right? What's the purpose of this test?

##########
File path: core/src/main/scala/kafka/api/ApiVersion.scala
##########
@@ -98,7 +98,9 @@ object ApiVersion {
     // No new APIs, equivalent to 2.4-IV1
     KAFKA_2_5_IV0,
     // Introduced StopReplicaRequest V3 containing the leader epoch for each partition (KIP-570)
-    KAFKA_2_6_IV0
+    KAFKA_2_6_IV0,
+    // Introduce feature versioning support (KIP-584)

Review comment:
       nit: use `Introduced` to align with previous comment?

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,243 @@
+/**
+ * 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.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.{StateChangeHandler, ZNodeChangeHandler}
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+import scala.util.control.Exception.ignoring
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked exactly
+     * once successfully. A subsequent invocation will raise an exception.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     * @throws   FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     * @throws   RuntimeException, if there was a failure in reading/deserializing the

Review comment:
       Sorry it's been a while since my last review, but have we discussed the recovery path when we hit a data corruption exception for the cluster? Is there a way to turn off the feature versioning completely to unblock, or we have a mechanism to wipe out ZK data?

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,102 @@
+/**
+ * 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}
+
+// 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) {
+  override def toString(): String = {
+    "FinalizedFeaturesAndEpoch(features=%s, epoch=%s)".format(features, epoch)
+  }
+}
+
+/**
+ * A common mutable cache containing the latest finalized features and epoch. By default the contents of
+ * the cache are empty. This cache needs to be populated at least once for it's contents to become
+ * non-empty. Currently the main reader of this cache is the read path that serves an ApiVersionsRequest,
+ * returning the features information in the response.
+ *
+ * @see FinalizedFeatureChangeListener
+ */
+object FinalizedFeatureCache extends Logging {
+  @volatile private var featuresAndEpoch: Option[FinalizedFeaturesAndEpoch] = Option.empty
+
+  /**
+   * @return   the latest known FinalizedFeaturesAndEpoch. If the returned value is empty, it means

Review comment:
       could be simplified as `the latest known FinalizedFeaturesAndEpoch or empty if not defined in the cache`

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -744,6 +782,90 @@ object DelegationTokenInfoZNode {
   def decode(bytes: Array[Byte]): Option[TokenInformation] = DelegationTokenManager.fromBytes(bytes)
 }
 
+object FeatureZNodeStatus extends Enumeration {
+  val Disabled, Enabled = Value
+
+  def withNameOpt(value: Int): Option[Value] = {
+    values.find(_.id == value)
+  }
+}
+
+case class FeatureZNode(status: FeatureZNodeStatus.Value, features: Features[FinalizedVersionRange]) {
+}
+
+object FeatureZNode {

Review comment:
       I think I'm no longer insisting on this point, as we could make this as a follow-up work. Filed JIRA here: https://issues.apache.org/jira/browse/KAFKA-10130

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,243 @@
+/**
+ * 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.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.{StateChangeHandler, ZNodeChangeHandler}
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+import scala.util.control.Exception.ignoring
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable

Review comment:
       I think we could remove `If the cache update is not successful, then, a suitable exception is raised...` which is pretty obvious.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,243 @@
+/**
+ * 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.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.{StateChangeHandler, ZNodeChangeHandler}
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+import scala.util.control.Exception.ignoring
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked exactly
+     * once successfully. A subsequent invocation will raise an exception.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     * @throws   FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     * @throws   RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      debug(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        featureZNode.status match {
+          case FeatureZNodeStatus.Disabled => {
+            info(s"Feature ZK node at path: $featureZkNodePath is in disabled status.")
+            FinalizedFeatureCache.clear()
+          }
+          case FeatureZNodeStatus.Enabled => {
+            FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+          }
+          case _ => throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully. This method returns
+     * immediately if an updateLatestOrThrow call had already completed successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             TimeoutException if the wait can not be completed in waitTimeMs
+     *                     milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (!notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        ignoring(classOf[InterruptedException]) {
+          queue.take.updateLatestOrThrow()
+        }
+      } catch {
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will eventually exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDataChange(): Unit = {
+      info(s"Feature ZK node updated at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDeletion(): Unit = {
+      warn(s"Feature ZK node deleted at path: $path")
+      // This event may happen, rarely (ex: ZK corruption or operational error).
+      // In such a case, we prefer to just log a warning and treat the case as if the node is absent,
+      // and populate the FinalizedFeatureCache with empty finalized features.
+      queue.add(new FeatureCacheUpdater(path))
+    }
+  }
+
+  object ZkStateChangeHandler extends StateChangeHandler {
+    val path: String = FeatureZNode.path
+
+    override val name: String = path
+
+    override def afterInitializingSession(): Unit = {
+      queue.add(new FeatureCacheUpdater(path))
+    }
+  }
+
+  private val queue = new LinkedBlockingQueue[FeatureCacheUpdater]
+
+  private val thread = new ChangeNotificationProcessorThread("feature-zk-node-event-process-thread")
+
+  /**
+   * This method initializes the feature ZK node change listener. Optionally, it also ensures to
+   * update the FinalizedFeatureCache once with the latest contents of the feature ZK node
+   * (if the node exists). This step helps ensure that feature incompatibilities (if any) in brokers
+   * are conveniently detected before the initOrThrow() method returns to the caller. If feature
+   * incompatibilities are detected, this method will throw an Exception to the caller, and the Broker
+   * will exit eventually.
+   *
+   * @param waitOnceForCacheUpdateMs   # of milli seconds to wait for feature cache to be updated once.
+   *                                   If this parameter <= 0, no wait operation happens.
+   *
+   * @throws Exception if feature incompatibility check could not be finished in a timely manner
+   */
+  def initOrThrow(waitOnceForCacheUpdateMs: Long): Unit = {
+    if (waitOnceForCacheUpdateMs <= 0) {
+      throw new IllegalArgumentException(
+        s"Expected waitOnceForCacheUpdateMs > 0, but provided: $waitOnceForCacheUpdateMs")
+    }
+
+    thread.start()
+    zkClient.registerStateChangeHandler(ZkStateChangeHandler)
+    zkClient.registerZNodeChangeHandlerAndCheckExistence(FeatureZNodeChangeHandler)
+    val ensureCacheUpdateOnce = new FeatureCacheUpdater(
+      FeatureZNodeChangeHandler.path, Some(new CountDownLatch(1)))
+    queue.add(ensureCacheUpdateOnce)
+    try {
+      ensureCacheUpdateOnce.awaitUpdateOrThrow(waitOnceForCacheUpdateMs)
+    } catch {
+      case e: Exception => {

Review comment:
       nit: {} not necessary

##########
File path: core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
##########
@@ -755,7 +755,7 @@ class KafkaConfigTest {
         case KafkaConfig.SaslLoginRefreshMinPeriodSecondsProp =>
         case KafkaConfig.SaslLoginRefreshBufferSecondsProp =>
 
-        // Security config
+        //Security config

Review comment:
       add the space back

##########
File path: core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala
##########
@@ -0,0 +1,215 @@
+/**
+ * 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

Review comment:
       nit: we could add a minor test to verify a negative `waitOnceForCacheUpdateMs` will throw

##########
File path: clients/src/test/java/org/apache/kafka/common/feature/FeaturesTest.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThrows;
+
+public class FeaturesTest {
+
+    @Test
+    public void testEmptyFeatures() {
+        Map<String, Map<String, Short>> emptyMap = new HashMap<>();
+
+        Features<FinalizedVersionRange> emptyFinalizedFeatures = Features.emptyFinalizedFeatures();
+        assertTrue(emptyFinalizedFeatures.features().isEmpty());
+        assertTrue(emptyFinalizedFeatures.toMap().isEmpty());
+        assertEquals(emptyFinalizedFeatures, Features.fromFinalizedFeaturesMap(emptyMap));
+
+        Features<SupportedVersionRange> emptySupportedFeatures = Features.emptySupportedFeatures();
+        assertTrue(emptySupportedFeatures.features().isEmpty());
+        assertTrue(emptySupportedFeatures.toMap().isEmpty());
+        assertEquals(emptySupportedFeatures, Features.fromSupportedFeaturesMap(emptyMap));
+    }
+
+    @Test
+    public void testNullFeatures() {
+        assertThrows(
+            NullPointerException.class,
+            () -> Features.finalizedFeatures(null));
+        assertThrows(
+            NullPointerException.class,
+            () -> Features.supportedFeatures(null));
+    }
+
+    @Test
+    public void testGetAllFeaturesAPI() {
+        SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2);
+        SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4);
+        Map<String, SupportedVersionRange> allFeatures =
+            mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2));
+        Features<SupportedVersionRange> features = Features.supportedFeatures(allFeatures);
+        assertEquals(allFeatures, features.features());
+    }
+
+    @Test
+    public void testGetAPI() {
+        SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2);
+        SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4);
+        Map<String, SupportedVersionRange> allFeatures = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2));
+        Features<SupportedVersionRange> features = Features.supportedFeatures(allFeatures);
+        assertEquals(v1, features.get("feature_1"));
+        assertEquals(v2, features.get("feature_2"));
+        assertNull(features.get("nonexistent_feature"));
+    }
+
+    @Test
+    public void testFromFeaturesMapToFeaturesMap() {
+        SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2);
+        SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4);
+        Map<String, SupportedVersionRange> allFeatures = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2));
+
+        Features<SupportedVersionRange> features = Features.supportedFeatures(allFeatures);
+
+        Map<String, Map<String, Short>> expected = mkMap(
+            mkEntry("feature_1", mkMap(mkEntry("min_version", (short) 1), mkEntry("max_version", (short) 2))),
+            mkEntry("feature_2", mkMap(mkEntry("min_version", (short) 3), mkEntry("max_version", (short) 4))));
+        assertEquals(expected, features.toMap());
+        assertEquals(features, Features.fromSupportedFeaturesMap(expected));
+    }
+
+    @Test
+    public void testFromToFinalizedFeaturesMap() {
+        FinalizedVersionRange v1 = new FinalizedVersionRange((short) 1, (short) 2);
+        FinalizedVersionRange v2 = new FinalizedVersionRange((short) 3, (short) 4);
+        Map<String, FinalizedVersionRange> allFeatures = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2));
+
+        Features<FinalizedVersionRange> features = Features.finalizedFeatures(allFeatures);
+
+        Map<String, Map<String, Short>> expected = mkMap(
+            mkEntry("feature_1", mkMap(mkEntry("min_version_level", (short) 1), mkEntry("max_version_level", (short) 2))),
+            mkEntry("feature_2", mkMap(mkEntry("min_version_level", (short) 3), mkEntry("max_version_level", (short) 4))));
+        assertEquals(expected, features.toMap());
+        assertEquals(features, Features.fromFinalizedFeaturesMap(expected));
+    }
+
+    @Test
+    public void testToStringFinalizedFeatures() {
+        FinalizedVersionRange v1 = new FinalizedVersionRange((short) 1, (short) 2);
+        FinalizedVersionRange v2 = new FinalizedVersionRange((short) 3, (short) 4);
+        Map<String, FinalizedVersionRange> allFeatures = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2));
+
+        Features<FinalizedVersionRange> features = Features.finalizedFeatures(allFeatures);
+
+        assertEquals(
+            "Features{(feature_1 -> FinalizedVersionRange[min_version_level:1, max_version_level:2]), (feature_2 -> FinalizedVersionRange[min_version_level:3, max_version_level:4])}",
+            features.toString());
+    }
+
+    @Test
+    public void testToStringSupportedFeatures() {
+        SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2);
+        SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4);
+        Map<String, SupportedVersionRange> allFeatures
+            = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2));
+
+        Features<SupportedVersionRange> features = Features.supportedFeatures(allFeatures);
+
+        assertEquals(
+            "Features{(feature_1 -> SupportedVersionRange[min_version:1, max_version:2]), (feature_2 -> SupportedVersionRange[min_version:3, max_version:4])}",
+            features.toString());
+    }
+
+    @Test
+    public void testSupportedFeaturesFromMapFailure() {
+        // This is invalid because 'max_version' key is missing.

Review comment:
       nit: instead of using comments, better to build this into the test name, for example:
   `testInvalidSuppportedFeaturesWithMissingMaxVersion`




----------------------------------------------------------------
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 #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/KafkaConfig.scala
##########
@@ -442,6 +445,8 @@ object KafkaConfig {
   val ControlledShutdownMaxRetriesProp = "controlled.shutdown.max.retries"
   val ControlledShutdownRetryBackoffMsProp = "controlled.shutdown.retry.backoff.ms"
   val ControlledShutdownEnableProp = "controlled.shutdown.enable"
+  /** ********* Features configuration ***********/
+  val FeatureChangeListenerCacheUpdateWaitTimeMsProp = "feature.listener.cache.update.wait.ms"

Review comment:
       This config has been eliminated 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] abbccdda commented on a change in pull request #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,232 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     *
+     *           FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     *
+     *           RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      var mayBeFeatureZNodeBytes: Option[Array[Byte]] = null
+      var version: Int = ZkVersion.UnknownVersion
+      try {
+        val result = zkClient.getDataAndVersion(featureZkNodePath)
+        mayBeFeatureZNodeBytes = result._1
+        version = result._2
+      } catch {
+        // Convert to RuntimeException, to avoid a confusion that there is no argument passed
+        // to the updateOrThrow() method.
+        case e: IllegalArgumentException => throw new RuntimeException(e)
+      }
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             RuntimeException if the thread was interrupted during wait
+     *
+     *                     TimeoutException if the wait can not be completed in waitTimeMs
+     *                     milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Change notification queue interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDataChange(): Unit = {
+      info(s"Feature ZK node updated at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDeletion(): Unit = {
+      warn(s"Feature ZK node deleted at path: $path")
+      // This event may happen, rarely (ex: operational error).

Review comment:
       Thanks, I don't think we need to be super paranoid with this rare scenario, but we should also be indicating this error state to the client suggesting that some manual fix is necessary. My proposed idea above is to add such an error state to the feature cache to refuse any further updates until we have: 1. a node creation event 2. restart of the broker (once the issue gets fixed), so this blocking behavior shall be ephemeral and recoverable from broker perspective. Make sense to cc @cmccabe and @hachikuji as well.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,232 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     *
+     *           FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     *
+     *           RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      var mayBeFeatureZNodeBytes: Option[Array[Byte]] = null
+      var version: Int = ZkVersion.UnknownVersion
+      try {
+        val result = zkClient.getDataAndVersion(featureZkNodePath)
+        mayBeFeatureZNodeBytes = result._1
+        version = result._2
+      } catch {
+        // Convert to RuntimeException, to avoid a confusion that there is no argument passed
+        // to the updateOrThrow() method.
+        case e: IllegalArgumentException => throw new RuntimeException(e)
+      }
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             RuntimeException if the thread was interrupted during wait
+     *
+     *                     TimeoutException if the wait can not be completed in waitTimeMs
+     *                     milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Change notification queue interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDataChange(): Unit = {
+      info(s"Feature ZK node updated at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDeletion(): Unit = {
+      warn(s"Feature ZK node deleted at path: $path")
+      // This event may happen, rarely (ex: operational error).

Review comment:
       Thanks, I don't think we need to be super paranoid with this rare scenario, but we should also be indicating this error state to the client suggesting that some manual fix is necessary. My proposed idea above is to add such an error state to the feature cache to refuse any further updates until we have: 1. a node creation event 2. restart of the broker (once the issue gets fixed), so this blocking behavior shall be ephemeral and recoverable from broker perspective. We don't have to implement this logic in the current PR, as we don't have a write path yet, just get a JIRA to track it sounds fine.
   
   Make sense to cc @cmccabe and @hachikuji 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 a change in pull request #8680: KAFKA-9755: Implement read path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ */
+public class Features<VersionRangeType extends VersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;

Review comment:
       Do you feel strongly about this?
   The reasons why I ask the question is:
   1. Caller is unlikely to pass `null`.
   2.  I looked over a number of other existing classes in Kafka, and there aren't any null checks for most constructor parameters.
   
   It will help me if you could share couple examples from existing code where the `null` check convention is followed in Kafka.

##########
File path: clients/src/test/java/org/apache/kafka/common/feature/FeaturesTest.java
##########
@@ -0,0 +1,216 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThrows;
+
+public class FeaturesTest {
+
+    @Test
+    public void testEmptyFeatures() {
+        Map<String, Map<String, Long>> emptyMap = new HashMap<>();
+
+        Features<VersionLevelRange> emptyFinalizedFeatures = Features.emptyFinalizedFeatures();
+        assertEquals(new HashMap<>(), emptyFinalizedFeatures.all());
+        assertEquals(emptyMap, emptyFinalizedFeatures.serialize());
+        assertEquals(emptyFinalizedFeatures, Features.deserializeFinalizedFeatures(emptyMap));
+
+        Features<VersionRange> emptySupportedFeatures = Features.emptySupportedFeatures();
+        assertEquals(new HashMap<>(), emptySupportedFeatures.all());
+        assertEquals(
+            new HashMap<String, HashMap<String, Long>>(),
+            emptySupportedFeatures.serialize());
+        assertEquals(emptySupportedFeatures, Features.deserializeSupportedFeatures(emptyMap));
+    }
+
+    @Test
+    public void testAllAPI() {
+        VersionRange v1 = new VersionRange(1, 2);
+        VersionRange v2 = new VersionRange(3, 4);
+        Map<String, VersionRange> allFeatures = new HashMap<String, VersionRange>() {
+            {
+                put("feature_1", v1);
+                put("feature_2", v2);
+            }
+        };
+        Features<VersionRange> features = Features.supportedFeatures(allFeatures);
+        assertEquals(allFeatures, features.all());
+    }
+
+    @Test
+    public void testGetAPI() {
+        VersionRange v1 = new VersionRange(1, 2);
+        VersionRange v2 = new VersionRange(3, 4);
+        Map<String, VersionRange> allFeatures = new HashMap<String, VersionRange>() {
+            {
+                put("feature_1", v1);
+                put("feature_2", v2);
+            }
+        };
+        Features<VersionRange> features = Features.supportedFeatures(allFeatures);
+        assertEquals(v1, features.get("feature_1"));
+        assertEquals(v2, features.get("feature_2"));
+        assertNull(features.get("nonexistent_feature"));
+    }
+
+    @Test
+    public void testSerializeDeserializeSupportedFeatures() {
+        VersionRange v1 = new VersionRange(1, 2);
+        VersionRange v2 = new VersionRange(3, 4);
+        Map<String, VersionRange> allFeatures = new HashMap<String, VersionRange>() {

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ */
+public class Features<VersionRangeType extends VersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to VersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "supported" features.
+     */
+    public static Features<VersionRange> supportedFeatures(Map<String, VersionRange> features) {
+        return new Features<VersionRange>(features);
+    }
+
+    /**
+     * @param features   Map of feature name to VersionLevelRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "finalized" features.
+     */
+    public static Features<VersionLevelRange> finalizedFeatures(Map<String, VersionLevelRange> features) {
+        return new Features<VersionLevelRange>(features);
+    }
+
+    public static Features<VersionLevelRange> emptyFinalizedFeatures() {

Review comment:
       Done. Yes, I've changed it to default visibility now.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ */
+public class Features<VersionRangeType extends VersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to VersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "supported" features.
+     */
+    public static Features<VersionRange> supportedFeatures(Map<String, VersionRange> features) {
+        return new Features<VersionRange>(features);
+    }
+
+    /**
+     * @param features   Map of feature name to VersionLevelRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "finalized" features.
+     */
+    public static Features<VersionLevelRange> finalizedFeatures(Map<String, VersionLevelRange> features) {
+        return new Features<VersionLevelRange>(features);
+    }
+
+    public static Features<VersionLevelRange> emptyFinalizedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public static Features<VersionRange> emptySupportedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+
+    public Map<String, VersionRangeType> all() {
+        return features;
+    }
+
+    public boolean empty() {
+        return features.isEmpty();
+    }
+
+    public VersionRangeType get(String feature) {
+        return all().get(feature);
+    }
+
+    public String toString() {
+        return String.format(
+            "Features{%s}",
+            features
+                .entrySet()
+                .stream()
+                .map(entry -> String.format("(%s -> %s)", entry.getKey(), entry.getValue()))
+                .collect(joining(", "))
+        );
+    }
+
+    /**
+     * @return   Serializes the underlying features to a map, and returns the same.
+     *           The returned value can be deserialized using one of the deserialize* APIs.
+     */
+    public Map<String, Map<String, Long>> serialize() {
+        return features.entrySet().stream().collect(
+            Collectors.toMap(
+                Map.Entry::getKey,
+                entry -> entry.getValue().serialize()));
+    }
+
+    /**
+     * Deserializes a map to Features<VersionLevelRange>.
+     *
+     * @param serialized   the serialized representation of a Features<VersionLevelRange> object,
+     *                     generated using the serialize() API.
+     *
+     * @return             the deserialized Features<VersionLevelRange> object
+     */
+    public static Features<VersionLevelRange> deserializeFinalizedFeatures(
+        Map<String, Map<String, Long>> serialized) {
+        return finalizedFeatures(serialized.entrySet().stream().collect(
+            Collectors.toMap(
+                Map.Entry::getKey,
+                entry -> VersionLevelRange.deserialize(entry.getValue()))));
+    }
+
+    /**
+     * Deserializes a map to Features<VersionRange>.
+     *
+     * @param serialized   the serialized representation of a Features<VersionRange> object,
+     *                     generated using the serialize() API.
+     *
+     * @return             the deserialized Features<VersionRange> object
+     */
+    public static Features<VersionRange> deserializeSupportedFeatures(
+        Map<String, Map<String, Long>> serialized) {
+        return supportedFeatures(serialized.entrySet().stream().collect(
+            Collectors.toMap(
+                Map.Entry::getKey,
+                entry -> VersionRange.deserialize(entry.getValue()))));
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (this == other) {

Review comment:
       Done. Good point! Added test as well.

##########
File path: clients/src/test/java/org/apache/kafka/common/feature/FeaturesTest.java
##########
@@ -0,0 +1,216 @@
+package org.apache.kafka.common.feature;

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,93 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionLevelRange}
+
+// 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[VersionLevelRange], epoch: Int) {
+
+  def isValid(newEpoch: Int): Boolean = {
+    newEpoch >= epoch
+  }
+
+  override def toString(): String = {
+    "FinalizedFeaturesAndEpoch(features=%s, epoch=%s)".format(features, epoch)
+  }
+}
+
+/**
+ * A mutable cache containing the latest finalized features and epoch. This cache is populated by a
+ * FinalizedFeatureChangeListener.
+ *
+ * Currently the main reader of this cache is the read path that serves an ApiVersionsRequest
+ * returning the features information in the response. In the future, as the feature versioning
+ * system in KIP-584 is used more widely, this cache could be read by other read paths trying to
+ * learn the finalized feature information.
+ */
+object FinalizedFeatureCache extends Logging {
+  @volatile private var featuresAndEpoch: Option[FinalizedFeaturesAndEpoch] = Option.empty
+
+  /**
+   * @return   the latest known FinalizedFeaturesAndEpoch. If the returned value is empty, it means
+   *           no FinalizedFeaturesAndEpoch exists in the cache at the time when this
+   *           method is invoked. This result could change in the future whenever the
+   *           updateOrThrow method is invoked.
+   */
+  def get: Option[FinalizedFeaturesAndEpoch] = {
+    featuresAndEpoch
+  }
+
+  def empty: Boolean = {
+    featuresAndEpoch.isEmpty
+  }
+
+  /**
+   * Clears all existing finalized features and epoch from the cache.
+   */
+  def clear(): Unit = {
+    featuresAndEpoch = Option.empty
+    info("Cleared cache")
+  }
+
+  /**
+   * Updates the cache to the latestFeatures, and updates the existing epoch to latestEpoch.
+   * Raises an exception when the operation is not successful.
+   *
+   * @param latestFeatures   the latest finalized features to be set in the cache
+   * @param latestEpoch      the latest epoch value to be set in the cache
+   *
+   * @throws                 FeatureCacheUpdateException if the cache update operation fails
+   *                         due to invalid parameters or incompatibilities with the broker's
+   *                         supported features. In such a case, the existing cache contents are
+   *                         not modified.
+   */
+  def updateOrThrow(latestFeatures: Features[VersionLevelRange], latestEpoch: Int): Unit = {
+    updateOrThrow(FinalizedFeaturesAndEpoch(latestFeatures, latestEpoch))
+  }
+
+  private def updateOrThrow(latest: FinalizedFeaturesAndEpoch): Unit = {
+    val existingStr = featuresAndEpoch.map(existing => existing.toString).getOrElse("<empty>")
+    if (!featuresAndEpoch.isEmpty && featuresAndEpoch.get.epoch > latest.epoch) {

Review comment:
       Done.

##########
File path: core/src/test/scala/unit/kafka/server/FinalizedFeatureCacheTest.scala
##########
@@ -0,0 +1,95 @@
+package kafka.server
+
+import org.apache.kafka.common.feature.{Features, VersionLevelRange, VersionRange}
+import org.junit.Assert.{assertEquals, assertThrows, assertTrue}
+import org.junit.{Before, Test}
+
+import scala.jdk.CollectionConverters._
+
+class FinalizedFeatureCacheTest {
+
+  @Before
+  def setUp(): Unit = {
+    FinalizedFeatureCache.clear()
+    SupportedFeatures.clear()
+  }
+
+  @Test
+  def testEmpty(): Unit = {
+    assertTrue(FinalizedFeatureCache.get.isEmpty)
+  }
+
+  @Test
+  def testUpdateOrThrowFailedDueToInvalidEpoch(): Unit = {
+    val supportedFeatures = Map[String, VersionRange](
+      "feature_1" -> new VersionRange(1, 4))
+    SupportedFeatures.update(Features.supportedFeatures(supportedFeatures.asJava))
+
+    val features = Map[String, VersionLevelRange](
+      "feature_1" -> new VersionLevelRange(1, 4))
+    val finalizedFeatures = Features.finalizedFeatures(features.asJava)
+
+    FinalizedFeatureCache.updateOrThrow(finalizedFeatures, 10)
+    assertEquals(finalizedFeatures, FinalizedFeatureCache.get.get.features)

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,93 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionLevelRange}
+
+// 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[VersionLevelRange], epoch: Int) {
+
+  def isValid(newEpoch: Int): Boolean = {
+    newEpoch >= epoch
+  }
+
+  override def toString(): String = {
+    "FinalizedFeaturesAndEpoch(features=%s, epoch=%s)".format(features, epoch)
+  }
+}
+
+/**
+ * A mutable cache containing the latest finalized features and epoch. This cache is populated by a
+ * FinalizedFeatureChangeListener.
+ *
+ * Currently the main reader of this cache is the read path that serves an ApiVersionsRequest
+ * returning the features information in the response. In the future, as the feature versioning
+ * system in KIP-584 is used more widely, this cache could be read by other read paths trying to
+ * learn the finalized feature information.
+ */
+object FinalizedFeatureCache extends Logging {
+  @volatile private var featuresAndEpoch: Option[FinalizedFeaturesAndEpoch] = Option.empty
+
+  /**
+   * @return   the latest known FinalizedFeaturesAndEpoch. If the returned value is empty, it means
+   *           no FinalizedFeaturesAndEpoch exists in the cache at the time when this
+   *           method is invoked. This result could change in the future whenever the
+   *           updateOrThrow method is invoked.
+   */
+  def get: Option[FinalizedFeaturesAndEpoch] = {
+    featuresAndEpoch
+  }
+
+  def empty: Boolean = {
+    featuresAndEpoch.isEmpty
+  }
+
+  /**
+   * Clears all existing finalized features and epoch from the cache.
+   */
+  def clear(): Unit = {
+    featuresAndEpoch = Option.empty
+    info("Cleared cache")
+  }
+
+  /**
+   * Updates the cache to the latestFeatures, and updates the existing epoch to latestEpoch.
+   * Raises an exception when the operation is not successful.
+   *
+   * @param latestFeatures   the latest finalized features to be set in the cache
+   * @param latestEpoch      the latest epoch value to be set in the cache
+   *
+   * @throws                 FeatureCacheUpdateException if the cache update operation fails
+   *                         due to invalid parameters or incompatibilities with the broker's
+   *                         supported features. In such a case, the existing cache contents are
+   *                         not modified.
+   */
+  def updateOrThrow(latestFeatures: Features[VersionLevelRange], latestEpoch: Int): Unit = {
+    updateOrThrow(FinalizedFeaturesAndEpoch(latestFeatures, latestEpoch))
+  }
+
+  private def updateOrThrow(latest: FinalizedFeaturesAndEpoch): Unit = {
+    val existingStr = featuresAndEpoch.map(existing => existing.toString).getOrElse("<empty>")
+    if (!featuresAndEpoch.isEmpty && featuresAndEpoch.get.epoch > latest.epoch) {
+      val errorMsg = ("FinalizedFeatureCache update failed due to invalid epoch in new finalized %s." +
+        " The existing finalized is %s").format(latest, existingStr)
+      throw new FeatureCacheUpdateException(errorMsg)
+    } else {
+      val incompatibleFeatures = SupportedFeatures.incompatibleFeatures(latest.features)
+      if (incompatibleFeatures.nonEmpty) {
+        val errorMsg = ("FinalizedFeatureCache updated failed since feature compatibility" +

Review comment:
       It is used intentionally to split the log message into 2 lines (for ~100-char readability limit per line). Otherwise the string will be huge and all in the same line.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ */
+public class Features<VersionRangeType extends VersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to VersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "supported" features.
+     */
+    public static Features<VersionRange> supportedFeatures(Map<String, VersionRange> features) {
+        return new Features<VersionRange>(features);
+    }
+
+    /**
+     * @param features   Map of feature name to VersionLevelRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "finalized" features.
+     */
+    public static Features<VersionLevelRange> finalizedFeatures(Map<String, VersionLevelRange> features) {
+        return new Features<VersionLevelRange>(features);
+    }
+
+    public static Features<VersionLevelRange> emptyFinalizedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public static Features<VersionRange> emptySupportedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+
+    public Map<String, VersionRangeType> all() {
+        return features;
+    }
+
+    public boolean empty() {
+        return features.isEmpty();
+    }
+
+    public VersionRangeType get(String feature) {
+        return all().get(feature);

Review comment:
       The underlying data structure is a `Map`. It would be simpler if this method just returns `null` if the feature doesn't exist. For example, that is how java's `Map.get` works, here is the javadoc: https://docs.oracle.com/javase/8/docs/api/java/util/Map.html#get-java.lang.Object-.
   
   Also, I've documented this method now (doc was previously absent).

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ */
+public class Features<VersionRangeType extends VersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to VersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "supported" features.
+     */
+    public static Features<VersionRange> supportedFeatures(Map<String, VersionRange> features) {
+        return new Features<VersionRange>(features);
+    }
+
+    /**
+     * @param features   Map of feature name to VersionLevelRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "finalized" features.
+     */
+    public static Features<VersionLevelRange> finalizedFeatures(Map<String, VersionLevelRange> features) {
+        return new Features<VersionLevelRange>(features);

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ */
+public class Features<VersionRangeType extends VersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to VersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "supported" features.
+     */
+    public static Features<VersionRange> supportedFeatures(Map<String, VersionRange> features) {
+        return new Features<VersionRange>(features);

Review comment:
       Done. Good point!

##########
File path: clients/src/test/java/org/apache/kafka/common/feature/FeaturesTest.java
##########
@@ -0,0 +1,216 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThrows;
+
+public class FeaturesTest {
+
+    @Test
+    public void testEmptyFeatures() {
+        Map<String, Map<String, Long>> emptyMap = new HashMap<>();
+
+        Features<VersionLevelRange> emptyFinalizedFeatures = Features.emptyFinalizedFeatures();
+        assertEquals(new HashMap<>(), emptyFinalizedFeatures.all());
+        assertEquals(emptyMap, emptyFinalizedFeatures.serialize());
+        assertEquals(emptyFinalizedFeatures, Features.deserializeFinalizedFeatures(emptyMap));
+
+        Features<VersionRange> emptySupportedFeatures = Features.emptySupportedFeatures();
+        assertEquals(new HashMap<>(), emptySupportedFeatures.all());
+        assertEquals(
+            new HashMap<String, HashMap<String, Long>>(),
+            emptySupportedFeatures.serialize());
+        assertEquals(emptySupportedFeatures, Features.deserializeSupportedFeatures(emptyMap));
+    }
+
+    @Test
+    public void testAllAPI() {

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/VersionRange.java
##########
@@ -0,0 +1,104 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Represents an immutable basic version range using 2 attributes: min and max of type long.
+ * The min and max attributes are expected to be >= 1, and with max >= min.
+ *
+ * The class also provides API to serialize/deserialize the version range to/from a map.
+ * The class allows for configurable labels for the min/max attributes, which can be specialized by
+ * sub-classes (if needed).
+ *
+ * NOTE: This is the backing class used to define the min/max versions for supported features.
+ */
+public class VersionRange {
+    // Label for the min version key, that's used only for serialization/deserialization purposes.
+    private static final String MIN_VERSION_KEY_LABEL = "min_version";
+
+    // Label for the max version key, that's used only for serialization/deserialization purposes.
+    private static final String MAX_VERSION_KEY_LABEL = "max_version";
+
+    private final String minKeyLabel;
+
+    private final long minValue;
+
+    private final String maxKeyLabel;
+
+    private final long maxValue;
+
+    protected VersionRange(String minKey, long minValue, String maxKeyLabel, long maxValue) {
+        if (minValue < 1 || maxValue < 1 || maxValue < minValue) {
+            throw new IllegalArgumentException(
+                String.format(
+                    "Expected minValue > 1, maxValue > 1 and maxValue >= minValue, but received" +
+                    " minValue: %d, maxValue: %d", minValue, maxValue));
+        }
+        this.minKeyLabel = minKey;
+        this.minValue = minValue;
+        this.maxKeyLabel = maxKeyLabel;
+        this.maxValue = maxValue;
+    }
+
+    public VersionRange(long minVersion, long maxVersion) {
+        this(MIN_VERSION_KEY_LABEL, minVersion, MAX_VERSION_KEY_LABEL, maxVersion);
+    }
+
+    public long min() {
+        return minValue;
+    }
+
+    public long max() {
+        return maxValue;
+    }
+
+    public String toString() {
+        return String.format("%s[%d, %d]", this.getClass().getSimpleName(), min(), max());
+    }
+
+    public Map<String, Long> serialize() {
+        return new HashMap<String, Long>() {
+            {
+                put(minKeyLabel, min());
+                put(maxKeyLabel, max());
+            }
+        };
+    }
+
+    public static VersionRange deserialize(Map<String, Long> serialized) {
+        return new VersionRange(
+            valueOrThrow(MIN_VERSION_KEY_LABEL, serialized),
+            valueOrThrow(MAX_VERSION_KEY_LABEL, serialized));
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (this == other) {
+            return true;
+        }
+        if (!(other instanceof VersionRange)) {
+            return false;
+        }
+
+        final VersionRange that = (VersionRange) other;
+        return Objects.equals(this.minKeyLabel, that.minKeyLabel) &&

Review comment:
       It provides slightly better convenience: `Object.equals` will also take care of the `null` checks for you.
   Also it turned out it was overkill to use `Objects.equals` for primitive type checks for `minValue` and `maxValue`. So I've simplified the code to use `==` those attributes.
   
   Good point!

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ */
+public class Features<VersionRangeType extends VersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to VersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "supported" features.
+     */
+    public static Features<VersionRange> supportedFeatures(Map<String, VersionRange> features) {
+        return new Features<VersionRange>(features);
+    }
+
+    /**
+     * @param features   Map of feature name to VersionLevelRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "finalized" features.
+     */
+    public static Features<VersionLevelRange> finalizedFeatures(Map<String, VersionLevelRange> features) {
+        return new Features<VersionLevelRange>(features);
+    }
+
+    public static Features<VersionLevelRange> emptyFinalizedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public static Features<VersionRange> emptySupportedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+
+    public Map<String, VersionRangeType> all() {
+        return features;
+    }
+
+    public boolean empty() {
+        return features.isEmpty();
+    }
+
+    public VersionRangeType get(String feature) {
+        return all().get(feature);
+    }
+
+    public String toString() {
+        return String.format(
+            "Features{%s}",
+            features
+                .entrySet()
+                .stream()
+                .map(entry -> String.format("(%s -> %s)", entry.getKey(), entry.getValue()))
+                .collect(joining(", "))
+        );
+    }
+
+    /**
+     * @return   Serializes the underlying features to a map, and returns the same.
+     *           The returned value can be deserialized using one of the deserialize* APIs.
+     */
+    public Map<String, Map<String, Long>> serialize() {
+        return features.entrySet().stream().collect(
+            Collectors.toMap(
+                Map.Entry::getKey,
+                entry -> entry.getValue().serialize()));
+    }
+
+    /**
+     * Deserializes a map to Features<VersionLevelRange>.

Review comment:
       Done.

##########
File path: clients/src/main/resources/common/message/ApiVersionsResponse.json
##########
@@ -42,6 +42,33 @@
         "about": "The maximum supported version, inclusive." }
     ]},
     { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true,
-      "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }
+      "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." },

Review comment:
       Are you sure? All newly added fields are tagged (i.e. optional).
   Going by [this documentation](https://cwiki.apache.org/confluence/display/KAFKA/KIP-482%3A+The+Kafka+Protocol+should+Support+Optional+Tagged+Fields#KIP-482:TheKafkaProtocolshouldSupportOptionalTaggedFields-TaggedFields) in KIP-482, it is not required to change the schema version whenever tagged fields are introduced.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ */
+public class Features<VersionRangeType extends VersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to VersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "supported" features.
+     */
+    public static Features<VersionRange> supportedFeatures(Map<String, VersionRange> features) {
+        return new Features<VersionRange>(features);
+    }
+
+    /**
+     * @param features   Map of feature name to VersionLevelRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "finalized" features.
+     */
+    public static Features<VersionLevelRange> finalizedFeatures(Map<String, VersionLevelRange> features) {
+        return new Features<VersionLevelRange>(features);
+    }
+
+    public static Features<VersionLevelRange> emptyFinalizedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public static Features<VersionRange> emptySupportedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+
+    public Map<String, VersionRangeType> all() {
+        return features;
+    }
+
+    public boolean empty() {
+        return features.isEmpty();
+    }
+
+    public VersionRangeType get(String feature) {
+        return all().get(feature);
+    }
+
+    public String toString() {
+        return String.format(
+            "Features{%s}",
+            features
+                .entrySet()
+                .stream()
+                .map(entry -> String.format("(%s -> %s)", entry.getKey(), entry.getValue()))
+                .collect(joining(", "))
+        );
+    }
+
+    /**
+     * @return   Serializes the underlying features to a map, and returns the same.

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/cluster/Broker.scala
##########
@@ -34,14 +36,19 @@ object Broker {
                                          brokerId: Int,
                                          endpoints: util.List[Endpoint],
                                          interBrokerEndpoint: Endpoint) extends AuthorizerServerInfo
+
+  def apply(id: Int, endPoints: Seq[EndPoint], rack: Option[String]): Broker = {
+    new Broker(id, endPoints, rack, emptySupportedFeatures)
+  }
 }
 
 /**
  * A Kafka broker.
- * A broker has an id, a collection of end-points, an optional rack and a listener to security protocol map.
+ * A broker has an id, a collection of end-points, an optional rack and a listener to security protocol map,

Review comment:
       Done. Good point!

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ */
+public class Features<VersionRangeType extends VersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to VersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "supported" features.
+     */
+    public static Features<VersionRange> supportedFeatures(Map<String, VersionRange> features) {
+        return new Features<VersionRange>(features);
+    }
+
+    /**
+     * @param features   Map of feature name to VersionLevelRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "finalized" features.
+     */
+    public static Features<VersionLevelRange> finalizedFeatures(Map<String, VersionLevelRange> features) {
+        return new Features<VersionLevelRange>(features);
+    }
+
+    public static Features<VersionLevelRange> emptyFinalizedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public static Features<VersionRange> emptySupportedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+
+    public Map<String, VersionRangeType> all() {

Review comment:
       Done. Good point!

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -113,14 +141,26 @@ public static ApiVersionsResponse fromStruct(Struct struct, short version) {
         }
     }
 
-    public static ApiVersionsResponse apiVersionsResponse(int throttleTimeMs, byte maxMagic) {
+    public static ApiVersionsResponse apiVersionsResponse(

Review comment:
       Done. Good point!

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/VersionRange.java
##########
@@ -0,0 +1,104 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Represents an immutable basic version range using 2 attributes: min and max of type long.
+ * The min and max attributes are expected to be >= 1, and with max >= min.
+ *
+ * The class also provides API to serialize/deserialize the version range to/from a map.
+ * The class allows for configurable labels for the min/max attributes, which can be specialized by
+ * sub-classes (if needed).
+ *
+ * NOTE: This is the backing class used to define the min/max versions for supported features.
+ */
+public class VersionRange {
+    // Label for the min version key, that's used only for serialization/deserialization purposes.
+    private static final String MIN_VERSION_KEY_LABEL = "min_version";
+
+    // Label for the max version key, that's used only for serialization/deserialization purposes.
+    private static final String MAX_VERSION_KEY_LABEL = "max_version";
+
+    private final String minKeyLabel;
+
+    private final long minValue;
+
+    private final String maxKeyLabel;
+
+    private final long maxValue;
+
+    protected VersionRange(String minKey, long minValue, String maxKeyLabel, long maxValue) {
+        if (minValue < 1 || maxValue < 1 || maxValue < minValue) {
+            throw new IllegalArgumentException(
+                String.format(
+                    "Expected minValue > 1, maxValue > 1 and maxValue >= minValue, but received" +
+                    " minValue: %d, maxValue: %d", minValue, maxValue));
+        }
+        this.minKeyLabel = minKey;
+        this.minValue = minValue;
+        this.maxKeyLabel = maxKeyLabel;
+        this.maxValue = maxValue;
+    }
+
+    public VersionRange(long minVersion, long maxVersion) {
+        this(MIN_VERSION_KEY_LABEL, minVersion, MAX_VERSION_KEY_LABEL, maxVersion);
+    }
+
+    public long min() {
+        return minValue;
+    }
+
+    public long max() {
+        return maxValue;
+    }
+
+    public String toString() {
+        return String.format("%s[%d, %d]", this.getClass().getSimpleName(), min(), max());
+    }
+
+    public Map<String, Long> serialize() {
+        return new HashMap<String, Long>() {
+            {
+                put(minKeyLabel, min());
+                put(maxKeyLabel, max());
+            }
+        };
+    }
+
+    public static VersionRange deserialize(Map<String, Long> serialized) {
+        return new VersionRange(
+            valueOrThrow(MIN_VERSION_KEY_LABEL, serialized),
+            valueOrThrow(MAX_VERSION_KEY_LABEL, serialized));
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (this == other) {
+            return true;
+        }
+        if (!(other instanceof VersionRange)) {

Review comment:
       Done. Also added a test. Good catch!

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ */
+public class Features<VersionRangeType extends VersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to VersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "supported" features.
+     */
+    public static Features<VersionRange> supportedFeatures(Map<String, VersionRange> features) {
+        return new Features<VersionRange>(features);
+    }
+
+    /**
+     * @param features   Map of feature name to VersionLevelRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "finalized" features.
+     */
+    public static Features<VersionLevelRange> finalizedFeatures(Map<String, VersionLevelRange> features) {
+        return new Features<VersionLevelRange>(features);
+    }
+
+    public static Features<VersionLevelRange> emptyFinalizedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public static Features<VersionRange> emptySupportedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+
+    public Map<String, VersionRangeType> all() {
+        return features;
+    }
+
+    public boolean empty() {
+        return features.isEmpty();
+    }
+
+    public VersionRangeType get(String feature) {
+        return all().get(feature);

Review comment:
       Done. Good point!

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -77,6 +93,18 @@ public boolean shouldClientThrottle(short version) {
         return version >= 2;
     }
 
+    public SupportedFeatureKey supportedFeature(String featureName) {

Review comment:
       Done.

##########
File path: clients/src/test/java/org/apache/kafka/common/feature/VersionLevelRangeTest.java
##########
@@ -0,0 +1,162 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class VersionLevelRangeTest {
+
+    @Test
+    public void testCreateFailDueToInvalidParams() {
+        // min and max can't be < 1.

Review comment:
       Done. Some of it is not required. Good point, I have removed the unnecessary testing now.
   We still need to check if exception is thrown in these 4 basic tests: min < 1, max < 1, min & max < 1 and max > min.

##########
File path: clients/src/test/java/org/apache/kafka/common/feature/FeaturesTest.java
##########
@@ -0,0 +1,216 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThrows;
+
+public class FeaturesTest {
+
+    @Test
+    public void testEmptyFeatures() {
+        Map<String, Map<String, Long>> emptyMap = new HashMap<>();
+
+        Features<VersionLevelRange> emptyFinalizedFeatures = Features.emptyFinalizedFeatures();
+        assertEquals(new HashMap<>(), emptyFinalizedFeatures.all());
+        assertEquals(emptyMap, emptyFinalizedFeatures.serialize());
+        assertEquals(emptyFinalizedFeatures, Features.deserializeFinalizedFeatures(emptyMap));
+
+        Features<VersionRange> emptySupportedFeatures = Features.emptySupportedFeatures();
+        assertEquals(new HashMap<>(), emptySupportedFeatures.all());
+        assertEquals(
+            new HashMap<String, HashMap<String, Long>>(),
+            emptySupportedFeatures.serialize());
+        assertEquals(emptySupportedFeatures, Features.deserializeSupportedFeatures(emptyMap));
+    }
+
+    @Test
+    public void testAllAPI() {
+        VersionRange v1 = new VersionRange(1, 2);
+        VersionRange v2 = new VersionRange(3, 4);
+        Map<String, VersionRange> allFeatures = new HashMap<String, VersionRange>() {
+            {
+                put("feature_1", v1);
+                put("feature_2", v2);
+            }
+        };
+        Features<VersionRange> features = Features.supportedFeatures(allFeatures);
+        assertEquals(allFeatures, features.all());
+    }
+
+    @Test
+    public void testGetAPI() {
+        VersionRange v1 = new VersionRange(1, 2);
+        VersionRange v2 = new VersionRange(3, 4);
+        Map<String, VersionRange> allFeatures = new HashMap<String, VersionRange>() {

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ */
+public class Features<VersionRangeType extends VersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to VersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "supported" features.
+     */
+    public static Features<VersionRange> supportedFeatures(Map<String, VersionRange> features) {
+        return new Features<VersionRange>(features);
+    }
+
+    /**
+     * @param features   Map of feature name to VersionLevelRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "finalized" features.
+     */
+    public static Features<VersionLevelRange> finalizedFeatures(Map<String, VersionLevelRange> features) {
+        return new Features<VersionLevelRange>(features);
+    }
+
+    public static Features<VersionLevelRange> emptyFinalizedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public static Features<VersionRange> emptySupportedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+

Review comment:
       Done. Removed it.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,93 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionLevelRange}
+
+// 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[VersionLevelRange], epoch: Int) {
+
+  def isValid(newEpoch: Int): Boolean = {

Review comment:
       Done. It was unused and I have eliminated it now.

##########
File path: core/src/main/scala/kafka/cluster/Broker.scala
##########
@@ -34,14 +36,19 @@ object Broker {
                                          brokerId: Int,
                                          endpoints: util.List[Endpoint],
                                          interBrokerEndpoint: Endpoint) extends AuthorizerServerInfo
+
+  def apply(id: Int, endPoints: Seq[EndPoint], rack: Option[String]): Broker = {

Review comment:
       No, this constructor overload was simply created to avoid a churn of test code at number of places adding the additional `SupportedFeatures` parameter. How do you feel about keeping it?

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -113,14 +141,26 @@ public static ApiVersionsResponse fromStruct(Struct struct, short version) {
         }
     }
 
-    public static ApiVersionsResponse apiVersionsResponse(int throttleTimeMs, byte maxMagic) {
+    public static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<VersionRange> latestSupportedFeatures,
+        Optional<Features<VersionLevelRange>> finalizedFeatures,
+        Optional<Long> finalizedFeaturesEpoch) {
         if (maxMagic == RecordBatch.CURRENT_MAGIC_VALUE && throttleTimeMs == DEFAULT_THROTTLE_TIME) {
             return DEFAULT_API_VERSIONS_RESPONSE;
         }
-        return createApiVersionsResponse(throttleTimeMs, maxMagic);
+        return createApiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, finalizedFeatures, finalizedFeaturesEpoch);
     }
 
-    public static ApiVersionsResponse createApiVersionsResponse(int throttleTimeMs, final byte minMagic) {
+    public static ApiVersionsResponse createApiVersionsResponse(

Review comment:
       The tests have been already added. Pls check out the tests added in `ApiVersionsResponseTest.java`, particularly: `shouldReturnFeatureKeysWhenMagicIsCurrentValueAndThrottleMsIsDefaultThrottle`.
   
   Let me know if this test does not look sufficient.

##########
File path: clients/src/test/java/org/apache/kafka/common/feature/VersionRangeTest.java
##########
@@ -0,0 +1,150 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+
+public class VersionRangeTest {
+    @Test
+    public void testFailDueToInvalidParams() {
+        // min and max can't be < 1.
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionRange(0, 0));
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionRange(-1, -1));
+        // min can't be < 1.
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionRange(0, 1));
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionRange(-1, 1));
+        // max can't be < 1.
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionRange(1, 0));
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionRange(1, -1));
+        // min can't be > max.
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionRange(2, 1));
+    }
+
+    @Test
+    public void testSerializeDeserializeTest() {
+        VersionRange versionRange = new VersionRange(1, 2);
+        assertEquals(1, versionRange.min());
+        assertEquals(2, versionRange.max());
+
+        Map<String, Long> serialized = versionRange.serialize();
+        assertEquals(
+            new HashMap<String, Long>() {
+                {
+                    put("min_version", versionRange.min());
+                    put("max_version", versionRange.max());
+                }
+            },
+            serialized
+        );
+
+        VersionRange deserialized = VersionRange.deserialize(serialized);
+        assertEquals(1, deserialized.min());
+        assertEquals(2, deserialized.max());
+        assertEquals(versionRange, deserialized);
+    }
+
+    @Test
+    public void testDeserializationFailureTest() {
+        // min_version can't be < 1.
+        Map<String, Long> invalidWithBadMinVersion = new HashMap<String, Long>() {
+            {
+                put("min_version", 0L);
+                put("max_version", 1L);
+            }
+        };
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> VersionRange.deserialize(invalidWithBadMinVersion));
+
+        // max_version can't be < 1.
+        Map<String, Long> invalidWithBadMaxVersion = new HashMap<String, Long>() {
+            {
+                put("min_version", 1L);
+                put("max_version", 0L);
+            }
+        };
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> VersionRange.deserialize(invalidWithBadMaxVersion));
+
+        // min_version and max_version can't be < 1.
+        Map<String, Long> invalidWithBadMinMaxVersion = new HashMap<String, Long>() {
+            {
+                put("min_version", 0L);
+                put("max_version", 0L);
+            }
+        };
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> VersionRange.deserialize(invalidWithBadMinMaxVersion));
+
+        // min_version can't be > max_version.
+        Map<String, Long> invalidWithLowerMaxVersion = new HashMap<String, Long>() {
+            {
+                put("min_version", 2L);
+                put("max_version", 1L);
+            }
+        };
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> VersionRange.deserialize(invalidWithLowerMaxVersion));
+
+        // min_version key missing.
+        Map<String, Long> invalidWithMinKeyMissing = new HashMap<String, Long>() {
+            {
+                put("max_version", 1L);
+            }
+        };
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> VersionRange.deserialize(invalidWithMinKeyMissing));
+
+        // max_version key missing.
+        Map<String, Long> invalidWithMaxKeyMissing = new HashMap<String, Long>() {
+            {
+                put("min_version", 1L);
+            }
+        };
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> VersionRange.deserialize(invalidWithMaxKeyMissing));
+    }
+
+    @Test
+    public void testToString() {
+        assertEquals("VersionRange[1, 1]", new VersionRange(1, 1).toString());
+        assertEquals("VersionRange[1, 2]", new VersionRange(1, 2).toString());
+    }
+
+    @Test
+    public void testEquals() {
+        assertTrue(new VersionRange(1, 1).equals(new VersionRange(1, 1)));
+        assertFalse(new VersionRange(1, 1).equals(new VersionRange(1, 2)));
+    }
+
+    @Test
+    public void testGetters() {

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/SupportedFeatures.scala
##########
@@ -0,0 +1,70 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionRange, VersionLevelRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A common 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.
+ */
+object SupportedFeatures extends Logging {
+  /**
+   * This is the latest features supported by the Broker.
+   * This is currently empty, but in the future as we define supported features, this map should be
+   * populated.
+   */
+  @volatile private var supportedFeatures = emptySupportedFeatures
+
+  /**
+   * Returns the latest features supported by the Broker.
+   */
+  def get: Features[VersionRange] = {
+    supportedFeatures
+  }
+
+  // Should be used only for testing.
+  def update(newFeatures: Features[VersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // Should be used only for testing.
+  def clear(): Unit = {
+    supportedFeatures = emptySupportedFeatures
+  }
+
+  /**
+   * Returns the set of feature names found to be incompatible between the latest features supported
+   * by the Broker, and the provided cluster-wide finalized features.
+   *
+   * @param finalized   The finalized features against which incompatibilities need to be checked for.
+   *
+   * @return            The set of incompatible feature names. If the returned set is empty, it
+   *                    means there were no feature incompatibilities found.
+   */
+  def incompatibleFeatures(finalized: Features[VersionLevelRange]): Set[String] = {
+    val supported = get
+
+    val incompatibilities = finalized.all.asScala.collect {
+      case (feature, versionLevels) => {
+        val supportedVersions = supported.get(feature);
+        if (supportedVersions == null) {
+          (feature, "{feature=%s, reason='Unsupported feature'}".format(feature))
+        } else if (!versionLevels.isCompatibleWith(supportedVersions)) {

Review comment:
       Done.

##########
File path: clients/src/test/java/org/apache/kafka/common/feature/VersionLevelRangeTest.java
##########
@@ -0,0 +1,162 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class VersionLevelRangeTest {
+
+    @Test
+    public void testCreateFailDueToInvalidParams() {
+        // min and max can't be < 1.
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionLevelRange(0, 0));
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionLevelRange(-1, -1));
+        // min can't be < 1.
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionLevelRange(0, 1));
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionLevelRange(-1, 1));
+        // max can't be < 1.
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionLevelRange(1, 0));
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionLevelRange(1, -1));
+        // min can't be > max.
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionLevelRange(2, 1));
+    }
+
+    @Test
+    public void testSerializeDeserialize() {
+        VersionLevelRange versionLevelRange = new VersionLevelRange(1, 2);
+        assertEquals(1, versionLevelRange.min());
+        assertEquals(2, versionLevelRange.max());
+
+        Map<String, Long> serialized = versionLevelRange.serialize();
+        assertEquals(
+            new HashMap<String, Long>() {
+                {
+                    put("min_version_level", versionLevelRange.min());
+                    put("max_version_level", versionLevelRange.max());
+                }
+            },
+            serialized
+        );
+
+        VersionLevelRange deserialized = VersionLevelRange.deserialize(serialized);
+        assertEquals(1, deserialized.min());
+        assertEquals(2, deserialized.max());
+        assertEquals(versionLevelRange, deserialized);
+    }
+
+    @Test
+    public void testDeserializationFailureTest() {
+        // min_version_level can't be < 1.
+        Map<String, Long> invalidWithBadMinVersion = new HashMap<String, Long>() {
+            {
+                put("min_version_level", 0L);
+                put("max_version_level", 1L);
+            }
+        };
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> VersionLevelRange.deserialize(invalidWithBadMinVersion));
+
+        // max_version_level can't be < 1.
+        Map<String, Long> invalidWithBadMaxVersion = new HashMap<String, Long>() {
+            {
+                put("min_version_level", 1L);
+                put("max_version_level", 0L);
+            }
+        };
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> VersionLevelRange.deserialize(invalidWithBadMaxVersion));
+
+        // min_version_level and max_version_level can't be < 1.
+        Map<String, Long> invalidWithBadMinMaxVersion = new HashMap<String, Long>() {
+            {
+                put("min_version_level", 0L);
+                put("max_version_level", 0L);
+            }
+        };
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> VersionLevelRange.deserialize(invalidWithBadMinMaxVersion));
+
+        // min_version_level can't be > max_version_level.
+        Map<String, Long> invalidWithLowerMaxVersion = new HashMap<String, Long>() {
+            {
+                put("min_version_level", 2L);
+                put("max_version_level", 1L);
+            }
+        };
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> VersionLevelRange.deserialize(invalidWithLowerMaxVersion));
+
+        // min_version_level key missing.
+        Map<String, Long> invalidWithMinKeyMissing = new HashMap<String, Long>() {
+            {
+                put("max_version_level", 1L);
+            }
+        };
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> VersionLevelRange.deserialize(invalidWithMinKeyMissing));
+
+        // max_version_level key missing.
+        Map<String, Long> invalidWithMaxKeyMissing = new HashMap<String, Long>() {
+            {
+                put("min_version_level", 1L);
+            }
+        };
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> VersionLevelRange.deserialize(invalidWithMaxKeyMissing));
+    }
+
+    @Test
+    public void testToString() {
+        assertEquals("VersionLevelRange[1, 1]", new VersionLevelRange(1, 1).toString());
+        assertEquals("VersionLevelRange[1, 2]", new VersionLevelRange(1, 2).toString());
+    }
+
+    @Test
+    public void testEquals() {
+        assertTrue(new VersionLevelRange(1, 1).equals(new VersionLevelRange(1, 1)));
+        assertFalse(new VersionLevelRange(1, 1).equals(new VersionLevelRange(1, 2)));
+    }
+
+    @Test
+    public void testIsCompatibleWith() {
+        assertTrue(new VersionLevelRange(1, 1).isCompatibleWith(new VersionRange(1, 1)));
+        assertTrue(new VersionLevelRange(2, 3).isCompatibleWith(new VersionRange(1, 4)));
+        assertTrue(new VersionLevelRange(1, 4).isCompatibleWith(new VersionRange(1, 4)));
+
+        assertFalse(new VersionLevelRange(1, 4).isCompatibleWith(new VersionRange(2, 3)));
+        assertFalse(new VersionLevelRange(1, 4).isCompatibleWith(new VersionRange(2, 4)));
+        assertFalse(new VersionLevelRange(2, 4).isCompatibleWith(new VersionRange(2, 3)));
+    }
+
+    @Test
+    public void testGetters() {
+        assertEquals(1, new VersionLevelRange(1, 2).min());
+        assertEquals(2, new VersionLevelRange(1, 2).max());
+    }
+}

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,93 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionLevelRange}
+
+// 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[VersionLevelRange], epoch: Int) {
+
+  def isValid(newEpoch: Int): Boolean = {
+    newEpoch >= epoch
+  }
+
+  override def toString(): String = {
+    "FinalizedFeaturesAndEpoch(features=%s, epoch=%s)".format(features, epoch)
+  }
+}
+
+/**
+ * A mutable cache containing the latest finalized features and epoch. This cache is populated by a
+ * FinalizedFeatureChangeListener.
+ *
+ * Currently the main reader of this cache is the read path that serves an ApiVersionsRequest
+ * returning the features information in the response. In the future, as the feature versioning
+ * system in KIP-584 is used more widely, this cache could be read by other read paths trying to
+ * learn the finalized feature information.
+ */
+object FinalizedFeatureCache extends Logging {
+  @volatile private var featuresAndEpoch: Option[FinalizedFeaturesAndEpoch] = Option.empty
+
+  /**
+   * @return   the latest known FinalizedFeaturesAndEpoch. If the returned value is empty, it means
+   *           no FinalizedFeaturesAndEpoch exists in the cache at the time when this
+   *           method is invoked. This result could change in the future whenever the
+   *           updateOrThrow method is invoked.
+   */
+  def get: Option[FinalizedFeaturesAndEpoch] = {
+    featuresAndEpoch
+  }
+
+  def empty: Boolean = {
+    featuresAndEpoch.isEmpty
+  }
+
+  /**
+   * Clears all existing finalized features and epoch from the cache.
+   */
+  def clear(): Unit = {
+    featuresAndEpoch = Option.empty
+    info("Cleared cache")
+  }
+
+  /**
+   * Updates the cache to the latestFeatures, and updates the existing epoch to latestEpoch.
+   * Raises an exception when the operation is not successful.
+   *
+   * @param latestFeatures   the latest finalized features to be set in the cache
+   * @param latestEpoch      the latest epoch value to be set in the cache
+   *
+   * @throws                 FeatureCacheUpdateException if the cache update operation fails
+   *                         due to invalid parameters or incompatibilities with the broker's
+   *                         supported features. In such a case, the existing cache contents are
+   *                         not modified.
+   */
+  def updateOrThrow(latestFeatures: Features[VersionLevelRange], latestEpoch: Int): Unit = {
+    updateOrThrow(FinalizedFeaturesAndEpoch(latestFeatures, latestEpoch))
+  }
+
+  private def updateOrThrow(latest: FinalizedFeaturesAndEpoch): Unit = {
+    val existingStr = featuresAndEpoch.map(existing => existing.toString).getOrElse("<empty>")

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,93 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionLevelRange}
+
+// 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[VersionLevelRange], epoch: Int) {
+
+  def isValid(newEpoch: Int): Boolean = {
+    newEpoch >= epoch
+  }
+
+  override def toString(): String = {
+    "FinalizedFeaturesAndEpoch(features=%s, epoch=%s)".format(features, epoch)
+  }
+}
+
+/**
+ * A mutable cache containing the latest finalized features and epoch. This cache is populated by a
+ * FinalizedFeatureChangeListener.

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -81,17 +83,27 @@ object BrokerIdsZNode {
 object BrokerInfo {
 
   /**
-   * Create a broker info with v4 json format (which includes multiple endpoints and rack) if
-   * the apiVersion is 0.10.0.X or above. Register the broker with v2 json format otherwise.
+   * - Create a broker info with v5 json format if the apiVersion is 2.6.x or above.
+   * - Create a broker info with v4 json format (which includes multiple endpoints and rack) if
+   *   the apiVersion is 0.10.0.X or above but lesser than 2.6.x.
+   * - Register the broker with v2 json format otherwise.
    *
    * Due to KAFKA-3100, 0.9.0.0 broker and old clients will break if JSON version is above 2.
    *
-   * We include v2 to make it possible for the broker to migrate from 0.9.0.0 to 0.10.0.X or above without having to
-   * upgrade to 0.9.0.1 first (clients have to be upgraded to 0.9.0.1 in any case).
+   * We include v2 to make it possible for the broker to migrate from 0.9.0.0 to 0.10.0.X or above
+   * without having to upgrade to 0.9.0.1 first (clients have to be upgraded to 0.9.0.1 in
+   * any case).
    */
   def apply(broker: Broker, apiVersion: ApiVersion, jmxPort: Int): BrokerInfo = {
-    // see method documentation for the reason why we do this
-    val version = if (apiVersion >= KAFKA_0_10_0_IV1) 4 else 2
+    val version = {

Review comment:
       Done. Good point!

##########
File path: core/src/main/scala/kafka/server/SupportedFeatures.scala
##########
@@ -0,0 +1,70 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionRange, VersionLevelRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A common 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.
+ */
+object SupportedFeatures extends Logging {
+  /**
+   * This is the latest features supported by the Broker.
+   * This is currently empty, but in the future as we define supported features, this map should be
+   * populated.
+   */
+  @volatile private var supportedFeatures = emptySupportedFeatures
+
+  /**
+   * Returns the latest features supported by the Broker.
+   */
+  def get: Features[VersionRange] = {
+    supportedFeatures
+  }
+
+  // Should be used only for testing.
+  def update(newFeatures: Features[VersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // Should be used only for testing.
+  def clear(): Unit = {
+    supportedFeatures = emptySupportedFeatures
+  }
+
+  /**
+   * Returns the set of feature names found to be incompatible between the latest features supported
+   * by the Broker, and the provided cluster-wide finalized features.
+   *
+   * @param finalized   The finalized features against which incompatibilities need to be checked for.
+   *
+   * @return            The set of incompatible feature names. If the returned set is empty, it
+   *                    means there were no feature incompatibilities found.
+   */
+  def incompatibleFeatures(finalized: Features[VersionLevelRange]): Set[String] = {
+    val supported = get

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/SupportedFeatures.scala
##########
@@ -0,0 +1,70 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionRange, VersionLevelRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A common 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.
+ */
+object SupportedFeatures extends Logging {

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/SupportedFeatures.scala
##########
@@ -0,0 +1,70 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionRange, VersionLevelRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A common 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.
+ */
+object SupportedFeatures extends Logging {
+  /**
+   * This is the latest features supported by the Broker.
+   * This is currently empty, but in the future as we define supported features, this map should be
+   * populated.
+   */
+  @volatile private var supportedFeatures = emptySupportedFeatures
+
+  /**
+   * Returns the latest features supported by the Broker.
+   */
+  def get: Features[VersionRange] = {
+    supportedFeatures
+  }
+
+  // Should be used only for testing.
+  def update(newFeatures: Features[VersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // Should be used only for testing.
+  def clear(): Unit = {
+    supportedFeatures = emptySupportedFeatures
+  }
+
+  /**
+   * Returns the set of feature names found to be incompatible between the latest features supported
+   * by the Broker, and the provided cluster-wide finalized features.
+   *
+   * @param finalized   The finalized features against which incompatibilities need to be checked for.

Review comment:
       Good point. I have improved the doc now. Let me know how you feel about it.

##########
File path: core/src/main/scala/kafka/zk/KafkaZkClient.scala
##########
@@ -1567,6 +1567,36 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
     createRecursive(path, data = null, throwIfPathExists = false)
   }
 
+  // Visible for testing.
+  def createFeatureZNode(nodeContents: FeatureZNode): Unit = {

Review comment:
       Yes, this will get used in the future. For example the write path will use it.

##########
File path: core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala
##########
@@ -149,6 +153,53 @@ class BrokerEndPointTest {
     assertEquals(None, broker.rack)
   }
 
+  @Test
+  def testFromJsonV5(): Unit = {

Review comment:
       In my understanding, this is an impossible case. Because, we always write features into the JSON only in v5 or above. That is why, there is no test for it. Let me know how you feel about it.

##########
File path: core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala
##########
@@ -149,6 +153,53 @@ class BrokerEndPointTest {
     assertEquals(None, broker.rack)
   }
 
+  @Test
+  def testFromJsonV5(): Unit = {
+    val json = """{
+      "version":5,
+      "host":"localhost",
+      "port":9092,
+      "jmx_port":9999,
+      "timestamp":"2233345666",
+      "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}}
+    }"""
+    val broker = parseBrokerJson(1, json)
+    assertEquals(1, broker.id)
+    val brokerEndPoint = broker.brokerEndPoint(new ListenerName("CLIENT"))
+    assertEquals("host1", brokerEndPoint.host)
+    assertEquals(9092, brokerEndPoint.port)
+    assertEquals(Some("dc1"), broker.rack)
+    assertEquals(Features.supportedFeatures(
+      Map[String, VersionRange](
+        "feature1" -> new VersionRange(1, 2),
+        "feature2" -> new VersionRange(2, 4)).asJava),
+      broker.features)
+  }
+
+  @Test
+  def testFromJsonV4WithNoFeatures(): Unit = {

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,93 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionLevelRange}
+
+// 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[VersionLevelRange], epoch: Int) {
+
+  def isValid(newEpoch: Int): Boolean = {
+    newEpoch >= epoch
+  }
+
+  override def toString(): String = {
+    "FinalizedFeaturesAndEpoch(features=%s, epoch=%s)".format(features, epoch)
+  }
+}
+
+/**
+ * A mutable cache containing the latest finalized features and epoch. This cache is populated by a
+ * FinalizedFeatureChangeListener.
+ *
+ * Currently the main reader of this cache is the read path that serves an ApiVersionsRequest
+ * returning the features information in the response. In the future, as the feature versioning
+ * system in KIP-584 is used more widely, this cache could be read by other read paths trying to
+ * learn the finalized feature information.
+ */
+object FinalizedFeatureCache extends Logging {
+  @volatile private var featuresAndEpoch: Option[FinalizedFeaturesAndEpoch] = Option.empty
+
+  /**
+   * @return   the latest known FinalizedFeaturesAndEpoch. If the returned value is empty, it means
+   *           no FinalizedFeaturesAndEpoch exists in the cache at the time when this
+   *           method is invoked. This result could change in the future whenever the
+   *           updateOrThrow method is invoked.
+   */
+  def get: Option[FinalizedFeaturesAndEpoch] = {
+    featuresAndEpoch
+  }
+
+  def empty: Boolean = {
+    featuresAndEpoch.isEmpty
+  }
+
+  /**
+   * Clears all existing finalized features and epoch from the cache.
+   */
+  def clear(): Unit = {
+    featuresAndEpoch = Option.empty
+    info("Cleared cache")
+  }
+
+  /**
+   * Updates the cache to the latestFeatures, and updates the existing epoch to latestEpoch.
+   * Raises an exception when the operation is not successful.
+   *
+   * @param latestFeatures   the latest finalized features to be set in the cache
+   * @param latestEpoch      the latest epoch value to be set in the cache
+   *
+   * @throws                 FeatureCacheUpdateException if the cache update operation fails
+   *                         due to invalid parameters or incompatibilities with the broker's
+   *                         supported features. In such a case, the existing cache contents are
+   *                         not modified.
+   */
+  def updateOrThrow(latestFeatures: Features[VersionLevelRange], latestEpoch: Int): Unit = {
+    updateOrThrow(FinalizedFeaturesAndEpoch(latestFeatures, latestEpoch))
+  }
+
+  private def updateOrThrow(latest: FinalizedFeaturesAndEpoch): Unit = {
+    val existingStr = featuresAndEpoch.map(existing => existing.toString).getOrElse("<empty>")
+    if (!featuresAndEpoch.isEmpty && featuresAndEpoch.get.epoch > latest.epoch) {
+      val errorMsg = ("FinalizedFeatureCache update failed due to invalid epoch in new finalized %s." +
+        " The existing finalized is %s").format(latest, existingStr)
+      throw new FeatureCacheUpdateException(errorMsg)
+    } else {
+      val incompatibleFeatures = SupportedFeatures.incompatibleFeatures(latest.features)
+      if (incompatibleFeatures.nonEmpty) {
+        val errorMsg = ("FinalizedFeatureCache updated failed since feature compatibility" +
+          " checks failed! Supported %s has incompatibilities with the latest finalized %s." +
+          " The incompatible features are: %s.").format(
+          SupportedFeatures.get, latest, incompatibleFeatures)
+        throw new FeatureCacheUpdateException(errorMsg)
+      }
+    }
+    val logMsg = "Updated cache from existing finalized %s to latest finalized %s".format(

Review comment:
       It is used intentionally to split the log message into 2 lines (for ~100-char readability limit per line). Otherwise the string will be huge and all in the same line.

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -225,7 +255,12 @@ object BrokerIdZNode {
           }
 
         val rack = brokerInfo.get(RackKey).flatMap(_.to[Option[String]])
-        BrokerInfo(Broker(id, endpoints, rack), version, jmxPort)
+        val features = FeatureZNode.asJavaMap(brokerInfo

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,200 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             - RuntimeException if the thread was interrupted during wait
+     *                     - TimeoutException if the wait can not be completed in waitTimeMs
+     *                       milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    private def processNotification(): Unit = {
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")
+      processNotification()
+    }
+
+    override def handleDataChange(): Unit = {
+      info(s"Feature ZK node updated at path: $path")
+      processNotification()
+    }
+
+    override def handleDeletion(): Unit = {
+      warn(s"Feature ZK node deleted at path: $path")
+      processNotification()

Review comment:
       I have added comments now to the code.
   
   The idea I had was that this event may happen, rarely (ex: operational error).
   In such a case, we do not want to kill the brokers, so we just log a warning and treat the case as if the node is absent, and populate the cache with empty features.
   
   So, this case is actually handled inside `FeatureCacheUpdater.updateLatestOrThrow()`.
   The call to read ZK node will return `ZkVersion.UnknownVersion` whenever the node does not exist in ZK, and I've explicitly handled this returned version.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,93 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionLevelRange}
+
+// Raised whenever there was an error in updating the FinalizedFeatureCache with features.

Review comment:
       As we discussed over slack today, this exception is already handled in `ChangeNotificationProcessorThread.doWork()` method defined in `FinalizedFeatureChangeListener.scala`. Basically, the ZK change notification processor thread exits the Broker with a fatal error (non-zero exit code) when this exception (or any exception) is caught while trying to update `FinalizedFeatureCache`.

##########
File path: core/src/main/scala/kafka/server/KafkaServer.scala
##########
@@ -210,6 +215,14 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP
         /* setup zookeeper */
         initZkClient(time)
 
+        /* initialize features */
+        _featureChangeListener = new FinalizedFeatureChangeListener(_zkClient)
+        if (config.interBrokerProtocolVersion >= KAFKA_2_6_IV1) {
+          // The feature versioning system (KIP-584) is active only when:
+          // config.interBrokerProtocolVersion is >= KAFKA_2_6_IV1.
+          _featureChangeListener.initOrThrow(60000)

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/VersionLevelRange.java
##########
@@ -0,0 +1,39 @@
+package org.apache.kafka.common.feature;
+
+import java.util.Map;
+
+/**
+ * A specialization of VersionRange representing a range of version levels. The main specialization
+ * is that the class uses different serialization keys for min/max attributes.
+ *
+ * NOTE: This is the backing class used to define the min/max version levels for finalized features.
+ */
+public class VersionLevelRange extends VersionRange {

Review comment:
       Done. Good point!
    - I have now created 3 classes as you proposed. `BaseVersionRange` is the base class, and, `SupportedVersionRange` & `FinalizedVersionRange` are it's child classes.
    - The key labels couldn't be made into abstract functions since these constants are needed within `deserialize()` which is a static method defined in the child classes.




----------------------------------------------------------------
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 #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -22,7 +22,7 @@ import java.util.Properties
 
 import com.fasterxml.jackson.annotation.JsonProperty
 import com.fasterxml.jackson.core.JsonProcessingException
-import kafka.api.{ApiVersion, KAFKA_0_10_0_IV1, LeaderAndIsr}
+import kafka.api.{ApiVersion, KAFKA_0_10_0_IV1, KAFKA_2_6_IV1, LeaderAndIsr}

Review comment:
       This won't make 2.6.0 release. So, perhaps we should use KAFKA_2_7 or whatever the next release is?

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Represents an immutable basic version range using 2 attributes: min and max, each of type long.
+ * The min and max attributes need to satisfy 2 rules:
+ *  - they are each expected to be >= 1, as we only consider positive version values to be valid.
+ *  - max should be >= min.
+ *
+ * The class also provides API to serialize/deserialize the version range to/from a map.
+ * The class allows for configurable labels for the min/max attributes, which can be specialized by
+ * sub-classes (if needed).
+ */
+class BaseVersionRange {
+    // Non-empty label for the min version key, that's used only for serialization/deserialization purposes.
+    private final String minKeyLabel;
+
+    // The value of the minimum version.
+    private final long minValue;

Review comment:
       I missed this in the KIP, but it seems that long is overkilling for version. The version in request is short and the version in ZK data is int. So, perhaps this should just be short?

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,85 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange}
+
+// 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) {
+  override def toString(): String = {
+    "FinalizedFeaturesAndEpoch(features=%s, epoch=%s)".format(features, epoch)
+  }
+}
+
+/**
+ * A common mutable cache containing the latest finalized features and epoch. By default the contents of
+ * the cache are empty. This cache needs to be populated at least once for it's contents to become
+ * non-empty. Currently the main reader of this cache is the read path that serves an ApiVersionsRequest,
+ * returning the features information in the response.
+ *
+ * @see FinalizedFeatureChangeListener
+ */
+object FinalizedFeatureCache extends Logging {
+  @volatile private var featuresAndEpoch: Option[FinalizedFeaturesAndEpoch] = Option.empty
+
+  /**
+   * @return   the latest known FinalizedFeaturesAndEpoch. If the returned value is empty, it means
+   *           no FinalizedFeaturesAndEpoch exists in the cache at the time when this
+   *           method is invoked. This result could change in the future whenever the
+   *           updateOrThrow method is invoked.
+   */
+  def get: Option[FinalizedFeaturesAndEpoch] = {
+    featuresAndEpoch
+  }
+
+  def empty: Boolean = {

Review comment:
       empty => isEmpty ?

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,219 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked exactly
+     * once successfully. A subsequent invocation will raise an exception.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     * @throws   FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     * @throws   RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      debug(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status, clearing feature cache.")
+          FinalizedFeatureCache.clear()
+        } else if (featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully. This method returns
+     * immediately if an updateLatestOrThrow call had already completed successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             RuntimeException if the thread was interrupted during wait
+     *
+     *                     TimeoutException if the wait can not be completed in waitTimeMs
+     *                     milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(

Review comment:
       InterruptedException can be thrown if the thread is shut down explicitly. In this case, we probably don't want to throw RuntimeException to the caller.

##########
File path: clients/src/main/resources/common/message/ApiVersionsResponse.json
##########
@@ -42,6 +42,33 @@
         "about": "The maximum supported version, inclusive." }
     ]},
     { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true,
-      "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }
+      "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." },
+    { "name":  "SupportedFeatures", "type": "[]SupportedFeatureKey",
+      "versions":  "3+", "tag": 0, "taggedVersions": "3+",
+      "about": "Features supported by the broker.",
+      "fields":  [
+        { "name": "Name", "type": "string", "versions": "3+", "mapKey": true,
+          "about": "The name of the feature." },
+        { "name": "MinVersion", "type": "int64", "versions": "3+",
+          "about": "The minimum supported version for the feature." },
+        { "name": "MaxVersion", "type": "int64", "versions": "3+",
+          "about": "The maximum supported version for the feature." }
+      ]
+    },
+    {"name": "FinalizedFeaturesEpoch", "type": "int64", "versions": "3+",

Review comment:
       Hmm, the epoch returned from ZK is int32. Does FinalizedFeaturesEpoch need to be int64?

##########
File path: core/src/main/scala/kafka/cluster/Broker.scala
##########
@@ -54,7 +64,7 @@ case class Broker(id: Int, endPoints: Seq[EndPoint], rack: Option[String]) {
     s"$id : ${endPointsMap.values.mkString("(",",",")")} : ${rack.orNull}"
 
   def this(id: Int, host: String, port: Int, listenerName: ListenerName, protocol: SecurityProtocol) = {
-    this(id, Seq(EndPoint(host, port, listenerName, protocol)), None)
+    this(id, Seq(EndPoint(host, port, listenerName, protocol)), None, emptySupportedFeatures)

Review comment:
       Should we include the new field in toString()?

##########
File path: core/src/main/scala/kafka/server/KafkaConfig.scala
##########
@@ -442,6 +445,8 @@ object KafkaConfig {
   val ControlledShutdownMaxRetriesProp = "controlled.shutdown.max.retries"
   val ControlledShutdownRetryBackoffMsProp = "controlled.shutdown.retry.backoff.ms"
   val ControlledShutdownEnableProp = "controlled.shutdown.enable"
+  /** ********* Features configuration ***********/
+  val FeatureChangeListenerCacheUpdateWaitTimeMsProp = "feature.listener.cache.update.wait.ms"

Review comment:
       Hmm, could we just use config.zkConnectionTimeoutMs for this, instead of introducing a new config?

##########
File path: core/src/main/scala/kafka/server/SupportedFeatures.scala
##########
@@ -0,0 +1,76 @@
+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 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.
+ *
+ * NOTE: the update() and clear() APIs of this class should be used only for testing purposes.
+ */
+object SupportedFeatures extends Logging {
+
+  /**
+   * This is the latest features supported by the Broker.
+   * This is currently empty, but in the future as we define supported features, this map should be
+   * populated.
+   */
+  @volatile private var supportedFeatures = emptySupportedFeatures
+
+  /**
+   * Returns a reference to the latest features supported by the Broker.
+   */
+  def get: Features[SupportedVersionRange] = {
+    supportedFeatures
+  }
+
+  // For testing only.
+  def update(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  def clear(): Unit = {
+    supportedFeatures = emptySupportedFeatures
+  }
+
+  /**
+   * 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] = {
+    val incompatibilities = finalized.features.asScala.collect {
+      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 (incompatibilities.nonEmpty) {
+      warn("Feature incompatibilities seen: " + incompatibilities.map{ case(_, _, errorReason) => errorReason })
+    }
+    Features.finalizedFeatures(incompatibilities.map(item => (item._1, item._2)).toMap.asJava)

Review comment:
       Could we use map {case(feature, versionLevel, _) => ...} to avoid unnamed references like _1? 

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -744,6 +782,161 @@ object DelegationTokenInfoZNode {
   def decode(bytes: Array[Byte]): Option[TokenInformation] = DelegationTokenManager.fromBytes(bytes)
 }
 
+/**
+ * Represents the status of the FeatureZNode.
+ *
+ * Enabled  -> This status means the feature versioning system (KIP-584) is enabled, and, the
+ *             finalized features stored in the FeatureZNode are active. This status is written by
+ *             the controller to the FeatureZNode only when the broker IBP config is greater than
+ *             or equal to KAFKA_2_6_IV1.
+ *
+ * Disabled -> This status means the feature versioning system (KIP-584) is disabled, and, the
+ *             the finalized features stored in the FeatureZNode is not relevant. This status is
+ *             written by the controller to the FeatureZNode only when the broker IBP config
+ *             is less than KAFKA_2_6_IV1.
+ *
+ * The purpose behind the FeatureZNodeStatus is that it helps differentiates between the following
+ * cases:
+ *
+ * 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_6_IV1. In such a case, the
+ *    controller will start up and notice that the FeatureZNode is absent in the new cluster.
+ *    To handle the requirement, the controller will create a FeatureZNode (with enabled status)
+ *    containing the entire list of supported features as its finalized features.
+ *
+ * 2. Cluster upgrade:
+ *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_6_IV1, but
+ *    the Broker binary has been upgraded to a state where it 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 enable
+ *    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_6_IV1. The reason is that enabling all the possible
+ *    features immediately after an upgrade could be harmful to the cluster.
+ *    In such a case:
+ *      - Before the Broker upgrade (i.e. IBP config set to less than KAFKA_2_6_IV1), the controller
+ *        will start up and check if the FeatureZNode is absent. If true, then it will react by
+ *        creating a FeatureZNode with disabled status and empty features.
+ *      - After the Broker upgrade (i.e. IBP config set to greater than or equal to KAFKA_2_6_IV1),
+ *        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.
+ *
+ * 2. Cluster downgrade:
+ *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+ *    KAFKA_2_6_IV1. Then, the user decided to downgrade the cluster by setting IBP config to a
+ *    value less than KAFKA_2_6_IV1. 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.
+ */
+object FeatureZNodeStatus extends Enumeration {
+  val Disabled, Enabled = Value
+
+  def withNameOpt(value: Int): Option[Value] = {
+    values.find(_.id == value)
+  }
+}
+
+/**
+ * Represents the contents of the ZK node containing finalized feature information.
+ *
+ * @param status     the status of the ZK node
+ * @param features   the cluster-wide finalized features
+ */
+case class FeatureZNode(status: FeatureZNodeStatus.Value, features: Features[FinalizedVersionRange]) {
+}
+
+object FeatureZNode {
+  private val VersionKey = "version"
+  private val StatusKey = "status"
+  private val FeaturesKey = "features"
+
+  // V0 contains 'version', 'status' and 'features' keys.
+  val V0 = 0
+  val CurrentVersion = V0
+
+  def path = "/feature"
+
+  def asJavaMap(scalaMap: Map[String, Map[String, Long]]): util.Map[String, util.Map[String, java.lang.Long]] = {
+    scalaMap
+      .view.mapValues(_.view.mapValues(scalaLong => java.lang.Long.valueOf(scalaLong)).toMap.asJava)
+      .toMap
+      .asJava
+  }
+
+  /**
+   * Encodes a FeatureZNode to JSON.
+   *
+   * @param featureZNode   FeatureZNode to be encoded
+   *
+   * @return               JSON representation of the FeatureZNode, as an Array[Byte]
+   */
+  def encode(featureZNode: FeatureZNode): Array[Byte] = {
+    val jsonMap = collection.mutable.Map(
+      VersionKey -> CurrentVersion,
+      StatusKey -> featureZNode.status.id,

Review comment:
       The KIP doesn't seems to include this field. Could we add it to the KIP wiki?

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,85 @@
+package kafka.server

Review comment:
       missing license header

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -196,6 +219,19 @@ object BrokerIdZNode {
     *   "listener_security_protocol_map":{"CLIENT":"SSL", "REPLICATION":"PLAINTEXT"},
     *   "rack":"dc1"
     * }
+    *
+    * Version 5 (current) JSON schema for a broker is:
+    * {
+    *   "version":5,
+    *   "host":"localhost",
+    *   "port":9092,
+    *   "jmx_port":9999,
+    *   "timestamp":"2233345666",
+    *   "endpoints":["CLIENT://host1:9092", "REPLICATION://host1:9093"],
+    *   "listener_security_protocol_map":{"CLIENT":"SSL", "REPLICATION":"PLAINTEXT"},

Review comment:
       The existing comments seem incorrect since we don't store listener_security_protocol_map in ZK.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Represents an immutable basic version range using 2 attributes: min and max, each of type long.
+ * The min and max attributes need to satisfy 2 rules:
+ *  - they are each expected to be >= 1, as we only consider positive version values to be valid.
+ *  - max should be >= min.
+ *
+ * The class also provides API to serialize/deserialize the version range to/from a map.
+ * The class allows for configurable labels for the min/max attributes, which can be specialized by
+ * sub-classes (if needed).
+ */
+class BaseVersionRange {
+    // Non-empty label for the min version key, that's used only for serialization/deserialization purposes.
+    private final String minKeyLabel;
+
+    // The value of the minimum version.
+    private final long minValue;
+
+    // Non-empty label for the max version key, that's used only for serialization/deserialization purposes.
+    private final String maxKeyLabel;
+
+    // The value of the maximum version.
+    private final long maxValue;
+
+    /**
+     * Raises an exception unless the following condition is met:
+     * minValue >= 1 and maxValue >= 1 and maxValue >= minValue.
+     *
+     * @param minKeyLabel   Label for the min version key, that's used only for
+     *                      serialization/deserialization purposes.
+     * @param minValue      The minimum version value.
+     * @param maxKeyLabel   Label for the max version key, that's used only for
+     *                      serialization/deserialization purposes.
+     * @param maxValue      The maximum version value.
+     *
+     * @throws IllegalArgumentException   If any of the following conditions are true:
+     *                                     - (minValue < 1) OR (maxValue < 1) OR (maxValue < minValue).
+     *                                     - minKeyLabel is empty, OR, minKeyLabel is empty.
+     */
+    protected BaseVersionRange(String minKeyLabel, long minValue, String maxKeyLabel, long maxValue) {
+        if (minValue < 1 || maxValue < 1 || maxValue < minValue) {
+            throw new IllegalArgumentException(
+                String.format(
+                    "Expected minValue > 1, maxValue > 1 and maxValue >= minValue, but received" +
+                    " minValue: %d, maxValue: %d", minValue, maxValue));
+        }
+        if (minKeyLabel.isEmpty()) {
+            throw new IllegalArgumentException("Expected minKeyLabel to be non-empty.");
+        }
+        if (maxKeyLabel.isEmpty()) {
+            throw new IllegalArgumentException("Expected maxKeyLabel to be non-empty.");
+        }
+        this.minKeyLabel = minKeyLabel;
+        this.minValue = minValue;
+        this.maxKeyLabel = maxKeyLabel;
+        this.maxValue = maxValue;
+    }
+
+    public long min() {
+        return minValue;
+    }
+
+    public long max() {
+        return maxValue;
+    }
+
+    public String toString() {
+        return String.format("%s[%d, %d]", this.getClass().getSimpleName(), min(), max());
+    }
+
+    public Map<String, Long> serialize() {

Review comment:
       serialize typically means generating binary data. Perhaps this is better called toMap()?

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Represents an immutable basic version range using 2 attributes: min and max, each of type long.
+ * The min and max attributes need to satisfy 2 rules:
+ *  - they are each expected to be >= 1, as we only consider positive version values to be valid.
+ *  - max should be >= min.
+ *
+ * The class also provides API to serialize/deserialize the version range to/from a map.
+ * The class allows for configurable labels for the min/max attributes, which can be specialized by
+ * sub-classes (if needed).
+ */
+class BaseVersionRange {
+    // Non-empty label for the min version key, that's used only for serialization/deserialization purposes.
+    private final String minKeyLabel;
+
+    // The value of the minimum version.
+    private final long minValue;
+
+    // Non-empty label for the max version key, that's used only for serialization/deserialization purposes.
+    private final String maxKeyLabel;
+
+    // The value of the maximum version.
+    private final long maxValue;
+
+    /**
+     * Raises an exception unless the following condition is met:
+     * minValue >= 1 and maxValue >= 1 and maxValue >= minValue.
+     *
+     * @param minKeyLabel   Label for the min version key, that's used only for
+     *                      serialization/deserialization purposes.
+     * @param minValue      The minimum version value.
+     * @param maxKeyLabel   Label for the max version key, that's used only for
+     *                      serialization/deserialization purposes.
+     * @param maxValue      The maximum version value.
+     *
+     * @throws IllegalArgumentException   If any of the following conditions are true:
+     *                                     - (minValue < 1) OR (maxValue < 1) OR (maxValue < minValue).
+     *                                     - minKeyLabel is empty, OR, minKeyLabel is empty.
+     */
+    protected BaseVersionRange(String minKeyLabel, long minValue, String maxKeyLabel, long maxValue) {
+        if (minValue < 1 || maxValue < 1 || maxValue < minValue) {
+            throw new IllegalArgumentException(
+                String.format(
+                    "Expected minValue > 1, maxValue > 1 and maxValue >= minValue, but received" +
+                    " minValue: %d, maxValue: %d", minValue, maxValue));
+        }
+        if (minKeyLabel.isEmpty()) {
+            throw new IllegalArgumentException("Expected minKeyLabel to be non-empty.");
+        }
+        if (maxKeyLabel.isEmpty()) {
+            throw new IllegalArgumentException("Expected maxKeyLabel to be non-empty.");
+        }
+        this.minKeyLabel = minKeyLabel;
+        this.minValue = minValue;
+        this.maxKeyLabel = maxKeyLabel;
+        this.maxValue = maxValue;
+    }
+
+    public long min() {
+        return minValue;
+    }
+
+    public long max() {
+        return maxValue;
+    }
+
+    public String toString() {
+        return String.format("%s[%d, %d]", this.getClass().getSimpleName(), min(), max());

Review comment:
       Should we include the label too?

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Represents an immutable basic version range using 2 attributes: min and max, each of type long.
+ * The min and max attributes need to satisfy 2 rules:
+ *  - they are each expected to be >= 1, as we only consider positive version values to be valid.
+ *  - max should be >= min.
+ *
+ * The class also provides API to serialize/deserialize the version range to/from a map.
+ * The class allows for configurable labels for the min/max attributes, which can be specialized by
+ * sub-classes (if needed).
+ */
+class BaseVersionRange {
+    // Non-empty label for the min version key, that's used only for serialization/deserialization purposes.
+    private final String minKeyLabel;
+
+    // The value of the minimum version.
+    private final long minValue;
+
+    // Non-empty label for the max version key, that's used only for serialization/deserialization purposes.
+    private final String maxKeyLabel;
+
+    // The value of the maximum version.
+    private final long maxValue;
+
+    /**
+     * Raises an exception unless the following condition is met:
+     * minValue >= 1 and maxValue >= 1 and maxValue >= minValue.
+     *
+     * @param minKeyLabel   Label for the min version key, that's used only for
+     *                      serialization/deserialization purposes.
+     * @param minValue      The minimum version value.
+     * @param maxKeyLabel   Label for the max version key, that's used only for
+     *                      serialization/deserialization purposes.
+     * @param maxValue      The maximum version value.
+     *
+     * @throws IllegalArgumentException   If any of the following conditions are true:
+     *                                     - (minValue < 1) OR (maxValue < 1) OR (maxValue < minValue).
+     *                                     - minKeyLabel is empty, OR, minKeyLabel is empty.
+     */
+    protected BaseVersionRange(String minKeyLabel, long minValue, String maxKeyLabel, long maxValue) {
+        if (minValue < 1 || maxValue < 1 || maxValue < minValue) {
+            throw new IllegalArgumentException(
+                String.format(
+                    "Expected minValue > 1, maxValue > 1 and maxValue >= minValue, but received" +

Review comment:
       minValue > 1, maxValue > 1 => minValue >= 1, maxValue >= 1

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,219 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked exactly
+     * once successfully. A subsequent invocation will raise an exception.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     * @throws   FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     * @throws   RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      debug(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status, clearing feature cache.")
+          FinalizedFeatureCache.clear()
+        } else if (featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully. This method returns
+     * immediately if an updateLatestOrThrow call had already completed successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             RuntimeException if the thread was interrupted during wait
+     *
+     *                     TimeoutException if the wait can not be completed in waitTimeMs
+     *                     milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Change notification queue interrupted", e)

Review comment:
       If this thread is being closed, the InterruptedException is expected and we don't need to log this.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,219 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked exactly
+     * once successfully. A subsequent invocation will raise an exception.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     * @throws   FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     * @throws   RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      debug(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status, clearing feature cache.")
+          FinalizedFeatureCache.clear()
+        } else if (featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully. This method returns
+     * immediately if an updateLatestOrThrow call had already completed successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             RuntimeException if the thread was interrupted during wait
+     *
+     *                     TimeoutException if the wait can not be completed in waitTimeMs
+     *                     milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Change notification queue interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDataChange(): Unit = {
+      info(s"Feature ZK node updated at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDeletion(): Unit = {
+      warn(s"Feature ZK node deleted at path: $path")
+      // This event may happen, rarely (ex: ZK corruption or operational error).
+      // In such a case, we prefer to just log a warning and treat the case as if the node is absent,
+      // and populate the FinalizedFeatureCache with empty finalized features.
+      queue.add(new FeatureCacheUpdater(path))
+    }
+  }
+
+  private val queue = new LinkedBlockingQueue[FeatureCacheUpdater]
+
+  private val thread = new ChangeNotificationProcessorThread("feature-zk-node-event-process-thread")
+
+  /**
+   * This method initializes the feature ZK node change listener. Optionally, it also ensures to
+   * update the FinalizedFeatureCache once with the latest contents of the feature ZK node
+   * (if the node exists). This step helps ensure that feature incompatibilities (if any) in brokers
+   * are conveniently detected before the initOrThrow() method returns to the caller. If feature
+   * incompatibilities are detected, this method will throw an Exception to the caller, and the Broker
+   * will exit eventually.
+   *
+   * @param waitOnceForCacheUpdateMs   # of milli seconds to wait for feature cache to be updated once.
+   *                                   If this parameter <= 0, no wait operation happens.
+   *
+   * @throws Exception if feature incompatibility check could not be finished in a timely manner
+   */
+  def initOrThrow(waitOnceForCacheUpdateMs: Long): Unit = {
+    thread.start()
+    zkClient.registerZNodeChangeHandlerAndCheckExistence(FeatureZNodeChangeHandler)
+
+    if (waitOnceForCacheUpdateMs > 0) {

Review comment:
       Hmm, is waitOnceForCacheUpdateMs <=0 supported? In that case, it seems that we still need to read the /features path in ZK?

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,219 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked exactly
+     * once successfully. A subsequent invocation will raise an exception.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     * @throws   FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     * @throws   RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      debug(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status, clearing feature cache.")
+          FinalizedFeatureCache.clear()
+        } else if (featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully. This method returns
+     * immediately if an updateLatestOrThrow call had already completed successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             RuntimeException if the thread was interrupted during wait
+     *
+     *                     TimeoutException if the wait can not be completed in waitTimeMs
+     *                     milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Change notification queue interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)

Review comment:
       Hmm, this just kills the thread, but not the broker as the comment says. Also, not sure about killing the broker. We probably should just log an error and continue since this is not necessarily fatal.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -113,14 +127,44 @@ public static ApiVersionsResponse fromStruct(Struct struct, short version) {
         }
     }
 
-    public static ApiVersionsResponse apiVersionsResponse(int throttleTimeMs, byte maxMagic) {
+    public static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures) {
+        return apiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, Optional.empty(), Optional.empty());
+    }
+
+    public static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures,
+        Features<FinalizedVersionRange> finalizedFeatures,
+        long finalizedFeaturesEpoch) {
+        return apiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, Optional.of(finalizedFeatures), Optional.of(finalizedFeaturesEpoch));
+    }
+
+    private static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures,
+        Optional<Features<FinalizedVersionRange>> finalizedFeatures,
+        Optional<Long> finalizedFeaturesEpoch) {
         if (maxMagic == RecordBatch.CURRENT_MAGIC_VALUE && throttleTimeMs == DEFAULT_THROTTLE_TIME) {
             return DEFAULT_API_VERSIONS_RESPONSE;
         }
-        return createApiVersionsResponse(throttleTimeMs, maxMagic);
+        return createApiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, finalizedFeatures, finalizedFeaturesEpoch);
     }
 
-    public static ApiVersionsResponse createApiVersionsResponse(int throttleTimeMs, final byte minMagic) {
+    public static ApiVersionsResponse createApiVersionsResponse(
+        int throttleTimeMs,
+        final byte minMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures,
+        Optional<Features<FinalizedVersionRange>> finalizedFeatures,
+        Optional<Long> finalizedFeaturesEpoch

Review comment:
       Should we add public methods for accessing those fields?

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -113,14 +127,44 @@ public static ApiVersionsResponse fromStruct(Struct struct, short version) {
         }
     }
 
-    public static ApiVersionsResponse apiVersionsResponse(int throttleTimeMs, byte maxMagic) {
+    public static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures) {
+        return apiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, Optional.empty(), Optional.empty());
+    }
+
+    public static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures,
+        Features<FinalizedVersionRange> finalizedFeatures,
+        long finalizedFeaturesEpoch) {
+        return apiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, Optional.of(finalizedFeatures), Optional.of(finalizedFeaturesEpoch));
+    }
+
+    private static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures,
+        Optional<Features<FinalizedVersionRange>> finalizedFeatures,

Review comment:
       Hmm, why is FinalizedFeaturesEpoch an optional but latestSupportedFeatures is not?

##########
File path: core/src/main/scala/kafka/cluster/Broker.scala
##########
@@ -34,14 +36,22 @@ object Broker {
                                          brokerId: Int,
                                          endpoints: util.List[Endpoint],
                                          interBrokerEndpoint: Endpoint) extends AuthorizerServerInfo
+
+  def apply(id: Int, endPoints: Seq[EndPoint], rack: Option[String]): Broker = {
+    new Broker(id, endPoints, rack, emptySupportedFeatures)
+  }
 }
 
 /**
  * A Kafka broker.
- * A broker has an id, a collection of end-points, an optional rack and a listener to security protocol map.
- * Each end-point is (host, port, listenerName).
+ *
+ * @param id          a broker id
+ * @param endPoints   a collection of: end-point and a listener to security protocol map.

Review comment:
       It doesn't seem we store security protocol map in the broker registration.




----------------------------------------------------------------
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 #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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


   test 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 pull request #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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


   ok to test


----------------------------------------------------------------
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 #8680: KAFKA-9755: Implement read path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/feature/VersionLevelRange.java
##########
@@ -0,0 +1,39 @@
+package org.apache.kafka.common.feature;
+
+import java.util.Map;
+
+/**
+ * A specialization of VersionRange representing a range of version levels. The main specialization
+ * is that the class uses different serialization keys for min/max attributes.
+ *
+ * NOTE: This is the backing class used to define the min/max version levels for finalized features.
+ */
+public class VersionLevelRange extends VersionRange {

Review comment:
       In terms of naming, do you think `FinalizedVersionRange` is more explicit? Also when I look closer at the class hierarchy, I feel the sharing point between finalized version range and supported version range should be extracted to avoid weird inheritance. What I'm proposing is to have `VersionRange` as a super class with two subclasses: `SupportedVersionRange` and `FinalizedVersionRange`, and make `minKeyLabel` and `maxKeyLabel` abstract functions, WDYT?

##########
File path: clients/src/test/java/org/apache/kafka/common/feature/VersionRangeTest.java
##########
@@ -0,0 +1,150 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+
+public class VersionRangeTest {
+    @Test
+    public void testFailDueToInvalidParams() {
+        // min and max can't be < 1.
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionRange(0, 0));
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionRange(-1, -1));
+        // min can't be < 1.
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionRange(0, 1));
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionRange(-1, 1));
+        // max can't be < 1.
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionRange(1, 0));
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionRange(1, -1));
+        // min can't be > max.
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> new VersionRange(2, 1));
+    }
+
+    @Test
+    public void testSerializeDeserializeTest() {
+        VersionRange versionRange = new VersionRange(1, 2);
+        assertEquals(1, versionRange.min());
+        assertEquals(2, versionRange.max());
+
+        Map<String, Long> serialized = versionRange.serialize();
+        assertEquals(
+            new HashMap<String, Long>() {
+                {
+                    put("min_version", versionRange.min());
+                    put("max_version", versionRange.max());
+                }
+            },
+            serialized
+        );
+
+        VersionRange deserialized = VersionRange.deserialize(serialized);
+        assertEquals(1, deserialized.min());
+        assertEquals(2, deserialized.max());
+        assertEquals(versionRange, deserialized);
+    }
+
+    @Test
+    public void testDeserializationFailureTest() {
+        // min_version can't be < 1.
+        Map<String, Long> invalidWithBadMinVersion = new HashMap<String, Long>() {
+            {
+                put("min_version", 0L);
+                put("max_version", 1L);
+            }
+        };
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> VersionRange.deserialize(invalidWithBadMinVersion));
+
+        // max_version can't be < 1.
+        Map<String, Long> invalidWithBadMaxVersion = new HashMap<String, Long>() {
+            {
+                put("min_version", 1L);
+                put("max_version", 0L);
+            }
+        };
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> VersionRange.deserialize(invalidWithBadMaxVersion));
+
+        // min_version and max_version can't be < 1.
+        Map<String, Long> invalidWithBadMinMaxVersion = new HashMap<String, Long>() {
+            {
+                put("min_version", 0L);
+                put("max_version", 0L);
+            }
+        };
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> VersionRange.deserialize(invalidWithBadMinMaxVersion));
+
+        // min_version can't be > max_version.
+        Map<String, Long> invalidWithLowerMaxVersion = new HashMap<String, Long>() {
+            {
+                put("min_version", 2L);
+                put("max_version", 1L);
+            }
+        };
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> VersionRange.deserialize(invalidWithLowerMaxVersion));
+
+        // min_version key missing.
+        Map<String, Long> invalidWithMinKeyMissing = new HashMap<String, Long>() {
+            {
+                put("max_version", 1L);
+            }
+        };
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> VersionRange.deserialize(invalidWithMinKeyMissing));
+
+        // max_version key missing.
+        Map<String, Long> invalidWithMaxKeyMissing = new HashMap<String, Long>() {
+            {
+                put("min_version", 1L);
+            }
+        };
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> VersionRange.deserialize(invalidWithMaxKeyMissing));
+    }
+
+    @Test
+    public void testToString() {
+        assertEquals("VersionRange[1, 1]", new VersionRange(1, 1).toString());
+        assertEquals("VersionRange[1, 2]", new VersionRange(1, 2).toString());
+    }
+
+    @Test
+    public void testEquals() {
+        assertTrue(new VersionRange(1, 1).equals(new VersionRange(1, 1)));
+        assertFalse(new VersionRange(1, 1).equals(new VersionRange(1, 2)));
+    }
+
+    @Test
+    public void testGetters() {

Review comment:
       nit: testMinMax, and we could reuse the same `new VersionRange(1, 2)` by only creating it once. 

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/VersionRange.java
##########
@@ -0,0 +1,104 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Represents an immutable basic version range using 2 attributes: min and max of type long.
+ * The min and max attributes are expected to be >= 1, and with max >= min.
+ *
+ * The class also provides API to serialize/deserialize the version range to/from a map.
+ * The class allows for configurable labels for the min/max attributes, which can be specialized by
+ * sub-classes (if needed).
+ *
+ * NOTE: This is the backing class used to define the min/max versions for supported features.
+ */
+public class VersionRange {
+    // Label for the min version key, that's used only for serialization/deserialization purposes.
+    private static final String MIN_VERSION_KEY_LABEL = "min_version";
+
+    // Label for the max version key, that's used only for serialization/deserialization purposes.
+    private static final String MAX_VERSION_KEY_LABEL = "max_version";
+
+    private final String minKeyLabel;
+
+    private final long minValue;
+
+    private final String maxKeyLabel;
+
+    private final long maxValue;
+
+    protected VersionRange(String minKey, long minValue, String maxKeyLabel, long maxValue) {
+        if (minValue < 1 || maxValue < 1 || maxValue < minValue) {
+            throw new IllegalArgumentException(
+                String.format(
+                    "Expected minValue > 1, maxValue > 1 and maxValue >= minValue, but received" +
+                    " minValue: %d, maxValue: %d", minValue, maxValue));
+        }
+        this.minKeyLabel = minKey;
+        this.minValue = minValue;
+        this.maxKeyLabel = maxKeyLabel;
+        this.maxValue = maxValue;
+    }
+
+    public VersionRange(long minVersion, long maxVersion) {
+        this(MIN_VERSION_KEY_LABEL, minVersion, MAX_VERSION_KEY_LABEL, maxVersion);
+    }
+
+    public long min() {
+        return minValue;
+    }
+
+    public long max() {
+        return maxValue;
+    }
+
+    public String toString() {
+        return String.format("%s[%d, %d]", this.getClass().getSimpleName(), min(), max());
+    }
+
+    public Map<String, Long> serialize() {
+        return new HashMap<String, Long>() {
+            {
+                put(minKeyLabel, min());
+                put(maxKeyLabel, max());
+            }
+        };
+    }
+
+    public static VersionRange deserialize(Map<String, Long> serialized) {
+        return new VersionRange(
+            valueOrThrow(MIN_VERSION_KEY_LABEL, serialized),
+            valueOrThrow(MAX_VERSION_KEY_LABEL, serialized));
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (this == other) {
+            return true;
+        }
+        if (!(other instanceof VersionRange)) {

Review comment:
       Need to check null

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,93 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionLevelRange}
+
+// 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[VersionLevelRange], epoch: Int) {
+
+  def isValid(newEpoch: Int): Boolean = {
+    newEpoch >= epoch
+  }
+
+  override def toString(): String = {
+    "FinalizedFeaturesAndEpoch(features=%s, epoch=%s)".format(features, epoch)
+  }
+}
+
+/**
+ * A mutable cache containing the latest finalized features and epoch. This cache is populated by a
+ * FinalizedFeatureChangeListener.

Review comment:
       Could we add a reference to the class?

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -81,17 +83,27 @@ object BrokerIdsZNode {
 object BrokerInfo {
 
   /**
-   * Create a broker info with v4 json format (which includes multiple endpoints and rack) if
-   * the apiVersion is 0.10.0.X or above. Register the broker with v2 json format otherwise.
+   * - Create a broker info with v5 json format if the apiVersion is 2.6.x or above.
+   * - Create a broker info with v4 json format (which includes multiple endpoints and rack) if
+   *   the apiVersion is 0.10.0.X or above but lesser than 2.6.x.
+   * - Register the broker with v2 json format otherwise.
    *
    * Due to KAFKA-3100, 0.9.0.0 broker and old clients will break if JSON version is above 2.
    *
-   * We include v2 to make it possible for the broker to migrate from 0.9.0.0 to 0.10.0.X or above without having to
-   * upgrade to 0.9.0.1 first (clients have to be upgraded to 0.9.0.1 in any case).
+   * We include v2 to make it possible for the broker to migrate from 0.9.0.0 to 0.10.0.X or above
+   * without having to upgrade to 0.9.0.1 first (clients have to be upgraded to 0.9.0.1 in
+   * any case).
    */
   def apply(broker: Broker, apiVersion: ApiVersion, jmxPort: Int): BrokerInfo = {
-    // see method documentation for the reason why we do this
-    val version = if (apiVersion >= KAFKA_0_10_0_IV1) 4 else 2
+    val version = {

Review comment:
       I don't think we need a nested if-else:
   ```
    val version = {
         if (apiVersion >= KAFKA_2_6_IV1) 
           5
         else if (apiVersion >= KAFKA_0_10_0_IV1)
           4
         else
           2
       }
   ```

##########
File path: core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala
##########
@@ -149,6 +153,53 @@ class BrokerEndPointTest {
     assertEquals(None, broker.rack)
   }
 
+  @Test
+  def testFromJsonV5(): Unit = {
+    val json = """{
+      "version":5,
+      "host":"localhost",
+      "port":9092,
+      "jmx_port":9999,
+      "timestamp":"2233345666",
+      "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}}
+    }"""
+    val broker = parseBrokerJson(1, json)
+    assertEquals(1, broker.id)
+    val brokerEndPoint = broker.brokerEndPoint(new ListenerName("CLIENT"))
+    assertEquals("host1", brokerEndPoint.host)
+    assertEquals(9092, brokerEndPoint.port)
+    assertEquals(Some("dc1"), broker.rack)
+    assertEquals(Features.supportedFeatures(
+      Map[String, VersionRange](
+        "feature1" -> new VersionRange(1, 2),
+        "feature2" -> new VersionRange(2, 4)).asJava),
+      broker.features)
+  }
+
+  @Test
+  def testFromJsonV4WithNoFeatures(): Unit = {

Review comment:
       nit: This test could move closer to testFromJsonV4WithNoRack

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,93 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionLevelRange}
+
+// 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[VersionLevelRange], epoch: Int) {
+
+  def isValid(newEpoch: Int): Boolean = {
+    newEpoch >= epoch
+  }
+
+  override def toString(): String = {
+    "FinalizedFeaturesAndEpoch(features=%s, epoch=%s)".format(features, epoch)
+  }
+}
+
+/**
+ * A mutable cache containing the latest finalized features and epoch. This cache is populated by a
+ * FinalizedFeatureChangeListener.
+ *
+ * Currently the main reader of this cache is the read path that serves an ApiVersionsRequest
+ * returning the features information in the response. In the future, as the feature versioning
+ * system in KIP-584 is used more widely, this cache could be read by other read paths trying to
+ * learn the finalized feature information.
+ */
+object FinalizedFeatureCache extends Logging {
+  @volatile private var featuresAndEpoch: Option[FinalizedFeaturesAndEpoch] = Option.empty
+
+  /**
+   * @return   the latest known FinalizedFeaturesAndEpoch. If the returned value is empty, it means
+   *           no FinalizedFeaturesAndEpoch exists in the cache at the time when this
+   *           method is invoked. This result could change in the future whenever the
+   *           updateOrThrow method is invoked.
+   */
+  def get: Option[FinalizedFeaturesAndEpoch] = {
+    featuresAndEpoch
+  }
+
+  def empty: Boolean = {
+    featuresAndEpoch.isEmpty
+  }
+
+  /**
+   * Clears all existing finalized features and epoch from the cache.
+   */
+  def clear(): Unit = {
+    featuresAndEpoch = Option.empty
+    info("Cleared cache")
+  }
+
+  /**
+   * Updates the cache to the latestFeatures, and updates the existing epoch to latestEpoch.
+   * Raises an exception when the operation is not successful.
+   *
+   * @param latestFeatures   the latest finalized features to be set in the cache
+   * @param latestEpoch      the latest epoch value to be set in the cache
+   *
+   * @throws                 FeatureCacheUpdateException if the cache update operation fails
+   *                         due to invalid parameters or incompatibilities with the broker's
+   *                         supported features. In such a case, the existing cache contents are
+   *                         not modified.
+   */
+  def updateOrThrow(latestFeatures: Features[VersionLevelRange], latestEpoch: Int): Unit = {
+    updateOrThrow(FinalizedFeaturesAndEpoch(latestFeatures, latestEpoch))
+  }
+
+  private def updateOrThrow(latest: FinalizedFeaturesAndEpoch): Unit = {
+    val existingStr = featuresAndEpoch.map(existing => existing.toString).getOrElse("<empty>")
+    if (!featuresAndEpoch.isEmpty && featuresAndEpoch.get.epoch > latest.epoch) {
+      val errorMsg = ("FinalizedFeatureCache update failed due to invalid epoch in new finalized %s." +
+        " The existing finalized is %s").format(latest, existingStr)
+      throw new FeatureCacheUpdateException(errorMsg)
+    } else {
+      val incompatibleFeatures = SupportedFeatures.incompatibleFeatures(latest.features)
+      if (incompatibleFeatures.nonEmpty) {
+        val errorMsg = ("FinalizedFeatureCache updated failed since feature compatibility" +

Review comment:
       nit: this errorMsg val seems redundant.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,200 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             - RuntimeException if the thread was interrupted during wait
+     *                     - TimeoutException if the wait can not be completed in waitTimeMs
+     *                       milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    private def processNotification(): Unit = {
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")
+      processNotification()
+    }
+
+    override def handleDataChange(): Unit = {
+      info(s"Feature ZK node updated at path: $path")
+      processNotification()
+    }
+
+    override def handleDeletion(): Unit = {
+      warn(s"Feature ZK node deleted at path: $path")
+      processNotification()

Review comment:
       Does this event actually happen? Will we hit illegal state exception in `updateLatestOrThrow`?

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -113,14 +141,26 @@ public static ApiVersionsResponse fromStruct(Struct struct, short version) {
         }
     }
 
-    public static ApiVersionsResponse apiVersionsResponse(int throttleTimeMs, byte maxMagic) {
+    public static ApiVersionsResponse apiVersionsResponse(

Review comment:
       Note this function is public, which suggests there could be external dependency that we need to take care of. The safer approach is to keep this static function and create a separate one with augmented parameters. cc @ijuma for validation.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ */
+public class Features<VersionRangeType extends VersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to VersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "supported" features.
+     */
+    public static Features<VersionRange> supportedFeatures(Map<String, VersionRange> features) {
+        return new Features<VersionRange>(features);
+    }
+
+    /**
+     * @param features   Map of feature name to VersionLevelRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "finalized" features.
+     */
+    public static Features<VersionLevelRange> finalizedFeatures(Map<String, VersionLevelRange> features) {
+        return new Features<VersionLevelRange>(features);
+    }
+
+    public static Features<VersionLevelRange> emptyFinalizedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public static Features<VersionRange> emptySupportedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+
+    public Map<String, VersionRangeType> all() {

Review comment:
       I gave it more thought, and wonder whether we could just call this function `features` to be more consistent with our convention for getters.

##########
File path: clients/src/test/java/org/apache/kafka/common/feature/VersionLevelRangeTest.java
##########
@@ -0,0 +1,162 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class VersionLevelRangeTest {
+
+    @Test
+    public void testCreateFailDueToInvalidParams() {
+        // min and max can't be < 1.

Review comment:
       Does L17-23 really necessary for testing?

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/VersionRange.java
##########
@@ -0,0 +1,104 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Represents an immutable basic version range using 2 attributes: min and max of type long.
+ * The min and max attributes are expected to be >= 1, and with max >= min.
+ *
+ * The class also provides API to serialize/deserialize the version range to/from a map.
+ * The class allows for configurable labels for the min/max attributes, which can be specialized by
+ * sub-classes (if needed).
+ *
+ * NOTE: This is the backing class used to define the min/max versions for supported features.
+ */
+public class VersionRange {
+    // Label for the min version key, that's used only for serialization/deserialization purposes.
+    private static final String MIN_VERSION_KEY_LABEL = "min_version";
+
+    // Label for the max version key, that's used only for serialization/deserialization purposes.
+    private static final String MAX_VERSION_KEY_LABEL = "max_version";
+
+    private final String minKeyLabel;
+
+    private final long minValue;
+
+    private final String maxKeyLabel;
+
+    private final long maxValue;
+
+    protected VersionRange(String minKey, long minValue, String maxKeyLabel, long maxValue) {
+        if (minValue < 1 || maxValue < 1 || maxValue < minValue) {
+            throw new IllegalArgumentException(
+                String.format(
+                    "Expected minValue > 1, maxValue > 1 and maxValue >= minValue, but received" +
+                    " minValue: %d, maxValue: %d", minValue, maxValue));
+        }
+        this.minKeyLabel = minKey;
+        this.minValue = minValue;
+        this.maxKeyLabel = maxKeyLabel;
+        this.maxValue = maxValue;
+    }
+
+    public VersionRange(long minVersion, long maxVersion) {
+        this(MIN_VERSION_KEY_LABEL, minVersion, MAX_VERSION_KEY_LABEL, maxVersion);
+    }
+
+    public long min() {
+        return minValue;
+    }
+
+    public long max() {
+        return maxValue;
+    }
+
+    public String toString() {
+        return String.format("%s[%d, %d]", this.getClass().getSimpleName(), min(), max());
+    }
+
+    public Map<String, Long> serialize() {
+        return new HashMap<String, Long>() {
+            {
+                put(minKeyLabel, min());
+                put(maxKeyLabel, max());
+            }
+        };
+    }
+
+    public static VersionRange deserialize(Map<String, Long> serialized) {
+        return new VersionRange(
+            valueOrThrow(MIN_VERSION_KEY_LABEL, serialized),
+            valueOrThrow(MAX_VERSION_KEY_LABEL, serialized));
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (this == other) {
+            return true;
+        }
+        if (!(other instanceof VersionRange)) {
+            return false;
+        }
+
+        final VersionRange that = (VersionRange) other;
+        return Objects.equals(this.minKeyLabel, that.minKeyLabel) &&

Review comment:
       Is there a difference between `Objects.equals` and `this.minKeyLabel.equals(that.minKeyLabel)`?

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -77,6 +93,18 @@ public boolean shouldClientThrottle(short version) {
         return version >= 2;
     }
 
+    public SupportedFeatureKey supportedFeature(String featureName) {

Review comment:
       I think we could delay the addition for these helpers until we actually need them.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,93 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionLevelRange}
+
+// 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[VersionLevelRange], epoch: Int) {
+
+  def isValid(newEpoch: Int): Boolean = {

Review comment:
       Is this function being used?

##########
File path: core/src/main/scala/kafka/zk/KafkaZkClient.scala
##########
@@ -1567,6 +1567,36 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
     createRecursive(path, data = null, throwIfPathExists = false)
   }
 
+  // Visible for testing.
+  def createFeatureZNode(nodeContents: FeatureZNode): Unit = {

Review comment:
       Do you expect these helper functions actually to be used in production logic with subsequent PRs? 

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,93 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionLevelRange}
+
+// Raised whenever there was an error in updating the FinalizedFeatureCache with features.

Review comment:
       It seems that we don't have the handling logic for this FeatureCacheUpdateException. Do we think this is fatal?

##########
File path: core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala
##########
@@ -149,6 +153,53 @@ class BrokerEndPointTest {
     assertEquals(None, broker.rack)
   }
 
+  @Test
+  def testFromJsonV5(): Unit = {

Review comment:
       What would happen if we are dealing with a V4 json map containing features?

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -225,7 +255,12 @@ object BrokerIdZNode {
           }
 
         val rack = brokerInfo.get(RackKey).flatMap(_.to[Option[String]])
-        BrokerInfo(Broker(id, endpoints, rack), version, jmxPort)
+        val features = FeatureZNode.asJavaMap(brokerInfo

Review comment:
       Could we make feature extraction as a helper function?

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,93 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionLevelRange}
+
+// Raised whenever there was an error in updating the FinalizedFeatureCache with features.

Review comment:
       Might worth getting a ticket to define the handling strategy for such exception, and in general how `updateOrThrow` will be used. 

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,93 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionLevelRange}
+
+// 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[VersionLevelRange], epoch: Int) {
+
+  def isValid(newEpoch: Int): Boolean = {
+    newEpoch >= epoch
+  }
+
+  override def toString(): String = {
+    "FinalizedFeaturesAndEpoch(features=%s, epoch=%s)".format(features, epoch)
+  }
+}
+
+/**
+ * A mutable cache containing the latest finalized features and epoch. This cache is populated by a
+ * FinalizedFeatureChangeListener.
+ *
+ * Currently the main reader of this cache is the read path that serves an ApiVersionsRequest
+ * returning the features information in the response. In the future, as the feature versioning
+ * system in KIP-584 is used more widely, this cache could be read by other read paths trying to
+ * learn the finalized feature information.
+ */
+object FinalizedFeatureCache extends Logging {
+  @volatile private var featuresAndEpoch: Option[FinalizedFeaturesAndEpoch] = Option.empty
+
+  /**
+   * @return   the latest known FinalizedFeaturesAndEpoch. If the returned value is empty, it means
+   *           no FinalizedFeaturesAndEpoch exists in the cache at the time when this
+   *           method is invoked. This result could change in the future whenever the
+   *           updateOrThrow method is invoked.
+   */
+  def get: Option[FinalizedFeaturesAndEpoch] = {
+    featuresAndEpoch
+  }
+
+  def empty: Boolean = {
+    featuresAndEpoch.isEmpty
+  }
+
+  /**
+   * Clears all existing finalized features and epoch from the cache.
+   */
+  def clear(): Unit = {
+    featuresAndEpoch = Option.empty
+    info("Cleared cache")
+  }
+
+  /**
+   * Updates the cache to the latestFeatures, and updates the existing epoch to latestEpoch.
+   * Raises an exception when the operation is not successful.
+   *
+   * @param latestFeatures   the latest finalized features to be set in the cache
+   * @param latestEpoch      the latest epoch value to be set in the cache
+   *
+   * @throws                 FeatureCacheUpdateException if the cache update operation fails
+   *                         due to invalid parameters or incompatibilities with the broker's
+   *                         supported features. In such a case, the existing cache contents are
+   *                         not modified.
+   */
+  def updateOrThrow(latestFeatures: Features[VersionLevelRange], latestEpoch: Int): Unit = {
+    updateOrThrow(FinalizedFeaturesAndEpoch(latestFeatures, latestEpoch))
+  }
+
+  private def updateOrThrow(latest: FinalizedFeaturesAndEpoch): Unit = {
+    val existingStr = featuresAndEpoch.map(existing => existing.toString).getOrElse("<empty>")

Review comment:
       s/existingStr/oldFeatureAndEpoch

##########
File path: core/src/main/scala/kafka/server/SupportedFeatures.scala
##########
@@ -0,0 +1,70 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionRange, VersionLevelRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A common 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.
+ */
+object SupportedFeatures extends Logging {
+  /**
+   * This is the latest features supported by the Broker.
+   * This is currently empty, but in the future as we define supported features, this map should be
+   * populated.
+   */
+  @volatile private var supportedFeatures = emptySupportedFeatures
+
+  /**
+   * Returns the latest features supported by the Broker.
+   */
+  def get: Features[VersionRange] = {
+    supportedFeatures
+  }
+
+  // Should be used only for testing.
+  def update(newFeatures: Features[VersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // Should be used only for testing.
+  def clear(): Unit = {
+    supportedFeatures = emptySupportedFeatures
+  }
+
+  /**
+   * Returns the set of feature names found to be incompatible between the latest features supported
+   * by the Broker, and the provided cluster-wide finalized features.
+   *
+   * @param finalized   The finalized features against which incompatibilities need to be checked for.
+   *
+   * @return            The set of incompatible feature names. If the returned set is empty, it
+   *                    means there were no feature incompatibilities found.
+   */
+  def incompatibleFeatures(finalized: Features[VersionLevelRange]): Set[String] = {
+    val supported = get

Review comment:
       This is only used on L53, maybe we could just use supportedFeatures instead

##########
File path: core/src/test/scala/unit/kafka/server/FinalizedFeatureCacheTest.scala
##########
@@ -0,0 +1,95 @@
+package kafka.server
+
+import org.apache.kafka.common.feature.{Features, VersionLevelRange, VersionRange}
+import org.junit.Assert.{assertEquals, assertThrows, assertTrue}
+import org.junit.{Before, Test}
+
+import scala.jdk.CollectionConverters._
+
+class FinalizedFeatureCacheTest {
+
+  @Before
+  def setUp(): Unit = {
+    FinalizedFeatureCache.clear()
+    SupportedFeatures.clear()
+  }
+
+  @Test
+  def testEmpty(): Unit = {
+    assertTrue(FinalizedFeatureCache.get.isEmpty)
+  }
+
+  @Test
+  def testUpdateOrThrowFailedDueToInvalidEpoch(): Unit = {
+    val supportedFeatures = Map[String, VersionRange](
+      "feature_1" -> new VersionRange(1, 4))
+    SupportedFeatures.update(Features.supportedFeatures(supportedFeatures.asJava))
+
+    val features = Map[String, VersionLevelRange](
+      "feature_1" -> new VersionLevelRange(1, 4))
+    val finalizedFeatures = Features.finalizedFeatures(features.asJava)
+
+    FinalizedFeatureCache.updateOrThrow(finalizedFeatures, 10)
+    assertEquals(finalizedFeatures, FinalizedFeatureCache.get.get.features)

Review comment:
       Should we test `isDefined` before calling `get`?

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,93 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionLevelRange}
+
+// 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[VersionLevelRange], epoch: Int) {
+
+  def isValid(newEpoch: Int): Boolean = {
+    newEpoch >= epoch
+  }
+
+  override def toString(): String = {
+    "FinalizedFeaturesAndEpoch(features=%s, epoch=%s)".format(features, epoch)
+  }
+}
+
+/**
+ * A mutable cache containing the latest finalized features and epoch. This cache is populated by a
+ * FinalizedFeatureChangeListener.
+ *
+ * Currently the main reader of this cache is the read path that serves an ApiVersionsRequest
+ * returning the features information in the response. In the future, as the feature versioning
+ * system in KIP-584 is used more widely, this cache could be read by other read paths trying to
+ * learn the finalized feature information.
+ */
+object FinalizedFeatureCache extends Logging {
+  @volatile private var featuresAndEpoch: Option[FinalizedFeaturesAndEpoch] = Option.empty
+
+  /**
+   * @return   the latest known FinalizedFeaturesAndEpoch. If the returned value is empty, it means
+   *           no FinalizedFeaturesAndEpoch exists in the cache at the time when this
+   *           method is invoked. This result could change in the future whenever the
+   *           updateOrThrow method is invoked.
+   */
+  def get: Option[FinalizedFeaturesAndEpoch] = {
+    featuresAndEpoch
+  }
+
+  def empty: Boolean = {
+    featuresAndEpoch.isEmpty
+  }
+
+  /**
+   * Clears all existing finalized features and epoch from the cache.
+   */
+  def clear(): Unit = {
+    featuresAndEpoch = Option.empty
+    info("Cleared cache")
+  }
+
+  /**
+   * Updates the cache to the latestFeatures, and updates the existing epoch to latestEpoch.
+   * Raises an exception when the operation is not successful.
+   *
+   * @param latestFeatures   the latest finalized features to be set in the cache
+   * @param latestEpoch      the latest epoch value to be set in the cache
+   *
+   * @throws                 FeatureCacheUpdateException if the cache update operation fails
+   *                         due to invalid parameters or incompatibilities with the broker's
+   *                         supported features. In such a case, the existing cache contents are
+   *                         not modified.
+   */
+  def updateOrThrow(latestFeatures: Features[VersionLevelRange], latestEpoch: Int): Unit = {
+    updateOrThrow(FinalizedFeaturesAndEpoch(latestFeatures, latestEpoch))
+  }
+
+  private def updateOrThrow(latest: FinalizedFeaturesAndEpoch): Unit = {
+    val existingStr = featuresAndEpoch.map(existing => existing.toString).getOrElse("<empty>")
+    if (!featuresAndEpoch.isEmpty && featuresAndEpoch.get.epoch > latest.epoch) {
+      val errorMsg = ("FinalizedFeatureCache update failed due to invalid epoch in new finalized %s." +
+        " The existing finalized is %s").format(latest, existingStr)
+      throw new FeatureCacheUpdateException(errorMsg)
+    } else {
+      val incompatibleFeatures = SupportedFeatures.incompatibleFeatures(latest.features)
+      if (incompatibleFeatures.nonEmpty) {
+        val errorMsg = ("FinalizedFeatureCache updated failed since feature compatibility" +
+          " checks failed! Supported %s has incompatibilities with the latest finalized %s." +
+          " The incompatible features are: %s.").format(
+          SupportedFeatures.get, latest, incompatibleFeatures)
+        throw new FeatureCacheUpdateException(errorMsg)
+      }
+    }
+    val logMsg = "Updated cache from existing finalized %s to latest finalized %s".format(

Review comment:
       This val seems redundant.

##########
File path: core/src/main/scala/kafka/server/SupportedFeatures.scala
##########
@@ -0,0 +1,70 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionRange, VersionLevelRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A common 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.
+ */
+object SupportedFeatures extends Logging {
+  /**
+   * This is the latest features supported by the Broker.
+   * This is currently empty, but in the future as we define supported features, this map should be
+   * populated.
+   */
+  @volatile private var supportedFeatures = emptySupportedFeatures
+
+  /**
+   * Returns the latest features supported by the Broker.
+   */
+  def get: Features[VersionRange] = {
+    supportedFeatures
+  }
+
+  // Should be used only for testing.
+  def update(newFeatures: Features[VersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // Should be used only for testing.
+  def clear(): Unit = {
+    supportedFeatures = emptySupportedFeatures
+  }
+
+  /**
+   * Returns the set of feature names found to be incompatible between the latest features supported
+   * by the Broker, and the provided cluster-wide finalized features.
+   *
+   * @param finalized   The finalized features against which incompatibilities need to be checked for.
+   *
+   * @return            The set of incompatible feature names. If the returned set is empty, it
+   *                    means there were no feature incompatibilities found.
+   */
+  def incompatibleFeatures(finalized: Features[VersionLevelRange]): Set[String] = {
+    val supported = get
+
+    val incompatibilities = finalized.all.asScala.collect {
+      case (feature, versionLevels) => {
+        val supportedVersions = supported.get(feature);
+        if (supportedVersions == null) {
+          (feature, "{feature=%s, reason='Unsupported feature'}".format(feature))
+        } else if (!versionLevels.isCompatibleWith(supportedVersions)) {

Review comment:
       nit: maybe rename to `incompatibleWith` and flip the boolean

##########
File path: core/src/main/scala/kafka/server/KafkaServer.scala
##########
@@ -210,6 +215,14 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP
         /* setup zookeeper */
         initZkClient(time)
 
+        /* initialize features */
+        _featureChangeListener = new FinalizedFeatureChangeListener(_zkClient)
+        if (config.interBrokerProtocolVersion >= KAFKA_2_6_IV1) {
+          // The feature versioning system (KIP-584) is active only when:
+          // config.interBrokerProtocolVersion is >= KAFKA_2_6_IV1.
+          _featureChangeListener.initOrThrow(60000)

Review comment:
       Could we make this parameter configurable?

##########
File path: core/src/main/scala/kafka/server/SupportedFeatures.scala
##########
@@ -0,0 +1,70 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionRange, VersionLevelRange}
+import org.apache.kafka.common.feature.Features._
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * A common 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.
+ */
+object SupportedFeatures extends Logging {
+  /**
+   * This is the latest features supported by the Broker.
+   * This is currently empty, but in the future as we define supported features, this map should be
+   * populated.
+   */
+  @volatile private var supportedFeatures = emptySupportedFeatures
+
+  /**
+   * Returns the latest features supported by the Broker.
+   */
+  def get: Features[VersionRange] = {
+    supportedFeatures
+  }
+
+  // Should be used only for testing.
+  def update(newFeatures: Features[VersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // Should be used only for testing.
+  def clear(): Unit = {
+    supportedFeatures = emptySupportedFeatures
+  }
+
+  /**
+   * Returns the set of feature names found to be incompatible between the latest features supported
+   * by the Broker, and the provided cluster-wide finalized features.
+   *
+   * @param finalized   The finalized features against which incompatibilities need to be checked for.

Review comment:
       This comment is a bit vague to me, what are you referring by `incompatibilities`?




----------------------------------------------------------------
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 #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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


   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 #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,232 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     *
+     *           FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     *
+     *           RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      var mayBeFeatureZNodeBytes: Option[Array[Byte]] = null
+      var version: Int = ZkVersion.UnknownVersion
+      try {
+        val result = zkClient.getDataAndVersion(featureZkNodePath)
+        mayBeFeatureZNodeBytes = result._1
+        version = result._2
+      } catch {
+        // Convert to RuntimeException, to avoid a confusion that there is no argument passed
+        // to the updateOrThrow() method.
+        case e: IllegalArgumentException => throw new RuntimeException(e)
+      }
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             RuntimeException if the thread was interrupted during wait
+     *
+     *                     TimeoutException if the wait can not be completed in waitTimeMs
+     *                     milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Change notification queue interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDataChange(): Unit = {
+      info(s"Feature ZK node updated at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDeletion(): Unit = {
+      warn(s"Feature ZK node deleted at path: $path")
+      // This event may happen, rarely (ex: operational error).

Review comment:
       The deletion of ZNode is a rare case, it should never happen in reality unless it is ZK corruption, or rarely an operational error that deletes some ZK nodes. It's not easy to prevent damage in such a case. From a correctness standpoint, imagine what would happen if the feature ZNode gets deleted, and, afterwards a broker restarts. It will start with empty cache, so the damage is done. Therefore, it seems that even if we add a special logic here, we can not prevent damage if the source of truth is lost.
   
   Two things to note here:
   1. The client should anyway ignore older stale epoch responses, if it had seen newer epochs that are greater. In that spirit, the client can be also made to treat the absence of finalized features in an `ApiVersionsResponse` just like a stale epoch case, if, it had seen at least one valid `ApiVersionsResponse` earlier (i.e. at least one response with some valid epoch). 
   
   2. Deletion of individual finalized feature is actually supported in [KIP-584](https://cwiki.apache.org/confluence/display/KAFKA/KIP-584%3A+Versioning+scheme+for+features), but not deletion of the entire ZNode. Search for the word 'deletion' in the KIP write-up. If needed, this deletion functionality could be extended to provide the ability to delete all features too.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [kafka] kowshik commented on a change in pull request #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being <VersionRangeType>.
+ * Also provides API to convert the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ * @see SupportedVersionRange
+ * @see FinalizedVersionRange
+ */
+public class Features<VersionRangeType extends BaseVersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to a type of VersionRange.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        Objects.requireNonNull(features,"Provided features can not be null.");
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to SupportedVersionRange.
+     *
+     * @return           Returns a new Features object representing supported features.
+     */
+    public static Features<SupportedVersionRange> supportedFeatures(Map<String, SupportedVersionRange> features) {
+        return new Features<>(features);
+    }
+
+    /**
+     * @param features   Map of feature name to FinalizedVersionRange.
+     *
+     * @return           Returns a new Features object representing finalized features.
+     */
+    public static Features<FinalizedVersionRange> finalizedFeatures(Map<String, FinalizedVersionRange> features) {
+        return new Features<>(features);
+    }
+
+    // Visible for testing.
+    public static Features<FinalizedVersionRange> emptyFinalizedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public static Features<SupportedVersionRange> emptySupportedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public Map<String, VersionRangeType> features() {
+        return features;
+    }
+
+    public boolean empty() {
+        return features.isEmpty();
+    }
+
+    /**
+     * @param  feature   name of the feature
+     *
+     * @return           the VersionRangeType corresponding to the feature name, or null if the
+     *                   feature is absent
+     */
+    public VersionRangeType get(String feature) {
+        return features.get(feature);
+    }
+
+    public String toString() {
+        return String.format(
+            "Features{%s}",
+            features
+                .entrySet()
+                .stream()
+                .map(entry -> String.format("(%s -> %s)", entry.getKey(), entry.getValue()))
+                .collect(joining(", "))
+        );
+    }
+
+    /**
+     * @return   A map representation of the underlying features. The returned value can be converted
+     *           back to Features using one of the from*FeaturesMap() APIs of this class.
+     */
+    public Map<String, Map<String, Short>> toMap() {
+        return features.entrySet().stream().collect(
+            Collectors.toMap(
+                Map.Entry::getKey,
+                entry -> entry.getValue().toMap()));
+    }
+
+    /**
+     * An interface that defines behavior to convert from a Map to an object of type BaseVersionRange.
+     */
+    private interface MapToBaseVersionRangeConverter<V extends BaseVersionRange> {
+
+        /**
+         * Convert the map representation of an object of type <V>, to an object of type <V>.
+         *
+         * @param  baseVersionRangeMap   the map representation of a BaseVersionRange object.
+         *
+         * @return                       the object of type <V>
+         */
+        V fromMap(Map<String, Short> baseVersionRangeMap);
+    }
+
+    private static <V extends BaseVersionRange> Features<V> fromFeaturesMap(
+        Map<String, Map<String, Short>> featuresMap, MapToBaseVersionRangeConverter<V> converter) {
+        return new Features<>(featuresMap.entrySet().stream().collect(
+            Collectors.toMap(
+                Map.Entry::getKey,
+                entry -> converter.fromMap(entry.getValue()))));
+    }
+
+    /**
+     * Converts from a map to Features<FinalizedVersionRange>.
+     *
+     * @param featuresMap  the map representation of a Features<FinalizedVersionRange> object,
+     *                     generated using the toMap() API.
+     *
+     * @return             the Features<FinalizedVersionRange> object
+     */
+    public static Features<FinalizedVersionRange> fromFinalizedFeaturesMap(
+        Map<String, Map<String, Short>> featuresMap) {
+        return fromFeaturesMap(featuresMap, FinalizedVersionRange::fromMap);
+    }
+
+    /**
+     * Converts from a map to Features<SupportedVersionRange>.
+     *
+     * @param featuresMap  the map representation of a Features<SupportedVersionRange> object,
+     *                     generated using the toMap() API.
+     *
+     * @return             the Features<SupportedVersionRange> object
+     */
+    public static Features<SupportedVersionRange> fromSupportedFeaturesMap(
+        Map<String, Map<String, Short>> featuresMap) {
+        return fromFeaturesMap(featuresMap, SupportedVersionRange::fromMap);
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (this == other) {
+            return true;
+        }
+        if (other == null || !(other instanceof Features)) {

Review comment:
       Done. Good point.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -113,14 +128,45 @@ public static ApiVersionsResponse fromStruct(Struct struct, short version) {
         }
     }
 
-    public static ApiVersionsResponse apiVersionsResponse(int throttleTimeMs, byte maxMagic) {
+    public static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures) {
+        return apiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, Features.emptyFinalizedFeatures(), UNKNOWN_FINALIZED_FEATURES_EPOCH);
+    }
+
+    public static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures,
+        Features<FinalizedVersionRange> finalizedFeatures,
+        int finalizedFeaturesEpoch) {
         if (maxMagic == RecordBatch.CURRENT_MAGIC_VALUE && throttleTimeMs == DEFAULT_THROTTLE_TIME) {
             return DEFAULT_API_VERSIONS_RESPONSE;
         }
-        return createApiVersionsResponse(throttleTimeMs, maxMagic);
+        return createApiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, finalizedFeatures, finalizedFeaturesEpoch);
     }
 
-    public static ApiVersionsResponse createApiVersionsResponse(int throttleTimeMs, final byte minMagic) {
+    public static ApiVersionsResponse createApiVersionsResponseWithEmptyFeatures(

Review comment:
       Done.

##########
File path: core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala
##########
@@ -149,6 +153,53 @@ class BrokerEndPointTest {
     assertEquals(None, broker.rack)
   }
 
+  @Test
+  def testFromJsonV4WithNoFeatures(): Unit = {

Review comment:
       It checks backwards compatibility i.e. it checks whether the deserialization code (V5-based) can correctly deserialize V4 such that features are assigned empty value by default..

##########
File path: clients/src/test/java/org/apache/kafka/common/feature/FeaturesTest.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThrows;
+
+public class FeaturesTest {
+
+    @Test
+    public void testEmptyFeatures() {
+        Map<String, Map<String, Short>> emptyMap = new HashMap<>();
+
+        Features<FinalizedVersionRange> emptyFinalizedFeatures = Features.emptyFinalizedFeatures();
+        assertTrue(emptyFinalizedFeatures.features().isEmpty());
+        assertTrue(emptyFinalizedFeatures.toMap().isEmpty());
+        assertEquals(emptyFinalizedFeatures, Features.fromFinalizedFeaturesMap(emptyMap));
+
+        Features<SupportedVersionRange> emptySupportedFeatures = Features.emptySupportedFeatures();
+        assertTrue(emptySupportedFeatures.features().isEmpty());
+        assertTrue(emptySupportedFeatures.toMap().isEmpty());
+        assertEquals(emptySupportedFeatures, Features.fromSupportedFeaturesMap(emptyMap));
+    }
+
+    @Test
+    public void testNullFeatures() {
+        assertThrows(
+            NullPointerException.class,
+            () -> Features.finalizedFeatures(null));
+        assertThrows(
+            NullPointerException.class,
+            () -> Features.supportedFeatures(null));
+    }
+
+    @Test
+    public void testGetAllFeaturesAPI() {
+        SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2);
+        SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4);
+        Map<String, SupportedVersionRange> allFeatures =
+            mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2));
+        Features<SupportedVersionRange> features = Features.supportedFeatures(allFeatures);
+        assertEquals(allFeatures, features.features());
+    }
+
+    @Test
+    public void testGetAPI() {
+        SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2);
+        SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4);
+        Map<String, SupportedVersionRange> allFeatures = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2));
+        Features<SupportedVersionRange> features = Features.supportedFeatures(allFeatures);
+        assertEquals(v1, features.get("feature_1"));
+        assertEquals(v2, features.get("feature_2"));
+        assertNull(features.get("nonexistent_feature"));
+    }
+
+    @Test
+    public void testFromFeaturesMapToFeaturesMap() {
+        SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2);
+        SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4);
+        Map<String, SupportedVersionRange> allFeatures = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2));
+
+        Features<SupportedVersionRange> features = Features.supportedFeatures(allFeatures);
+
+        Map<String, Map<String, Short>> expected = mkMap(
+            mkEntry("feature_1", mkMap(mkEntry("min_version", (short) 1), mkEntry("max_version", (short) 2))),
+            mkEntry("feature_2", mkMap(mkEntry("min_version", (short) 3), mkEntry("max_version", (short) 4))));
+        assertEquals(expected, features.toMap());
+        assertEquals(features, Features.fromSupportedFeaturesMap(expected));
+    }
+
+    @Test
+    public void testFromToFinalizedFeaturesMap() {
+        FinalizedVersionRange v1 = new FinalizedVersionRange((short) 1, (short) 2);
+        FinalizedVersionRange v2 = new FinalizedVersionRange((short) 3, (short) 4);
+        Map<String, FinalizedVersionRange> allFeatures = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2));
+
+        Features<FinalizedVersionRange> features = Features.finalizedFeatures(allFeatures);
+
+        Map<String, Map<String, Short>> expected = mkMap(
+            mkEntry("feature_1", mkMap(mkEntry("min_version_level", (short) 1), mkEntry("max_version_level", (short) 2))),
+            mkEntry("feature_2", mkMap(mkEntry("min_version_level", (short) 3), mkEntry("max_version_level", (short) 4))));
+        assertEquals(expected, features.toMap());
+        assertEquals(features, Features.fromFinalizedFeaturesMap(expected));
+    }
+
+    @Test
+    public void testToStringFinalizedFeatures() {
+        FinalizedVersionRange v1 = new FinalizedVersionRange((short) 1, (short) 2);
+        FinalizedVersionRange v2 = new FinalizedVersionRange((short) 3, (short) 4);
+        Map<String, FinalizedVersionRange> allFeatures = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2));
+
+        Features<FinalizedVersionRange> features = Features.finalizedFeatures(allFeatures);
+
+        assertEquals(
+            "Features{(feature_1 -> FinalizedVersionRange[min_version_level:1, max_version_level:2]), (feature_2 -> FinalizedVersionRange[min_version_level:3, max_version_level:4])}",
+            features.toString());
+    }
+
+    @Test
+    public void testToStringSupportedFeatures() {
+        SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2);
+        SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4);
+        Map<String, SupportedVersionRange> allFeatures
+            = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2));
+
+        Features<SupportedVersionRange> features = Features.supportedFeatures(allFeatures);
+
+        assertEquals(
+            "Features{(feature_1 -> SupportedVersionRange[min_version:1, max_version:2]), (feature_2 -> SupportedVersionRange[min_version:3, max_version:4])}",
+            features.toString());
+    }
+
+    @Test
+    public void testSupportedFeaturesFromMapFailure() {
+        // This is invalid because 'max_version' key is missing.

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,243 @@
+/**
+ * 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.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.{StateChangeHandler, ZNodeChangeHandler}
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+import scala.util.control.Exception.ignoring
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked exactly
+     * once successfully. A subsequent invocation will raise an exception.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     * @throws   FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     * @throws   RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      debug(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        featureZNode.status match {
+          case FeatureZNodeStatus.Disabled => {
+            info(s"Feature ZK node at path: $featureZkNodePath is in disabled status.")
+            FinalizedFeatureCache.clear()
+          }
+          case FeatureZNodeStatus.Enabled => {
+            FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)

Review comment:
       1. Re: out-of-order updates from ZK:
   I don't understand. When a watch fires from ZK, we react by issuing a ZK read operation to obtain the latest value of the ZK node (see L75). It is impossible that we get a stale read from ZK after watch fires on the client side.
   
   2. Re: broker death:
   The exception thrown here almost always indicates a feature incompatibility, and, that means the broker can cause damage if it sticks around (because feature bumps are breaking changes and you can not allow an incompatible broker to stick around in the cluster). That is why I felt it is better to kill the broker in such a rare incompatibility case. Note that after the controller has finalized features, there should be no brokers in the cluster with incompatibilites, so death here makes sense.
   
   Note: I have also explained point #2 in this comment: https://github.com/apache/kafka/pull/8680/files#r434443007.

##########
File path: core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
##########
@@ -755,7 +755,7 @@ class KafkaConfigTest {
         case KafkaConfig.SaslLoginRefreshMinPeriodSecondsProp =>
         case KafkaConfig.SaslLoginRefreshBufferSecondsProp =>
 
-        // Security config
+        //Security config

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,243 @@
+/**
+ * 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.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.{StateChangeHandler, ZNodeChangeHandler}
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+import scala.util.control.Exception.ignoring
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked exactly
+     * once successfully. A subsequent invocation will raise an exception.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     * @throws   FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     * @throws   RuntimeException, if there was a failure in reading/deserializing the

Review comment:
       It's very rare especially when controller is the only entity writing to the ZK node. I have now modified the code to handle this case and clear the cache. Perhaps that's better than crashing the broker in such a case. Remediation will need human intervention in fixing the ZK node. We can provide tooling if required.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,102 @@
+/**
+ * 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}
+
+// 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) {
+  override def toString(): String = {
+    "FinalizedFeaturesAndEpoch(features=%s, epoch=%s)".format(features, epoch)
+  }
+}
+
+/**
+ * A common mutable cache containing the latest finalized features and epoch. By default the contents of
+ * the cache are empty. This cache needs to be populated at least once for it's contents to become
+ * non-empty. Currently the main reader of this cache is the read path that serves an ApiVersionsRequest,
+ * returning the features information in the response.
+ *
+ * @see FinalizedFeatureChangeListener
+ */
+object FinalizedFeatureCache extends Logging {
+  @volatile private var featuresAndEpoch: Option[FinalizedFeaturesAndEpoch] = Option.empty
+
+  /**
+   * @return   the latest known FinalizedFeaturesAndEpoch. If the returned value is empty, it means

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -744,6 +782,90 @@ object DelegationTokenInfoZNode {
   def decode(bytes: Array[Byte]): Option[TokenInformation] = DelegationTokenManager.fromBytes(bytes)
 }
 
+object FeatureZNodeStatus extends Enumeration {
+  val Disabled, Enabled = Value
+
+  def withNameOpt(value: Int): Option[Value] = {
+    values.find(_.id == value)
+  }
+}
+
+case class FeatureZNode(status: FeatureZNodeStatus.Value, features: Features[FinalizedVersionRange]) {
+}
+
+object FeatureZNode {

Review comment:
       Thanks. Good idea to leave a jira. I have linked it to KAFKA-9755.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Represents an immutable basic version range using 2 attributes: min and max, each of type short.
+ * The min and max attributes need to satisfy 2 rules:
+ *  - they are each expected to be >= 1, as we only consider positive version values to be valid.
+ *  - max should be >= min.
+ *
+ * The class also provides API to convert the version range to a map.
+ * The class allows for configurable labels for the min/max attributes, which can be specialized by
+ * sub-classes (if needed).
+ */
+class BaseVersionRange {
+    // Non-empty label for the min version key, that's used only to convert to/from a map.
+    private final String minKeyLabel;
+
+    // The value of the minimum version.
+    private final short minValue;
+
+    // Non-empty label for the max version key, that's used only to convert to/from a map.
+    private final String maxKeyLabel;
+
+    // The value of the maximum version.
+    private final short maxValue;
+
+    /**
+     * Raises an exception unless the following condition is met:
+     * minValue >= 1 and maxValue >= 1 and maxValue >= minValue.
+     *
+     * @param minKeyLabel   Label for the min version key, that's used only to convert to/from a map.
+     * @param minValue      The minimum version value.
+     * @param maxKeyLabel   Label for the max version key, that's used only to convert to/from a map.
+     * @param maxValue      The maximum version value.
+     *
+     * @throws IllegalArgumentException   If any of the following conditions are true:
+     *                                     - (minValue < 1) OR (maxValue < 1) OR (maxValue < minValue).
+     *                                     - minKeyLabel is empty, OR, minKeyLabel is empty.
+     */
+    protected BaseVersionRange(String minKeyLabel, short minValue, String maxKeyLabel, short maxValue) {
+        if (minValue < 1 || maxValue < 1 || maxValue < minValue) {
+            throw new IllegalArgumentException(
+                String.format(
+                    "Expected minValue >= 1, maxValue >= 1 and maxValue >= minValue, but received" +
+                    " minValue: %d, maxValue: %d", minValue, maxValue));
+        }
+        if (minKeyLabel.isEmpty()) {
+            throw new IllegalArgumentException("Expected minKeyLabel to be non-empty.");
+        }
+        if (maxKeyLabel.isEmpty()) {
+            throw new IllegalArgumentException("Expected maxKeyLabel to be non-empty.");
+        }
+        this.minKeyLabel = minKeyLabel;
+        this.minValue = minValue;
+        this.maxKeyLabel = maxKeyLabel;
+        this.maxValue = maxValue;
+    }
+
+    public short min() {
+        return minValue;
+    }
+
+    public short max() {
+        return maxValue;
+    }
+
+    public String toString() {
+        return String.format("%s[%s:%d, %s:%d]", this.getClass().getSimpleName(), this.minKeyLabel, min(), this.maxKeyLabel, max());
+    }
+
+    public Map<String, Short> toMap() {
+        return new HashMap<String, Short>() {

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -113,14 +128,45 @@ public static ApiVersionsResponse fromStruct(Struct struct, short version) {
         }
     }
 
-    public static ApiVersionsResponse apiVersionsResponse(int throttleTimeMs, byte maxMagic) {
+    public static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures) {
+        return apiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, Features.emptyFinalizedFeatures(), UNKNOWN_FINALIZED_FEATURES_EPOCH);
+    }
+
+    public static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures,
+        Features<FinalizedVersionRange> finalizedFeatures,
+        int finalizedFeaturesEpoch) {
         if (maxMagic == RecordBatch.CURRENT_MAGIC_VALUE && throttleTimeMs == DEFAULT_THROTTLE_TIME) {
             return DEFAULT_API_VERSIONS_RESPONSE;
         }
-        return createApiVersionsResponse(throttleTimeMs, maxMagic);
+        return createApiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, finalizedFeatures, finalizedFeaturesEpoch);
     }
 
-    public static ApiVersionsResponse createApiVersionsResponse(int throttleTimeMs, final byte minMagic) {
+    public static ApiVersionsResponse createApiVersionsResponseWithEmptyFeatures(
+        int throttleTimeMs,
+        final byte minMagic) {
+        return createApiVersionsResponse(
+            throttleTimeMs,
+            minMagic,
+            Features.emptySupportedFeatures(),
+            Features.emptyFinalizedFeatures(),
+            UNKNOWN_FINALIZED_FEATURES_EPOCH);
+    }
+
+    public static ApiVersionsResponse createApiVersionsResponse(
+        int throttleTimeMs,

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,102 @@
+/**
+ * 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}
+
+// 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) {
+  override def toString(): String = {
+    "FinalizedFeaturesAndEpoch(features=%s, epoch=%s)".format(features, epoch)
+  }
+}
+
+/**
+ * A common mutable cache containing the latest finalized features and epoch. By default the contents of
+ * the cache are empty. This cache needs to be populated at least once for it's contents to become

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/api/ApiVersion.scala
##########
@@ -98,7 +98,9 @@ object ApiVersion {
     // No new APIs, equivalent to 2.4-IV1
     KAFKA_2_5_IV0,
     // Introduced StopReplicaRequest V3 containing the leader epoch for each partition (KIP-570)
-    KAFKA_2_6_IV0
+    KAFKA_2_6_IV0,
+    // Introduce feature versioning support (KIP-584)

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,243 @@
+/**
+ * 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.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.{StateChangeHandler, ZNodeChangeHandler}
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+import scala.util.control.Exception.ignoring
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,243 @@
+/**
+ * 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.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.{StateChangeHandler, ZNodeChangeHandler}
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+import scala.util.control.Exception.ignoring
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked exactly
+     * once successfully. A subsequent invocation will raise an exception.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     * @throws   FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     * @throws   RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      debug(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        featureZNode.status match {
+          case FeatureZNodeStatus.Disabled => {
+            info(s"Feature ZK node at path: $featureZkNodePath is in disabled status.")
+            FinalizedFeatureCache.clear()
+          }
+          case FeatureZNodeStatus.Enabled => {
+            FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+          }
+          case _ => throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully. This method returns
+     * immediately if an updateLatestOrThrow call had already completed successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             TimeoutException if the wait can not be completed in waitTimeMs
+     *                     milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (!notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        ignoring(classOf[InterruptedException]) {
+          queue.take.updateLatestOrThrow()
+        }
+      } catch {
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will eventually exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDataChange(): Unit = {
+      info(s"Feature ZK node updated at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDeletion(): Unit = {
+      warn(s"Feature ZK node deleted at path: $path")
+      // This event may happen, rarely (ex: ZK corruption or operational error).
+      // In such a case, we prefer to just log a warning and treat the case as if the node is absent,
+      // and populate the FinalizedFeatureCache with empty finalized features.
+      queue.add(new FeatureCacheUpdater(path))
+    }
+  }
+
+  object ZkStateChangeHandler extends StateChangeHandler {
+    val path: String = FeatureZNode.path
+
+    override val name: String = path
+
+    override def afterInitializingSession(): Unit = {
+      queue.add(new FeatureCacheUpdater(path))
+    }
+  }
+
+  private val queue = new LinkedBlockingQueue[FeatureCacheUpdater]
+
+  private val thread = new ChangeNotificationProcessorThread("feature-zk-node-event-process-thread")
+
+  /**
+   * This method initializes the feature ZK node change listener. Optionally, it also ensures to
+   * update the FinalizedFeatureCache once with the latest contents of the feature ZK node
+   * (if the node exists). This step helps ensure that feature incompatibilities (if any) in brokers
+   * are conveniently detected before the initOrThrow() method returns to the caller. If feature
+   * incompatibilities are detected, this method will throw an Exception to the caller, and the Broker
+   * will exit eventually.
+   *
+   * @param waitOnceForCacheUpdateMs   # of milli seconds to wait for feature cache to be updated once.
+   *                                   If this parameter <= 0, no wait operation happens.
+   *
+   * @throws Exception if feature incompatibility check could not be finished in a timely manner
+   */
+  def initOrThrow(waitOnceForCacheUpdateMs: Long): Unit = {
+    if (waitOnceForCacheUpdateMs <= 0) {
+      throw new IllegalArgumentException(
+        s"Expected waitOnceForCacheUpdateMs > 0, but provided: $waitOnceForCacheUpdateMs")
+    }
+
+    thread.start()
+    zkClient.registerStateChangeHandler(ZkStateChangeHandler)
+    zkClient.registerZNodeChangeHandlerAndCheckExistence(FeatureZNodeChangeHandler)
+    val ensureCacheUpdateOnce = new FeatureCacheUpdater(
+      FeatureZNodeChangeHandler.path, Some(new CountDownLatch(1)))
+    queue.add(ensureCacheUpdateOnce)
+    try {
+      ensureCacheUpdateOnce.awaitUpdateOrThrow(waitOnceForCacheUpdateMs)
+    } catch {
+      case e: Exception => {

Review comment:
       It's a 2-line block.

##########
File path: core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala
##########
@@ -0,0 +1,215 @@
+/**
+ * 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

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -744,6 +781,165 @@ object DelegationTokenInfoZNode {
   def decode(bytes: Array[Byte]): Option[TokenInformation] = DelegationTokenManager.fromBytes(bytes)
 }
 
+/**
+ * Represents the status of the FeatureZNode.
+ *
+ * Enabled  -> This status means the feature versioning system (KIP-584) is enabled, and, the
+ *             finalized features stored in the FeatureZNode are active. This status is written by
+ *             the controller to the FeatureZNode only when the broker IBP config is greater than
+ *             or equal to KAFKA_2_7_IV0.
+ *
+ * Disabled -> This status means the feature versioning system (KIP-584) is disabled, and, the
+ *             the finalized features stored in the FeatureZNode is not relevant. This status is
+ *             written by the controller to the FeatureZNode only when the broker IBP config
+ *             is less than KAFKA_2_7_IV0.
+ *
+ * The purpose behind the FeatureZNodeStatus is that it helps differentiates between the following
+ * cases:
+ *
+ * 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. In such a case, the
+ *    controller will start up and notice that the FeatureZNode is absent in the new cluster.
+ *    To handle the requirement, the controller will create a FeatureZNode (with enabled status)
+ *    containing the entire list of supported features as its finalized features.
+ *
+ * 2. Cluster upgrade:
+ *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, but
+ *    the Broker binary has been upgraded to a state where it 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 enable
+ *    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. The reason is that enabling all the possible
+ *    features immediately after an upgrade could be harmful to the cluster.
+ *    In such a case:
+ *      - Before the Broker 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 true, then it will react by
+ *        creating a FeatureZNode with disabled status and empty features.
+ *      - After the Broker 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. Cluster 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.
+ */
+object FeatureZNodeStatus extends Enumeration {
+  val Disabled, Enabled = Value
+
+  def withNameOpt(value: Int): Option[Value] = {
+    values.find(_.id == value)
+  }
+}
+
+/**
+ * Represents the contents of the ZK node containing finalized feature information.
+ *
+ * @param status     the status of the ZK node
+ * @param features   the cluster-wide finalized features
+ */
+case class FeatureZNode(status: FeatureZNodeStatus.Value, features: Features[FinalizedVersionRange]) {
+}
+
+object FeatureZNode {
+  private val VersionKey = "version"
+  private val StatusKey = "status"
+  private val FeaturesKey = "features"
+
+  // V1 contains 'version', 'status' and 'features' keys.
+  val V1 = 1
+  val CurrentVersion = V1
+
+  def path = "/feature"
+
+  def asJavaMap(scalaMap: Map[String, Map[String, Short]]): util.Map[String, util.Map[String, java.lang.Short]] = {
+    scalaMap
+      .view.mapValues(_.view.mapValues(scalaShort => java.lang.Short.valueOf(scalaShort)).toMap.asJava)
+      .toMap
+      .asJava
+  }
+
+  /**
+   * Encodes a FeatureZNode to JSON.
+   *
+   * @param featureZNode   FeatureZNode to be encoded
+   *
+   * @return               JSON representation of the FeatureZNode, as an Array[Byte]
+   */
+  def encode(featureZNode: FeatureZNode): Array[Byte] = {
+    val jsonMap = collection.mutable.Map(
+      VersionKey -> CurrentVersion,
+      StatusKey -> featureZNode.status.id,
+      FeaturesKey -> featureZNode.features.toMap)
+    Json.encodeAsBytes(jsonMap.asJava)
+  }
+
+  /**
+   * Decodes the contents of the feature ZK node from Array[Byte] to a FeatureZNode.
+   *
+   * @param jsonBytes   the contents of the feature ZK node
+   *
+   * @return            the FeatureZNode created from jsonBytes
+   *
+   * @throws IllegalArgumentException   if the Array[Byte] can not be decoded.
+   */
+  def decode(jsonBytes: Array[Byte]): FeatureZNode = {
+    Json.tryParseBytes(jsonBytes) match {
+      case Right(js) =>
+        val featureInfo = js.asJsonObject
+        val version = featureInfo(VersionKey).to[Int]
+        if (version < V1) {
+          throw new IllegalArgumentException(s"Unsupported version: $version of feature information: " +
+            s"${new String(jsonBytes, UTF_8)}")
+        }
+
+        val featuresMap = featureInfo
+          .get(FeaturesKey)
+          .flatMap(_.to[Option[Map[String, Map[String, Int]]]])
+
+        if (featuresMap.isEmpty) {
+          throw new IllegalArgumentException("Features map can not be absent in: " +
+            s"${new String(jsonBytes, UTF_8)}")
+        }
+        val features = asJavaMap(
+          featuresMap
+            .map(theMap => theMap.view.mapValues(_.view.mapValues(_.asInstanceOf[Short]).toMap).toMap)
+            .getOrElse(Map[String, Map[String, Short]]()))
+
+        val statusInt = featureInfo
+          .get(StatusKey)
+          .flatMap(_.to[Option[Int]])
+        if (statusInt.isEmpty) {
+          throw new IllegalArgumentException("Status can not be absent in feature information: " +
+            s"${new String(jsonBytes, UTF_8)}")
+        }
+        val status = FeatureZNodeStatus.withNameOpt(statusInt.get)
+        if (status.isEmpty) {
+          throw new IllegalArgumentException(
+            s"Malformed status: $statusInt  found in feature information: ${new String(jsonBytes, UTF_8)}")

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 #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Represents an immutable basic version range using 2 attributes: min and max, each of type long.
+ * The min and max attributes need to satisfy 2 rules:
+ *  - they are each expected to be >= 1, as we only consider positive version values to be valid.
+ *  - max should be >= min.
+ *
+ * The class also provides API to serialize/deserialize the version range to/from a map.
+ * The class allows for configurable labels for the min/max attributes, which can be specialized by
+ * sub-classes (if needed).
+ */
+class BaseVersionRange {
+    // Non-empty label for the min version key, that's used only for serialization/deserialization purposes.
+    private final String minKeyLabel;
+
+    // The value of the minimum version.
+    private final long minValue;

Review comment:
       Done. I have made it `int16` now. Great point.

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -22,7 +22,7 @@ import java.util.Properties
 
 import com.fasterxml.jackson.annotation.JsonProperty
 import com.fasterxml.jackson.core.JsonProcessingException
-import kafka.api.{ApiVersion, KAFKA_0_10_0_IV1, LeaderAndIsr}
+import kafka.api.{ApiVersion, KAFKA_0_10_0_IV1, KAFKA_2_6_IV1, LeaderAndIsr}

Review comment:
       Done. Made it KAFKA_2_7_IV0.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Represents an immutable basic version range using 2 attributes: min and max, each of type long.
+ * The min and max attributes need to satisfy 2 rules:
+ *  - they are each expected to be >= 1, as we only consider positive version values to be valid.
+ *  - max should be >= min.
+ *
+ * The class also provides API to serialize/deserialize the version range to/from a map.
+ * The class allows for configurable labels for the min/max attributes, which can be specialized by
+ * sub-classes (if needed).
+ */
+class BaseVersionRange {
+    // Non-empty label for the min version key, that's used only for serialization/deserialization purposes.
+    private final String minKeyLabel;
+
+    // The value of the minimum version.
+    private final long minValue;
+
+    // Non-empty label for the max version key, that's used only for serialization/deserialization purposes.
+    private final String maxKeyLabel;
+
+    // The value of the maximum version.
+    private final long maxValue;
+
+    /**
+     * Raises an exception unless the following condition is met:
+     * minValue >= 1 and maxValue >= 1 and maxValue >= minValue.
+     *
+     * @param minKeyLabel   Label for the min version key, that's used only for
+     *                      serialization/deserialization purposes.
+     * @param minValue      The minimum version value.
+     * @param maxKeyLabel   Label for the max version key, that's used only for
+     *                      serialization/deserialization purposes.
+     * @param maxValue      The maximum version value.
+     *
+     * @throws IllegalArgumentException   If any of the following conditions are true:
+     *                                     - (minValue < 1) OR (maxValue < 1) OR (maxValue < minValue).
+     *                                     - minKeyLabel is empty, OR, minKeyLabel is empty.
+     */
+    protected BaseVersionRange(String minKeyLabel, long minValue, String maxKeyLabel, long maxValue) {
+        if (minValue < 1 || maxValue < 1 || maxValue < minValue) {
+            throw new IllegalArgumentException(
+                String.format(
+                    "Expected minValue > 1, maxValue > 1 and maxValue >= minValue, but received" +
+                    " minValue: %d, maxValue: %d", minValue, maxValue));
+        }
+        if (minKeyLabel.isEmpty()) {
+            throw new IllegalArgumentException("Expected minKeyLabel to be non-empty.");
+        }
+        if (maxKeyLabel.isEmpty()) {
+            throw new IllegalArgumentException("Expected maxKeyLabel to be non-empty.");
+        }
+        this.minKeyLabel = minKeyLabel;
+        this.minValue = minValue;
+        this.maxKeyLabel = maxKeyLabel;
+        this.maxValue = maxValue;
+    }
+
+    public long min() {
+        return minValue;
+    }
+
+    public long max() {
+        return maxValue;
+    }
+
+    public String toString() {
+        return String.format("%s[%d, %d]", this.getClass().getSimpleName(), min(), max());

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/cluster/Broker.scala
##########
@@ -54,7 +64,7 @@ case class Broker(id: Int, endPoints: Seq[EndPoint], rack: Option[String]) {
     s"$id : ${endPointsMap.values.mkString("(",",",")")} : ${rack.orNull}"
 
   def this(id: Int, host: String, port: Int, listenerName: ListenerName, protocol: SecurityProtocol) = {
-    this(id, Seq(EndPoint(host, port, listenerName, protocol)), None)
+    this(id, Seq(EndPoint(host, port, listenerName, protocol)), None, emptySupportedFeatures)

Review comment:
       Done. Nice catch!

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,85 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange}
+
+// 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) {
+  override def toString(): String = {
+    "FinalizedFeaturesAndEpoch(features=%s, epoch=%s)".format(features, epoch)
+  }
+}
+
+/**
+ * A common mutable cache containing the latest finalized features and epoch. By default the contents of
+ * the cache are empty. This cache needs to be populated at least once for it's contents to become
+ * non-empty. Currently the main reader of this cache is the read path that serves an ApiVersionsRequest,
+ * returning the features information in the response.
+ *
+ * @see FinalizedFeatureChangeListener
+ */
+object FinalizedFeatureCache extends Logging {
+  @volatile private var featuresAndEpoch: Option[FinalizedFeaturesAndEpoch] = Option.empty
+
+  /**
+   * @return   the latest known FinalizedFeaturesAndEpoch. If the returned value is empty, it means
+   *           no FinalizedFeaturesAndEpoch exists in the cache at the time when this
+   *           method is invoked. This result could change in the future whenever the
+   *           updateOrThrow method is invoked.
+   */
+  def get: Option[FinalizedFeaturesAndEpoch] = {
+    featuresAndEpoch
+  }
+
+  def empty: Boolean = {

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -196,6 +219,19 @@ object BrokerIdZNode {
     *   "listener_security_protocol_map":{"CLIENT":"SSL", "REPLICATION":"PLAINTEXT"},
     *   "rack":"dc1"
     * }
+    *
+    * Version 5 (current) JSON schema for a broker is:
+    * {
+    *   "version":5,
+    *   "host":"localhost",
+    *   "port":9092,
+    *   "jmx_port":9999,
+    *   "timestamp":"2233345666",
+    *   "endpoints":["CLIENT://host1:9092", "REPLICATION://host1:9093"],
+    *   "listener_security_protocol_map":{"CLIENT":"SSL", "REPLICATION":"PLAINTEXT"},

Review comment:
       Done. Removed. Great catch!

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Represents an immutable basic version range using 2 attributes: min and max, each of type long.
+ * The min and max attributes need to satisfy 2 rules:
+ *  - they are each expected to be >= 1, as we only consider positive version values to be valid.
+ *  - max should be >= min.
+ *
+ * The class also provides API to serialize/deserialize the version range to/from a map.
+ * The class allows for configurable labels for the min/max attributes, which can be specialized by
+ * sub-classes (if needed).
+ */
+class BaseVersionRange {
+    // Non-empty label for the min version key, that's used only for serialization/deserialization purposes.
+    private final String minKeyLabel;
+
+    // The value of the minimum version.
+    private final long minValue;
+
+    // Non-empty label for the max version key, that's used only for serialization/deserialization purposes.
+    private final String maxKeyLabel;
+
+    // The value of the maximum version.
+    private final long maxValue;
+
+    /**
+     * Raises an exception unless the following condition is met:
+     * minValue >= 1 and maxValue >= 1 and maxValue >= minValue.
+     *
+     * @param minKeyLabel   Label for the min version key, that's used only for
+     *                      serialization/deserialization purposes.
+     * @param minValue      The minimum version value.
+     * @param maxKeyLabel   Label for the max version key, that's used only for
+     *                      serialization/deserialization purposes.
+     * @param maxValue      The maximum version value.
+     *
+     * @throws IllegalArgumentException   If any of the following conditions are true:
+     *                                     - (minValue < 1) OR (maxValue < 1) OR (maxValue < minValue).
+     *                                     - minKeyLabel is empty, OR, minKeyLabel is empty.
+     */
+    protected BaseVersionRange(String minKeyLabel, long minValue, String maxKeyLabel, long maxValue) {
+        if (minValue < 1 || maxValue < 1 || maxValue < minValue) {
+            throw new IllegalArgumentException(
+                String.format(
+                    "Expected minValue > 1, maxValue > 1 and maxValue >= minValue, but received" +
+                    " minValue: %d, maxValue: %d", minValue, maxValue));
+        }
+        if (minKeyLabel.isEmpty()) {
+            throw new IllegalArgumentException("Expected minKeyLabel to be non-empty.");
+        }
+        if (maxKeyLabel.isEmpty()) {
+            throw new IllegalArgumentException("Expected maxKeyLabel to be non-empty.");
+        }
+        this.minKeyLabel = minKeyLabel;
+        this.minValue = minValue;
+        this.maxKeyLabel = maxKeyLabel;
+        this.maxValue = maxValue;
+    }
+
+    public long min() {
+        return minValue;
+    }
+
+    public long max() {
+        return maxValue;
+    }
+
+    public String toString() {
+        return String.format("%s[%d, %d]", this.getClass().getSimpleName(), min(), max());
+    }
+
+    public Map<String, Long> serialize() {

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -113,14 +127,44 @@ public static ApiVersionsResponse fromStruct(Struct struct, short version) {
         }
     }
 
-    public static ApiVersionsResponse apiVersionsResponse(int throttleTimeMs, byte maxMagic) {
+    public static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures) {
+        return apiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, Optional.empty(), Optional.empty());
+    }
+
+    public static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures,
+        Features<FinalizedVersionRange> finalizedFeatures,
+        long finalizedFeaturesEpoch) {
+        return apiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, Optional.of(finalizedFeatures), Optional.of(finalizedFeaturesEpoch));
+    }
+
+    private static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures,
+        Optional<Features<FinalizedVersionRange>> finalizedFeatures,
+        Optional<Long> finalizedFeaturesEpoch) {
         if (maxMagic == RecordBatch.CURRENT_MAGIC_VALUE && throttleTimeMs == DEFAULT_THROTTLE_TIME) {
             return DEFAULT_API_VERSIONS_RESPONSE;
         }
-        return createApiVersionsResponse(throttleTimeMs, maxMagic);
+        return createApiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, finalizedFeatures, finalizedFeaturesEpoch);
     }
 
-    public static ApiVersionsResponse createApiVersionsResponse(int throttleTimeMs, final byte minMagic) {
+    public static ApiVersionsResponse createApiVersionsResponse(
+        int throttleTimeMs,
+        final byte minMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures,
+        Optional<Features<FinalizedVersionRange>> finalizedFeatures,
+        Optional<Long> finalizedFeaturesEpoch

Review comment:
       I had added such APIs previously. But @abbccdda wanted these removed, as they are not currently unused. Please refer to this comment: https://github.com/apache/kafka/pull/8680#discussion_r426931875.
   Please, let me know, and I can add them back if you prefer.

##########
File path: core/src/main/scala/kafka/cluster/Broker.scala
##########
@@ -34,14 +36,22 @@ object Broker {
                                          brokerId: Int,
                                          endpoints: util.List[Endpoint],
                                          interBrokerEndpoint: Endpoint) extends AuthorizerServerInfo
+
+  def apply(id: Int, endPoints: Seq[EndPoint], rack: Option[String]): Broker = {
+    new Broker(id, endPoints, rack, emptySupportedFeatures)
+  }
 }
 
 /**
  * A Kafka broker.
- * A broker has an id, a collection of end-points, an optional rack and a listener to security protocol map.
- * Each end-point is (host, port, listenerName).
+ *
+ * @param id          a broker id
+ * @param endPoints   a collection of: end-point and a listener to security protocol map.

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/KafkaConfig.scala
##########
@@ -442,6 +445,8 @@ object KafkaConfig {
   val ControlledShutdownMaxRetriesProp = "controlled.shutdown.max.retries"
   val ControlledShutdownRetryBackoffMsProp = "controlled.shutdown.retry.backoff.ms"
   val ControlledShutdownEnableProp = "controlled.shutdown.enable"
+  /** ********* Features configuration ***********/
+  val FeatureChangeListenerCacheUpdateWaitTimeMsProp = "feature.listener.cache.update.wait.ms"

Review comment:
       Done. Great point!

##########
File path: clients/src/main/resources/common/message/ApiVersionsResponse.json
##########
@@ -42,6 +42,33 @@
         "about": "The maximum supported version, inclusive." }
     ]},
     { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true,
-      "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }
+      "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." },
+    { "name":  "SupportedFeatures", "type": "[]SupportedFeatureKey",
+      "versions":  "3+", "tag": 0, "taggedVersions": "3+",
+      "about": "Features supported by the broker.",
+      "fields":  [
+        { "name": "Name", "type": "string", "versions": "3+", "mapKey": true,
+          "about": "The name of the feature." },
+        { "name": "MinVersion", "type": "int64", "versions": "3+",
+          "about": "The minimum supported version for the feature." },
+        { "name": "MaxVersion", "type": "int64", "versions": "3+",
+          "about": "The maximum supported version for the feature." }
+      ]
+    },
+    {"name": "FinalizedFeaturesEpoch", "type": "int64", "versions": "3+",

Review comment:
       Done. Changed to `int32` now. Great point!

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Represents an immutable basic version range using 2 attributes: min and max, each of type long.
+ * The min and max attributes need to satisfy 2 rules:
+ *  - they are each expected to be >= 1, as we only consider positive version values to be valid.
+ *  - max should be >= min.
+ *
+ * The class also provides API to serialize/deserialize the version range to/from a map.
+ * The class allows for configurable labels for the min/max attributes, which can be specialized by
+ * sub-classes (if needed).
+ */
+class BaseVersionRange {
+    // Non-empty label for the min version key, that's used only for serialization/deserialization purposes.
+    private final String minKeyLabel;
+
+    // The value of the minimum version.
+    private final long minValue;
+
+    // Non-empty label for the max version key, that's used only for serialization/deserialization purposes.
+    private final String maxKeyLabel;
+
+    // The value of the maximum version.
+    private final long maxValue;
+
+    /**
+     * Raises an exception unless the following condition is met:
+     * minValue >= 1 and maxValue >= 1 and maxValue >= minValue.
+     *
+     * @param minKeyLabel   Label for the min version key, that's used only for
+     *                      serialization/deserialization purposes.
+     * @param minValue      The minimum version value.
+     * @param maxKeyLabel   Label for the max version key, that's used only for
+     *                      serialization/deserialization purposes.
+     * @param maxValue      The maximum version value.
+     *
+     * @throws IllegalArgumentException   If any of the following conditions are true:
+     *                                     - (minValue < 1) OR (maxValue < 1) OR (maxValue < minValue).
+     *                                     - minKeyLabel is empty, OR, minKeyLabel is empty.
+     */
+    protected BaseVersionRange(String minKeyLabel, long minValue, String maxKeyLabel, long maxValue) {
+        if (minValue < 1 || maxValue < 1 || maxValue < minValue) {
+            throw new IllegalArgumentException(
+                String.format(
+                    "Expected minValue > 1, maxValue > 1 and maxValue >= minValue, but received" +

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,85 @@
+package kafka.server

Review comment:
       Done.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
##########
@@ -113,14 +127,44 @@ public static ApiVersionsResponse fromStruct(Struct struct, short version) {
         }
     }
 
-    public static ApiVersionsResponse apiVersionsResponse(int throttleTimeMs, byte maxMagic) {
+    public static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures) {
+        return apiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, Optional.empty(), Optional.empty());
+    }
+
+    public static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures,
+        Features<FinalizedVersionRange> finalizedFeatures,
+        long finalizedFeaturesEpoch) {
+        return apiVersionsResponse(
+            throttleTimeMs, maxMagic, latestSupportedFeatures, Optional.of(finalizedFeatures), Optional.of(finalizedFeaturesEpoch));
+    }
+
+    private static ApiVersionsResponse apiVersionsResponse(
+        int throttleTimeMs,
+        byte maxMagic,
+        Features<SupportedVersionRange> latestSupportedFeatures,
+        Optional<Features<FinalizedVersionRange>> finalizedFeatures,

Review comment:
       It's because non-existing supported features can be represented by an empty map (i.e. broker does not advertise any features). But on the other hand, non-existing finalized features can not be represented by empty map alone, as we need a suitable epoch value that indicates the absence of finalized features. To address this case, I saw 2 ways:
   1) Provide a negative epoch value indicating absence of finalized features, OR
   2) Represent using an empty `Optional` for both finalized features and epoch.
   
   I chose the latter approach. Please, let me know if you have concerns.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,219 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked exactly
+     * once successfully. A subsequent invocation will raise an exception.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     * @throws   FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     * @throws   RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      debug(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status, clearing feature cache.")
+          FinalizedFeatureCache.clear()
+        } else if (featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully. This method returns
+     * immediately if an updateLatestOrThrow call had already completed successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             RuntimeException if the thread was interrupted during wait
+     *
+     *                     TimeoutException if the wait can not be completed in waitTimeMs
+     *                     milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(

Review comment:
       Done. Removed the catch clause and exception wrapping.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,219 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked exactly
+     * once successfully. A subsequent invocation will raise an exception.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     * @throws   FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     * @throws   RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      debug(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status, clearing feature cache.")
+          FinalizedFeatureCache.clear()
+        } else if (featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully. This method returns
+     * immediately if an updateLatestOrThrow call had already completed successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             RuntimeException if the thread was interrupted during wait
+     *
+     *                     TimeoutException if the wait can not be completed in waitTimeMs
+     *                     milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Change notification queue interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)

Review comment:
       It kills the broker because `ShutdownableThread` catches `FatalExitError` and triggers exit sequence: https://github.com/apache/kafka/blob/b8d609c207ed3d1e678c2f1eb6f3cae637f92c30/core/src/main/scala/kafka/utils/ShutdownableThread.scala#L98-L102
   
   I have updated the comment to use the word "eventually".
   Regarding logging fatal and continuing -- the exception caught here almost always indicates a feature incompatibility, and, that means the broker can cause damage if it sticks around. That is why I felt it is better to kill the broker in such a rare incompatibility case.
   
   Please, let me know your thoughts.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,219 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked exactly
+     * once successfully. A subsequent invocation will raise an exception.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     * @throws   FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     * @throws   RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      debug(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status, clearing feature cache.")
+          FinalizedFeatureCache.clear()
+        } else if (featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully. This method returns
+     * immediately if an updateLatestOrThrow call had already completed successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             RuntimeException if the thread was interrupted during wait
+     *
+     *                     TimeoutException if the wait can not be completed in waitTimeMs
+     *                     milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Change notification queue interrupted", e)

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,219 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked exactly
+     * once successfully. A subsequent invocation will raise an exception.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     * @throws   FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     * @throws   RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      debug(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status, clearing feature cache.")
+          FinalizedFeatureCache.clear()
+        } else if (featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully. This method returns
+     * immediately if an updateLatestOrThrow call had already completed successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             RuntimeException if the thread was interrupted during wait
+     *
+     *                     TimeoutException if the wait can not be completed in waitTimeMs
+     *                     milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Change notification queue interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDataChange(): Unit = {
+      info(s"Feature ZK node updated at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDeletion(): Unit = {
+      warn(s"Feature ZK node deleted at path: $path")
+      // This event may happen, rarely (ex: ZK corruption or operational error).
+      // In such a case, we prefer to just log a warning and treat the case as if the node is absent,
+      // and populate the FinalizedFeatureCache with empty finalized features.
+      queue.add(new FeatureCacheUpdater(path))
+    }
+  }
+
+  private val queue = new LinkedBlockingQueue[FeatureCacheUpdater]
+
+  private val thread = new ChangeNotificationProcessorThread("feature-zk-node-event-process-thread")
+
+  /**
+   * This method initializes the feature ZK node change listener. Optionally, it also ensures to
+   * update the FinalizedFeatureCache once with the latest contents of the feature ZK node
+   * (if the node exists). This step helps ensure that feature incompatibilities (if any) in brokers
+   * are conveniently detected before the initOrThrow() method returns to the caller. If feature
+   * incompatibilities are detected, this method will throw an Exception to the caller, and the Broker
+   * will exit eventually.
+   *
+   * @param waitOnceForCacheUpdateMs   # of milli seconds to wait for feature cache to be updated once.
+   *                                   If this parameter <= 0, no wait operation happens.
+   *
+   * @throws Exception if feature incompatibility check could not be finished in a timely manner
+   */
+  def initOrThrow(waitOnceForCacheUpdateMs: Long): Unit = {
+    thread.start()
+    zkClient.registerZNodeChangeHandlerAndCheckExistence(FeatureZNodeChangeHandler)
+
+    if (waitOnceForCacheUpdateMs > 0) {

Review comment:
       Done. I have changed the code disallowing values <= 0.

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -744,6 +782,161 @@ object DelegationTokenInfoZNode {
   def decode(bytes: Array[Byte]): Option[TokenInformation] = DelegationTokenManager.fromBytes(bytes)
 }
 
+/**
+ * Represents the status of the FeatureZNode.
+ *
+ * Enabled  -> This status means the feature versioning system (KIP-584) is enabled, and, the
+ *             finalized features stored in the FeatureZNode are active. This status is written by
+ *             the controller to the FeatureZNode only when the broker IBP config is greater than
+ *             or equal to KAFKA_2_6_IV1.
+ *
+ * Disabled -> This status means the feature versioning system (KIP-584) is disabled, and, the
+ *             the finalized features stored in the FeatureZNode is not relevant. This status is
+ *             written by the controller to the FeatureZNode only when the broker IBP config
+ *             is less than KAFKA_2_6_IV1.
+ *
+ * The purpose behind the FeatureZNodeStatus is that it helps differentiates between the following
+ * cases:
+ *
+ * 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_6_IV1. In such a case, the
+ *    controller will start up and notice that the FeatureZNode is absent in the new cluster.
+ *    To handle the requirement, the controller will create a FeatureZNode (with enabled status)
+ *    containing the entire list of supported features as its finalized features.
+ *
+ * 2. Cluster upgrade:
+ *    Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_6_IV1, but
+ *    the Broker binary has been upgraded to a state where it 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 enable
+ *    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_6_IV1. The reason is that enabling all the possible
+ *    features immediately after an upgrade could be harmful to the cluster.
+ *    In such a case:
+ *      - Before the Broker upgrade (i.e. IBP config set to less than KAFKA_2_6_IV1), the controller
+ *        will start up and check if the FeatureZNode is absent. If true, then it will react by
+ *        creating a FeatureZNode with disabled status and empty features.
+ *      - After the Broker upgrade (i.e. IBP config set to greater than or equal to KAFKA_2_6_IV1),
+ *        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.
+ *
+ * 2. Cluster downgrade:
+ *    Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
+ *    KAFKA_2_6_IV1. Then, the user decided to downgrade the cluster by setting IBP config to a
+ *    value less than KAFKA_2_6_IV1. 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.
+ */
+object FeatureZNodeStatus extends Enumeration {
+  val Disabled, Enabled = Value
+
+  def withNameOpt(value: Int): Option[Value] = {
+    values.find(_.id == value)
+  }
+}
+
+/**
+ * Represents the contents of the ZK node containing finalized feature information.
+ *
+ * @param status     the status of the ZK node
+ * @param features   the cluster-wide finalized features
+ */
+case class FeatureZNode(status: FeatureZNodeStatus.Value, features: Features[FinalizedVersionRange]) {
+}
+
+object FeatureZNode {
+  private val VersionKey = "version"
+  private val StatusKey = "status"
+  private val FeaturesKey = "features"
+
+  // V0 contains 'version', 'status' and 'features' keys.
+  val V0 = 0
+  val CurrentVersion = V0
+
+  def path = "/feature"
+
+  def asJavaMap(scalaMap: Map[String, Map[String, Long]]): util.Map[String, util.Map[String, java.lang.Long]] = {
+    scalaMap
+      .view.mapValues(_.view.mapValues(scalaLong => java.lang.Long.valueOf(scalaLong)).toMap.asJava)
+      .toMap
+      .asJava
+  }
+
+  /**
+   * Encodes a FeatureZNode to JSON.
+   *
+   * @param featureZNode   FeatureZNode to be encoded
+   *
+   * @return               JSON representation of the FeatureZNode, as an Array[Byte]
+   */
+  def encode(featureZNode: FeatureZNode): Array[Byte] = {
+    val jsonMap = collection.mutable.Map(
+      VersionKey -> CurrentVersion,
+      StatusKey -> featureZNode.status.id,

Review comment:
       Sure. I will be happy to follow up on this. Trying to understand the process -- should I update the KIP and send an email as FYI to `dev@kafka.apache.org` ?

##########
File path: core/src/main/scala/kafka/server/SupportedFeatures.scala
##########
@@ -0,0 +1,76 @@
+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 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.
+ *
+ * NOTE: the update() and clear() APIs of this class should be used only for testing purposes.
+ */
+object SupportedFeatures extends Logging {
+
+  /**
+   * This is the latest features supported by the Broker.
+   * This is currently empty, but in the future as we define supported features, this map should be
+   * populated.
+   */
+  @volatile private var supportedFeatures = emptySupportedFeatures
+
+  /**
+   * Returns a reference to the latest features supported by the Broker.
+   */
+  def get: Features[SupportedVersionRange] = {
+    supportedFeatures
+  }
+
+  // For testing only.
+  def update(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // For testing only.
+  def clear(): Unit = {
+    supportedFeatures = emptySupportedFeatures
+  }
+
+  /**
+   * 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] = {
+    val incompatibilities = finalized.features.asScala.collect {
+      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 (incompatibilities.nonEmpty) {
+      warn("Feature incompatibilities seen: " + incompatibilities.map{ case(_, _, errorReason) => errorReason })
+    }
+    Features.finalizedFeatures(incompatibilities.map(item => (item._1, item._2)).toMap.asJava)

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 #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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



##########
File path: clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Represents an immutable basic version range using 2 attributes: min and max of type long.
+ * The min and max attributes are expected to be >= 1, and with max >= min.
+ *
+ * The class also provides API to serialize/deserialize the version range to/from a map.
+ * The class allows for configurable labels for the min/max attributes, which can be specialized by
+ * sub-classes (if needed).
+ */
+class BaseVersionRange {

Review comment:
       Do we want to get a unit test class for `BaseVersionRange`?

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ * @see SupportedVersionRange
+ * @see FinalizedVersionRange
+ */
+public class Features<VersionRangeType extends BaseVersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to VersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "supported" features.
+     */
+    public static Features<SupportedVersionRange> supportedFeatures(Map<String, SupportedVersionRange> features) {
+        return new Features<>(features);
+    }
+
+    /**
+     * @param features   Map of feature name to FinalizedVersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "finalized" features.
+     */
+    public static Features<FinalizedVersionRange> finalizedFeatures(Map<String, FinalizedVersionRange> features) {
+        return new Features<>(features);
+    }
+
+    // Visible for testing.
+    public static Features<FinalizedVersionRange> emptyFinalizedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public static Features<SupportedVersionRange> emptySupportedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public Map<String, VersionRangeType> features() {
+        return features;
+    }
+
+    public boolean empty() {
+        return features.isEmpty();
+    }
+
+    /**
+     * @param  feature   name of the feature
+     *
+     * @return           the VersionRangeType corresponding to the feature name, or null if absent
+     */
+    public VersionRangeType get(String feature) {
+        return features.get(feature);
+    }
+
+    public String toString() {
+        return String.format(
+            "Features{%s}",
+            features
+                .entrySet()
+                .stream()
+                .map(entry -> String.format("(%s -> %s)", entry.getKey(), entry.getValue()))
+                .collect(joining(", "))
+        );
+    }
+
+    /**
+     * @return   A map with underlying features serialized. The returned value can be deserialized
+     *           using one of the deserialize* APIs.

Review comment:
       `deserialize()`? I think the second sentence is redundant.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/FinalizedVersionRange.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.feature;
+
+import java.util.Map;
+
+/**
+ * A specialization of {@link BaseVersionRange} representing a range of version levels.
+ * NOTE: This is the backing class used to define the min/max version levels for finalized features.
+ */
+public class FinalizedVersionRange extends BaseVersionRange {
+    // Label for the min version key, that's used only for serialization/deserialization purposes.
+    private static final String MIN_VERSION_LEVEL_KEY_LABEL = "min_version_level";
+
+    // Label for the max version key, that's used only for serialization/deserialization purposes.
+    private static final String MAX_VERSION_LEVEL_KEY_LABEL = "max_version_level";
+
+    public FinalizedVersionRange(long minVersionLevel, long maxVersionLevel) {
+        super(MIN_VERSION_LEVEL_KEY_LABEL, minVersionLevel, MAX_VERSION_LEVEL_KEY_LABEL, maxVersionLevel);
+    }
+
+    public static FinalizedVersionRange deserialize(Map<String, Long> serialized) {
+        return new FinalizedVersionRange(
+            BaseVersionRange.valueOrThrow(MIN_VERSION_LEVEL_KEY_LABEL, serialized),
+            BaseVersionRange.valueOrThrow(MAX_VERSION_LEVEL_KEY_LABEL, serialized));
+    }
+
+    private boolean isCompatibleWith(BaseVersionRange versionRange) {

Review comment:
       Should be `SupportedVersionRange` 

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,143 @@
+package org.apache.kafka.common.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ */
+public class Features<VersionRangeType extends VersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;

Review comment:
       Yea, the reasoning is that we have `get` call blindly look up inside `features` which in this case null is not valid. And I don't feel passing `null` makes sense for the caller, correct?

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/FinalizedVersionRange.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.feature;
+
+import java.util.Map;
+
+/**
+ * A specialization of {@link BaseVersionRange} representing a range of version levels.
+ * NOTE: This is the backing class used to define the min/max version levels for finalized features.
+ */
+public class FinalizedVersionRange extends BaseVersionRange {
+    // Label for the min version key, that's used only for serialization/deserialization purposes.
+    private static final String MIN_VERSION_LEVEL_KEY_LABEL = "min_version_level";
+
+    // Label for the max version key, that's used only for serialization/deserialization purposes.
+    private static final String MAX_VERSION_LEVEL_KEY_LABEL = "max_version_level";
+
+    public FinalizedVersionRange(long minVersionLevel, long maxVersionLevel) {
+        super(MIN_VERSION_LEVEL_KEY_LABEL, minVersionLevel, MAX_VERSION_LEVEL_KEY_LABEL, maxVersionLevel);
+    }
+
+    public static FinalizedVersionRange deserialize(Map<String, Long> serialized) {
+        return new FinalizedVersionRange(
+            BaseVersionRange.valueOrThrow(MIN_VERSION_LEVEL_KEY_LABEL, serialized),
+            BaseVersionRange.valueOrThrow(MAX_VERSION_LEVEL_KEY_LABEL, serialized));
+    }
+
+    private boolean isCompatibleWith(BaseVersionRange versionRange) {
+        return min() >= versionRange.min() && max() <= versionRange.max();

Review comment:
       Just for the sake of argument, I feel we could remove this method and just test:
   ```
   min() < supportedVersionRange.min() || max() > supportedVersionRange.max()
   ```
   for incompatibility.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,203 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             - RuntimeException if the thread was interrupted during wait
+     *                     - TimeoutException if the wait can not be completed in waitTimeMs
+     *                       milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()

Review comment:
       Is it possible to have no enqueued updater, and cause this function block the thread indefinitely? 

##########
File path: clients/src/test/java/org/apache/kafka/common/feature/FinalizedVersionRangeTest.java
##########
@@ -0,0 +1,135 @@
+package org.apache.kafka.common.feature;

Review comment:
       Missing header

##########
File path: clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java
##########
@@ -17,13 +17,19 @@
 
 package org.apache.kafka.common.requests;
 
+import org.apache.kafka.common.feature.Features;

Review comment:
       Seems we didn't trigger style check on this new class.

##########
File path: clients/src/test/java/org/apache/kafka/common/feature/FinalizedVersionRangeTest.java
##########
@@ -0,0 +1,135 @@
+package org.apache.kafka.common.feature;
+
+import java.util.Map;
+
+import org.junit.Test;
+
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class FinalizedVersionRangeTest {

Review comment:
       What's the difference between this test class and its super class test case? Same question for `SupportedVersionRangeTest`

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,93 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, VersionLevelRange}
+
+// 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[VersionLevelRange], epoch: Int) {
+
+  def isValid(newEpoch: Int): Boolean = {
+    newEpoch >= epoch
+  }
+
+  override def toString(): String = {
+    "FinalizedFeaturesAndEpoch(features=%s, epoch=%s)".format(features, epoch)
+  }
+}
+
+/**
+ * A mutable cache containing the latest finalized features and epoch. This cache is populated by a
+ * FinalizedFeatureChangeListener.
+ *
+ * Currently the main reader of this cache is the read path that serves an ApiVersionsRequest
+ * returning the features information in the response. In the future, as the feature versioning
+ * system in KIP-584 is used more widely, this cache could be read by other read paths trying to
+ * learn the finalized feature information.
+ */
+object FinalizedFeatureCache extends Logging {
+  @volatile private var featuresAndEpoch: Option[FinalizedFeaturesAndEpoch] = Option.empty
+
+  /**
+   * @return   the latest known FinalizedFeaturesAndEpoch. If the returned value is empty, it means
+   *           no FinalizedFeaturesAndEpoch exists in the cache at the time when this
+   *           method is invoked. This result could change in the future whenever the
+   *           updateOrThrow method is invoked.
+   */
+  def get: Option[FinalizedFeaturesAndEpoch] = {
+    featuresAndEpoch
+  }
+
+  def empty: Boolean = {
+    featuresAndEpoch.isEmpty
+  }
+
+  /**
+   * Clears all existing finalized features and epoch from the cache.
+   */
+  def clear(): Unit = {
+    featuresAndEpoch = Option.empty
+    info("Cleared cache")
+  }
+
+  /**
+   * Updates the cache to the latestFeatures, and updates the existing epoch to latestEpoch.
+   * Raises an exception when the operation is not successful.
+   *
+   * @param latestFeatures   the latest finalized features to be set in the cache
+   * @param latestEpoch      the latest epoch value to be set in the cache
+   *
+   * @throws                 FeatureCacheUpdateException if the cache update operation fails
+   *                         due to invalid parameters or incompatibilities with the broker's
+   *                         supported features. In such a case, the existing cache contents are
+   *                         not modified.
+   */
+  def updateOrThrow(latestFeatures: Features[VersionLevelRange], latestEpoch: Int): Unit = {
+    updateOrThrow(FinalizedFeaturesAndEpoch(latestFeatures, latestEpoch))
+  }
+
+  private def updateOrThrow(latest: FinalizedFeaturesAndEpoch): Unit = {
+    val existingStr = featuresAndEpoch.map(existing => existing.toString).getOrElse("<empty>")
+    if (!featuresAndEpoch.isEmpty && featuresAndEpoch.get.epoch > latest.epoch) {
+      val errorMsg = ("FinalizedFeatureCache update failed due to invalid epoch in new finalized %s." +
+        " The existing finalized is %s").format(latest, existingStr)
+      throw new FeatureCacheUpdateException(errorMsg)
+    } else {
+      val incompatibleFeatures = SupportedFeatures.incompatibleFeatures(latest.features)
+      if (incompatibleFeatures.nonEmpty) {
+        val errorMsg = ("FinalizedFeatureCache updated failed since feature compatibility" +
+          " checks failed! Supported %s has incompatibilities with the latest finalized %s." +
+          " The incompatible features are: %s.").format(
+          SupportedFeatures.get, latest, incompatibleFeatures)
+        throw new FeatureCacheUpdateException(errorMsg)
+      }
+    }
+    val logMsg = "Updated cache from existing finalized %s to latest finalized %s".format(

Review comment:
       Could we move this logic as part of inner else? Like:
   ```
   else {
         val incompatibleFeatures = SupportedFeatures.incompatibleFeatures(latest.features)
         if (incompatibleFeatures.nonEmpty) {
           val errorMsg = ("FinalizedFeatureCache updated failed since feature compatibility" +
             " checks failed! Supported %s has incompatibilities with the latest finalized %s." +
             " The incompatible features are: %s.").format(
             SupportedFeatures.get, latest, incompatibleFeatures)
           throw new FeatureCacheUpdateException(errorMsg)
         } else {
           val logMsg = "Updated cache from existing finalized %s to latest finalized %s".format(
             oldFeatureAndEpoch, latest)
           featuresAndEpoch = Some(latest)
           info(logMsg)
         }
       }
   ```
   It makes the if-else logic more tight.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,203 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             - RuntimeException if the thread was interrupted during wait

Review comment:
       I don't think this is scala accepted comment style to add `-`, do you see a warning?

##########
File path: core/src/main/scala/kafka/server/KafkaConfig.scala
##########
@@ -812,6 +817,8 @@ object KafkaConfig {
   val ControlledShutdownMaxRetriesDoc = "Controlled shutdown can fail for multiple reasons. This determines the number of retries when such failure happens"
   val ControlledShutdownRetryBackoffMsDoc = "Before each retry, the system needs time to recover from the state that caused the previous failure (Controller fail over, replica lag etc). This config determines the amount of time to wait before retrying."
   val ControlledShutdownEnableDoc = "Enable controlled shutdown of the server"
+  /** ********* Feature configuration ***********/
+  val FeatureChangeListenerCacheUpdateWaitTimeMsDoc = "# of milli seconds to wait for feature cache to be updated once."

Review comment:
       `wait time for the first feature cache update upon initialization`

##########
File path: core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala
##########
@@ -811,8 +828,16 @@ class KafkaZkClientTest extends ZooKeeperTestHarness {
     assertEquals(Seq.empty, zkClient.getSortedBrokerList)
     assertEquals(None, zkClient.getBroker(0))
 
-    val brokerInfo0 = createBrokerInfo(0, "test.host0", 9998, SecurityProtocol.PLAINTEXT)
-    val brokerInfo1 = createBrokerInfo(1, "test.host1", 9999, SecurityProtocol.SSL)
+    val brokerInfo0 = createBrokerInfo(

Review comment:
       If we are not validating the features by extracting them, I think we do not need to pass in a non-empty feature list?

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/FinalizedVersionRange.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.feature;
+
+import java.util.Map;
+
+/**
+ * A specialization of {@link BaseVersionRange} representing a range of version levels.
+ * NOTE: This is the backing class used to define the min/max version levels for finalized features.
+ */
+public class FinalizedVersionRange extends BaseVersionRange {
+    // Label for the min version key, that's used only for serialization/deserialization purposes.
+    private static final String MIN_VERSION_LEVEL_KEY_LABEL = "min_version_level";
+
+    // Label for the max version key, that's used only for serialization/deserialization purposes.
+    private static final String MAX_VERSION_LEVEL_KEY_LABEL = "max_version_level";
+
+    public FinalizedVersionRange(long minVersionLevel, long maxVersionLevel) {
+        super(MIN_VERSION_LEVEL_KEY_LABEL, minVersionLevel, MAX_VERSION_LEVEL_KEY_LABEL, maxVersionLevel);
+    }
+
+    public static FinalizedVersionRange deserialize(Map<String, Long> serialized) {
+        return new FinalizedVersionRange(
+            BaseVersionRange.valueOrThrow(MIN_VERSION_LEVEL_KEY_LABEL, serialized),
+            BaseVersionRange.valueOrThrow(MAX_VERSION_LEVEL_KEY_LABEL, serialized));
+    }
+
+    private boolean isCompatibleWith(BaseVersionRange versionRange) {
+        return min() >= versionRange.min() && max() <= versionRange.max();
+    }
+
+    /**
+     * 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.
+     *
+     * @param versionRange   the SupportedVersionRange to be checked

Review comment:
       nit: supportedVersionRange

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -146,7 +162,14 @@ object BrokerIdZNode {
     val plaintextEndpoint = broker.endPoints.find(_.securityProtocol == SecurityProtocol.PLAINTEXT).getOrElse(
       new EndPoint(null, -1, null, null))
     encode(brokerInfo.version, plaintextEndpoint.host, plaintextEndpoint.port, broker.endPoints, brokerInfo.jmxPort,
-      broker.rack)
+      broker.rack, broker.features)
+  }
+
+  def asJavaMap(brokerInfo: JsonObject): util.Map[String, util.Map[String, java.lang.Long]] = {

Review comment:
       s/asJavaMap/featuresAsJavaMap

##########
File path: core/src/main/scala/kafka/server/SupportedFeatures.scala
##########
@@ -0,0 +1,74 @@
+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 common 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.
+ */
+object SupportedFeatures extends Logging {
+
+  /**
+   * This is the latest features supported by the Broker.
+   * This is currently empty, but in the future as we define supported features, this map should be
+   * populated.
+   */
+  @volatile private var supportedFeatures = emptySupportedFeatures
+
+  /**
+   * Returns the latest features supported by the Broker.
+   */
+  def get: Features[SupportedVersionRange] = {
+    supportedFeatures
+  }
+
+  // Should be used only for testing.
+  def update(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // Should be used only for testing.
+  def clear(): Unit = {
+    supportedFeatures = emptySupportedFeatures
+  }
+
+  /**
+   * 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 cluster-wide finalized feature. This can happen because a provided
+   * cluster-wide 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 set of incompatible feature names. If the returned set is empty, it
+   *                    means there were no feature incompatibilities found.
+   */
+  def incompatibleFeatures(finalized: Features[FinalizedVersionRange]): Set[String] = {

Review comment:
       I'm slightly inclined to return a set of features instead of just strings, and make the string conversion as a helper. But I leave this up to you to decide, and we could always adapt the function to make it more useful in other scenarios as needed.

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -81,17 +83,26 @@ object BrokerIdsZNode {
 object BrokerInfo {
 
   /**
-   * Create a broker info with v4 json format (which includes multiple endpoints and rack) if
-   * the apiVersion is 0.10.0.X or above. Register the broker with v2 json format otherwise.
+   * - Create a broker info with v5 json format if the apiVersion is 2.6.x or above.
+   * - Create a broker info with v4 json format (which includes multiple endpoints and rack) if
+   *   the apiVersion is 0.10.0.X or above but lesser than 2.6.x.
+   * - Register the broker with v2 json format otherwise.
    *
    * Due to KAFKA-3100, 0.9.0.0 broker and old clients will break if JSON version is above 2.
    *
-   * We include v2 to make it possible for the broker to migrate from 0.9.0.0 to 0.10.0.X or above without having to
-   * upgrade to 0.9.0.1 first (clients have to be upgraded to 0.9.0.1 in any case).
+   * We include v2 to make it possible for the broker to migrate from 0.9.0.0 to 0.10.0.X or above
+   * without having to upgrade to 0.9.0.1 first (clients have to be upgraded to 0.9.0.1 in
+   * any case).
    */
   def apply(broker: Broker, apiVersion: ApiVersion, jmxPort: Int): BrokerInfo = {
-    // see method documentation for the reason why we do this
-    val version = if (apiVersion >= KAFKA_0_10_0_IV1) 4 else 2
+    val version = {
+      if (apiVersion >= KAFKA_0_10_0_IV1)

Review comment:
       aha, the order is wrong for `KAFKA_0_10_0_IV1` and `KAFKA_2_6_IV1`

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -744,6 +782,90 @@ object DelegationTokenInfoZNode {
   def decode(bytes: Array[Byte]): Option[TokenInformation] = DelegationTokenManager.fromBytes(bytes)
 }
 
+object FeatureZNodeStatus extends Enumeration {
+  val Disabled, Enabled = Value
+
+  def withNameOpt(value: Int): Option[Value] = {
+    values.find(_.id == value)
+  }
+}
+
+case class FeatureZNode(status: FeatureZNodeStatus.Value, features: Features[FinalizedVersionRange]) {
+}
+
+object FeatureZNode {
+  private val VersionKey = "version"
+  private val StatusKey = "status"
+  private val FeaturesKey = "features"
+
+  // Version0 contains 'version', 'status' and 'features' keys.
+  val Version0 = 0
+  val CurrentVersion = Version0
+
+  def path = "/feature"
+
+  def asJavaMap(scalaMap: Map[String, Map[String, Long]]): util.Map[String, util.Map[String, java.lang.Long]] = {
+    scalaMap
+      .view.mapValues(_.view.mapValues(scalaLong => java.lang.Long.valueOf(scalaLong)).toMap.asJava)
+      .toMap
+      .asJava
+  }
+
+  def encode(featureZNode: FeatureZNode): Array[Byte] = {
+    val jsonMap = collection.mutable.Map(
+      VersionKey -> CurrentVersion,
+      StatusKey -> featureZNode.status.id,
+      FeaturesKey -> featureZNode.features.serialize)
+    Json.encodeAsBytes(jsonMap.asJava)
+  }
+
+  def decode(jsonBytes: Array[Byte]): FeatureZNode = {
+    Json.tryParseBytes(jsonBytes) match {
+      case Right(js) =>
+        val featureInfo = js.asJsonObject
+        val version = featureInfo(VersionKey).to[Int]
+        if (version < Version0 || version > CurrentVersion) {
+          throw new KafkaException(s"Unsupported version: $version of feature information: " +
+            s"${new String(jsonBytes, UTF_8)}")
+        }
+
+        val featuresMap = featureInfo
+          .get(FeaturesKey)
+          .flatMap(_.to[Option[Map[String, Map[String, Long]]]])
+        if (featuresMap.isEmpty) {
+          throw new KafkaException("Features map can not be absent in: " +
+            s"${new String(jsonBytes, UTF_8)}")
+        }
+        val features = asJavaMap(featuresMap.get)
+
+        val statusInt = featureInfo
+          .get(StatusKey)
+          .flatMap(_.to[Option[Int]])
+        if (statusInt.isEmpty) {
+          throw new KafkaException("Status can not be absent in feature information: " +
+            s"${new String(jsonBytes, UTF_8)}")
+        }
+        val status = FeatureZNodeStatus.withNameOpt(statusInt.get)
+        if (status.isEmpty) {

Review comment:
       Could we log statusInt here as well? Also I feel the exception should be thrown from  `FeatureZNodeStatus.withNameOpt`

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/Features.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * Represents an immutable dictionary with key being feature name, and value being VersionRangeType.
+ * Also provides API to serialize/deserialize the features and their version ranges to/from a map.
+ *
+ * This class can be instantiated only using its factory functions, with the important ones being:
+ * Features.supportedFeatures(...) and Features.finalizedFeatures(...).
+ *
+ * @param <VersionRangeType> is the type of version range.
+ * @see SupportedVersionRange
+ * @see FinalizedVersionRange
+ */
+public class Features<VersionRangeType extends BaseVersionRange> {
+    private final Map<String, VersionRangeType> features;
+
+    /**
+     * Constructor is made private, as for readability it is preferred the caller uses one of the
+     * static factory functions for instantiation (see below).
+     *
+     * @param features   Map of feature name to type of VersionRange, as the backing data structure
+     *                   for the Features object.
+     */
+    private Features(Map<String, VersionRangeType> features) {
+        this.features = features;
+    }
+
+    /**
+     * @param features   Map of feature name to VersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "supported" features.
+     */
+    public static Features<SupportedVersionRange> supportedFeatures(Map<String, SupportedVersionRange> features) {
+        return new Features<>(features);
+    }
+
+    /**
+     * @param features   Map of feature name to FinalizedVersionRange, as the backing data structure
+     *                   for the Features object.
+     * @return           Returns a new Features object representing "finalized" features.
+     */
+    public static Features<FinalizedVersionRange> finalizedFeatures(Map<String, FinalizedVersionRange> features) {
+        return new Features<>(features);
+    }
+
+    // Visible for testing.
+    public static Features<FinalizedVersionRange> emptyFinalizedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public static Features<SupportedVersionRange> emptySupportedFeatures() {
+        return new Features<>(new HashMap<>());
+    }
+
+    public Map<String, VersionRangeType> features() {
+        return features;
+    }
+
+    public boolean empty() {
+        return features.isEmpty();
+    }
+
+    /**
+     * @param  feature   name of the feature
+     *
+     * @return           the VersionRangeType corresponding to the feature name, or null if absent
+     */
+    public VersionRangeType get(String feature) {
+        return features.get(feature);
+    }
+
+    public String toString() {
+        return String.format(
+            "Features{%s}",
+            features
+                .entrySet()
+                .stream()
+                .map(entry -> String.format("(%s -> %s)", entry.getKey(), entry.getValue()))
+                .collect(joining(", "))
+        );
+    }
+
+    /**
+     * @return   A map with underlying features serialized. The returned value can be deserialized
+     *           using one of the deserialize* APIs.
+     */
+    public Map<String, Map<String, Long>> serialize() {
+        return features.entrySet().stream().collect(
+            Collectors.toMap(
+                Map.Entry::getKey,
+                entry -> entry.getValue().serialize()));
+    }
+
+    /**
+     * Deserialize a map to Features<FinalizedVersionRange>.
+     *
+     * @param serialized   the serialized representation of a Features<FinalizedVersionRange> object,
+     *                     generated using the serialize() API.
+     *
+     * @return             the deserialized Features<FinalizedVersionRange> object
+     */
+    public static Features<FinalizedVersionRange> deserializeFinalizedFeatures(
+        Map<String, Map<String, Long>> serialized) {
+        return finalizedFeatures(serialized.entrySet().stream().collect(
+            Collectors.toMap(
+                Map.Entry::getKey,
+                entry -> FinalizedVersionRange.deserialize(entry.getValue()))));
+    }
+
+    /**
+     * Deserializes a map to Features<SupportedVersionRange>.
+     *
+     * @param serialized   the serialized representation of a Features<SupportedVersionRange> object,
+     *                     generated using the serialize() API.
+     *
+     * @return             the deserialized Features<SupportedVersionRange> object
+     */
+    public static Features<SupportedVersionRange> deserializeSupportedFeatures(

Review comment:
       Maybe I'm a bit too obsessive about code duplication, but after I made an attempt I thought we could actually have the internal deserialization logic shared between `deserializeFinalizedFeatures` and `deserializeSupportedFeatures` by making a template
   ```
    public static Features<FinalizedVersionRange> deserializeFinalizedFeatures(Map<String, Map<String, Long>> serialized) {
           return deserializeFeatures(serialized, FinalizedVersionRange::deserialize);
       }
   
       public static Features<SupportedVersionRange> deserializeSupportedFeatures(
           Map<String, Map<String, Long>> serialized) {
           return deserializeFeatures(serialized, SupportedVersionRange::deserialize);
       }
           
       
       private interface Deserializer<V> {
           V deserialize(Map<String, Long> serialized);
       }
   
   
       private static <V extends BaseVersionRange> Features<V> deserializeFeatures(Map<String, Map<String, Long>> serialized, Deserializer<V> deserializer) {
           return new Features<>(serialized.entrySet().stream().collect(
               Collectors.toMap(
                   Map.Entry::getKey,
                   entry -> deserializer.deserialize(entry.getValue()))));
       }
   ```

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
##########
@@ -0,0 +1,88 @@
+package kafka.server
+
+import kafka.utils.Logging
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange}
+
+// 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) {
+  override def toString(): String = {
+    "FinalizedFeaturesAndEpoch(features=%s, epoch=%s)".format(features, epoch)
+  }
+}
+
+/**
+ * A mutable cache containing the latest finalized features and epoch. This cache is populated by a
+ * {@link FinalizedFeatureChangeListener}.
+ *
+ * Currently the main reader of this cache is the read path that serves an ApiVersionsRequest

Review comment:
       I think we don't need to talk about future work inside the comment, just making it clear that the read path for serving ApiVersionsRequest is the only reader as of now.

##########
File path: clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.feature;
+
+import java.util.Map;
+
+/**
+ * A specialization of VersionRange representing a range of versions.
+ * NOTE: This is the backing class used to define the min/max versions for supported features.

Review comment:
       Why this is a `NOTE`? Could we just comment like:
   ```
   An extended BaseVersionRange representing the min/max versions for supported features.
   ```

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,203 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification

Review comment:
       nit: provide

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,203 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      if (version == ZkVersion.UnknownVersion) {

Review comment:
       Does the version field existence guarantee there is a valid feature data node or not? In fact, `getDataAndVersion` returns an optional data. I checked the getDataAndVersion caller `ProducerIdManager`, there is a handling for empty data which I feel we should have as well. 
   Additionally, I think since we haven't implemented the write path yet, could we get a ticket to write down a short description on how the write path shall look like, by defining the different cases like:
   ```
   empty dataBytes, valid version 
   valid dataBytes, valid version 
   empty dataBytes, unknown version 
   valid dataBytes, unknown version 
   ```
   if that makes sense, so that we could keep track of the design decisions we made in the read path PR when implementing the write path.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,203 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             - RuntimeException if the thread was interrupted during wait
+     *                     - TimeoutException if the wait can not be completed in waitTimeMs
+     *                       milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    private def processNotification(): Unit = {
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")

Review comment:
       Do we need the comment to be on info level?

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,203 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             - RuntimeException if the thread was interrupted during wait
+     *                     - TimeoutException if the wait can not be completed in waitTimeMs
+     *                       milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    private def processNotification(): Unit = {

Review comment:
       feel neutral about this helper function

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,203 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             - RuntimeException if the thread was interrupted during wait
+     *                     - TimeoutException if the wait can not be completed in waitTimeMs
+     *                       milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    private def processNotification(): Unit = {
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")
+      processNotification()
+    }
+
+    override def handleDataChange(): Unit = {
+      info(s"Feature ZK node updated at path: $path")
+      processNotification()
+    }
+
+    override def handleDeletion(): Unit = {
+      warn(s"Feature ZK node deleted at path: $path")
+      // This event may happen, rarely (ex: operational error).
+      // In such a case, we prefer to just log a warning and treat the case as if the node is absent,
+      // and populate the FinalizedFeatureCache with empty finalized features.
+      processNotification()
+    }
+  }
+
+  private val queue = new LinkedBlockingQueue[FeatureCacheUpdater]
+
+  private val thread = new ChangeNotificationProcessorThread("feature-zk-node-event-process-thread")
+
+  /**
+   * This method initializes the feature ZK node change listener. Optionally, it also ensures to
+   * update the FinalizedFeatureCache once with the latest contents of the feature ZK node
+   * (if the node exists). This step helps ensure that feature incompatibilities (if any) in brokers
+   * are conveniently detected before the initOrThrow() method returns to the caller. If feature
+   * incompatibilities are detected, this method will throw an Exception to the caller, and the Broker
+   * would exit eventually.
+   *
+   * @param waitOnceForCacheUpdateMs   # of milli seconds to wait for feature cache to be updated once.
+   *                                   If this parameter <= 0, no wait operation happens.
+   *
+   * @throws Exception if feature incompatibility check could not be finished in a timely manner
+   */
+  def initOrThrow(waitOnceForCacheUpdateMs: Long): Unit = {
+    thread.start()
+    zkClient.registerZNodeChangeHandlerAndCheckExistence(FeatureZNodeChangeHandler)
+
+    if (waitOnceForCacheUpdateMs > 0) {
+      val barrier = new CountDownLatch(1)

Review comment:
       nit: don't feel strong about having this parameter

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,203 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             - RuntimeException if the thread was interrupted during wait
+     *                     - TimeoutException if the wait can not be completed in waitTimeMs
+     *                       milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    private def processNotification(): Unit = {
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")
+      processNotification()
+    }
+
+    override def handleDataChange(): Unit = {
+      info(s"Feature ZK node updated at path: $path")
+      processNotification()
+    }
+
+    override def handleDeletion(): Unit = {
+      warn(s"Feature ZK node deleted at path: $path")
+      // This event may happen, rarely (ex: operational error).
+      // In such a case, we prefer to just log a warning and treat the case as if the node is absent,
+      // and populate the FinalizedFeatureCache with empty finalized features.
+      processNotification()
+    }
+  }
+
+  private val queue = new LinkedBlockingQueue[FeatureCacheUpdater]
+
+  private val thread = new ChangeNotificationProcessorThread("feature-zk-node-event-process-thread")
+
+  /**
+   * This method initializes the feature ZK node change listener. Optionally, it also ensures to
+   * update the FinalizedFeatureCache once with the latest contents of the feature ZK node
+   * (if the node exists). This step helps ensure that feature incompatibilities (if any) in brokers
+   * are conveniently detected before the initOrThrow() method returns to the caller. If feature
+   * incompatibilities are detected, this method will throw an Exception to the caller, and the Broker
+   * would exit eventually.
+   *
+   * @param waitOnceForCacheUpdateMs   # of milli seconds to wait for feature cache to be updated once.
+   *                                   If this parameter <= 0, no wait operation happens.
+   *
+   * @throws Exception if feature incompatibility check could not be finished in a timely manner
+   */
+  def initOrThrow(waitOnceForCacheUpdateMs: Long): Unit = {
+    thread.start()
+    zkClient.registerZNodeChangeHandlerAndCheckExistence(FeatureZNodeChangeHandler)

Review comment:
       For an educational question, does the zkClient have a separate thread to do the node change monitoring?

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,203 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {

Review comment:
       Could we summary the possible thrown error code in the comment as well? For example, does a JSON deserialization error should be treated as fatal?

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -744,6 +782,90 @@ object DelegationTokenInfoZNode {
   def decode(bytes: Array[Byte]): Option[TokenInformation] = DelegationTokenManager.fromBytes(bytes)
 }
 
+object FeatureZNodeStatus extends Enumeration {
+  val Disabled, Enabled = Value
+
+  def withNameOpt(value: Int): Option[Value] = {
+    values.find(_.id == value)
+  }
+}
+
+case class FeatureZNode(status: FeatureZNodeStatus.Value, features: Features[FinalizedVersionRange]) {
+}
+
+object FeatureZNode {

Review comment:
       I feel we might worth creating a separate thread discussing whether we could get some benefit of the automated protocol generation framework here, as I think this could be easily represented as JSON if we define it in the common package like other RPC data. The difficulty right now is mostly on the serialization and deserialization for feature itself, but these could have workarounds if we want to do so.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,203 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             - RuntimeException if the thread was interrupted during wait
+     *                     - TimeoutException if the wait can not be completed in waitTimeMs
+     *                       milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    private def processNotification(): Unit = {
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")
+      processNotification()
+    }
+
+    override def handleDataChange(): Unit = {
+      info(s"Feature ZK node updated at path: $path")
+      processNotification()
+    }
+
+    override def handleDeletion(): Unit = {
+      warn(s"Feature ZK node deleted at path: $path")
+      // This event may happen, rarely (ex: operational error).
+      // In such a case, we prefer to just log a warning and treat the case as if the node is absent,
+      // and populate the FinalizedFeatureCache with empty finalized features.
+      processNotification()
+    }
+  }
+
+  private val queue = new LinkedBlockingQueue[FeatureCacheUpdater]
+
+  private val thread = new ChangeNotificationProcessorThread("feature-zk-node-event-process-thread")
+
+  /**
+   * This method initializes the feature ZK node change listener. Optionally, it also ensures to
+   * update the FinalizedFeatureCache once with the latest contents of the feature ZK node
+   * (if the node exists). This step helps ensure that feature incompatibilities (if any) in brokers
+   * are conveniently detected before the initOrThrow() method returns to the caller. If feature
+   * incompatibilities are detected, this method will throw an Exception to the caller, and the Broker
+   * would exit eventually.
+   *
+   * @param waitOnceForCacheUpdateMs   # of milli seconds to wait for feature cache to be updated once.
+   *                                   If this parameter <= 0, no wait operation happens.
+   *
+   * @throws Exception if feature incompatibility check could not be finished in a timely manner
+   */
+  def initOrThrow(waitOnceForCacheUpdateMs: Long): Unit = {
+    thread.start()
+    zkClient.registerZNodeChangeHandlerAndCheckExistence(FeatureZNodeChangeHandler)
+
+    if (waitOnceForCacheUpdateMs > 0) {
+      val barrier = new CountDownLatch(1)
+      val ensureCacheUpdateOnce = new FeatureCacheUpdater(FeatureZNodeChangeHandler.path, Some(barrier))
+      queue.add(ensureCacheUpdateOnce)
+      try {
+        ensureCacheUpdateOnce.awaitUpdateOrThrow(waitOnceForCacheUpdateMs)
+      } catch {
+        case e: Exception => {
+          close()
+          throw e
+        }
+      }
+    }
+  }
+
+  /**
+   * Closes the feature ZK node change listener by unregistering the listener from ZK client,
+   * clearing the queue and shutting down the ChangeNotificationProcessorThread.
+   */
+  def close(): Unit = {
+    zkClient.unregisterZNodeChangeHandler(FeatureZNodeChangeHandler.path)

Review comment:
       Does the order matter here? I was wondering if there is any concurrent issue if we unregister before the queue and thread get cleaned up.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,203 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             - RuntimeException if the thread was interrupted during wait
+     *                     - TimeoutException if the wait can not be completed in waitTimeMs
+     *                       milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Interrupted", e)

Review comment:
       `Feature cache update gets interrupted`

##########
File path: core/src/main/scala/kafka/server/KafkaServer.scala
##########
@@ -210,6 +215,14 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP
         /* setup zookeeper */
         initZkClient(time)
 
+        /* initialize features */
+        _featureChangeListener = new FinalizedFeatureChangeListener(_zkClient)
+        if (config.interBrokerProtocolVersion >= KAFKA_2_6_IV1) {
+          // The feature versioning system (KIP-584) is active only when:

Review comment:
       I think the comment is not necessary, since we have already commented on `KAFKA_2_6_IV1`

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,203 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             - RuntimeException if the thread was interrupted during wait
+     *                     - TimeoutException if the wait can not be completed in waitTimeMs
+     *                       milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    private def processNotification(): Unit = {
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")
+      processNotification()
+    }
+
+    override def handleDataChange(): Unit = {
+      info(s"Feature ZK node updated at path: $path")
+      processNotification()
+    }
+
+    override def handleDeletion(): Unit = {
+      warn(s"Feature ZK node deleted at path: $path")
+      // This event may happen, rarely (ex: operational error).
+      // In such a case, we prefer to just log a warning and treat the case as if the node is absent,
+      // and populate the FinalizedFeatureCache with empty finalized features.
+      processNotification()
+    }
+  }
+
+  private val queue = new LinkedBlockingQueue[FeatureCacheUpdater]
+
+  private val thread = new ChangeNotificationProcessorThread("feature-zk-node-event-process-thread")
+
+  /**
+   * This method initializes the feature ZK node change listener. Optionally, it also ensures to
+   * update the FinalizedFeatureCache once with the latest contents of the feature ZK node
+   * (if the node exists). This step helps ensure that feature incompatibilities (if any) in brokers
+   * are conveniently detected before the initOrThrow() method returns to the caller. If feature
+   * incompatibilities are detected, this method will throw an Exception to the caller, and the Broker
+   * would exit eventually.
+   *
+   * @param waitOnceForCacheUpdateMs   # of milli seconds to wait for feature cache to be updated once.
+   *                                   If this parameter <= 0, no wait operation happens.
+   *
+   * @throws Exception if feature incompatibility check could not be finished in a timely manner
+   */
+  def initOrThrow(waitOnceForCacheUpdateMs: Long): Unit = {
+    thread.start()
+    zkClient.registerZNodeChangeHandlerAndCheckExistence(FeatureZNodeChangeHandler)
+
+    if (waitOnceForCacheUpdateMs > 0) {
+      val barrier = new CountDownLatch(1)
+      val ensureCacheUpdateOnce = new FeatureCacheUpdater(FeatureZNodeChangeHandler.path, Some(barrier))
+      queue.add(ensureCacheUpdateOnce)
+      try {
+        ensureCacheUpdateOnce.awaitUpdateOrThrow(waitOnceForCacheUpdateMs)
+      } catch {
+        case e: Exception => {
+          close()
+          throw e
+        }
+      }
+    }
+  }
+
+  /**
+   * Closes the feature ZK node change listener by unregistering the listener from ZK client,
+   * clearing the queue and shutting down the ChangeNotificationProcessorThread.
+   */
+  def close(): Unit = {
+    zkClient.unregisterZNodeChangeHandler(FeatureZNodeChangeHandler.path)
+    queue.clear()
+    thread.shutdown()
+    thread.join()
+  }
+
+  // Useful for testing.

Review comment:
       We could just comment `For testing only`

##########
File path: core/src/main/scala/kafka/server/SupportedFeatures.scala
##########
@@ -0,0 +1,74 @@
+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 common 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.
+ */
+object SupportedFeatures extends Logging {
+
+  /**
+   * This is the latest features supported by the Broker.
+   * This is currently empty, but in the future as we define supported features, this map should be
+   * populated.
+   */
+  @volatile private var supportedFeatures = emptySupportedFeatures
+
+  /**
+   * Returns the latest features supported by the Broker.
+   */
+  def get: Features[SupportedVersionRange] = {
+    supportedFeatures
+  }
+
+  // Should be used only for testing.
+  def update(newFeatures: Features[SupportedVersionRange]): Unit = {
+    supportedFeatures = newFeatures
+  }
+
+  // Should be used only for testing.
+  def clear(): Unit = {
+    supportedFeatures = emptySupportedFeatures
+  }
+
+  /**
+   * 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 cluster-wide finalized feature. This can happen because a provided
+   * cluster-wide 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 set of incompatible feature names. If the returned set is empty, it
+   *                    means there were no feature incompatibilities found.
+   */
+  def incompatibleFeatures(finalized: Features[FinalizedVersionRange]): Set[String] = {
+    val incompatibilities = finalized.features.asScala.collect {
+      case (feature, versionLevels) => {
+        val supportedVersions = supportedFeatures.get(feature);
+        if (supportedVersions == null) {
+          (feature, "{feature=%s, reason='Unsupported feature'}".format(feature))
+        } else if (versionLevels.isIncompatibleWith(supportedVersions)) {
+          (feature, "{feature=%s, reason='Finalized %s is incompatible with supported %s'}".format(
+            feature, versionLevels, supportedVersions))
+        } else {
+          (feature, null)
+        }
+      }
+    }.filter(entry => entry._2 != null)
+
+    if (incompatibilities.nonEmpty) {

Review comment:
       This logging is duplicate

##########
File path: core/src/main/scala/kafka/server/SupportedFeatures.scala
##########
@@ -0,0 +1,74 @@
+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 common 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.
+ */
+object SupportedFeatures extends Logging {
+
+  /**
+   * This is the latest features supported by the Broker.
+   * This is currently empty, but in the future as we define supported features, this map should be
+   * populated.
+   */
+  @volatile private var supportedFeatures = emptySupportedFeatures
+
+  /**
+   * Returns the latest features supported by the Broker.

Review comment:
       nit: Returns a reference to the latest features supported by the broker.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,200 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to propvide notification
+   *                            when an update operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {
+          FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+        } else {
+          throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully.
+     * NOTE: The method returns immediately if an updateLatestOrThrow call has already completed
+     * successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             - RuntimeException if the thread was interrupted during wait
+     *                     - TimeoutException if the wait can not be completed in waitTimeMs
+     *                       milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        var success = false
+        try {
+          success = notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)
+        } catch {
+          case e: InterruptedException =>
+            throw new RuntimeException(
+              "Unable to wait for FinalizedFeatureCache update to finish.", e)
+        }
+
+        if (!success) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        queue.take.updateLatestOrThrow()
+      } catch {
+        case e: InterruptedException => info(s"Interrupted", e)
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    private def processNotification(): Unit = {
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")
+      processNotification()
+    }
+
+    override def handleDataChange(): Unit = {
+      info(s"Feature ZK node updated at path: $path")
+      processNotification()
+    }
+
+    override def handleDeletion(): Unit = {
+      warn(s"Feature ZK node deleted at path: $path")
+      processNotification()

Review comment:
       I think even if this is an operational error, the cluster is at risk of violating the feature semantics previously enabled, which is different from an unknown feature version from the beginning. I feel we should just exit in fatal error for this case, but would open for discussion.

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -744,6 +782,90 @@ object DelegationTokenInfoZNode {
   def decode(bytes: Array[Byte]): Option[TokenInformation] = DelegationTokenManager.fromBytes(bytes)
 }
 
+object FeatureZNodeStatus extends Enumeration {
+  val Disabled, Enabled = Value
+
+  def withNameOpt(value: Int): Option[Value] = {
+    values.find(_.id == value)
+  }
+}
+
+case class FeatureZNode(status: FeatureZNodeStatus.Value, features: Features[FinalizedVersionRange]) {
+}
+
+object FeatureZNode {
+  private val VersionKey = "version"
+  private val StatusKey = "status"
+  private val FeaturesKey = "features"
+
+  // Version0 contains 'version', 'status' and 'features' keys.
+  val Version0 = 0

Review comment:
       Could we name it V0 for simplicity?

##########
File path: core/src/main/scala/kafka/zk/KafkaZkClient.scala
##########
@@ -1567,6 +1567,36 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
     createRecursive(path, data = null, throwIfPathExists = false)
   }
 
+  // Visible for testing.
+  def createFeatureZNode(nodeContents: FeatureZNode): Unit = {

Review comment:
       If that's the case, I feel we could remove the testing only comment.

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -744,6 +782,90 @@ object DelegationTokenInfoZNode {
   def decode(bytes: Array[Byte]): Option[TokenInformation] = DelegationTokenManager.fromBytes(bytes)
 }
 
+object FeatureZNodeStatus extends Enumeration {
+  val Disabled, Enabled = Value
+
+  def withNameOpt(value: Int): Option[Value] = {
+    values.find(_.id == value)
+  }
+}
+
+case class FeatureZNode(status: FeatureZNodeStatus.Value, features: Features[FinalizedVersionRange]) {
+}
+
+object FeatureZNode {
+  private val VersionKey = "version"
+  private val StatusKey = "status"
+  private val FeaturesKey = "features"
+
+  // Version0 contains 'version', 'status' and 'features' keys.
+  val Version0 = 0
+  val CurrentVersion = Version0
+
+  def path = "/feature"
+
+  def asJavaMap(scalaMap: Map[String, Map[String, Long]]): util.Map[String, util.Map[String, java.lang.Long]] = {
+    scalaMap
+      .view.mapValues(_.view.mapValues(scalaLong => java.lang.Long.valueOf(scalaLong)).toMap.asJava)
+      .toMap
+      .asJava
+  }
+
+  def encode(featureZNode: FeatureZNode): Array[Byte] = {
+    val jsonMap = collection.mutable.Map(
+      VersionKey -> CurrentVersion,
+      StatusKey -> featureZNode.status.id,
+      FeaturesKey -> featureZNode.features.serialize)
+    Json.encodeAsBytes(jsonMap.asJava)
+  }
+
+  def decode(jsonBytes: Array[Byte]): FeatureZNode = {
+    Json.tryParseBytes(jsonBytes) match {
+      case Right(js) =>
+        val featureInfo = js.asJsonObject
+        val version = featureInfo(VersionKey).to[Int]
+        if (version < Version0 || version > CurrentVersion) {
+          throw new KafkaException(s"Unsupported version: $version of feature information: " +
+            s"${new String(jsonBytes, UTF_8)}")
+        }
+
+        val featuresMap = featureInfo
+          .get(FeaturesKey)
+          .flatMap(_.to[Option[Map[String, Map[String, Long]]]])
+        if (featuresMap.isEmpty) {
+          throw new KafkaException("Features map can not be absent in: " +
+            s"${new String(jsonBytes, UTF_8)}")
+        }
+        val features = asJavaMap(featuresMap.get)
+
+        val statusInt = featureInfo
+          .get(StatusKey)
+          .flatMap(_.to[Option[Int]])
+        if (statusInt.isEmpty) {
+          throw new KafkaException("Status can not be absent in feature information: " +

Review comment:
       Is there a more dedicated exception code for deserialization error? I feel the KafkaException is a bit too general compared with IllegalArgument

##########
File path: core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala
##########
@@ -0,0 +1,228 @@
+package kafka.server
+
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion, ZooKeeperTestHarness}
+import kafka.utils.{Exit, TestUtils}
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.internals.FatalExitError
+import org.junit.Assert.{assertEquals, assertFalse, assertNotEquals, assertThrows, assertTrue}
+import org.junit.{Before, Test}
+
+import scala.concurrent.TimeoutException
+import scala.jdk.CollectionConverters._
+
+class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness {

Review comment:
       Could we extract some common initialization logic for the tests to reduce duplication?

##########
File path: core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala
##########
@@ -0,0 +1,228 @@
+package kafka.server
+
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion, ZooKeeperTestHarness}
+import kafka.utils.{Exit, TestUtils}
+import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
+import org.apache.kafka.common.internals.FatalExitError
+import org.junit.Assert.{assertEquals, assertFalse, assertNotEquals, assertThrows, assertTrue}
+import org.junit.{Before, Test}
+
+import scala.concurrent.TimeoutException
+import scala.jdk.CollectionConverters._
+
+class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness {
+  @Before
+  override def setUp(): Unit = {
+    super.setUp()
+    FinalizedFeatureCache.clear()
+    SupportedFeatures.clear()
+  }
+
+  /**
+   * 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).
+   */
+  @Test
+  def testInitSuccessAndNotificationSuccess(): Unit = {
+    val supportedFeatures = Map[String, SupportedVersionRange](
+      "feature_1" -> new SupportedVersionRange(1, 4),
+      "feature_2" -> new SupportedVersionRange(1, 3))
+    SupportedFeatures.update(Features.supportedFeatures(supportedFeatures.asJava))
+
+    val initialFinalizedFeaturesMap = Map[String, FinalizedVersionRange](
+      "feature_1" -> new FinalizedVersionRange(2, 3))
+    val initialFinalizedFeatures = Features.finalizedFeatures(initialFinalizedFeaturesMap.asJava)
+    zkClient.createFeatureZNode(FeatureZNode(FeatureZNodeStatus.Enabled, initialFinalizedFeatures))
+    val (mayBeFeatureZNodeBytes, initialVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(initialVersion, ZkVersion.UnknownVersion)
+    assertFalse(mayBeFeatureZNodeBytes.isEmpty)
+
+    val listener = new FinalizedFeatureChangeListener(zkClient)
+    assertFalse(listener.isListenerInitiated)
+    assertTrue(FinalizedFeatureCache.empty)
+    listener.initOrThrow(15000)
+    assertTrue(listener.isListenerInitiated)
+    val mayBeNewCacheContent = FinalizedFeatureCache.get
+    assertFalse(mayBeNewCacheContent.isEmpty)
+    val newCacheContent = mayBeNewCacheContent.get
+    assertEquals(initialFinalizedFeatures, newCacheContent.features)
+    assertEquals(initialVersion, newCacheContent.epoch)
+
+    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 > initialVersion)
+    TestUtils.waitUntilTrue(() => {
+      FinalizedFeatureCache.get.get.equals(FinalizedFeaturesAndEpoch(updatedFinalizedFeatures, updatedVersion))
+    }, "Timed out waiting for FinalizedFeatureCache to be updated with new features")
+    assertTrue(listener.isListenerInitiated)
+  }
+
+  /**
+   * Tests that the listener can be initialized, and that it can process FeatureZNode deletion
+   * successfully.
+   */
+  @Test
+  def testFeatureZNodeDeleteNotificationProcessing(): Unit = {
+    val supportedFeatures = Map[String, SupportedVersionRange](
+      "feature_1" -> new SupportedVersionRange(1, 4),
+      "feature_2" -> new SupportedVersionRange(1, 3))
+    SupportedFeatures.update(Features.supportedFeatures(supportedFeatures.asJava))
+
+    val initialFinalizedFeaturesMap = Map[String, FinalizedVersionRange](
+      "feature_1" -> new FinalizedVersionRange(2, 3))
+    val initialFinalizedFeatures = Features.finalizedFeatures(initialFinalizedFeaturesMap.asJava)
+    zkClient.createFeatureZNode(FeatureZNode(FeatureZNodeStatus.Enabled, initialFinalizedFeatures))
+    val (mayBeFeatureZNodeBytes, initialVersion) = zkClient.getDataAndVersion(FeatureZNode.path)
+    assertNotEquals(initialVersion, ZkVersion.UnknownVersion)
+    assertFalse(mayBeFeatureZNodeBytes.isEmpty)
+
+    val listener = new FinalizedFeatureChangeListener(zkClient)
+    assertFalse(listener.isListenerInitiated)
+    assertTrue(FinalizedFeatureCache.empty)
+    listener.initOrThrow(15000)
+    assertTrue(listener.isListenerInitiated)
+    val mayBeNewCacheContent = FinalizedFeatureCache.get
+    assertFalse(mayBeNewCacheContent.isEmpty)
+    val newCacheContent = mayBeNewCacheContent.get
+    assertEquals(initialFinalizedFeatures, newCacheContent.features)
+    assertEquals(initialVersion, newCacheContent.epoch)
+
+    zkClient.deleteFeatureZNode()

Review comment:
       I'm a bit surprised, do we want to support feature znode deletion in long term?

##########
File path: core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
##########
@@ -776,6 +776,9 @@ class KafkaConfigTest {
         case KafkaConfig.KafkaMetricsReporterClassesProp => // ignore
         case KafkaConfig.KafkaMetricsPollingIntervalSecondsProp => //ignore
 
+        //Feature configuration

Review comment:
       nit: space




----------------------------------------------------------------
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 #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -81,17 +83,26 @@ object BrokerIdsZNode {
 object BrokerInfo {
 
   /**
-   * Create a broker info with v4 json format (which includes multiple endpoints and rack) if
-   * the apiVersion is 0.10.0.X or above. Register the broker with v2 json format otherwise.
+   * - Create a broker info with v5 json format if the apiVersion is 2.6.x or above.
+   * - Create a broker info with v4 json format (which includes multiple endpoints and rack) if
+   *   the apiVersion is 0.10.0.X or above but lesser than 2.6.x.

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/zk/ZkData.scala
##########
@@ -81,17 +83,26 @@ object BrokerIdsZNode {
 object BrokerInfo {
 
   /**
-   * Create a broker info with v4 json format (which includes multiple endpoints and rack) if
-   * the apiVersion is 0.10.0.X or above. Register the broker with v2 json format otherwise.
+   * - Create a broker info with v5 json format if the apiVersion is 2.6.x or above.

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,243 @@
+/**
+ * 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.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode, FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.{StateChangeHandler, ZNodeChangeHandler}
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+import scala.util.control.Exception.ignoring
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked exactly
+     * once successfully. A subsequent invocation will raise an exception.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     * @throws   FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     * @throws   RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      debug(s"Reading feature ZK node at path: $featureZkNodePath")
+      val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(featureZkNodePath)
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        featureZNode.status match {
+          case FeatureZNodeStatus.Disabled => {
+            info(s"Feature ZK node at path: $featureZkNodePath is in disabled status.")
+            FinalizedFeatureCache.clear()
+          }
+          case FeatureZNodeStatus.Enabled => {
+            FinalizedFeatureCache.updateOrThrow(featureZNode.features, version)
+          }
+          case _ => throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode")
+        }
+      }
+
+      maybeNotifyOnce.foreach(notifier => notifier.countDown())
+    }
+
+    /**
+     * Waits until at least a single updateLatestOrThrow completes successfully. This method returns
+     * immediately if an updateLatestOrThrow call had already completed successfully.
+     *
+     * @param waitTimeMs   the timeout for the wait operation
+     *
+     * @throws             TimeoutException if the wait can not be completed in waitTimeMs
+     *                     milli seconds
+     */
+    def awaitUpdateOrThrow(waitTimeMs: Long): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (!notifier.await(waitTimeMs, TimeUnit.MILLISECONDS)) {
+          throw new TimeoutException(
+            s"Timed out after waiting for ${waitTimeMs}ms for FeatureCache to be updated.")
+        }
+      })
+    }
+  }
+
+  /**
+   * A shutdownable thread to process feature node change notifications that are populated into the
+   * queue. If any change notification can not be processed successfully (unless it is due to an
+   * interrupt), the thread treats it as a fatal event and triggers Broker exit.
+   *
+   * @param name   name of the thread
+   */
+  private class ChangeNotificationProcessorThread(name: String) extends ShutdownableThread(name = name) {
+    override def doWork(): Unit = {
+      try {
+        ignoring(classOf[InterruptedException]) {
+          queue.take.updateLatestOrThrow()
+        }
+      } catch {
+        case e: Exception => {
+          error("Failed to process feature ZK node change event. The broker will eventually exit.", e)
+          throw new FatalExitError(1)
+        }
+      }
+    }
+  }
+
+  // Feature ZK node change handler.
+  object FeatureZNodeChangeHandler extends ZNodeChangeHandler {
+    override val path: String = FeatureZNode.path
+
+    override def handleCreation(): Unit = {
+      info(s"Feature ZK node created at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDataChange(): Unit = {
+      info(s"Feature ZK node updated at path: $path")
+      queue.add(new FeatureCacheUpdater(path))
+    }
+
+    override def handleDeletion(): Unit = {
+      warn(s"Feature ZK node deleted at path: $path")
+      // This event may happen, rarely (ex: ZK corruption or operational error).
+      // In such a case, we prefer to just log a warning and treat the case as if the node is absent,
+      // and populate the FinalizedFeatureCache with empty finalized features.
+      queue.add(new FeatureCacheUpdater(path))
+    }
+  }
+
+  object ZkStateChangeHandler extends StateChangeHandler {
+    val path: String = FeatureZNode.path
+
+    override val name: String = s"change-notification-$path"

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 #8680: KAFKA-10027: Implement read path for feature versioning system (KIP-584)

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



##########
File path: core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala
##########
@@ -0,0 +1,232 @@
+package kafka.server
+
+import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit}
+
+import kafka.utils.{Logging, ShutdownableThread}
+import kafka.zk.{FeatureZNode,FeatureZNodeStatus, KafkaZkClient, ZkVersion}
+import kafka.zookeeper.ZNodeChangeHandler
+import org.apache.kafka.common.internals.FatalExitError
+
+import scala.concurrent.TimeoutException
+
+/**
+ * Listens to changes in the ZK feature node, via the ZK client. Whenever a change notification
+ * is received from ZK, the feature cache in FinalizedFeatureCache is asynchronously updated
+ * to the latest features read from ZK. The cache updates are serialized through a single
+ * notification processor thread.
+ *
+ * @param zkClient     the Zookeeper client
+ */
+class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging {
+
+  /**
+   * Helper class used to update the FinalizedFeatureCache.
+   *
+   * @param featureZkNodePath   the path to the ZK feature node to be read
+   * @param maybeNotifyOnce     an optional latch that can be used to notify the caller when an
+   *                            updateOrThrow() operation is over
+   */
+  private class FeatureCacheUpdater(featureZkNodePath: String, maybeNotifyOnce: Option[CountDownLatch]) {
+
+    def this(featureZkNodePath: String) = this(featureZkNodePath, Option.empty)
+
+    /**
+     * Updates the feature cache in FinalizedFeatureCache with the latest features read from the
+     * ZK node in featureZkNodePath. If the cache update is not successful, then, a suitable
+     * exception is raised.
+     *
+     * NOTE: if a notifier was provided in the constructor, then, this method can be invoked
+     * only exactly once successfully.
+     *
+     * @throws   IllegalStateException, if a non-empty notifier was provided in the constructor, and
+     *           this method is called again after a successful previous invocation.
+     *
+     *           FeatureCacheUpdateException, if there was an error in updating the
+     *           FinalizedFeatureCache.
+     *
+     *           RuntimeException, if there was a failure in reading/deserializing the
+     *           contents of the feature ZK node.
+     */
+    def updateLatestOrThrow(): Unit = {
+      maybeNotifyOnce.foreach(notifier => {
+        if (notifier.getCount != 1) {
+          throw new IllegalStateException(
+            "Can not notify after updateLatestOrThrow was called more than once successfully.")
+        }
+      })
+
+      info(s"Reading feature ZK node at path: $featureZkNodePath")
+      var mayBeFeatureZNodeBytes: Option[Array[Byte]] = null
+      var version: Int = ZkVersion.UnknownVersion
+      try {
+        val result = zkClient.getDataAndVersion(featureZkNodePath)
+        mayBeFeatureZNodeBytes = result._1
+        version = result._2
+      } catch {
+        // Convert to RuntimeException, to avoid a confusion that there is no argument passed
+        // to the updateOrThrow() method.
+        case e: IllegalArgumentException => throw new RuntimeException(e)
+      }
+
+      // There are 4 cases:
+      //
+      // (empty dataBytes, valid version)       => The empty dataBytes will fail FeatureZNode deserialization.
+      //                                           FeatureZNode, when present in ZK, can not have empty contents.
+      // (non-empty dataBytes, valid version)   => This is a valid case, and should pass FeatureZNode deserialization
+      //                                           if dataBytes contains valid data.
+      // (empty dataBytes, unknown version)     => This is a valid case, and this can happen if the FeatureZNode
+      //                                           does not exist in ZK.
+      // (non-empty dataBytes, unknown version) => This case is impossible, since, KafkaZkClient.getDataAndVersion
+      //                                           API ensures that unknown version is returned only when the
+      //                                           ZK node is absent. Therefore dataBytes should be empty in such
+      //                                           a case.
+      if (version == ZkVersion.UnknownVersion) {
+        info(s"Feature ZK node at path: $featureZkNodePath does not exist")
+        FinalizedFeatureCache.clear()
+      } else {
+        val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
+        if (featureZNode.status == FeatureZNodeStatus.Disabled) {
+          info(s"Feature ZK node at path: $featureZkNodePath is in disabled status")
+          FinalizedFeatureCache.clear()
+        } else if(featureZNode.status == FeatureZNodeStatus.Enabled) {

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