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 2022/12/09 14:48:55 UTC

[GitHub] [kafka] dajac opened a new pull request, #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

dajac opened a new pull request, #12972:
URL: https://github.com/apache/kafka/pull/12972

   This patch adds the ConsumerGroupHeartbeat API alongside its new related errors. The API is gated by an internal configuration flag at the moment. Hence the API cannot be used by clients yet. The mechanism to gate the API is only meant to be used during the development of KIP-848. The API will be gated by Metadata Version or a dedicated feature flag in the future.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] guozhangwang commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "guozhangwang (via GitHub)" <gi...@apache.org>.
guozhangwang commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1099081262


##########
clients/src/main/resources/common/message/ConsumerGroupHeartbeatRequest.json:
##########
@@ -0,0 +1,66 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements.  See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//    http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT 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": "request",
+  "apiStability": "evolving",
+  "listeners": ["zkBroker", "broker"],
+  "name": "ConsumerGroupHeartbeatRequest",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId",
+      "about": "The group identifier." },
+    { "name": "MemberId", "type": "string", "versions": "0+",
+      "about": "The member id generated by the coordinator. The member id must be kept during the entire lifetime of the member." },
+    { "name": "MemberEpoch", "type": "int32", "versions": "0+",
+      "about": "The current member epoch; 0 to join the group; -1 to leave the group; -2 to indicate that the static member will rejoin." },
+    { "name": "InstanceId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null",
+      "about": "null if not provided or if it didn't change since the last heartbeat; the instance Id otherwise." },
+    { "name": "RackId", "type": "string", "versions": "0+",  "nullableVersions": "0+", "default": "null",
+      "about": "null if not provided or if it didn't change since the last heartbeat; the rack ID of consumer otherwise." },
+    { "name": "RebalanceTimeoutMs", "type": "int32", "versions": "0+", "default": -1,
+      "about": "-1 if it didn't chance since the last heartbeat; the maximum time in milliseconds that the coordinator will wait on the member to revoke its partitions otherwise." },
+    { "name": "SubscribedTopicNames", "type": "[]string", "versions": "0+", "nullableVersions": "0+", "default": "null",
+      "about": "null if it didn't change since the last heartbeat; the subscribed topic names otherwise." },
+    { "name": "SubscribedTopicRegex", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null",
+      "about": "null if it didn't change since the last heartbeat; the subscribed topic regex otherwise" },
+    { "name": "ServerAssignor", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null",
+      "about": "null if not used or if it didn't change since the last heartbeat; the server side assignor to use otherwise." },
+    { "name": "ClientAssignors", "type": "[]Assignor", "versions": "0+", "nullableVersions": "0+", "default": "null",
+      "about": "null if not used or if it didn't change since the last heartbeat; the list of client-side assignors otherwise.", "fields": [
+      { "name": "Name", "type": "string", "versions": "0+",
+        "about": "The name of the assignor." },
+      { "name": "MinimumVersion", "type": "int16", "versions": "0+",
+        "about": "The minimum supported version for the metadata." },
+      { "name": "MaximumVersion", "type": "int16", "versions": "0+",
+        "about": "The maximum supported version for the metadata." },
+      { "name": "Reason", "type": "int8", "versions": "0+",

Review Comment:
   It just occurs to me: shall we reserve some numbers for AK client usage? I know that in the KIP we did not do so, for Streams e.g. it can start with reason 0 for its own meaning. But thinking about this a bit more I'm wondering if some common reasons that are used by consumer clients directly should be reserved? E.g. re-discover coordinator after disconnect, subscription changes, etc.



##########
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:
   +1



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by GitBox <gi...@apache.org>.
dajac commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1066037931


##########
clients/src/main/java/org/apache/kafka/common/errors/FencedMemberEpoch.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 FencedMemberEpoch extends ApiException {

Review Comment:
   You're right. Let me fix this.



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

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

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


[GitHub] [kafka] jolshan commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by GitBox <gi...@apache.org>.
jolshan commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1072862469


##########
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
+  // - NOT_COORDINATOR
+  // - COORDINATOR_NOT_AVAILABLE
+  // - COORDINATOR_LOAD_IN_PROGRESS
+  // - INVALID_REQUEST
+  // - UNKNOWN_MEMBER_ID
+  // - FENCED_MEMBER_EPOCH
+  // - UNSUPPORTED_ASSIGNOR
+  // - UNRELEASED_INSTANCE_ID
+  // - GROUP_MAX_SIZE_REACHED

Review Comment:
   You are referring to GROUP_MAX_SIZE_REACHED?



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by GitBox <gi...@apache.org>.
jeffkbkim commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1081656322


##########
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
+  // - NOT_COORDINATOR
+  // - COORDINATOR_NOT_AVAILABLE
+  // - COORDINATOR_LOAD_IN_PROGRESS
+  // - INVALID_REQUEST
+  // - UNKNOWN_MEMBER_ID
+  // - FENCED_MEMBER_EPOCH
+  // - UNSUPPORTED_ASSIGNOR
+  // - UNRELEASED_INSTANCE_ID
+  // - GROUP_MAX_SIZE_REACHED
+  "fields": [
+    { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
+      "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": "ErrorCode", "type": "int16", "versions": "0+",
+      "about": "The top-level error code, or 0 if there was no error" },
+    { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null",
+      "about": "The top-level error message, or null if there was no error." },
+    { "name": "MemberId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null",
+      "about": "The member id generated by the coordinator. Only provided when the member joins with MemberEpoch == 0." },
+    { "name": "MemberEpoch", "type": "int32", "versions": "0+",
+      "about": "The member epoch." },
+    { "name": "ShouldComputeAssignment", "type": "bool", "versions": "0+",
+      "about": "True if the member should compute the assignment for the group." },
+    { "name": "HeartbeatIntervalMs", "type": "int32", "versions": "0+",
+      "about": "The heartbeat interval in milliseconds." },
+    { "name": "Assignment", "type": "Assignment", "versions": "0+", "nullableVersions": "0+", "default": "null",
+      "about": "null if not provided; the assignment otherwise.", "fields": [
+      { "name": "Error", "type": "int8", "versions": "0+",
+        "about": "The assigned error." },
+      { "name": "AssignedTopicPartitions", "type": "[]TopicPartitions", "versions": "0+",
+        "about": "The partitions assigned to the member that can be used immediately." },
+      { "name": "PendingTopicPartitions", "type": "[]TopicPartitions", "versions": "0+",
+        "about": "The partitions assigned to the member that cannot be used because they are not released by their former owners yet." },
+      { "name": "MetadataVersion", "type": "int16", "versions": "0+",
+        "about": "The version of the metadata." },
+      { "name": "MetadataBytes", "type": "bytes", "versions": "0+",
+        "about": "The assigned metadata." }
+    ]}
+  ],
+  "commonStructs": [

Review Comment:
   aha - thanks!



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by GitBox <gi...@apache.org>.
jeffkbkim commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1081655338


##########
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
+  // - NOT_COORDINATOR
+  // - COORDINATOR_NOT_AVAILABLE
+  // - COORDINATOR_LOAD_IN_PROGRESS
+  // - INVALID_REQUEST
+  // - UNKNOWN_MEMBER_ID
+  // - FENCED_MEMBER_EPOCH
+  // - UNSUPPORTED_ASSIGNOR
+  // - UNRELEASED_INSTANCE_ID
+  // - GROUP_MAX_SIZE_REACHED

Review Comment:
   yes



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by GitBox <gi...@apache.org>.
dajac commented on PR #12972:
URL: https://github.com/apache/kafka/pull/12972#issuecomment-1377592620

   @hachikuji What do you think about the _internal_ config name `unreleased.apis.enable` (boolean)? An alternative with your suggestion would be to have something like `apis.enabled` taking a list of "api stability type" to be exposed or just taking `stable` or `evolving` where `evolving` implies that `stable` apis are included 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.

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

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


[GitHub] [kafka] hachikuji commented on pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

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

   Would it be possible to make the release status part of the JSON spec? For example:
   ```json
     "apiStability": "evolving|stable"
   ```
   


-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on PR #12972:
URL: https://github.com/apache/kafka/pull/12972#issuecomment-1421105810

   @hachikuji Thanks for your review. I have addressed your comments.


-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1098641948


##########
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:
   Yeah, I agree.



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by GitBox <gi...@apache.org>.
jolshan commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1054893924


##########
clients/src/main/java/org/apache/kafka/common/errors/FencedMemberEpoch.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 FencedMemberEpoch extends ApiException {

Review Comment:
   I feel like we typically do that for classes that extend ApiException. (Same on other classes changed in this file.)



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

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

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


[GitHub] [kafka] jolshan commented on pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by GitBox <gi...@apache.org>.
jolshan commented on PR #12972:
URL: https://github.com/apache/kafka/pull/12972#issuecomment-1386129325

   I noticed that we don't consider the stability of responses. (I assume though that if the request is unstable, the response is too.) 
   
   Just curious if there are any potential gaps with this approach and the reasoning behind only the request stability.


-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by GitBox <gi...@apache.org>.
jolshan commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1072868862


##########
core/src/main/scala/kafka/server/ApiVersionManager.scala:
##########
@@ -86,14 +100,7 @@ class DefaultApiVersionManager(
         finalizedFeatures.features.map(kv => (kv._1, kv._2.asInstanceOf[java.lang.Short])).asJava,
         finalizedFeatures.epoch,
         controllerApiVersions.orNull,
-        listenerType)
-  }
-
-  override def enabledApis: collection.Set[ApiKeys] = {

Review Comment:
   Do we no longer need these methods? Or are they just handled elsewhere? (Ie, ApiKeys)



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1098634292


##########
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:
   Good idea.



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1100490932


##########
clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java:
##########
@@ -212,24 +248,32 @@ public static ApiVersionCollection collectApis(Set<ApiKeys> apiKeys) {
      * @param listenerType the listener type which constrains the set of exposed APIs
      * @param minRecordVersion min inter broker magic
      * @param activeControllerApiVersions controller ApiVersions
+     * @param enableUnstableLastVersion whether unstable versions should be advertised or not

Review Comment:
   i'm a bit confused on the word "last". does this mean we enable only if the last version is unstable? my understanding is we enable all unstable versions or we don't



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

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

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


[GitHub] [kafka] dajac commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1098633270


##########
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:
   Good idea.



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] hachikuji commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "hachikuji (via GitHub)" <gi...@apache.org>.
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1098630635


##########
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:
   Let me extend the comment.



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

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

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


[GitHub] [kafka] dajac commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1100495875


##########
clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java:
##########
@@ -212,24 +248,32 @@ public static ApiVersionCollection collectApis(Set<ApiKeys> apiKeys) {
      * @param listenerType the listener type which constrains the set of exposed APIs
      * @param minRecordVersion min inter broker magic
      * @param activeControllerApiVersions controller ApiVersions
+     * @param enableUnstableLastVersion whether unstable versions should be advertised or not

Review Comment:
   that's right but only the last version of an api could be unstable. this is where the "last" comes from.



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by GitBox <gi...@apache.org>.
jolshan commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1072861634


##########
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
+  // - NOT_COORDINATOR
+  // - COORDINATOR_NOT_AVAILABLE
+  // - COORDINATOR_LOAD_IN_PROGRESS
+  // - INVALID_REQUEST
+  // - UNKNOWN_MEMBER_ID
+  // - FENCED_MEMBER_EPOCH
+  // - UNSUPPORTED_ASSIGNOR
+  // - UNRELEASED_INSTANCE_ID
+  // - GROUP_MAX_SIZE_REACHED
+  "fields": [
+    { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
+      "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": "ErrorCode", "type": "int16", "versions": "0+",
+      "about": "The top-level error code, or 0 if there was no error" },
+    { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null",
+      "about": "The top-level error message, or null if there was no error." },
+    { "name": "MemberId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null",
+      "about": "The member id generated by the coordinator. Only provided when the member joins with MemberEpoch == 0." },
+    { "name": "MemberEpoch", "type": "int32", "versions": "0+",
+      "about": "The member epoch." },
+    { "name": "ShouldComputeAssignment", "type": "bool", "versions": "0+",
+      "about": "True if the member should compute the assignment for the group." },
+    { "name": "HeartbeatIntervalMs", "type": "int32", "versions": "0+",
+      "about": "The heartbeat interval in milliseconds." },
+    { "name": "Assignment", "type": "Assignment", "versions": "0+", "nullableVersions": "0+", "default": "null",
+      "about": "null if not provided; the assignment otherwise.", "fields": [
+      { "name": "Error", "type": "int8", "versions": "0+",
+        "about": "The assigned error." },
+      { "name": "AssignedTopicPartitions", "type": "[]TopicPartitions", "versions": "0+",
+        "about": "The partitions assigned to the member that can be used immediately." },
+      { "name": "PendingTopicPartitions", "type": "[]TopicPartitions", "versions": "0+",
+        "about": "The partitions assigned to the member that cannot be used because they are not released by their former owners yet." },
+      { "name": "MetadataVersion", "type": "int16", "versions": "0+",
+        "about": "The version of the metadata." },
+      { "name": "MetadataBytes", "type": "bytes", "versions": "0+",
+        "about": "The assigned metadata." }
+    ]}
+  ],
+  "commonStructs": [

Review Comment:
   See TopicPartitions used in line 52, 54. If we didn't have the common struct, we'd have to define the name and the nested fields each time.



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by GitBox <gi...@apache.org>.
dajac commented on PR #12972:
URL: https://github.com/apache/kafka/pull/12972#issuecomment-1377449336

   > Would it be possible to make the release status part of the JSON spec? For example:
   > 
   > ```json
   >   "apiStability": "evolving|stable"
   > ```
   
   That's a good idea. Let me do this.


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

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

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


[GitHub] [kafka] dajac commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1098634689


##########
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:
   We could but we have to add `enableUnstableLastVersion` to the trait as well. I guess that it is not an issue. Let me do it.



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

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

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


[GitHub] [kafka] dajac commented on pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on PR #12972:
URL: https://github.com/apache/kafka/pull/12972#issuecomment-1422991741

   Thanks @hachikuji!


-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by GitBox <gi...@apache.org>.
jolshan commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1070132569


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

Review Comment:
   nit: double space 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.

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

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


[GitHub] [kafka] dajac commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1092320492


##########
core/src/main/scala/kafka/server/ApiVersionManager.scala:
##########
@@ -86,14 +100,7 @@ class DefaultApiVersionManager(
         finalizedFeatures.features.map(kv => (kv._1, kv._2.asInstanceOf[java.lang.Short])).asJava,
         finalizedFeatures.epoch,
         controllerApiVersions.orNull,
-        listenerType)
-  }
-
-  override def enabledApis: collection.Set[ApiKeys] = {

Review Comment:
   Replaced the method by a val.



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] hachikuji commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "hachikuji (via GitHub)" <gi...@apache.org>.
hachikuji commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1100465745


##########
generator/src/main/java/org/apache/kafka/message/ApiMessageTypeGenerator.java:
##########
@@ -411,7 +409,7 @@ private void generateListenerTypesEnum() {
     }
 
     private void generateHighestStableVersion() {

Review Comment:
   nit: `generateHighestSupportedVersion`?



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac merged pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac merged PR #12972:
URL: https://github.com/apache/kafka/pull/12972


-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by GitBox <gi...@apache.org>.
dajac commented on PR #12972:
URL: https://github.com/apache/kafka/pull/12972#issuecomment-1377643457

   Yeah, that's an interesting though. That's true that the `apiStability` in the spec only works for new APIs. Let me check and see how feasible that is.


-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by GitBox <gi...@apache.org>.
dajac commented on PR #12972:
URL: https://github.com/apache/kafka/pull/12972#issuecomment-1344397829

   This PR incudes https://github.com/apache/kafka/pull/12969 for 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.

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

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


[GitHub] [kafka] hachikuji commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "hachikuji (via GitHub)" <gi...@apache.org>.
hachikuji commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1098020205


##########
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 in `KafkaApisTest`?



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

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

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


[GitHub] [kafka] dajac commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1099084820


##########
generator/src/main/java/org/apache/kafka/message/ApiMessageTypeGenerator.java:
##########
@@ -408,6 +410,23 @@ private void generateListenerTypesEnum() {
         buffer.printf("}%n");
     }
 
+    private void generateHighestStableVersion() {
+        buffer.printf("public short highestStableVersion(boolean enableUnstableLastVersion) {%n");

Review Comment:
   Yeah, you're right. I went with `highestSupportedVersion(boolean enableUnstableLastVersion)`. See my last commit.



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1098684544


##########
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:
   Added `highestStableVersion(enableUnstableLastVersion)` to `ApiMessageType`.



##########
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:
   Good idea.



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by GitBox <gi...@apache.org>.
dajac commented on PR #12972:
URL: https://github.com/apache/kafka/pull/12972#issuecomment-1384315580

   @hachikuji I played a bit with your suggesting and I personally prefer the current approach: `"apiStability": "evolving|stable"`. It is a bit odd to specify the unstable version because it is always the last one. Instead, I think that we could involve the current approach to apply  `apiStability` to the last version only. I would save this for another patch because it is not strictly required for 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.

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

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


[GitHub] [kafka] hachikuji commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "hachikuji (via GitHub)" <gi...@apache.org>.
hachikuji commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1096118406


##########
clients/src/main/java/org/apache/kafka/common/protocol/Errors.java:
##########
@@ -372,7 +375,10 @@ public enum Errors {
     FETCH_SESSION_TOPIC_ID_ERROR(106, "The fetch session encountered inconsistent topic ID usage", FetchSessionTopicIdException::new),
     INELIGIBLE_REPLICA(107, "The new ISR contains at least one ineligible replica.", IneligibleReplicaException::new),
     NEW_LEADER_ELECTED(108, "The AlterPartition request successfully updated the partition state but the leader has changed.", NewLeaderElectedException::new),
-    OFFSET_MOVED_TO_TIERED_STORAGE(109, "The requested offset is moved to tiered storage.", OffsetMovedToTieredStorageException::new);
+    OFFSET_MOVED_TO_TIERED_STORAGE(109, "The requested offset is moved to tiered storage.", OffsetMovedToTieredStorageException::new),
+    FENCED_MEMBER_EPOCH(110, "The member epoch is fenced by the group coordinator. The member must abandon all its partitions and rejoins.", FencedMemberEpochException::new),

Review Comment:
   nit: rejoin (singular)?



##########
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
+  // - NOT_COORDINATOR
+  // - COORDINATOR_NOT_AVAILABLE
+  // - COORDINATOR_LOAD_IN_PROGRESS
+  // - INVALID_REQUEST
+  // - UNKNOWN_MEMBER_ID
+  // - FENCED_MEMBER_EPOCH
+  // - UNSUPPORTED_ASSIGNOR
+  // - UNRELEASED_INSTANCE_ID
+  // - GROUP_MAX_SIZE_REACHED
+  "fields": [
+    { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
+      "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": "ErrorCode", "type": "int16", "versions": "0+",
+      "about": "The top-level error code, or 0 if there was no error" },
+    { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null",

Review Comment:
   I think we have an 'errorMessage' entity type.



##########
clients/src/main/resources/common/message/ConsumerGroupHeartbeatRequest.json:
##########
@@ -0,0 +1,69 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements.  See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//    http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+{
+  "apiKey": 68,
+  "type": "request",
+  "listeners": ["zkBroker", "broker"],
+  "name": "ConsumerGroupHeartbeatRequest",
+  // The ConsumerGroupHeartbeat API is added as part of KIP-848 and is still
+  // under developement. Hence, the API is not exposed by default by brokers
+  // unless explicitely enabled.
+  "latestVersionUnstable": true,
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId",
+      "about": "The group identifier." },
+    { "name": "MemberId", "type": "string", "versions": "0+",
+      "about": "The member id generated by the coordinator. The member id must be kept during the entire lifetime of the member." },
+    { "name": "MemberEpoch", "type": "int32", "versions": "0+",
+      "about": "The current member epoch; 0 to join the group; -1 to leave the group; -2 to indicate that the static member will rejoin." },
+    { "name": "InstanceId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null",
+      "about": "null if not provided or if it didn't change since the last heartbeat; the instance Id otherwise." },
+    { "name": "RackId", "type": "string", "versions": "0+",  "nullableVersions": "0+", "default": "null",
+      "about": "null if not provided or if it didn't change since the last heartbeat; the rack ID of consumer otherwise." },
+    { "name": "RebalanceTimeoutMs", "type": "int32", "versions": "0+", "default": -1,
+      "about": "-1 if it didn't chance since the last heartbeat; the maximum time in milliseconds that the coordinator will wait on the member to revoke its partitions otherwise." },
+    { "name": "SubscribedTopicNames", "type": "[]string", "versions": "0+", "nullableVersions": "0+", "default": "null",

Review Comment:
   Should set entity type `topicName`.



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1097095159


##########
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
+  // - NOT_COORDINATOR
+  // - COORDINATOR_NOT_AVAILABLE
+  // - COORDINATOR_LOAD_IN_PROGRESS
+  // - INVALID_REQUEST
+  // - UNKNOWN_MEMBER_ID
+  // - FENCED_MEMBER_EPOCH
+  // - UNSUPPORTED_ASSIGNOR
+  // - UNRELEASED_INSTANCE_ID
+  // - GROUP_MAX_SIZE_REACHED
+  "fields": [
+    { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
+      "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": "ErrorCode", "type": "int16", "versions": "0+",
+      "about": "The top-level error code, or 0 if there was no error" },
+    { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null",

Review Comment:
   We don't have it: https://github.com/apache/kafka/blob/trunk/generator/src/main/java/org/apache/kafka/message/EntityType.java#L22.



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] hachikuji commented on pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

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

   @dajac That could work too. Mainly I just want to get it into the spec. Another thought I had is explicitly calling out the unstable versions. For example:
   ```json
   "unstableVersions": "0"
   ```
   The advantage is that we could use this for version bumps 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.

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

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


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by GitBox <gi...@apache.org>.
jeffkbkim commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1072528536


##########
core/src/main/scala/kafka/server/KafkaConfig.scala:
##########
@@ -619,6 +619,9 @@ object KafkaConfig {
   val PasswordEncoderKeyLengthProp =  "password.encoder.key.length"
   val PasswordEncoderIterationsProp =  "password.encoder.iterations"
 
+  /** Internal Configurations **/
+  val UnreleasedApisEnableProd = "unreleased.apis.enable"

Review Comment:
   should this be Prop?



##########
clients/src/main/java/org/apache/kafka/common/protocol/Errors.java:
##########
@@ -372,7 +375,10 @@ public enum Errors {
     FETCH_SESSION_TOPIC_ID_ERROR(106, "The fetch session encountered inconsistent topic ID usage", FetchSessionTopicIdException::new),
     INELIGIBLE_REPLICA(107, "The new ISR contains at least one ineligible replica.", IneligibleReplicaException::new),
     NEW_LEADER_ELECTED(108, "The AlterPartition request successfully updated the partition state but the leader has changed.", NewLeaderElectedException::new),
-    OFFSET_MOVED_TO_TIERED_STORAGE(109, "The requested offset is moved to tiered storage.", OffsetMovedToTieredStorageException::new);
+    OFFSET_MOVED_TO_TIERED_STORAGE(109, "The requested offset is moved to tiered storage.", OffsetMovedToTieredStorageException::new),
+    FENCED_MEMBER_EPOCH(110, "The member epoch is fenced by the group coordinator. The member must abandon all its partitions and rejoins.", FencedMemberEpochException::new),
+    UNRELEASED_INSTANCE_ID(111, "The instance ID is still used by another member in the consumer group. That member must leave first.", UnreleasedInstanceIdException::new),
+    UNSUPPORTED_ASSIGNOR(112, "The assignor used by the member or its version range are not supported by the consumer group.", UnsupportedAssignorException::new);

Review Comment:
   nit: is not
   how's "The selected assignor or its version range is not supported by the consumer group."?



##########
core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala:
##########
@@ -68,14 +68,18 @@ abstract class AbstractApiVersionsRequestTest(cluster: ClusterInstance) {
     } finally socket.close()
   }
 
-  def validateApiVersionsResponse(apiVersionsResponse: ApiVersionsResponse, listenerName: ListenerName = cluster.clientListener()): Unit = {
+  def validateApiVersionsResponse(
+    apiVersionsResponse: ApiVersionsResponse,
+    listenerName: ListenerName = cluster.clientListener(),
+    shouldIncludeUnreleasedApi: Boolean = false

Review Comment:
   nit: Apis



##########
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
+  // - NOT_COORDINATOR
+  // - COORDINATOR_NOT_AVAILABLE
+  // - COORDINATOR_LOAD_IN_PROGRESS
+  // - INVALID_REQUEST
+  // - UNKNOWN_MEMBER_ID
+  // - FENCED_MEMBER_EPOCH
+  // - UNSUPPORTED_ASSIGNOR
+  // - UNRELEASED_INSTANCE_ID
+  // - GROUP_MAX_SIZE_REACHED

Review Comment:
   i noticed this is missing in the KIP. should we include it?



##########
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
+  // - NOT_COORDINATOR
+  // - COORDINATOR_NOT_AVAILABLE
+  // - COORDINATOR_LOAD_IN_PROGRESS
+  // - INVALID_REQUEST
+  // - UNKNOWN_MEMBER_ID
+  // - FENCED_MEMBER_EPOCH
+  // - UNSUPPORTED_ASSIGNOR
+  // - UNRELEASED_INSTANCE_ID
+  // - GROUP_MAX_SIZE_REACHED
+  "fields": [
+    { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
+      "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": "ErrorCode", "type": "int16", "versions": "0+",
+      "about": "The top-level error code, or 0 if there was no error" },
+    { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null",
+      "about": "The top-level error message, or null if there was no error." },
+    { "name": "MemberId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null",
+      "about": "The member id generated by the coordinator. Only provided when the member joins with MemberEpoch == 0." },
+    { "name": "MemberEpoch", "type": "int32", "versions": "0+",
+      "about": "The member epoch." },
+    { "name": "ShouldComputeAssignment", "type": "bool", "versions": "0+",
+      "about": "True if the member should compute the assignment for the group." },
+    { "name": "HeartbeatIntervalMs", "type": "int32", "versions": "0+",
+      "about": "The heartbeat interval in milliseconds." },
+    { "name": "Assignment", "type": "Assignment", "versions": "0+", "nullableVersions": "0+", "default": "null",
+      "about": "null if not provided; the assignment otherwise.", "fields": [
+      { "name": "Error", "type": "int8", "versions": "0+",
+        "about": "The assigned error." },
+      { "name": "AssignedTopicPartitions", "type": "[]TopicPartitions", "versions": "0+",
+        "about": "The partitions assigned to the member that can be used immediately." },
+      { "name": "PendingTopicPartitions", "type": "[]TopicPartitions", "versions": "0+",
+        "about": "The partitions assigned to the member that cannot be used because they are not released by their former owners yet." },
+      { "name": "MetadataVersion", "type": "int16", "versions": "0+",
+        "about": "The version of the metadata." },
+      { "name": "MetadataBytes", "type": "bytes", "versions": "0+",
+        "about": "The assigned metadata." }
+    ]}
+  ],
+  "commonStructs": [

Review Comment:
   what's the difference between fields and commonStructs?



##########
clients/src/main/resources/common/message/ConsumerGroupHeartbeatRequest.json:
##########
@@ -0,0 +1,66 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements.  See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//    http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT 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": "request",
+  "apiStability": "evolving",
+  "listeners": ["zkBroker", "broker"],
+  "name": "ConsumerGroupHeartbeatRequest",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId",
+      "about": "The group identifier." },
+    { "name": "MemberId", "type": "string", "versions": "0+",
+      "about": "The member id generated by the coordinator. The member id must be kept during the entire lifetime of the member." },
+    { "name": "MemberEpoch", "type": "int32", "versions": "0+",
+      "about": "The current member epoch; 0 to join the group; -1 to leave the group; -2 to indicate that the static member will rejoin." },
+    { "name": "InstanceId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null",
+      "about": "null if not provided or if it didn't change since the last heartbeat; the instance Id otherwise." },
+    { "name": "RackId", "type": "string", "versions": "0+",  "nullableVersions": "0+", "default": "null",
+      "about": "null if not provided or if it didn't change since the last heartbeat; the rack ID of consumer otherwise." },
+    { "name": "RebalanceTimeoutMs", "type": "int32", "versions": "0+", "default": -1,
+      "about": "-1 if it didn't chance since the last heartbeat; the maximum time in milliseconds that the coordinator will wait on the member to revoke its partitions otherwise." },
+    { "name": "SubscribedTopicNames", "type": "[]string", "versions": "0+", "nullableVersions": "0+", "default": "null",
+      "about": "null if it didn't change since the last heartbeat; the subscribed topic names otherwise." },
+    { "name": "SubscribedTopicRegex", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null",
+      "about": "null if it didn't change since the last heartbeat; the subscribed topic regex otherwise" },
+    { "name": "ServerAssignor", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null",
+      "about": "null if not used or if it didn't change since the last heartbeat; the server side assignor to use otherwise." },
+    { "name": "ClientAssignors", "type": "[]Assignor", "versions": "0+", "nullableVersions": "0+", "default": "null",
+      "about": "null if not used or if it didn't change since the last heartbeat; the list of client-side assignors otherwise.", "fields": [
+      { "name": "Name", "type": "string", "versions": "0+",
+        "about": "The name of the assignor." },
+      { "name": "MinimumVersion", "type": "int16", "versions": "0+",
+        "about": "The minimum supported version for the metadata." },
+      { "name": "MaximumVersion", "type": "int16", "versions": "0+",
+        "about": "The maximum supported version for the metadata." },
+      { "name": "Reason", "type": "int8", "versions": "0+",

Review Comment:
   can you provide an example of what we store in the byte?



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1092319133


##########
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
+  // - NOT_COORDINATOR
+  // - COORDINATOR_NOT_AVAILABLE
+  // - COORDINATOR_LOAD_IN_PROGRESS
+  // - INVALID_REQUEST
+  // - UNKNOWN_MEMBER_ID
+  // - FENCED_MEMBER_EPOCH
+  // - UNSUPPORTED_ASSIGNOR
+  // - UNRELEASED_INSTANCE_ID
+  // - GROUP_MAX_SIZE_REACHED

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.

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

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


[GitHub] [kafka] hachikuji commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "hachikuji (via GitHub)" <gi...@apache.org>.
hachikuji commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1099052637


##########
generator/src/main/java/org/apache/kafka/message/ApiMessageTypeGenerator.java:
##########
@@ -408,6 +410,23 @@ private void generateListenerTypesEnum() {
         buffer.printf("}%n");
     }
 
+    private void generateHighestStableVersion() {
+        buffer.printf("public short highestStableVersion(boolean enableUnstableLastVersion) {%n");

Review Comment:
   nit: It's a little odd to call this `highestStableVersion` and also accept a flag to enable unstable versions. Maybe we could replace the implementation of `highestSupportedVersion` to accept the flag.
   
   Or alternatively, the helper can be `highestStableVersion()` and we can drop the flag. Then we can check the flag in the code and use either `highestStableVersion` or `highestSupportedVersion` 



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] guozhangwang commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "guozhangwang (via GitHub)" <gi...@apache.org>.
guozhangwang commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1099085546


##########
core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala:
##########
@@ -44,6 +44,13 @@ class ApiVersionsRequestTest(cluster: ClusterInstance) extends AbstractApiVersio
     validateApiVersionsResponse(apiVersionsResponse)
   }
 
+  @ClusterTest(serverProperties = Array(new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true")))

Review Comment:
   Just curious: do we need the serverProperties given that we call the validate func below with specific `enableUnstableLastVersion` set already?



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1099087860


##########
core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala:
##########
@@ -44,6 +44,13 @@ class ApiVersionsRequestTest(cluster: ClusterInstance) extends AbstractApiVersio
     validateApiVersionsResponse(apiVersionsResponse)
   }
 
+  @ClusterTest(serverProperties = Array(new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true")))

Review Comment:
   Yes, we need both. `enableUnstableLastVersion = true` is to actually validate that the unstable is correctly exposed.



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by GitBox <gi...@apache.org>.
jolshan commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1054893630


##########
clients/src/main/java/org/apache/kafka/common/errors/FencedMemberEpoch.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 FencedMemberEpoch extends ApiException {

Review Comment:
   Do we want to add the word Exception to the end of the class name and file?



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

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

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


[GitHub] [kafka] dajac commented on pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on PR #12972:
URL: https://github.com/apache/kafka/pull/12972#issuecomment-1418729277

   @hachikuji Thanks for your comments. I have addressed them.


-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1100705771


##########
clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java:
##########
@@ -212,24 +248,32 @@ public static ApiVersionCollection collectApis(Set<ApiKeys> apiKeys) {
      * @param listenerType the listener type which constrains the set of exposed APIs
      * @param minRecordVersion min inter broker magic
      * @param activeControllerApiVersions controller ApiVersions
+     * @param enableUnstableLastVersion whether unstable versions should be advertised or not

Review Comment:
   Do we have a place where all this new unstable api usage is documented? It could be a bit confusing for folks who didn't take a look at 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.

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

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


[GitHub] [kafka] jolshan commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1098052600


##########
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:
   +1 to this idea



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1098685000


##########
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:
   That makes sense.



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1098628288


##########
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:
   That makes sense.



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1098642239


##########
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:
   Oops... Let me remove this one.



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by GitBox <gi...@apache.org>.
jolshan commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1070126393


##########
clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java:
##########
@@ -185,11 +215,11 @@ public static ApiVersionsResponse createApiVersionsResponse(
     }
 
     public static ApiVersionCollection filterApis(
-        RecordVersion minRecordVersion,
-        ApiMessageType.ListenerType listenerType
+        Set<ApiKeys> enabledApi,

Review Comment:
   nit: Do we want to make this plural `enabledApis`



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #12972: KAFKA-14391; Add ConsumerGroupHeartbeat API

Posted by GitBox <gi...@apache.org>.
jolshan commented on code in PR #12972:
URL: https://github.com/apache/kafka/pull/12972#discussion_r1072860670


##########
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
+  // - NOT_COORDINATOR
+  // - COORDINATOR_NOT_AVAILABLE
+  // - COORDINATOR_LOAD_IN_PROGRESS
+  // - INVALID_REQUEST
+  // - UNKNOWN_MEMBER_ID
+  // - FENCED_MEMBER_EPOCH
+  // - UNSUPPORTED_ASSIGNOR
+  // - UNRELEASED_INSTANCE_ID
+  // - GROUP_MAX_SIZE_REACHED
+  "fields": [
+    { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
+      "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": "ErrorCode", "type": "int16", "versions": "0+",
+      "about": "The top-level error code, or 0 if there was no error" },
+    { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null",
+      "about": "The top-level error message, or null if there was no error." },
+    { "name": "MemberId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null",
+      "about": "The member id generated by the coordinator. Only provided when the member joins with MemberEpoch == 0." },
+    { "name": "MemberEpoch", "type": "int32", "versions": "0+",
+      "about": "The member epoch." },
+    { "name": "ShouldComputeAssignment", "type": "bool", "versions": "0+",
+      "about": "True if the member should compute the assignment for the group." },
+    { "name": "HeartbeatIntervalMs", "type": "int32", "versions": "0+",
+      "about": "The heartbeat interval in milliseconds." },
+    { "name": "Assignment", "type": "Assignment", "versions": "0+", "nullableVersions": "0+", "default": "null",
+      "about": "null if not provided; the assignment otherwise.", "fields": [
+      { "name": "Error", "type": "int8", "versions": "0+",
+        "about": "The assigned error." },
+      { "name": "AssignedTopicPartitions", "type": "[]TopicPartitions", "versions": "0+",
+        "about": "The partitions assigned to the member that can be used immediately." },
+      { "name": "PendingTopicPartitions", "type": "[]TopicPartitions", "versions": "0+",
+        "about": "The partitions assigned to the member that cannot be used because they are not released by their former owners yet." },
+      { "name": "MetadataVersion", "type": "int16", "versions": "0+",
+        "about": "The version of the metadata." },
+      { "name": "MetadataBytes", "type": "bytes", "versions": "0+",
+        "about": "The assigned metadata." }
+    ]}
+  ],
+  "commonStructs": [

Review Comment:
   Common structs can be used multiple times when writing the json. It prevents repeated text in the fields.



-- 
This is an automated message from the 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-unsubscribe@kafka.apache.org

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