hachikuji commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1098014628


##########
clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java:
##########
@@ -212,6 +215,36 @@ public boolean isVersionSupported(short apiVersion) {
         return apiVersion >= oldestVersion() && apiVersion <= latestVersion();
     }
 
+    public boolean isVersionEnabled(short apiVersion, boolean 
enableUnstableLastVersion) {
+        // ApiVersions API is a particular case that we always accept any, even

Review Comment:
   It would be helpful to clarify the reason for this.



##########
core/src/main/scala/kafka/server/ApiVersionManager.scala:
##########
@@ -45,55 +45,70 @@ object ApiVersionManager {
       listenerType,
       forwardingManager,
       supportedFeatures,
-      metadataCache
+      metadataCache,
+      config.unstableApiVersionsEnabled
     )
   }
 }
 
 class SimpleApiVersionManager(
   val listenerType: ListenerType,
   val enabledApis: collection.Set[ApiKeys],
-  brokerFeatures: Features[SupportedVersionRange]
+  brokerFeatures: Features[SupportedVersionRange],
+  enableUnstableLastVersion: Boolean
 ) extends ApiVersionManager {
 
-  def this(listenerType: ListenerType) = {
-    this(listenerType, ApiKeys.apisForListener(listenerType).asScala, 
BrokerFeatures.defaultSupportedFeatures())
+  def this(
+    listenerType: ListenerType,
+    enableUnstableLastVersion: Boolean
+  ) = {
+    this(
+      listenerType,
+      ApiKeys.apisForListener(listenerType).asScala,
+      BrokerFeatures.defaultSupportedFeatures(),
+      enableUnstableLastVersion
+    )
   }
 
-  private val apiVersions = ApiVersionsResponse.collectApis(enabledApis.asJava)
+  private val apiVersions = 
ApiVersionsResponse.collectApis(enabledApis.asJava, enableUnstableLastVersion)
 
   override def apiVersionResponse(requestThrottleMs: Int): ApiVersionsResponse 
= {
     ApiVersionsResponse.createApiVersionsResponse(requestThrottleMs, 
apiVersions, brokerFeatures)
   }
+
+  override def isApiEnabled(apiKey: ApiKeys, apiVersion: Short): Boolean = {
+    apiKey != null && apiKey.inScope(listenerType) && 
apiKey.isVersionEnabled(apiVersion, enableUnstableLastVersion)
+  }
 }
 
 class DefaultApiVersionManager(
   val listenerType: ListenerType,
   forwardingManager: Option[ForwardingManager],
   features: BrokerFeatures,
-  metadataCache: MetadataCache
+  metadataCache: MetadataCache,
+  enableUnstableLastVersion: Boolean
 ) extends ApiVersionManager {
 
+  val enabledApis = ApiKeys.apisForListener(listenerType).asScala
+
   override def apiVersionResponse(throttleTimeMs: Int): ApiVersionsResponse = {
     val supportedFeatures = features.supportedFeatures
     val finalizedFeatures = metadataCache.features()
     val controllerApiVersions = 
forwardingManager.flatMap(_.controllerApiVersions)
 
     ApiVersionsResponse.createApiVersionsResponse(
-        throttleTimeMs,
-        metadataCache.metadataVersion().highestSupportedRecordVersion,
-        supportedFeatures,
-        finalizedFeatures.features.map(kv => (kv._1, 
kv._2.asInstanceOf[java.lang.Short])).asJava,
-        finalizedFeatures.epoch,
-        controllerApiVersions.orNull,
-        listenerType)
-  }
-
-  override def enabledApis: collection.Set[ApiKeys] = {
-    ApiKeys.apisForListener(listenerType).asScala
+      throttleTimeMs,
+      metadataCache.metadataVersion().highestSupportedRecordVersion,
+      supportedFeatures,
+      finalizedFeatures.features.map(kv => (kv._1, 
kv._2.asInstanceOf[java.lang.Short])).asJava,
+      finalizedFeatures.epoch,
+      controllerApiVersions.orNull,
+      listenerType,
+      enableUnstableLastVersion
+    )
   }
 
-  override def isApiEnabled(apiKey: ApiKeys): Boolean = {
-    apiKey.inScope(listenerType)
+  override def isApiEnabled(apiKey: ApiKeys, apiVersion: Short): Boolean = {
+    apiKey != null && apiKey.inScope(listenerType) && 
apiKey.isVersionEnabled(apiVersion, enableUnstableLastVersion)

Review Comment:
   Could we pull this implementation up to the `trait`? The implementation 
looks the same for `SimpleApiVersionManager`.



##########
generator/src/main/java/org/apache/kafka/message/RequestApiStabilityType.java:
##########
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.message;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public enum RequestApiStabilityType {

Review Comment:
   Do we still need this?



##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -3509,6 +3510,13 @@ class KafkaApis(val requestChannel: RequestChannel,
       )
   }
 
+  def handleConsumerGroupHeartbeat(request: RequestChannel.Request): 
CompletableFuture[Unit] = {
+    val consumerGroupHeartbeatRequest = 
request.body[ConsumerGroupHeartbeatRequest]
+    // KIP-848 is not implemented yet so return UNSUPPORTED_VERSION.
+    requestHelper.sendMaybeThrottle(request, 
consumerGroupHeartbeatRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))

Review Comment:
   Would it be overkill to have a unit test for this?



##########
clients/src/main/resources/common/message/ConsumerGroupHeartbeatResponse.json:
##########
@@ -0,0 +1,70 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements.  See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//    http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+{
+  "apiKey": 68,
+  "type": "response",
+  "name": "ConsumerGroupHeartbeatResponse",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  // Supported errors:
+  // - GROUP_AUTHORIZATION_FAILED

Review Comment:
   Maybe useful to annotate these to indicate which version they are enabled 
in. All of these will be 0+, but maybe we can set the pattern for future bumps.



##########
clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java:
##########
@@ -212,6 +215,36 @@ public boolean isVersionSupported(short apiVersion) {
         return apiVersion >= oldestVersion() && apiVersion <= latestVersion();
     }
 
+    public boolean isVersionEnabled(short apiVersion, boolean 
enableUnstableLastVersion) {
+        // ApiVersions API is a particular case that we always accept any, even
+        // unsupported, versions.
+        if (this == ApiKeys.API_VERSIONS) return true;
+
+        if (!messageType.latestVersionUnstable() || enableUnstableLastVersion) 
{

Review Comment:
   Would it make sense to move this logic into some generated helpers? For 
example, `latestStableVersion` or `latestVersion(boolean 
enableUnstableVersions)`.



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

Review Comment:
   We haven't done it for the other exception types, but I wonder if it makes 
sense to add the `@InterfaceStability.Evolving` to these types since they are 
public.
   



##########
core/src/test/scala/unit/kafka/server/ApiVersionManagerTest.scala:
##########
@@ -40,10 +40,15 @@ class ApiVersionManagerTest {
       listenerType = apiScope,
       forwardingManager = None,
       features = brokerFeatures,
-      metadataCache = metadataCache
+      metadataCache = metadataCache,
+      enableUnstableLastVersion = true

Review Comment:
   Does it make sense to add a test case when this flag is false. We could 
iterate the api keys and look for the presence of the `latestVersionUnstable` 
flag and then assert that the api is not enabled.



##########
generator/src/main/java/org/apache/kafka/message/MessageSpec.java:
##########
@@ -70,6 +74,12 @@ public MessageSpec(@JsonProperty("name") String name,
                 "messages with type `request`");
         }
         this.listeners = listeners;
+
+        if (latestVersionUnstable && type != MessageSpecType.REQUEST) {

Review Comment:
   Makes sense to restrict this for now, but it would be nice to make it more 
general in the future. I can imagine it being useful while experimenting with 
metadata record changes.



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

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

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

Reply via email to