From d6bff1f19ae393c0dd94fb847bf2f74ef12fd08a Mon Sep 17 00:00:00 2001 From: brenden20 Date: Mon, 3 Jun 2024 16:14:23 -0500 Subject: [PATCH 01/46] Revert "KAFKA-16713: Define initial set of RPCs for KIP-932 (#16022)" This reverts commit 8f82f14a483d827d4653fb27b09945df936f49f6. --- .../apache/kafka/common/ShareGroupState.java | 56 ---- .../errors/FencedStateEpochException.java | 28 -- .../errors/InvalidRecordStateException.java | 30 -- .../InvalidShareSessionEpochException.java | 28 -- .../errors/ShareSessionNotFoundException.java | 28 -- .../apache/kafka/common/protocol/ApiKeys.java | 6 +- .../apache/kafka/common/protocol/Errors.java | 10 +- .../common/requests/AbstractRequest.java | 8 - .../common/requests/AbstractResponse.java | 8 - .../requests/ShareAcknowledgeRequest.java | 127 --------- .../requests/ShareAcknowledgeResponse.java | 148 ---------- .../common/requests/ShareFetchMetadata.java | 121 -------- .../common/requests/ShareFetchRequest.java | 267 ------------------ .../common/requests/ShareFetchResponse.java | 212 -------------- .../requests/ShareGroupDescribeRequest.java | 100 ------- .../requests/ShareGroupDescribeResponse.java | 77 ----- .../requests/ShareGroupHeartbeatRequest.java | 86 ------ .../requests/ShareGroupHeartbeatResponse.java | 71 ----- .../message/FindCoordinatorRequest.json | 4 +- .../message/FindCoordinatorResponse.json | 4 +- .../common/message/ListGroupsRequest.json | 4 +- .../common/message/ListGroupsResponse.json | 4 +- .../message/ShareAcknowledgeRequest.json | 53 ---- .../message/ShareAcknowledgeResponse.json | 72 ----- .../common/message/ShareFetchRequest.json | 67 ----- .../common/message/ShareFetchResponse.json | 83 ------ .../message/ShareGroupDescribeRequest.json | 33 --- .../message/ShareGroupDescribeResponse.json | 87 ------ .../message/ShareGroupHeartbeatRequest.json | 39 --- .../message/ShareGroupHeartbeatResponse.json | 57 ---- .../common/requests/RequestResponseTest.java | 129 --------- .../kafka/network/RequestConvertToJson.scala | 8 - .../kafka/server/ApiVersionsRequestTest.scala | 2 +- .../unit/kafka/server/RequestQuotaTest.scala | 16 +- docs/security.html | 36 --- 35 files changed, 9 insertions(+), 2100 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/common/ShareGroupState.java delete mode 100644 clients/src/main/java/org/apache/kafka/common/errors/FencedStateEpochException.java delete mode 100644 clients/src/main/java/org/apache/kafka/common/errors/InvalidRecordStateException.java delete mode 100644 clients/src/main/java/org/apache/kafka/common/errors/InvalidShareSessionEpochException.java delete mode 100644 clients/src/main/java/org/apache/kafka/common/errors/ShareSessionNotFoundException.java delete mode 100644 clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeRequest.java delete mode 100644 clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeResponse.java delete mode 100644 clients/src/main/java/org/apache/kafka/common/requests/ShareFetchMetadata.java delete mode 100644 clients/src/main/java/org/apache/kafka/common/requests/ShareFetchRequest.java delete mode 100644 clients/src/main/java/org/apache/kafka/common/requests/ShareFetchResponse.java delete mode 100644 clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeRequest.java delete mode 100644 clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeResponse.java delete mode 100644 clients/src/main/java/org/apache/kafka/common/requests/ShareGroupHeartbeatRequest.java delete mode 100644 clients/src/main/java/org/apache/kafka/common/requests/ShareGroupHeartbeatResponse.java delete mode 100644 clients/src/main/resources/common/message/ShareAcknowledgeRequest.json delete mode 100644 clients/src/main/resources/common/message/ShareAcknowledgeResponse.json delete mode 100644 clients/src/main/resources/common/message/ShareFetchRequest.json delete mode 100644 clients/src/main/resources/common/message/ShareFetchResponse.json delete mode 100644 clients/src/main/resources/common/message/ShareGroupDescribeRequest.json delete mode 100644 clients/src/main/resources/common/message/ShareGroupDescribeResponse.json delete mode 100644 clients/src/main/resources/common/message/ShareGroupHeartbeatRequest.json delete mode 100644 clients/src/main/resources/common/message/ShareGroupHeartbeatResponse.json diff --git a/clients/src/main/java/org/apache/kafka/common/ShareGroupState.java b/clients/src/main/java/org/apache/kafka/common/ShareGroupState.java deleted file mode 100644 index 716421f3dea2a..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/ShareGroupState.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT 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; - -import java.util.Arrays; -import java.util.Locale; -import java.util.Map; -import java.util.function.Function; -import java.util.stream.Collectors; - -/** - * The share group state. - */ -public enum ShareGroupState { - UNKNOWN("Unknown"), - STABLE("Stable"), - DEAD("Dead"), - EMPTY("Empty"); - - private final static Map NAME_TO_ENUM = Arrays.stream(values()) - .collect(Collectors.toMap(state -> state.name.toUpperCase(Locale.ROOT), Function.identity())); - - private final String name; - - ShareGroupState(String name) { - this.name = name; - } - - /** - * Case-insensitive share group state lookup by string name. - */ - public static ShareGroupState parse(String name) { - ShareGroupState state = NAME_TO_ENUM.get(name.toUpperCase(Locale.ROOT)); - return state == null ? UNKNOWN : state; - } - - @Override - public String toString() { - return name; - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/FencedStateEpochException.java b/clients/src/main/java/org/apache/kafka/common/errors/FencedStateEpochException.java deleted file mode 100644 index 1e74bba199402..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/errors/FencedStateEpochException.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT 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; - -/** - * Thrown when the share coordinator rejected the request because the share-group state epoch did not match. - */ -public class FencedStateEpochException extends ApiException { - private static final long serialVersionUID = 1L; - - public FencedStateEpochException(String message) { - super(message); - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidRecordStateException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidRecordStateException.java deleted file mode 100644 index ae0fef5edeaef..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/errors/InvalidRecordStateException.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT 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; - -/** - * Thrown when the acknowledgement of delivery of a record could not be completed because the record - * state is invalid. - */ -public class InvalidRecordStateException extends ApiException { - - private static final long serialVersionUID = 1L; - - public InvalidRecordStateException(String message) { - super(message); - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidShareSessionEpochException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidShareSessionEpochException.java deleted file mode 100644 index e261d8b7a8e88..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/errors/InvalidShareSessionEpochException.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT 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; - -/** - * Thrown when the share session epoch is invalid. - */ -public class InvalidShareSessionEpochException extends RetriableException { - private static final long serialVersionUID = 1L; - - public InvalidShareSessionEpochException(String message) { - super(message); - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/ShareSessionNotFoundException.java b/clients/src/main/java/org/apache/kafka/common/errors/ShareSessionNotFoundException.java deleted file mode 100644 index 2b2249f8a5831..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/errors/ShareSessionNotFoundException.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT 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; - -/** - * Thrown when the share session was not found. - */ -public class ShareSessionNotFoundException extends RetriableException { - private static final long serialVersionUID = 1L; - - public ShareSessionNotFoundException(String message) { - super(message); - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index ffd5737ca3162..16bec4fb72dc6 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -118,11 +118,7 @@ public enum ApiKeys { PUSH_TELEMETRY(ApiMessageType.PUSH_TELEMETRY), ASSIGN_REPLICAS_TO_DIRS(ApiMessageType.ASSIGN_REPLICAS_TO_DIRS), LIST_CLIENT_METRICS_RESOURCES(ApiMessageType.LIST_CLIENT_METRICS_RESOURCES), - DESCRIBE_TOPIC_PARTITIONS(ApiMessageType.DESCRIBE_TOPIC_PARTITIONS), - SHARE_GROUP_HEARTBEAT(ApiMessageType.SHARE_GROUP_HEARTBEAT), - SHARE_GROUP_DESCRIBE(ApiMessageType.SHARE_GROUP_DESCRIBE), - SHARE_FETCH(ApiMessageType.SHARE_FETCH), - SHARE_ACKNOWLEDGE(ApiMessageType.SHARE_ACKNOWLEDGE); + DESCRIBE_TOPIC_PARTITIONS(ApiMessageType.DESCRIBE_TOPIC_PARTITIONS); private static final Map> APIS_BY_LISTENER = new EnumMap<>(ApiMessageType.ListenerType.class); diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index 10ae05aa850c9..900d191c8f9d4 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -41,7 +41,6 @@ import org.apache.kafka.common.errors.FencedInstanceIdException; import org.apache.kafka.common.errors.FencedLeaderEpochException; import org.apache.kafka.common.errors.FencedMemberEpochException; -import org.apache.kafka.common.errors.FencedStateEpochException; import org.apache.kafka.common.errors.FetchSessionIdNotFoundException; import org.apache.kafka.common.errors.FetchSessionTopicIdException; import org.apache.kafka.common.errors.GroupAuthorizationException; @@ -65,14 +64,12 @@ import org.apache.kafka.common.errors.InvalidPidMappingException; import org.apache.kafka.common.errors.InvalidPrincipalTypeException; import org.apache.kafka.common.errors.InvalidProducerEpochException; -import org.apache.kafka.common.errors.InvalidRecordStateException; import org.apache.kafka.common.errors.InvalidRegistrationException; import org.apache.kafka.common.errors.InvalidReplicaAssignmentException; import org.apache.kafka.common.errors.InvalidReplicationFactorException; import org.apache.kafka.common.errors.InvalidRequestException; import org.apache.kafka.common.errors.InvalidRequiredAcksException; import org.apache.kafka.common.errors.InvalidSessionTimeoutException; -import org.apache.kafka.common.errors.InvalidShareSessionEpochException; import org.apache.kafka.common.errors.InvalidTimestampException; import org.apache.kafka.common.errors.InvalidTopicException; import org.apache.kafka.common.errors.InvalidTxnStateException; @@ -112,7 +109,6 @@ import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.errors.SaslAuthenticationException; import org.apache.kafka.common.errors.SecurityDisabledException; -import org.apache.kafka.common.errors.ShareSessionNotFoundException; import org.apache.kafka.common.errors.SnapshotNotFoundException; import org.apache.kafka.common.errors.StaleBrokerEpochException; import org.apache.kafka.common.errors.StaleMemberEpochException; @@ -398,11 +394,7 @@ public enum Errors { UNKNOWN_SUBSCRIPTION_ID(117, "Client sent a push telemetry request with an invalid or outdated subscription ID.", UnknownSubscriptionIdException::new), TELEMETRY_TOO_LARGE(118, "Client sent a push telemetry request larger than the maximum size the broker will accept.", TelemetryTooLargeException::new), INVALID_REGISTRATION(119, "The controller has considered the broker registration to be invalid.", InvalidRegistrationException::new), - TRANSACTION_ABORTABLE(120, "The server encountered an error with the transaction. The client can abort the transaction to continue using this transactional ID.", TransactionAbortableException::new), - INVALID_RECORD_STATE(121, "The record state is invalid. The acknowledgement of delivery could not be completed.", InvalidRecordStateException::new), - SHARE_SESSION_NOT_FOUND(122, "The share session was not found.", ShareSessionNotFoundException::new), - INVALID_SHARE_SESSION_EPOCH(123, "The share session epoch is invalid.", InvalidShareSessionEpochException::new), - FENCED_STATE_EPOCH(124, "The share coordinator rejected the request because the share-group state epoch did not match.", FencedStateEpochException::new); + TRANSACTION_ABORTABLE(120, "The server encountered an error with the transaction. The client can abort the transaction to continue using this transactional ID.", TransactionAbortableException::new); private static final Logger log = LoggerFactory.getLogger(Errors.class); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java index 589e163992b22..b51221f5af642 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java @@ -326,14 +326,6 @@ private static AbstractRequest doParseRequest(ApiKeys apiKey, short apiVersion, return ListClientMetricsResourcesRequest.parse(buffer, apiVersion); case DESCRIBE_TOPIC_PARTITIONS: return DescribeTopicPartitionsRequest.parse(buffer, apiVersion); - case SHARE_GROUP_HEARTBEAT: - return ShareGroupHeartbeatRequest.parse(buffer, apiVersion); - case SHARE_GROUP_DESCRIBE: - return ShareGroupDescribeRequest.parse(buffer, apiVersion); - case SHARE_FETCH: - return ShareFetchRequest.parse(buffer, apiVersion); - case SHARE_ACKNOWLEDGE: - return ShareAcknowledgeRequest.parse(buffer, apiVersion); default: throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseRequest`, the " + "code should be updated to do so.", apiKey)); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java index 5534168098e9d..dbafdbf3bcb07 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java @@ -263,14 +263,6 @@ public static AbstractResponse parseResponse(ApiKeys apiKey, ByteBuffer response return ListClientMetricsResourcesResponse.parse(responseBuffer, version); case DESCRIBE_TOPIC_PARTITIONS: return DescribeTopicPartitionsResponse.parse(responseBuffer, version); - case SHARE_GROUP_HEARTBEAT: - return ShareGroupHeartbeatResponse.parse(responseBuffer, version); - case SHARE_GROUP_DESCRIBE: - return ShareGroupDescribeResponse.parse(responseBuffer, version); - case SHARE_FETCH: - return ShareFetchResponse.parse(responseBuffer, version); - case SHARE_ACKNOWLEDGE: - return ShareAcknowledgeResponse.parse(responseBuffer, version); default: throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseResponse`, the " + "code should be updated to do so.", apiKey)); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeRequest.java deleted file mode 100644 index 1b77b43be33c1..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeRequest.java +++ /dev/null @@ -1,127 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT 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.TopicIdPartition; -import org.apache.kafka.common.Uuid; -import org.apache.kafka.common.message.ShareAcknowledgeRequestData; -import org.apache.kafka.common.message.ShareAcknowledgeResponseData; -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; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class ShareAcknowledgeRequest extends AbstractRequest { - - public static class Builder extends AbstractRequest.Builder { - - private final ShareAcknowledgeRequestData data; - - public Builder(ShareAcknowledgeRequestData data) { - this(data, false); - } - - public Builder(ShareAcknowledgeRequestData data, boolean enableUnstableLastVersion) { - super(ApiKeys.SHARE_ACKNOWLEDGE, enableUnstableLastVersion); - this.data = data; - } - - public static ShareAcknowledgeRequest.Builder forConsumer(String groupId, ShareFetchMetadata metadata, - Map> acknowledgementsMap) { - ShareAcknowledgeRequestData data = new ShareAcknowledgeRequestData(); - data.setGroupId(groupId); - if (metadata != null) { - data.setMemberId(metadata.memberId().toString()); - data.setShareSessionEpoch(metadata.epoch()); - } - - // Build a map of topics to acknowledge keyed by topic ID, and within each a map of partitions keyed by index - Map> ackMap = new HashMap<>(); - - for (Map.Entry> acknowledgeEntry : acknowledgementsMap.entrySet()) { - TopicIdPartition tip = acknowledgeEntry.getKey(); - Map partMap = ackMap.computeIfAbsent(tip.topicId(), k -> new HashMap<>()); - ShareAcknowledgeRequestData.AcknowledgePartition ackPartition = partMap.get(tip.partition()); - if (ackPartition == null) { - ackPartition = new ShareAcknowledgeRequestData.AcknowledgePartition() - .setPartitionIndex(tip.partition()); - partMap.put(tip.partition(), ackPartition); - } - ackPartition.setAcknowledgementBatches(acknowledgeEntry.getValue()); - } - - // Finally, build up the data to fetch - data.setTopics(new ArrayList<>()); - ackMap.forEach((topicId, partMap) -> { - ShareAcknowledgeRequestData.AcknowledgeTopic ackTopic = new ShareAcknowledgeRequestData.AcknowledgeTopic() - .setTopicId(topicId) - .setPartitions(new ArrayList<>()); - data.topics().add(ackTopic); - - partMap.forEach((index, ackPartition) -> ackTopic.partitions().add(ackPartition)); - }); - - return new ShareAcknowledgeRequest.Builder(data, true); - } - - public ShareAcknowledgeRequestData data() { - return data; - } - - @Override - public ShareAcknowledgeRequest build(short version) { - return new ShareAcknowledgeRequest(data, version); - } - - @Override - public String toString() { - return data.toString(); - } - } - - private final ShareAcknowledgeRequestData data; - - public ShareAcknowledgeRequest(ShareAcknowledgeRequestData data, short version) { - super(ApiKeys.SHARE_ACKNOWLEDGE, version); - this.data = data; - } - - @Override - public ShareAcknowledgeRequestData data() { - return data; - } - - @Override - public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { - Errors error = Errors.forException(e); - return new ShareAcknowledgeResponse(new ShareAcknowledgeResponseData() - .setThrottleTimeMs(throttleTimeMs) - .setErrorCode(error.code())); - } - - public static ShareAcknowledgeRequest parse(ByteBuffer buffer, short version) { - return new ShareAcknowledgeRequest( - new ShareAcknowledgeRequestData(new ByteBufferAccessor(buffer), version), - version - ); - } -} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeResponse.java deleted file mode 100644 index 5cab233dccac8..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeResponse.java +++ /dev/null @@ -1,148 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT 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.Node; -import org.apache.kafka.common.TopicIdPartition; -import org.apache.kafka.common.Uuid; -import org.apache.kafka.common.message.ShareAcknowledgeResponseData; -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; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; - -/** - * Possible error codes. - * - {@link Errors#GROUP_AUTHORIZATION_FAILED} - * - {@link Errors#TOPIC_AUTHORIZATION_FAILED} - * - {@link Errors#UNKNOWN_TOPIC_OR_PARTITION} - * - {@link Errors#NOT_LEADER_OR_FOLLOWER} - * - {@link Errors#UNKNOWN_TOPIC_ID} - * - {@link Errors#INVALID_RECORD_STATE} - * - {@link Errors#KAFKA_STORAGE_ERROR} - * - {@link Errors#INVALID_REQUEST} - * - {@link Errors#UNKNOWN_SERVER_ERROR} - */ -public class ShareAcknowledgeResponse extends AbstractResponse { - - private final ShareAcknowledgeResponseData data; - - public ShareAcknowledgeResponse(ShareAcknowledgeResponseData data) { - super(ApiKeys.SHARE_ACKNOWLEDGE); - this.data = data; - } - - public Errors error() { - return Errors.forCode(data.errorCode()); - } - - @Override - public ShareAcknowledgeResponseData data() { - return data; - } - - @Override - public Map errorCounts() { - HashMap counts = new HashMap<>(); - updateErrorCounts(counts, Errors.forCode(data.errorCode())); - data.responses().forEach( - topic -> topic.partitions().forEach( - partition -> updateErrorCounts(counts, Errors.forCode(partition.errorCode())) - ) - ); - return counts; - } - - @Override - public int throttleTimeMs() { - return data.throttleTimeMs(); - } - - @Override - public void maybeSetThrottleTimeMs(int throttleTimeMs) { - data.setThrottleTimeMs(throttleTimeMs); - } - - public static ShareAcknowledgeResponse parse(ByteBuffer buffer, short version) { - return new ShareAcknowledgeResponse( - new ShareAcknowledgeResponseData(new ByteBufferAccessor(buffer), version) - ); - } - - private static boolean matchingTopic(ShareAcknowledgeResponseData.ShareAcknowledgeTopicResponse previousTopic, TopicIdPartition currentTopic) { - if (previousTopic == null) - return false; - return previousTopic.topicId().equals(currentTopic.topicId()); - } - - public static ShareAcknowledgeResponseData.PartitionData partitionResponse(TopicIdPartition topicIdPartition, Errors error) { - return partitionResponse(topicIdPartition.topicPartition().partition(), error); - } - - public static ShareAcknowledgeResponseData.PartitionData partitionResponse(int partition, Errors error) { - return new ShareAcknowledgeResponseData.PartitionData() - .setPartitionIndex(partition) - .setErrorCode(error.code()); - } - - public static ShareAcknowledgeResponse of(Errors error, - int throttleTimeMs, - LinkedHashMap responseData, - List nodeEndpoints) { - return new ShareAcknowledgeResponse(toMessage(error, throttleTimeMs, responseData.entrySet().iterator(), nodeEndpoints)); - } - - public static ShareAcknowledgeResponseData toMessage(Errors error, int throttleTimeMs, - Iterator> partIterator, - List nodeEndpoints) { - Map topicResponseList = new LinkedHashMap<>(); - while (partIterator.hasNext()) { - Map.Entry entry = partIterator.next(); - ShareAcknowledgeResponseData.PartitionData partitionData = entry.getValue(); - // Since PartitionData alone doesn't know the partition ID, we set it here - partitionData.setPartitionIndex(entry.getKey().topicPartition().partition()); - // Checking if the topic is already present in the map - if (topicResponseList.containsKey(entry.getKey().topicId())) { - topicResponseList.get(entry.getKey().topicId()).partitions().add(partitionData); - } else { - List partitionResponses = new ArrayList<>(); - partitionResponses.add(partitionData); - topicResponseList.put(entry.getKey().topicId(), new ShareAcknowledgeResponseData.ShareAcknowledgeTopicResponse() - .setTopicId(entry.getKey().topicId()) - .setPartitions(partitionResponses)); - } - } - ShareAcknowledgeResponseData data = new ShareAcknowledgeResponseData(); - // KafkaApis should only pass in node endpoints on error, otherwise this should be an empty list - nodeEndpoints.forEach(endpoint -> data.nodeEndpoints().add( - new ShareAcknowledgeResponseData.NodeEndpoint() - .setNodeId(endpoint.id()) - .setHost(endpoint.host()) - .setPort(endpoint.port()) - .setRack(endpoint.rack()))); - return data.setThrottleTimeMs(throttleTimeMs) - .setErrorCode(error.code()) - .setResponses(new ArrayList<>(topicResponseList.values())); - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchMetadata.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchMetadata.java deleted file mode 100644 index 4e5bcc2237e43..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchMetadata.java +++ /dev/null @@ -1,121 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT 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.Uuid; - -public class ShareFetchMetadata { - /** - * The first epoch. When used in a ShareFetch request, indicates that the client - * wants to create a session. - */ - public static final int INITIAL_EPOCH = 0; - - /** - * An invalid epoch. When used in a ShareFetch request, indicates that the client - * wants to close an existing session. - */ - public static final int FINAL_EPOCH = -1; - - /** - * - */ - public boolean isNewSession() { - return epoch == INITIAL_EPOCH; - } - - /** - * Returns true if this is a full share fetch request. - */ - public boolean isFull() { - return (this.epoch == INITIAL_EPOCH) || (this.epoch == FINAL_EPOCH); - } - - /** - * Returns the next epoch. - * - * @param prevEpoch The previous epoch. - * @return The next epoch. - */ - public static int nextEpoch(int prevEpoch) { - if (prevEpoch < 0) { - // The next epoch after FINAL_EPOCH is always FINAL_EPOCH itself. - return FINAL_EPOCH; - } else if (prevEpoch == Integer.MAX_VALUE) { - return 1; - } else { - return prevEpoch + 1; - } - } - - /** - * The member ID. - */ - private final Uuid memberId; - - /** - * The share session epoch. - */ - private final int epoch; - - public ShareFetchMetadata(Uuid memberId, int epoch) { - this.memberId = memberId; - this.epoch = epoch; - } - - public static ShareFetchMetadata initialEpoch(Uuid memberId) { - return new ShareFetchMetadata(memberId, INITIAL_EPOCH); - } - - public ShareFetchMetadata nextEpoch() { - return new ShareFetchMetadata(memberId, nextEpoch(epoch)); - } - - public ShareFetchMetadata nextCloseExistingAttemptNew() { - return new ShareFetchMetadata(memberId, INITIAL_EPOCH); - } - - public ShareFetchMetadata finalEpoch() { - return new ShareFetchMetadata(memberId, FINAL_EPOCH); - } - - public Uuid memberId() { - return memberId; - } - - public int epoch() { - return epoch; - } - - public boolean isFinalEpoch() { - return epoch == FINAL_EPOCH; - } - - public String toString() { - StringBuilder bld = new StringBuilder(); - bld.append("(memberId=").append(memberId).append(", "); - if (epoch == INITIAL_EPOCH) { - bld.append("epoch=INITIAL)"); - } else if (epoch == FINAL_EPOCH) { - bld.append("epoch=FINAL)"); - } else { - bld.append("epoch=").append(epoch).append(")"); - } - return bld.toString(); - } -} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchRequest.java deleted file mode 100644 index 385e802a691a9..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchRequest.java +++ /dev/null @@ -1,267 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT 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.TopicIdPartition; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.Uuid; -import org.apache.kafka.common.message.ShareFetchRequestData; -import org.apache.kafka.common.message.ShareFetchResponseData; -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; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; - -public class ShareFetchRequest extends AbstractRequest { - - public static class Builder extends AbstractRequest.Builder { - - private final ShareFetchRequestData data; - - public Builder(ShareFetchRequestData data) { - this(data, false); - } - - public Builder(ShareFetchRequestData data, boolean enableUnstableLastVersion) { - super(ApiKeys.SHARE_FETCH, enableUnstableLastVersion); - this.data = data; - } - - public static Builder forConsumer(String groupId, ShareFetchMetadata metadata, - int maxWait, int minBytes, int maxBytes, int fetchSize, - List send, List forget, - Map> acknowledgementsMap) { - ShareFetchRequestData data = new ShareFetchRequestData(); - data.setGroupId(groupId); - int ackOnlyPartitionMaxBytes = fetchSize; - boolean isClosingShareSession = false; - if (metadata != null) { - data.setMemberId(metadata.memberId().toString()); - data.setShareSessionEpoch(metadata.epoch()); - if (metadata.isFinalEpoch()) { - isClosingShareSession = true; - ackOnlyPartitionMaxBytes = 0; - } - } - data.setMaxWaitMs(maxWait); - data.setMinBytes(minBytes); - data.setMaxBytes(maxBytes); - - // Build a map of topics to fetch keyed by topic ID, and within each a map of partitions keyed by index - Map> fetchMap = new HashMap<>(); - - // First, start by adding the list of topic-partitions we are fetching - if (!isClosingShareSession) { - for (TopicIdPartition tip : send) { - Map partMap = fetchMap.computeIfAbsent(tip.topicId(), k -> new HashMap<>()); - ShareFetchRequestData.FetchPartition fetchPartition = new ShareFetchRequestData.FetchPartition() - .setPartitionIndex(tip.partition()) - .setPartitionMaxBytes(fetchSize); - partMap.put(tip.partition(), fetchPartition); - } - } - - // Next, add acknowledgements that we are piggybacking onto the fetch. Generally, the list of - // topic-partitions will be a subset, but if the assignment changes, there might be new entries to add - for (Map.Entry> acknowledgeEntry : acknowledgementsMap.entrySet()) { - TopicIdPartition tip = acknowledgeEntry.getKey(); - Map partMap = fetchMap.computeIfAbsent(tip.topicId(), k -> new HashMap<>()); - ShareFetchRequestData.FetchPartition fetchPartition = partMap.get(tip.partition()); - if (fetchPartition == null) { - fetchPartition = new ShareFetchRequestData.FetchPartition() - .setPartitionIndex(tip.partition()) - .setPartitionMaxBytes(ackOnlyPartitionMaxBytes); - partMap.put(tip.partition(), fetchPartition); - } - fetchPartition.setAcknowledgementBatches(acknowledgeEntry.getValue()); - } - - // Build up the data to fetch - if (!fetchMap.isEmpty()) { - data.setTopics(new ArrayList<>()); - fetchMap.forEach((topicId, partMap) -> { - ShareFetchRequestData.FetchTopic fetchTopic = new ShareFetchRequestData.FetchTopic() - .setTopicId(topicId) - .setPartitions(new ArrayList<>()); - partMap.forEach((index, fetchPartition) -> fetchTopic.partitions().add(fetchPartition)); - data.topics().add(fetchTopic); - }); - } - - // And finally, forget the topic-partitions that are no longer in the session - if (!forget.isEmpty()) { - Map> forgetMap = new HashMap<>(); - for (TopicIdPartition tip : forget) { - List partList = forgetMap.computeIfAbsent(tip.topicId(), k -> new ArrayList<>()); - partList.add(tip.partition()); - } - data.setForgottenTopicsData(new ArrayList<>()); - forgetMap.forEach((topicId, partList) -> { - ShareFetchRequestData.ForgottenTopic forgetTopic = new ShareFetchRequestData.ForgottenTopic() - .setTopicId(topicId) - .setPartitions(new ArrayList<>()); - partList.forEach(index -> forgetTopic.partitions().add(index)); - data.forgottenTopicsData().add(forgetTopic); - }); - } - - return new Builder(data, true); - } - - public ShareFetchRequestData data() { - return data; - } - - @Override - public ShareFetchRequest build(short version) { - return new ShareFetchRequest(data, version); - } - - @Override - public String toString() { - return data.toString(); - } - } - - private final ShareFetchRequestData data; - private volatile LinkedHashMap shareFetchData = null; - private volatile List toForget = null; - - public ShareFetchRequest(ShareFetchRequestData data, short version) { - super(ApiKeys.SHARE_FETCH, version); - this.data = data; - } - - @Override - public ShareFetchRequestData data() { - return data; - } - - @Override - public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { - Errors error = Errors.forException(e); - return new ShareFetchResponse(new ShareFetchResponseData() - .setThrottleTimeMs(throttleTimeMs) - .setErrorCode(error.code())); - } - - public static ShareFetchRequest parse(ByteBuffer buffer, short version) { - return new ShareFetchRequest( - new ShareFetchRequestData(new ByteBufferAccessor(buffer), version), - version - ); - } - - public static final class SharePartitionData { - public final Uuid topicId; - public final int maxBytes; - - public SharePartitionData( - Uuid topicId, - int maxBytes - ) { - this.topicId = topicId; - this.maxBytes = maxBytes; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - ShareFetchRequest.SharePartitionData that = (ShareFetchRequest.SharePartitionData) o; - return Objects.equals(topicId, that.topicId) && - maxBytes == that.maxBytes; - } - - @Override - public int hashCode() { - return Objects.hash(topicId, maxBytes); - } - - @Override - public String toString() { - return "SharePartitionData(" + - "topicId=" + topicId + - ", maxBytes=" + maxBytes + - ')'; - } - } - - public int minBytes() { - return data.minBytes(); - } - - public int maxBytes() { - return data.maxBytes(); - } - - public int maxWait() { - return data.maxWaitMs(); - } - - public Map shareFetchData(Map topicNames) { - if (shareFetchData == null) { - synchronized (this) { - if (shareFetchData == null) { - // Assigning the lazy-initialized `shareFetchData` in the last step - // to avoid other threads accessing a half-initialized object. - final LinkedHashMap shareFetchDataTmp = new LinkedHashMap<>(); - data.topics().forEach(shareFetchTopic -> { - String name = topicNames.get(shareFetchTopic.topicId()); - shareFetchTopic.partitions().forEach(shareFetchPartition -> { - // Topic name may be null here if the topic name was unable to be resolved using the topicNames map. - shareFetchDataTmp.put(new TopicIdPartition(shareFetchTopic.topicId(), new TopicPartition(name, shareFetchPartition.partitionIndex())), - new ShareFetchRequest.SharePartitionData( - shareFetchTopic.topicId(), - shareFetchPartition.partitionMaxBytes() - ) - ); - }); - }); - shareFetchData = shareFetchDataTmp; - } - } - } - return shareFetchData; - } - - public List forgottenTopics(Map topicNames) { - if (toForget == null) { - synchronized (this) { - if (toForget == null) { - // Assigning the lazy-initialized `toForget` in the last step - // to avoid other threads accessing a half-initialized object. - final List toForgetTmp = new ArrayList<>(); - data.forgottenTopicsData().forEach(forgottenTopic -> { - String name = topicNames.get(forgottenTopic.topicId()); - // Topic name may be null here if the topic name was unable to be resolved using the topicNames map. - forgottenTopic.partitions().forEach(partitionId -> toForgetTmp.add(new TopicIdPartition(forgottenTopic.topicId(), new TopicPartition(name, partitionId)))); - }); - toForget = toForgetTmp; - } - } - } - return toForget; - } -} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchResponse.java deleted file mode 100644 index b33969e0efa41..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchResponse.java +++ /dev/null @@ -1,212 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT 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.Node; -import org.apache.kafka.common.TopicIdPartition; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.Uuid; -import org.apache.kafka.common.message.ShareFetchResponseData; -import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; -import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ObjectSerializationCache; -import org.apache.kafka.common.record.MemoryRecords; -import org.apache.kafka.common.record.Records; - -import java.nio.ByteBuffer; -import java.util.LinkedHashMap; -import java.util.Map; -import java.util.Iterator; -import java.util.Collections; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; - - -/** - * Possible error codes. - * - {@link Errors#GROUP_AUTHORIZATION_FAILED} - * - {@link Errors#TOPIC_AUTHORIZATION_FAILED} - * - {@link Errors#UNKNOWN_TOPIC_OR_PARTITION} - * - {@link Errors#NOT_LEADER_OR_FOLLOWER} - * - {@link Errors#UNKNOWN_TOPIC_ID} - * - {@link Errors#INVALID_RECORD_STATE} - * - {@link Errors#KAFKA_STORAGE_ERROR} - * - {@link Errors#CORRUPT_MESSAGE} - * - {@link Errors#INVALID_REQUEST} - * - {@link Errors#UNKNOWN_SERVER_ERROR} - */ -public class ShareFetchResponse extends AbstractResponse { - - private final ShareFetchResponseData data; - - private volatile LinkedHashMap responseData = null; - - public ShareFetchResponse(ShareFetchResponseData data) { - super(ApiKeys.SHARE_FETCH); - this.data = data; - } - - public Errors error() { - return Errors.forCode(data.errorCode()); - } - - @Override - public ShareFetchResponseData data() { - return data; - } - - @Override - public Map errorCounts() { - HashMap counts = new HashMap<>(); - updateErrorCounts(counts, Errors.forCode(data.errorCode())); - data.responses().forEach( - topic -> topic.partitions().forEach( - partition -> updateErrorCounts(counts, Errors.forCode(partition.errorCode())) - ) - ); - return counts; - } - - public LinkedHashMap responseData(Map topicNames) { - if (responseData == null) { - synchronized (this) { - // Assigning the lazy-initialized `responseData` in the last step - // to avoid other threads accessing a half-initialized object. - if (responseData == null) { - final LinkedHashMap responseDataTmp = new LinkedHashMap<>(); - data.responses().forEach(topicResponse -> { - String name = topicNames.get(topicResponse.topicId()); - if (name != null) { - topicResponse.partitions().forEach(partitionData -> responseDataTmp.put(new TopicIdPartition(topicResponse.topicId(), - new TopicPartition(name, partitionData.partitionIndex())), partitionData)); - } - }); - responseData = responseDataTmp; - } - } - } - return responseData; - } - - @Override - public int throttleTimeMs() { - return data.throttleTimeMs(); - } - - @Override - public void maybeSetThrottleTimeMs(int throttleTimeMs) { - data.setThrottleTimeMs(throttleTimeMs); - } - - public static ShareFetchResponse parse(ByteBuffer buffer, short version) { - return new ShareFetchResponse( - new ShareFetchResponseData(new ByteBufferAccessor(buffer), version) - ); - } - - /** - * Returns `partition.records` as `Records` (instead of `BaseRecords`). If `records` is `null`, returns `MemoryRecords.EMPTY`. - * - *

If this response was deserialized after a share fetch, this method should never fail. An example where this would - * fail is a down-converted response (e.g. LazyDownConversionRecords) on the broker (before it's serialized and - * sent on the wire). - * - * @param partition partition data - * @return Records or empty record if the records in PartitionData is null. - */ - public static Records recordsOrFail(ShareFetchResponseData.PartitionData partition) { - if (partition.records() == null) return MemoryRecords.EMPTY; - if (partition.records() instanceof Records) return (Records) partition.records(); - throw new ClassCastException("The record type is " + partition.records().getClass().getSimpleName() + ", which is not a subtype of " + - Records.class.getSimpleName() + ". This method is only safe to call if the `ShareFetchResponse` was deserialized from bytes."); - } - - /** - * Convenience method to find the size of a response. - * - * @param version The version of the request - * @param partIterator The partition iterator. - * @return The response size in bytes. - */ - public static int sizeOf(short version, - Iterator> partIterator) { - // Since the throttleTimeMs and metadata field sizes are constant and fixed, we can - // use arbitrary values here without affecting the result. - ShareFetchResponseData data = toMessage(Errors.NONE, 0, partIterator, Collections.emptyList()); - ObjectSerializationCache cache = new ObjectSerializationCache(); - return 4 + data.size(cache, version); - } - - /** - * @return The size in bytes of the records. 0 is returned if records of input partition is null. - */ - public static int recordsSize(ShareFetchResponseData.PartitionData partition) { - return partition.records() == null ? 0 : partition.records().sizeInBytes(); - } - - public static ShareFetchResponse of(Errors error, - int throttleTimeMs, - LinkedHashMap responseData, - List nodeEndpoints) { - return new ShareFetchResponse(toMessage(error, throttleTimeMs, responseData.entrySet().iterator(), nodeEndpoints)); - } - - public static ShareFetchResponseData toMessage(Errors error, int throttleTimeMs, - Iterator> partIterator, - List nodeEndpoints) { - Map topicResponseList = new LinkedHashMap<>(); - while (partIterator.hasNext()) { - Map.Entry entry = partIterator.next(); - ShareFetchResponseData.PartitionData partitionData = entry.getValue(); - // Since PartitionData alone doesn't know the partition ID, we set it here - partitionData.setPartitionIndex(entry.getKey().topicPartition().partition()); - // Checking if the topic is already present in the map - if (topicResponseList.containsKey(entry.getKey().topicId())) { - topicResponseList.get(entry.getKey().topicId()).partitions().add(partitionData); - } else { - List partitionResponses = new ArrayList<>(); - partitionResponses.add(partitionData); - topicResponseList.put(entry.getKey().topicId(), new ShareFetchResponseData.ShareFetchableTopicResponse() - .setTopicId(entry.getKey().topicId()) - .setPartitions(partitionResponses)); - } - } - ShareFetchResponseData data = new ShareFetchResponseData(); - // KafkaApis should only pass in node endpoints on error, otherwise this should be an empty list - nodeEndpoints.forEach(endpoint -> data.nodeEndpoints().add( - new ShareFetchResponseData.NodeEndpoint() - .setNodeId(endpoint.id()) - .setHost(endpoint.host()) - .setPort(endpoint.port()) - .setRack(endpoint.rack()))); - return data.setThrottleTimeMs(throttleTimeMs) - .setErrorCode(error.code()) - .setResponses(new ArrayList<>(topicResponseList.values())); - } - - public static ShareFetchResponseData.PartitionData partitionResponse(TopicIdPartition topicIdPartition, Errors error) { - return partitionResponse(topicIdPartition.topicPartition().partition(), error); - } - - public static ShareFetchResponseData.PartitionData partitionResponse(int partition, Errors error) { - return new ShareFetchResponseData.PartitionData() - .setPartitionIndex(partition) - .setErrorCode(error.code()); - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeRequest.java deleted file mode 100644 index 25c02e4a83c5e..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeRequest.java +++ /dev/null @@ -1,100 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT 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.ShareGroupDescribeRequestData; -import org.apache.kafka.common.message.ShareGroupDescribeResponseData; -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; -import java.util.List; -import java.util.stream.Collectors; - -public class ShareGroupDescribeRequest extends AbstractRequest { - - public static class Builder extends AbstractRequest.Builder { - - private final ShareGroupDescribeRequestData data; - - public Builder(ShareGroupDescribeRequestData data) { - this(data, false); - } - - public Builder(ShareGroupDescribeRequestData data, boolean enableUnstableLastVersion) { - super(ApiKeys.SHARE_GROUP_DESCRIBE, enableUnstableLastVersion); - this.data = data; - } - - @Override - public ShareGroupDescribeRequest build(short version) { - return new ShareGroupDescribeRequest(data, version); - } - - @Override - public String toString() { - return data.toString(); - } - } - - private final ShareGroupDescribeRequestData data; - - public ShareGroupDescribeRequest(ShareGroupDescribeRequestData data, short version) { - super(ApiKeys.SHARE_GROUP_DESCRIBE, version); - this.data = data; - } - - @Override - public ShareGroupDescribeResponse getErrorResponse(int throttleTimeMs, Throwable e) { - ShareGroupDescribeResponseData data = new ShareGroupDescribeResponseData() - .setThrottleTimeMs(throttleTimeMs); - // Set error for each group - short errorCode = Errors.forException(e).code(); - this.data.groupIds().forEach( - groupId -> data.groups().add( - new ShareGroupDescribeResponseData.DescribedGroup() - .setGroupId(groupId) - .setErrorCode(errorCode) - ) - ); - return new ShareGroupDescribeResponse(data); - } - - @Override - public ShareGroupDescribeRequestData data() { - return data; - } - - public static ShareGroupDescribeRequest parse(ByteBuffer buffer, short version) { - return new ShareGroupDescribeRequest( - new ShareGroupDescribeRequestData(new ByteBufferAccessor(buffer), version), - version - ); - } - - public static List getErrorDescribedGroupList( - List groupIds, - Errors error - ) { - return groupIds.stream() - .map(groupId -> new ShareGroupDescribeResponseData.DescribedGroup() - .setGroupId(groupId) - .setErrorCode(error.code()) - ).collect(Collectors.toList()); - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeResponse.java deleted file mode 100644 index 95dd371eedfa7..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeResponse.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT 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.ShareGroupDescribeResponseData; -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; -import java.util.HashMap; -import java.util.Map; - -/** - * Possible error codes. - * - * - {@link Errors#GROUP_AUTHORIZATION_FAILED} - * - {@link Errors#NOT_COORDINATOR} - * - {@link Errors#COORDINATOR_NOT_AVAILABLE} - * - {@link Errors#COORDINATOR_LOAD_IN_PROGRESS} - * - {@link Errors#INVALID_REQUEST} - * - {@link Errors#INVALID_GROUP_ID} - * - {@link Errors#GROUP_ID_NOT_FOUND} - */ -public class ShareGroupDescribeResponse extends AbstractResponse { - - private final ShareGroupDescribeResponseData data; - - public ShareGroupDescribeResponse(ShareGroupDescribeResponseData data) { - super(ApiKeys.SHARE_GROUP_DESCRIBE); - this.data = data; - } - - @Override - public ShareGroupDescribeResponseData data() { - return data; - } - - @Override - public Map errorCounts() { - HashMap counts = new HashMap<>(); - data.groups().forEach( - group -> updateErrorCounts(counts, Errors.forCode(group.errorCode())) - ); - return counts; - } - - @Override - public int throttleTimeMs() { - return data.throttleTimeMs(); - } - - @Override - public void maybeSetThrottleTimeMs(int throttleTimeMs) { - data.setThrottleTimeMs(throttleTimeMs); - } - - public static ShareGroupDescribeResponse parse(ByteBuffer buffer, short version) { - return new ShareGroupDescribeResponse( - new ShareGroupDescribeResponseData(new ByteBufferAccessor(buffer), version) - ); - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupHeartbeatRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupHeartbeatRequest.java deleted file mode 100644 index 7e112ef29dd14..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupHeartbeatRequest.java +++ /dev/null @@ -1,86 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.common.requests; - -import java.nio.ByteBuffer; - -import org.apache.kafka.common.message.ShareGroupHeartbeatRequestData; -import org.apache.kafka.common.message.ShareGroupHeartbeatResponseData; -import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; -import org.apache.kafka.common.protocol.Errors; - -public class ShareGroupHeartbeatRequest extends AbstractRequest { - /** - * A member epoch of -1 means that the member wants to leave the group. - */ - public static final int LEAVE_GROUP_MEMBER_EPOCH = -1; - - /** - * A member epoch of 0 means that the member wants to join the group. - */ - public static final int JOIN_GROUP_MEMBER_EPOCH = 0; - - public static class Builder extends AbstractRequest.Builder { - private final ShareGroupHeartbeatRequestData data; - - public Builder(ShareGroupHeartbeatRequestData data) { - this(data, true); - } - - public Builder(ShareGroupHeartbeatRequestData data, boolean enableUnstableLastVersion) { - super(ApiKeys.SHARE_GROUP_HEARTBEAT, enableUnstableLastVersion); - this.data = data; - } - - @Override - public ShareGroupHeartbeatRequest build(short version) { - return new ShareGroupHeartbeatRequest(data, version); - } - - @Override - public String toString() { - return data.toString(); - } - } - - private final ShareGroupHeartbeatRequestData data; - - public ShareGroupHeartbeatRequest(ShareGroupHeartbeatRequestData data, short version) { - super(ApiKeys.SHARE_GROUP_HEARTBEAT, version); - this.data = data; - } - - @Override - public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { - return new ShareGroupHeartbeatResponse( - new ShareGroupHeartbeatResponseData() - .setThrottleTimeMs(throttleTimeMs) - .setErrorCode(Errors.forException(e).code()) - ); - } - - @Override - public ShareGroupHeartbeatRequestData data() { - return data; - } - - public static ShareGroupHeartbeatRequest parse(ByteBuffer buffer, short version) { - return new ShareGroupHeartbeatRequest(new ShareGroupHeartbeatRequestData( - new ByteBufferAccessor(buffer), version), version); - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupHeartbeatResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupHeartbeatResponse.java deleted file mode 100644 index de05d44aebecb..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupHeartbeatResponse.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT 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.ShareGroupHeartbeatResponseData; -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; -import java.util.Collections; -import java.util.Map; - -/** - * Possible error codes. - * - * - {@link Errors#GROUP_AUTHORIZATION_FAILED} - * - {@link Errors#NOT_COORDINATOR} - * - {@link Errors#COORDINATOR_NOT_AVAILABLE} - * - {@link Errors#COORDINATOR_LOAD_IN_PROGRESS} - * - {@link Errors#INVALID_REQUEST} - * - {@link Errors#UNKNOWN_MEMBER_ID} - * - {@link Errors#GROUP_MAX_SIZE_REACHED} - */ -public class ShareGroupHeartbeatResponse extends AbstractResponse { - private final ShareGroupHeartbeatResponseData data; - - public ShareGroupHeartbeatResponse(ShareGroupHeartbeatResponseData data) { - super(ApiKeys.SHARE_GROUP_HEARTBEAT); - this.data = data; - } - - @Override - public ShareGroupHeartbeatResponseData data() { - return data; - } - - @Override - public Map errorCounts() { - return Collections.singletonMap(Errors.forCode(data.errorCode()), 1); - } - - @Override - public int throttleTimeMs() { - return data.throttleTimeMs(); - } - - @Override - public void maybeSetThrottleTimeMs(int throttleTimeMs) { - data.setThrottleTimeMs(throttleTimeMs); - } - - public static ShareGroupHeartbeatResponse parse(ByteBuffer buffer, short version) { - return new ShareGroupHeartbeatResponse(new ShareGroupHeartbeatResponseData( - new ByteBufferAccessor(buffer), version)); - } -} diff --git a/clients/src/main/resources/common/message/FindCoordinatorRequest.json b/clients/src/main/resources/common/message/FindCoordinatorRequest.json index 43e6fe5014b26..42b2f4c891ad5 100644 --- a/clients/src/main/resources/common/message/FindCoordinatorRequest.json +++ b/clients/src/main/resources/common/message/FindCoordinatorRequest.json @@ -27,9 +27,7 @@ // Version 4 adds support for batching via CoordinatorKeys (KIP-699) // // Version 5 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). - // - // Version 6 adds support for share groups (KIP-932). - "validVersions": "0-6", + "validVersions": "0-5", "deprecatedVersions": "0", "flexibleVersions": "3+", "fields": [ diff --git a/clients/src/main/resources/common/message/FindCoordinatorResponse.json b/clients/src/main/resources/common/message/FindCoordinatorResponse.json index be0479f908c96..860d655a252b2 100644 --- a/clients/src/main/resources/common/message/FindCoordinatorResponse.json +++ b/clients/src/main/resources/common/message/FindCoordinatorResponse.json @@ -26,9 +26,7 @@ // Version 4 adds support for batching via Coordinators (KIP-699) // // Version 5 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). - // - // Version 6 adds support for share groups (KIP-932). - "validVersions": "0-6", + "validVersions": "0-5", "flexibleVersions": "3+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, diff --git a/clients/src/main/resources/common/message/ListGroupsRequest.json b/clients/src/main/resources/common/message/ListGroupsRequest.json index a872165d516cf..32defaa203382 100644 --- a/clients/src/main/resources/common/message/ListGroupsRequest.json +++ b/clients/src/main/resources/common/message/ListGroupsRequest.json @@ -25,9 +25,7 @@ // Version 4 adds the StatesFilter field (KIP-518). // // Version 5 adds the TypesFilter field (KIP-848). - // - // Version 6 adds support for share groups (KIP-932). - "validVersions": "0-6", + "validVersions": "0-5", "flexibleVersions": "3+", "fields": [ { "name": "StatesFilter", "type": "[]string", "versions": "4+", diff --git a/clients/src/main/resources/common/message/ListGroupsResponse.json b/clients/src/main/resources/common/message/ListGroupsResponse.json index 77f1c89e34a38..fc4077c080f46 100644 --- a/clients/src/main/resources/common/message/ListGroupsResponse.json +++ b/clients/src/main/resources/common/message/ListGroupsResponse.json @@ -27,9 +27,7 @@ // Version 4 adds the GroupState field (KIP-518). // // Version 5 adds the GroupType field (KIP-848). - // - // Version 6 adds support for share groups (KIP-932). - "validVersions": "0-6", + "validVersions": "0-5", "flexibleVersions": "3+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, diff --git a/clients/src/main/resources/common/message/ShareAcknowledgeRequest.json b/clients/src/main/resources/common/message/ShareAcknowledgeRequest.json deleted file mode 100644 index db534cb4c1c13..0000000000000 --- a/clients/src/main/resources/common/message/ShareAcknowledgeRequest.json +++ /dev/null @@ -1,53 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one or more -// contributor license agreements. See the NOTICE file distributed with -// this work for additional information regarding copyright ownership. -// The ASF licenses this file to You under the Apache License, Version 2.0 -// (the "License"); you may not use this file except in compliance with -// the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT 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": 79, - "type": "request", - "listeners": ["broker"], - "name": "ShareAcknowledgeRequest", - "validVersions": "0", - "flexibleVersions": "0+", - // The ShareAcknowledgeRequest API is added as part of KIP-932 and is still under - // development. Hence, the API is not exposed by default by brokers unless - // explicitly enabled. - "latestVersionUnstable": true, - "fields": [ - { "name": "GroupId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", "entityType": "groupId", - "about": "The group identifier." }, - { "name": "MemberId", "type": "string", "versions": "0+", "nullableVersions": "0+", - "about": "The member ID." }, - { "name": "ShareSessionEpoch", "type": "int32", "versions": "0+", - "about": "The current share session epoch: 0 to open a share session; -1 to close it; otherwise increments for consecutive requests." }, - { "name": "Topics", "type": "[]AcknowledgeTopic", "versions": "0+", - "about": "The topics containing records to acknowledge.", "fields": [ - { "name": "TopicId", "type": "uuid", "versions": "0+", "about": "The unique topic ID."}, - { "name": "Partitions", "type": "[]AcknowledgePartition", "versions": "0+", - "about": "The partitions containing records to acknowledge.", "fields": [ - { "name": "PartitionIndex", "type": "int32", "versions": "0+", - "about": "The partition index." }, - { "name": "AcknowledgementBatches", "type": "[]AcknowledgementBatch", "versions": "0+", - "about": "Record batches to acknowledge.", "fields": [ - { "name": "FirstOffset", "type": "int64", "versions": "0+", - "about": "First offset of batch of records to acknowledge."}, - { "name": "LastOffset", "type": "int64", "versions": "0+", - "about": "Last offset (inclusive) of batch of records to acknowledge."}, - { "name": "AcknowledgeTypes", "type": "[]int8", "versions": "0+", - "about": "Array of acknowledge types - 0:Gap,1:Accept,2:Release,3:Reject."} - ]} - ]} - ]} - ] -} \ No newline at end of file diff --git a/clients/src/main/resources/common/message/ShareAcknowledgeResponse.json b/clients/src/main/resources/common/message/ShareAcknowledgeResponse.json deleted file mode 100644 index 638ca10c64b3b..0000000000000 --- a/clients/src/main/resources/common/message/ShareAcknowledgeResponse.json +++ /dev/null @@ -1,72 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one or more -// contributor license agreements. See the NOTICE file distributed with -// this work for additional information regarding copyright ownership. -// The ASF licenses this file to You under the Apache License, Version 2.0 -// (the "License"); you may not use this file except in compliance with -// the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT 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": 79, - "type": "response", - "name": "ShareAcknowledgeResponse", - "validVersions": "0", - "flexibleVersions": "0+", - // Supported errors: - // - GROUP_AUTHORIZATION_FAILED (version 0+) - // - TOPIC_AUTHORIZATION_FAILED (version 0+) - // - UNKNOWN_TOPIC_OR_PARTITION (version 0+) - // - SHARE_SESSION_NOT_FOUND (version 0+) - // - INVALID_SHARE_SESSION_EPOCH (version 0+) - // - NOT_LEADER_OR_FOLLOWER (version 0+) - // - UNKNOWN_TOPIC_ID (version 0+) - // - INVALID_RECORD_STATE (version 0+) - // - KAFKA_STORAGE_ERROR (version 0+) - // - INVALID_REQUEST (version 0+) - // - UNKNOWN_SERVER_ERROR (version 0+) - "fields": [ - { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "ignorable": true, - "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, - { "name": "ErrorCode", "type": "int16", "versions": "0+", "ignorable": true, - "about": "The top level response error code." }, - { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", - "about": "The top-level error message, or null if there was no error." }, - { "name": "Responses", "type": "[]ShareAcknowledgeTopicResponse", "versions": "0+", - "about": "The response topics.", "fields": [ - { "name": "TopicId", "type": "uuid", "versions": "0+", "ignorable": true, "about": "The unique topic ID."}, - { "name": "Partitions", "type": "[]PartitionData", "versions": "0+", - "about": "The topic partitions.", "fields": [ - { "name": "PartitionIndex", "type": "int32", "versions": "0+", - "about": "The partition index." }, - { "name": "ErrorCode", "type": "int16", "versions": "0+", - "about": "The error code, or 0 if there was no error." }, - { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", - "about": "The error message, or null if there was no error." }, - { "name": "CurrentLeader", "type": "LeaderIdAndEpoch", "versions": "0+", "fields": [ - { "name": "LeaderId", "type": "int32", "versions": "0+", - "about": "The ID of the current leader or -1 if the leader is unknown." }, - { "name": "LeaderEpoch", "type": "int32", "versions": "0+", - "about": "The latest known leader epoch." } - ]} - ]} - ]}, - { "name": "NodeEndpoints", "type": "[]NodeEndpoint", "versions": "0+", - "about": "Endpoints for all current leaders enumerated in PartitionData with error NOT_LEADER_OR_FOLLOWER.", "fields": [ - { "name": "NodeId", "type": "int32", "versions": "0+", - "mapKey": true, "entityType": "brokerId", "about": "The ID of the associated node." }, - { "name": "Host", "type": "string", "versions": "0+", - "about": "The node's hostname." }, - { "name": "Port", "type": "int32", "versions": "0+", - "about": "The node's port." }, - { "name": "Rack", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", - "about": "The rack of the node, or null if it has not been assigned to a rack." } - ]} - ] -} \ No newline at end of file diff --git a/clients/src/main/resources/common/message/ShareFetchRequest.json b/clients/src/main/resources/common/message/ShareFetchRequest.json deleted file mode 100644 index d0b59dcb26a80..0000000000000 --- a/clients/src/main/resources/common/message/ShareFetchRequest.json +++ /dev/null @@ -1,67 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one or more -// contributor license agreements. See the NOTICE file distributed with -// this work for additional information regarding copyright ownership. -// The ASF licenses this file to You under the Apache License, Version 2.0 -// (the "License"); you may not use this file except in compliance with -// the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT 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": 78, - "type": "request", - "listeners": ["broker"], - "name": "ShareFetchRequest", - "validVersions": "0", - "flexibleVersions": "0+", - // The ShareFetchRequest API is added as part of KIP-932 and is still under - // development. Hence, the API is not exposed by default by brokers unless - // explicitly enabled. - "latestVersionUnstable": true, - "fields": [ - { "name": "GroupId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", "entityType": "groupId", - "about": "The group identifier." }, - { "name": "MemberId", "type": "string", "versions": "0+", "nullableVersions": "0+", - "about": "The member ID." }, - { "name": "ShareSessionEpoch", "type": "int32", "versions": "0+", - "about": "The current share session epoch: 0 to open a share session; -1 to close it; otherwise increments for consecutive requests." }, - { "name": "MaxWaitMs", "type": "int32", "versions": "0+", - "about": "The maximum time in milliseconds to wait for the response." }, - { "name": "MinBytes", "type": "int32", "versions": "0+", - "about": "The minimum bytes to accumulate in the response." }, - { "name": "MaxBytes", "type": "int32", "versions": "0+", "default": "0x7fffffff", "ignorable": true, - "about": "The maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored." }, - { "name": "Topics", "type": "[]FetchTopic", "versions": "0+", - "about": "The topics to fetch.", "fields": [ - { "name": "TopicId", "type": "uuid", "versions": "0+", "ignorable": true, "about": "The unique topic ID."}, - { "name": "Partitions", "type": "[]FetchPartition", "versions": "0+", - "about": "The partitions to fetch.", "fields": [ - { "name": "PartitionIndex", "type": "int32", "versions": "0+", - "about": "The partition index." }, - { "name": "PartitionMaxBytes", "type": "int32", "versions": "0+", - "about": "The maximum bytes to fetch from this partition. 0 when only acknowledgement with no fetching is required. See KIP-74 for cases where this limit may not be honored." }, - { "name": "AcknowledgementBatches", "type": "[]AcknowledgementBatch", "versions": "0+", - "about": "Record batches to acknowledge.", "fields": [ - { "name": "FirstOffset", "type": "int64", "versions": "0+", - "about": "First offset of batch of records to acknowledge."}, - { "name": "LastOffset", "type": "int64", "versions": "0+", - "about": "Last offset (inclusive) of batch of records to acknowledge."}, - { "name": "AcknowledgeTypes", "type": "[]int8", "versions": "0+", - "about": "Array of acknowledge types - 0:Gap,1:Accept,2:Release,3:Reject."} - ]} - ]} - ]}, - { "name": "ForgottenTopicsData", "type": "[]ForgottenTopic", "versions": "0+", "ignorable": false, - "about": "The partitions to remove from this share session.", "fields": [ - { "name": "TopicId", "type": "uuid", "versions": "0+", "ignorable": true, "about": "The unique topic ID."}, - { "name": "Partitions", "type": "[]int32", "versions": "0+", - "about": "The partitions indexes to forget." } - ]} - ] -} \ No newline at end of file diff --git a/clients/src/main/resources/common/message/ShareFetchResponse.json b/clients/src/main/resources/common/message/ShareFetchResponse.json deleted file mode 100644 index 5338e1208a7bc..0000000000000 --- a/clients/src/main/resources/common/message/ShareFetchResponse.json +++ /dev/null @@ -1,83 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one or more -// contributor license agreements. See the NOTICE file distributed with -// this work for additional information regarding copyright ownership. -// The ASF licenses this file to You under the Apache License, Version 2.0 -// (the "License"); you may not use this file except in compliance with -// the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT 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": 78, - "type": "response", - "name": "ShareFetchResponse", - "validVersions": "0", - "flexibleVersions": "0+", - // Supported errors for ErrorCode and AcknowledgeErrorCode: - // - GROUP_AUTHORIZATION_FAILED (version 0+) - // - TOPIC_AUTHORIZATION_FAILED (version 0+) - // - SHARE_SESSION_NOT_FOUND (version 0+) - // - INVALID_SHARE_SESSION_EPOCH (version 0+) - // - UNKNOWN_TOPIC_OR_PARTITION (version 0+) - // - NOT_LEADER_OR_FOLLOWER (version 0+) - // - UNKNOWN_TOPIC_ID (version 0+) - // - INVALID_RECORD_STATE (version 0+) - only for AcknowledgeErrorCode - // - KAFKA_STORAGE_ERROR (version 0+) - // - CORRUPT_MESSAGE (version 0+) - // - INVALID_REQUEST (version 0+) - // - UNKNOWN_SERVER_ERROR (version 0+) - "fields": [ - { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "ignorable": true, - "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, - { "name": "ErrorCode", "type": "int16", "versions": "0+", "ignorable": true, - "about": "The top-level response error code." }, - { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", - "about": "The top-level error message, or null if there was no error." }, - { "name": "Responses", "type": "[]ShareFetchableTopicResponse", "versions": "0+", - "about": "The response topics.", "fields": [ - { "name": "TopicId", "type": "uuid", "versions": "0+", "ignorable": true, "about": "The unique topic ID."}, - { "name": "Partitions", "type": "[]PartitionData", "versions": "0+", - "about": "The topic partitions.", "fields": [ - { "name": "PartitionIndex", "type": "int32", "versions": "0+", - "about": "The partition index." }, - { "name": "ErrorCode", "type": "int16", "versions": "0+", - "about": "The fetch error code, or 0 if there was no fetch error." }, - { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", - "about": "The fetch error message, or null if there was no fetch error." }, - { "name": "AcknowledgeErrorCode", "type": "int16", "versions": "0+", - "about": "The acknowledge error code, or 0 if there was no acknowledge error." }, - { "name": "AcknowledgeErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", - "about": "The acknowledge error message, or null if there was no acknowledge error." }, - { "name": "CurrentLeader", "type": "LeaderIdAndEpoch", "versions": "0+", "fields": [ - { "name": "LeaderId", "type": "int32", "versions": "0+", - "about": "The ID of the current leader or -1 if the leader is unknown." }, - { "name": "LeaderEpoch", "type": "int32", "versions": "0+", - "about": "The latest known leader epoch." } - ]}, - { "name": "Records", "type": "records", "versions": "0+", "nullableVersions": "0+", "about": "The record data."}, - { "name": "AcquiredRecords", "type": "[]AcquiredRecords", "versions": "0+", "about": "The acquired records.", "fields": [ - {"name": "FirstOffset", "type": "int64", "versions": "0+", "about": "The earliest offset in this batch of acquired records."}, - {"name": "LastOffset", "type": "int64", "versions": "0+", "about": "The last offset of this batch of acquired records."}, - {"name": "DeliveryCount", "type": "int16", "versions": "0+", "about": "The delivery count of this batch of acquired records."} - ]} - ]} - ]}, - { "name": "NodeEndpoints", "type": "[]NodeEndpoint", "versions": "0+", - "about": "Endpoints for all current leaders enumerated in PartitionData with error NOT_LEADER_OR_FOLLOWER.", "fields": [ - { "name": "NodeId", "type": "int32", "versions": "0+", - "mapKey": true, "entityType": "brokerId", "about": "The ID of the associated node." }, - { "name": "Host", "type": "string", "versions": "0+", - "about": "The node's hostname." }, - { "name": "Port", "type": "int32", "versions": "0+", - "about": "The node's port." }, - { "name": "Rack", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", - "about": "The rack of the node, or null if it has not been assigned to a rack." } - ]} - ] -} \ No newline at end of file diff --git a/clients/src/main/resources/common/message/ShareGroupDescribeRequest.json b/clients/src/main/resources/common/message/ShareGroupDescribeRequest.json deleted file mode 100644 index c95790c9b198f..0000000000000 --- a/clients/src/main/resources/common/message/ShareGroupDescribeRequest.json +++ /dev/null @@ -1,33 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one or more -// contributor license agreements. See the NOTICE file distributed with -// this work for additional information regarding copyright ownership. -// The ASF licenses this file to You under the Apache License, Version 2.0 -// (the "License"); you may not use this file except in compliance with -// the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT 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": 77, - "type": "request", - "listeners": ["broker"], - "name": "ShareGroupDescribeRequest", - "validVersions": "0", - "flexibleVersions": "0+", - // The ShareGroupDescribeRequest API is added as part of KIP-932 and is still under - // development. Hence, the API is not exposed by default by brokers unless - // explicitly enabled. - "latestVersionUnstable": true, - "fields": [ - { "name": "GroupIds", "type": "[]string", "versions": "0+", "entityType": "groupId", - "about": "The ids of the groups to describe" }, - { "name": "IncludeAuthorizedOperations", "type": "bool", "versions": "0+", - "about": "Whether to include authorized operations." } - ] -} \ No newline at end of file diff --git a/clients/src/main/resources/common/message/ShareGroupDescribeResponse.json b/clients/src/main/resources/common/message/ShareGroupDescribeResponse.json deleted file mode 100644 index c093b788bfc2f..0000000000000 --- a/clients/src/main/resources/common/message/ShareGroupDescribeResponse.json +++ /dev/null @@ -1,87 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one or more -// contributor license agreements. See the NOTICE file distributed with -// this work for additional information regarding copyright ownership. -// The ASF licenses this file to You under the Apache License, Version 2.0 -// (the "License"); you may not use this file except in compliance with -// the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT 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": 77, - "type": "response", - "name": "ShareGroupDescribeResponse", - "validVersions": "0", - "flexibleVersions": "0+", - // Supported errors: - // - GROUP_AUTHORIZATION_FAILED (version 0+) - // - NOT_COORDINATOR (version 0+) - // - COORDINATOR_NOT_AVAILABLE (version 0+) - // - COORDINATOR_LOAD_IN_PROGRESS (version 0+) - // - INVALID_REQUEST (version 0+) - // - INVALID_GROUP_ID (version 0+) - // - GROUP_ID_NOT_FOUND (version 0+) - "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": "Groups", "type": "[]DescribedGroup", "versions": "0+", - "about": "Each described group.", - "fields": [ - { "name": "ErrorCode", "type": "int16", "versions": "0+", - "about": "The describe error, 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": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId", - "about": "The group ID string." }, - { "name": "GroupState", "type": "string", "versions": "0+", - "about": "The group state string, or the empty string." }, - { "name": "GroupEpoch", "type": "int32", "versions": "0+", - "about": "The group epoch." }, - { "name": "AssignmentEpoch", "type": "int32", "versions": "0+", - "about": "The assignment epoch." }, - { "name": "AssignorName", "type": "string", "versions": "0+", - "about": "The selected assignor." }, - { "name": "Members", "type": "[]Member", "versions": "0+", - "about": "The members.", - "fields": [ - { "name": "MemberId", "type": "string", "versions": "0+", - "about": "The member ID." }, - { "name": "RackId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", - "about": "The member rack ID." }, - { "name": "MemberEpoch", "type": "int32", "versions": "0+", - "about": "The current member epoch." }, - { "name": "ClientId", "type": "string", "versions": "0+", - "about": "The client ID." }, - { "name": "ClientHost", "type": "string", "versions": "0+", - "about": "The client host." }, - { "name": "SubscribedTopicNames", "type": "[]string", "versions": "0+", "entityType": "topicName", - "about": "The subscribed topic names." }, - { "name": "Assignment", "type": "Assignment", "versions": "0+", - "about": "The current assignment." } - ]}, - { "name": "AuthorizedOperations", "type": "int32", "versions": "0+", "default": "-2147483648", - "about": "32-bit bitfield to represent authorized operations for this group." } - ] - } - ], - "commonStructs": [ - { "name": "TopicPartitions", "versions": "0+", "fields": [ - { "name": "TopicId", "type": "uuid", "versions": "0+", - "about": "The topic ID." }, - { "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName", - "about": "The topic name." }, - { "name": "Partitions", "type": "[]int32", "versions": "0+", - "about": "The partitions." } - ]}, - { "name": "Assignment", "versions": "0+", "fields": [ - { "name": "TopicPartitions", "type": "[]TopicPartitions", "versions": "0+", - "about": "The assigned topic-partitions to the member." } - ]} - ] -} \ No newline at end of file diff --git a/clients/src/main/resources/common/message/ShareGroupHeartbeatRequest.json b/clients/src/main/resources/common/message/ShareGroupHeartbeatRequest.json deleted file mode 100644 index 7d28c116454d3..0000000000000 --- a/clients/src/main/resources/common/message/ShareGroupHeartbeatRequest.json +++ /dev/null @@ -1,39 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one or more -// contributor license agreements. See the NOTICE file distributed with -// this work for additional information regarding copyright ownership. -// The ASF licenses this file to You under the Apache License, Version 2.0 -// (the "License"); you may not use this file except in compliance with -// the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT 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": 76, - "type": "request", - "listeners": ["broker"], - "name": "ShareGroupHeartbeatRequest", - "validVersions": "0", - "flexibleVersions": "0+", - // The ShareGroupHeartbeatRequest API is added as part of KIP-932 and is still under - // development. Hence, the API is not exposed by default by brokers unless - // explicitly enabled. - "latestVersionUnstable": true, - "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." }, - { "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": "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." } - ] -} \ No newline at end of file diff --git a/clients/src/main/resources/common/message/ShareGroupHeartbeatResponse.json b/clients/src/main/resources/common/message/ShareGroupHeartbeatResponse.json deleted file mode 100644 index e692839f29bf9..0000000000000 --- a/clients/src/main/resources/common/message/ShareGroupHeartbeatResponse.json +++ /dev/null @@ -1,57 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one or more -// contributor license agreements. See the NOTICE file distributed with -// this work for additional information regarding copyright ownership. -// The ASF licenses this file to You under the Apache License, Version 2.0 -// (the "License"); you may not use this file except in compliance with -// the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT 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": 76, - "type": "response", - "name": "ShareGroupHeartbeatResponse", - "validVersions": "0", - "flexibleVersions": "0+", - // Supported errors: - // - GROUP_AUTHORIZATION_FAILED (version 0+) - // - NOT_COORDINATOR (version 0+) - // - COORDINATOR_NOT_AVAILABLE (version 0+) - // - COORDINATOR_LOAD_IN_PROGRESS (version 0+) - // - INVALID_REQUEST (version 0+) - // - UNKNOWN_MEMBER_ID (version 0+) - // - GROUP_MAX_SIZE_REACHED (version 0+) - "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": "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": "TopicPartitions", "type": "[]TopicPartitions", "versions": "0+", - "about": "The partitions assigned to the member." } - ]} - ], - "commonStructs": [ - { "name": "TopicPartitions", "versions": "0+", "fields": [ - { "name": "TopicId", "type": "uuid", "versions": "0+", - "about": "The topic ID." }, - { "name": "Partitions", "type": "[]int32", "versions": "0+", - "about": "The partitions." } - ]} - ] -} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 82487bd418429..512a7cea76681 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.ElectionType; import org.apache.kafka.common.IsolationLevel; import org.apache.kafka.common.Node; -import org.apache.kafka.common.ShareGroupState; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.Uuid; @@ -211,14 +210,6 @@ import org.apache.kafka.common.message.SaslAuthenticateResponseData; import org.apache.kafka.common.message.SaslHandshakeRequestData; import org.apache.kafka.common.message.SaslHandshakeResponseData; -import org.apache.kafka.common.message.ShareAcknowledgeRequestData; -import org.apache.kafka.common.message.ShareAcknowledgeResponseData; -import org.apache.kafka.common.message.ShareFetchRequestData; -import org.apache.kafka.common.message.ShareFetchResponseData; -import org.apache.kafka.common.message.ShareGroupDescribeRequestData; -import org.apache.kafka.common.message.ShareGroupDescribeResponseData; -import org.apache.kafka.common.message.ShareGroupHeartbeatRequestData; -import org.apache.kafka.common.message.ShareGroupHeartbeatResponseData; import org.apache.kafka.common.message.StopReplicaRequestData.StopReplicaPartitionState; import org.apache.kafka.common.message.StopReplicaRequestData.StopReplicaTopicState; import org.apache.kafka.common.message.StopReplicaResponseData; @@ -1010,10 +1001,6 @@ public void testErrorCountsIncludesNone() { assertEquals(1, createTxnOffsetCommitResponse().errorCounts().get(Errors.NONE)); assertEquals(1, createUpdateMetadataResponse().errorCounts().get(Errors.NONE)); assertEquals(1, createWriteTxnMarkersResponse().errorCounts().get(Errors.NONE)); - assertEquals(1, createShareGroupHeartbeatResponse().errorCounts().get(Errors.NONE)); - assertEquals(1, createShareGroupDescribeResponse().errorCounts().get(Errors.NONE)); - assertEquals(2, createShareFetchResponse().errorCounts().get(Errors.NONE)); - assertEquals(2, createShareAcknowledgeResponse().errorCounts().get(Errors.NONE)); } private AbstractRequest getRequest(ApiKeys apikey, short version) { @@ -1094,10 +1081,6 @@ private AbstractRequest getRequest(ApiKeys apikey, short version) { case ASSIGN_REPLICAS_TO_DIRS: return createAssignReplicasToDirsRequest(version); case LIST_CLIENT_METRICS_RESOURCES: return createListClientMetricsResourcesRequest(version); case DESCRIBE_TOPIC_PARTITIONS: return createDescribeTopicPartitionsRequest(version); - case SHARE_GROUP_HEARTBEAT: return createShareGroupHeartbeatRequest(version); - case SHARE_GROUP_DESCRIBE: return createShareGroupDescribeRequest(version); - case SHARE_FETCH: return createShareFetchRequest(version); - case SHARE_ACKNOWLEDGE: return createShareAcknowledgeRequest(version); default: throw new IllegalArgumentException("Unknown API key " + apikey); } } @@ -1180,10 +1163,6 @@ private AbstractResponse getResponse(ApiKeys apikey, short version) { case ASSIGN_REPLICAS_TO_DIRS: return createAssignReplicasToDirsResponse(); case LIST_CLIENT_METRICS_RESOURCES: return createListClientMetricsResourcesResponse(); case DESCRIBE_TOPIC_PARTITIONS: return createDescribeTopicPartitionsResponse(); - case SHARE_GROUP_HEARTBEAT: return createShareGroupHeartbeatResponse(); - case SHARE_GROUP_DESCRIBE: return createShareGroupDescribeResponse(); - case SHARE_FETCH: return createShareFetchResponse(); - case SHARE_ACKNOWLEDGE: return createShareAcknowledgeResponse(); default: throw new IllegalArgumentException("Unknown API key " + apikey); } } @@ -1351,114 +1330,6 @@ private ConsumerGroupHeartbeatResponse createConsumerGroupHeartbeatResponse() { return new ConsumerGroupHeartbeatResponse(data); } - private ShareGroupHeartbeatRequest createShareGroupHeartbeatRequest(short version) { - ShareGroupHeartbeatRequestData data = new ShareGroupHeartbeatRequestData() - .setGroupId("group") - .setMemberId("memberid") - .setMemberEpoch(10) - .setRackId("rackid") - .setSubscribedTopicNames(Arrays.asList("foo", "bar")); - return new ShareGroupHeartbeatRequest.Builder(data).build(version); - } - - private ShareGroupHeartbeatResponse createShareGroupHeartbeatResponse() { - ShareGroupHeartbeatResponseData data = new ShareGroupHeartbeatResponseData() - .setErrorCode(Errors.NONE.code()) - .setThrottleTimeMs(1000) - .setMemberId("memberid") - .setMemberEpoch(11) - .setAssignment(new ShareGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Arrays.asList( - new ShareGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(Uuid.randomUuid()) - .setPartitions(Arrays.asList(0, 1, 2)), - new ShareGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(Uuid.randomUuid()) - .setPartitions(Arrays.asList(3, 4, 5)) - )) - ); - return new ShareGroupHeartbeatResponse(data); - } - - private ShareGroupDescribeRequest createShareGroupDescribeRequest(short version) { - ShareGroupDescribeRequestData data = new ShareGroupDescribeRequestData() - .setGroupIds(Collections.singletonList("group")) - .setIncludeAuthorizedOperations(false); - return new ShareGroupDescribeRequest.Builder(data).build(version); - } - - private ShareGroupDescribeResponse createShareGroupDescribeResponse() { - ShareGroupDescribeResponseData data = new ShareGroupDescribeResponseData() - .setGroups(Collections.singletonList( - new ShareGroupDescribeResponseData.DescribedGroup() - .setGroupId("group") - .setErrorCode((short) 0) - .setErrorMessage(Errors.forCode((short) 0).message()) - .setGroupState(ShareGroupState.EMPTY.toString()) - .setMembers(new ArrayList<>(0)) - )) - .setThrottleTimeMs(1000); - return new ShareGroupDescribeResponse(data); - } - - private ShareFetchRequest createShareFetchRequest(short version) { - ShareFetchRequestData data = new ShareFetchRequestData() - .setGroupId("group") - .setMemberId(Uuid.randomUuid().toString()) - .setTopics(singletonList(new ShareFetchRequestData.FetchTopic() - .setTopicId(Uuid.randomUuid()) - .setPartitions(singletonList(new ShareFetchRequestData.FetchPartition() - .setPartitionIndex(0))))); - return new ShareFetchRequest.Builder(data).build(version); - } - - private ShareFetchResponse createShareFetchResponse() { - ShareFetchResponseData data = new ShareFetchResponseData(); - MemoryRecords records = MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("blah".getBytes())); - ShareFetchResponseData.PartitionData partition = new ShareFetchResponseData.PartitionData() - .setPartitionIndex(0) - .setErrorCode(Errors.NONE.code()) - .setRecords(records) - .setAcquiredRecords(singletonList(new ShareFetchResponseData.AcquiredRecords() - .setFirstOffset(0) - .setLastOffset(0) - .setDeliveryCount((short) 1))); - ShareFetchResponseData.ShareFetchableTopicResponse response = new ShareFetchResponseData.ShareFetchableTopicResponse() - .setTopicId(Uuid.randomUuid()) - .setPartitions(singletonList(partition)); - - data.setResponses(singletonList(response)); - data.setThrottleTimeMs(345); - data.setErrorCode(Errors.NONE.code()); - return new ShareFetchResponse(data); - } - - private ShareAcknowledgeRequest createShareAcknowledgeRequest(short version) { - ShareAcknowledgeRequestData data = new ShareAcknowledgeRequestData() - .setMemberId(Uuid.randomUuid().toString()) - .setTopics(singletonList(new ShareAcknowledgeRequestData.AcknowledgeTopic() - .setTopicId(Uuid.randomUuid()) - .setPartitions(singletonList(new ShareAcknowledgeRequestData.AcknowledgePartition() - .setPartitionIndex(0) - .setAcknowledgementBatches(singletonList(new ShareAcknowledgeRequestData.AcknowledgementBatch() - .setFirstOffset(0) - .setLastOffset(0) - .setAcknowledgeTypes(Collections.singletonList((byte) 0)))))))); - return new ShareAcknowledgeRequest.Builder(data).build(version); - } - - private ShareAcknowledgeResponse createShareAcknowledgeResponse() { - ShareAcknowledgeResponseData data = new ShareAcknowledgeResponseData(); - data.setResponses(singletonList(new ShareAcknowledgeResponseData.ShareAcknowledgeTopicResponse() - .setTopicId(Uuid.randomUuid()) - .setPartitions(singletonList(new ShareAcknowledgeResponseData.PartitionData() - .setPartitionIndex(0) - .setErrorCode(Errors.NONE.code()))))); - data.setThrottleTimeMs(345); - data.setErrorCode(Errors.NONE.code()); - return new ShareAcknowledgeResponse(data); - } - private ControllerRegistrationRequest createControllerRegistrationRequest(short version) { ControllerRegistrationRequestData data = new ControllerRegistrationRequestData(). setControllerId(3). diff --git a/core/src/main/scala/kafka/network/RequestConvertToJson.scala b/core/src/main/scala/kafka/network/RequestConvertToJson.scala index 0900b94ef9f4f..54986f52c85a3 100644 --- a/core/src/main/scala/kafka/network/RequestConvertToJson.scala +++ b/core/src/main/scala/kafka/network/RequestConvertToJson.scala @@ -95,10 +95,6 @@ object RequestConvertToJson { case req: RenewDelegationTokenRequest => RenewDelegationTokenRequestDataJsonConverter.write(req.data, request.version) case req: SaslAuthenticateRequest => SaslAuthenticateRequestDataJsonConverter.write(req.data, request.version) case req: SaslHandshakeRequest => SaslHandshakeRequestDataJsonConverter.write(req.data, request.version) - case req: ShareAcknowledgeRequest => ShareAcknowledgeRequestDataJsonConverter.write(req.data, request.version) - case req: ShareFetchRequest => ShareFetchRequestDataJsonConverter.write(req.data, request.version) - case req: ShareGroupDescribeRequest => ShareGroupDescribeRequestDataJsonConverter.write(req.data, request.version) - case req: ShareGroupHeartbeatRequest => ShareGroupHeartbeatRequestDataJsonConverter.write(req.data, request.version) case req: StopReplicaRequest => StopReplicaRequestDataJsonConverter.write(req.data, request.version) case req: SyncGroupRequest => SyncGroupRequestDataJsonConverter.write(req.data, request.version) case req: TxnOffsetCommitRequest => TxnOffsetCommitRequestDataJsonConverter.write(req.data, request.version) @@ -182,10 +178,6 @@ object RequestConvertToJson { case res: RenewDelegationTokenResponse => RenewDelegationTokenResponseDataJsonConverter.write(res.data, version) case res: SaslAuthenticateResponse => SaslAuthenticateResponseDataJsonConverter.write(res.data, version) case res: SaslHandshakeResponse => SaslHandshakeResponseDataJsonConverter.write(res.data, version) - case res: ShareAcknowledgeResponse => ShareAcknowledgeResponseDataJsonConverter.write(res.data, version) - case res: ShareFetchResponse => ShareFetchResponseDataJsonConverter.write(res.data, version) - case res: ShareGroupDescribeResponse => ShareGroupDescribeResponseDataJsonConverter.write(res.data, version) - case res: ShareGroupHeartbeatResponse => ShareGroupHeartbeatResponseDataJsonConverter.write(res.data, version) case res: StopReplicaResponse => StopReplicaResponseDataJsonConverter.write(res.data, version) case res: SyncGroupResponse => SyncGroupResponseDataJsonConverter.write(res.data, version) case res: TxnOffsetCommitResponse => TxnOffsetCommitResponseDataJsonConverter.write(res.data, version) diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala index 3a69669d349e5..a7415b5d50a2e 100644 --- a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala @@ -95,7 +95,7 @@ class ApiVersionsRequestTest(cluster: ClusterInstance) extends AbstractApiVersio @ClusterTemplate("testApiVersionsRequestIncludesUnreleasedApisTemplate") @ClusterTest(types = Array(Type.KRAFT, Type.CO_KRAFT), serverProperties = Array( - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true"), + new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "false"), new ClusterConfigProperty(key = "unstable.feature.versions.enable", value = "true"), )) def testApiVersionsRequestIncludesUnreleasedApis(): Unit = { diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 97efd9bcf4cc0..456d075f91655 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -704,10 +704,10 @@ class RequestQuotaTest extends BaseRequestTest { new ConsumerGroupDescribeRequest.Builder(new ConsumerGroupDescribeRequestData(), true) case ApiKeys.GET_TELEMETRY_SUBSCRIPTIONS => - new GetTelemetrySubscriptionsRequest.Builder(new GetTelemetrySubscriptionsRequestData()) + new GetTelemetrySubscriptionsRequest.Builder(new GetTelemetrySubscriptionsRequestData(), true) case ApiKeys.PUSH_TELEMETRY => - new PushTelemetryRequest.Builder(new PushTelemetryRequestData()) + new PushTelemetryRequest.Builder(new PushTelemetryRequestData(), true) case ApiKeys.ASSIGN_REPLICAS_TO_DIRS => new AssignReplicasToDirsRequest.Builder(new AssignReplicasToDirsRequestData()) @@ -718,18 +718,6 @@ class RequestQuotaTest extends BaseRequestTest { case ApiKeys.DESCRIBE_TOPIC_PARTITIONS => new DescribeTopicPartitionsRequest.Builder(new DescribeTopicPartitionsRequestData()) - case ApiKeys.SHARE_GROUP_HEARTBEAT => - new ShareGroupHeartbeatRequest.Builder(new ShareGroupHeartbeatRequestData(), true) - - case ApiKeys.SHARE_GROUP_DESCRIBE => - new ShareGroupDescribeRequest.Builder(new ShareGroupDescribeRequestData(), true) - - case ApiKeys.SHARE_FETCH => - new ShareFetchRequest.Builder(new ShareFetchRequestData(), true) - - case ApiKeys.SHARE_ACKNOWLEDGE => - new ShareAcknowledgeRequest.Builder(new ShareAcknowledgeRequestData(), true) - case _ => throw new IllegalArgumentException("Unsupported API key " + apiKey) } diff --git a/docs/security.html b/docs/security.html index e3495f4b5188b..7eb0c2cb346f8 100644 --- a/docs/security.html +++ b/docs/security.html @@ -2267,42 +2267,6 @@

groupInfo; + private RequestManagers requestManagers; + private LogContext logContext; + private ConsumerConfig config; + private OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; @BeforeEach public void setUp() { - setUp(ConsumerTestBuilder.createDefaultGroupInformation()); - } - - private void setUp(Optional groupInfo) { - testBuilder = new ConsumerTestBuilder(groupInfo, true, false); - time = testBuilder.time; - coordinatorRequestManager = testBuilder.coordinatorRequestManager.orElseThrow(IllegalStateException::new); - heartbeatRequestManager = testBuilder.heartbeatRequestManager.orElseThrow(IllegalStateException::new); - heartbeatRequestState = testBuilder.heartbeatRequestState.orElseThrow(IllegalStateException::new); - heartbeatState = testBuilder.heartbeatState.orElseThrow(IllegalStateException::new); - backgroundEventHandler = testBuilder.backgroundEventHandler; - subscriptions = testBuilder.subscriptions; - membershipManager = testBuilder.membershipManager.orElseThrow(IllegalStateException::new); - metadata = testBuilder.metadata; - metrics = new Metrics(time); + setUp(createDefaultGroupInformation()); + } + + private void setUp(Optional groupInfo) { + this.time = new MockTime(); + this.metrics = new Metrics(time); + this.logContext = new LogContext(); + this.groupInfo = groupInfo; + this.pollTimer = mock(Timer.class); + this.coordinatorRequestManager = mock(CoordinatorRequestManager.class); + this.heartbeatRequestState = mock(HeartbeatRequestState.class); + this.heartbeatState = mock(HeartbeatState.class); + this.backgroundEventHandler = mock(BackgroundEventHandler.class); + this.subscriptions = mock(SubscriptionState.class); + this.membershipManager = mock(MembershipManagerImpl.class); + this.metadata = mock(ConsumerMetadata.class); + this.config = mock(ConsumerConfig.class); + this.offsetCommitCallbackInvoker = mock(OffsetCommitCallbackInvoker.class); + + this.heartbeatRequestManager = new HeartbeatRequestManager( + logContext, + time, + config, + coordinatorRequestManager, + subscriptions, + membershipManager, + backgroundEventHandler, + metrics + ); + + this.requestManagers = new RequestManagers( + logContext, + mock(OffsetsRequestManager.class), + mock(TopicMetadataRequestManager.class), + mock(FetchRequestManager.class), + Optional.empty(), Optional.empty(), + Optional.of(heartbeatRequestManager), + Optional.empty() + ); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); + Map> map = new HashMap<>(); + LocalAssignment local = new LocalAssignment(0, map); + when(membershipManager.currentAssignment()).thenReturn(local); } private void resetWithZeroHeartbeatInterval(Optional groupInstanceId) { cleanup(); - ConsumerTestBuilder.GroupInformation gi = new ConsumerTestBuilder.GroupInformation( + GroupInformation gi = new GroupInformation( DEFAULT_GROUP_ID, groupInstanceId, 0, @@ -147,33 +181,41 @@ private void resetWithZeroHeartbeatInterval(Optional groupInstanceId) { @AfterEach public void cleanup() { - if (testBuilder != null) { - testBuilder.close(); + if (heartbeatRequestManager != null) { + closeQuietly(requestManagers, RequestManagers.class.getSimpleName()); } } @Test public void testHeartbeatOnStartup() { + when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size()); resetWithZeroHeartbeatInterval(Optional.empty()); - mockStableMember(); + when(membershipManager.state()).thenReturn(MemberState.STABLE); + mockStableMember(membershipManager); assertEquals(0, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size()); // Ensure we do not resend the request without the first request being completed + when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); NetworkClientDelegate.PollResult result2 = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result2.unsentRequests.size()); } + // TODO @Test public void testSuccessfulHeartbeatTiming() { - mockStableMember(); + when(membershipManager.state()).thenReturn(MemberState.STABLE); + mockStableMember(membershipManager); + + when(membershipManager.isLeavingGroup()).thenReturn(true); + when(heartbeatRequestState.canSendRequest(time.milliseconds())).thenReturn(false); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size(), - "No heartbeat should be sent while interval has not expired"); + "No heartbeat should be sent while interval has not expired"); assertEquals(heartbeatRequestState.timeToNextHeartbeatMs(time.milliseconds()), result.timeUntilNextPollMs); assertNextHeartbeatTiming(DEFAULT_HEARTBEAT_INTERVAL_MS); @@ -181,25 +223,25 @@ public void testSuccessfulHeartbeatTiming() { assertEquals(1, result.unsentRequests.size(), "A heartbeat should be sent when interval expires"); NetworkClientDelegate.UnsentRequest inflightReq = result.unsentRequests.get(0); assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, - heartbeatRequestState.timeToNextHeartbeatMs(time.milliseconds()), - "Heartbeat timer was not reset to the interval when the heartbeat request was sent."); + heartbeatRequestState.timeToNextHeartbeatMs(time.milliseconds()), + "Heartbeat timer was not reset to the interval when the heartbeat request was sent."); long partOfInterval = DEFAULT_HEARTBEAT_INTERVAL_MS / 3; time.sleep(partOfInterval); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size(), - "No heartbeat should be sent while only part of the interval has passed"); + "No heartbeat should be sent while only part of the interval has passed"); assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS - partOfInterval, - heartbeatRequestState.timeToNextHeartbeatMs(time.milliseconds()), - "Time to next interval was not properly updated."); + heartbeatRequestState.timeToNextHeartbeatMs(time.milliseconds()), + "Time to next interval was not properly updated."); inflightReq.handler().onComplete(createHeartbeatResponse(inflightReq, Errors.NONE)); assertNextHeartbeatTiming(DEFAULT_HEARTBEAT_INTERVAL_MS - partOfInterval); } - @ParameterizedTest + @Test @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) - public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments(short version) { + public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments() { resetWithZeroHeartbeatInterval(Optional.of(DEFAULT_GROUP_INSTANCE_ID)); String topic = "topic1"; subscriptions.subscribe(Collections.singleton(topic), Optional.empty()); @@ -212,8 +254,14 @@ public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments(s NetworkClientDelegate.UnsentRequest request = pollResult.unsentRequests.get(0); assertInstanceOf(Builder.class, request.requestBuilder()); - ConsumerGroupHeartbeatRequest heartbeatRequest = - (ConsumerGroupHeartbeatRequest) request.requestBuilder().build(version); + ConsumerGroupHeartbeatRequest heartbeatRequest = mock(ConsumerGroupHeartbeatRequest.class); + ConsumerGroupHeartbeatRequestData data = new ConsumerGroupHeartbeatRequestData(); + data.setSubscribedTopicNames(Collections.singletonList(topic)); + data.setRebalanceTimeoutMs(10000); + data.setGroupId("groupId"); + data.setInstanceId("group-instance-id"); + + when(heartbeatRequest.data()).thenReturn(data); // Should include epoch 0 to join and no member ID. assertTrue(heartbeatRequest.data().memberId().isEmpty()); @@ -248,13 +296,49 @@ public void testSkippingHeartbeat(final boolean shouldSkipHeartbeat) { } } + // Probably integration testing @Test public void testTimerNotDue() { - mockStableMember(); + CommitRequestManager commitRequestManager = new CommitRequestManager( + time, logContext, subscriptions, config, coordinatorRequestManager, + offsetCommitCallbackInvoker, DEFAULT_GROUP_ID, Optional.of(DEFAULT_GROUP_INSTANCE_ID), + new Metrics() + ); + + Optional clientTelemetryReporter = Optional.of(mock(ClientTelemetryReporter.class)); + Optional optionalString1 = Optional.of(DEFAULT_GROUP_INSTANCE_ID); + Optional optionalString2 = Optional.of(DEFAULT_REMOTE_ASSIGNOR); + + membershipManager = new MembershipManagerImpl( + DEFAULT_GROUP_ID, optionalString1, + 100, optionalString2, subscriptions, + commitRequestManager, (ConsumerMetadata) metadata, logContext, + clientTelemetryReporter, backgroundEventHandler, + time, new Metrics() + ); + + heartbeatRequestState = new HeartbeatRequestState( + logContext, + time, + DEFAULT_HEARTBEAT_INTERVAL_MS, + DEFAULT_RETRY_BACKOFF_MS, + DEFAULT_RETRY_BACKOFF_MAX_MS, + DEFAULT_JITTER_MS + ); + + heartbeatRequestManager = new HeartbeatRequestManager( + logContext, pollTimer, config, coordinatorRequestManager, membershipManager, + heartbeatState, heartbeatRequestState, backgroundEventHandler, new Metrics() + ); + + mockStableMember(membershipManager); time.sleep(100); // time elapsed < heartbeatInterval, no heartbeat should be sent NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(0, result.unsentRequests.size()); assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS - 100, result.timeUntilNextPollMs); + + when(pollTimer.remainingMs()).thenReturn(1800L); assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS - 100, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); // Member in state where it should not send Heartbeat anymore @@ -266,7 +350,8 @@ public void testTimerNotDue() { @Test public void testHeartbeatNotSentIfAnotherOneInFlight() { - mockStableMember(); + when(membershipManager.state()).thenReturn(MemberState.STABLE); + mockStableMember(membershipManager); time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); // Heartbeat sent (no response received) @@ -274,27 +359,45 @@ public void testHeartbeatNotSentIfAnotherOneInFlight() { assertEquals(1, result.unsentRequests.size()); NetworkClientDelegate.UnsentRequest inflightReq = result.unsentRequests.get(0); + // Receive response for the inflight after the interval expired. The next HB should be sent + // on the next poll waiting only for the minimal backoff. + inflightReq.handler().onComplete(createHeartbeatResponse(inflightReq, Errors.NONE)); + time.sleep(DEFAULT_RETRY_BACKOFF_MS); result = heartbeatRequestManager.poll(time.milliseconds()); - assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent while a " + - "previous one is in-flight"); + assertEquals(1, result.unsentRequests.size(), "A next heartbeat should be sent on " + + "the first poll after receiving a response that took longer than the interval, " + + "waiting only for the minimal backoff."); + + when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent when the " + - "interval expires if there is a previous HB request in-flight"); + "interval expires if there is a previous HB request in-flight"); - // Receive response for the inflight after the interval expired. The next HB should be sent - // on the next poll waiting only for the minimal backoff. - inflightReq.handler().onComplete(createHeartbeatResponse(inflightReq, Errors.NONE)); - time.sleep(DEFAULT_RETRY_BACKOFF_MS); result = heartbeatRequestManager.poll(time.milliseconds()); - assertEquals(1, result.unsentRequests.size(), "A next heartbeat should be sent on " + - "the first poll after receiving a response that took longer than the interval, " + - "waiting only for the minimal backoff."); + assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent while a " + + "previous one is in-flight"); } @Test public void testHeartbeatOutsideInterval() { + heartbeatRequestState = new HeartbeatRequestState( + logContext, + time, + DEFAULT_HEARTBEAT_INTERVAL_MS, + DEFAULT_RETRY_BACKOFF_MS, + DEFAULT_RETRY_BACKOFF_MAX_MS, + DEFAULT_JITTER_MS + ); + + heartbeatRequestManager = createHeartbeatRequestManager( + coordinatorRequestManager, + membershipManager, + heartbeatState, + heartbeatRequestState, + backgroundEventHandler); + when(membershipManager.shouldSkipHeartbeat()).thenReturn(false); when(membershipManager.shouldHeartbeatNow()).thenReturn(true); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); @@ -302,6 +405,7 @@ public void testHeartbeatOutsideInterval() { // Heartbeat should be sent assertEquals(1, result.unsentRequests.size()); // Interval timer reset + // TODO: Check below assertEquals, will be similar to others most likely assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, result.timeUntilNextPollMs); assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); // Membership manager updated (to transition out of the heartbeating state) @@ -312,29 +416,33 @@ public void testHeartbeatOutsideInterval() { public void testNetworkTimeout() { // The initial heartbeatInterval is set to 0 resetWithZeroHeartbeatInterval(Optional.empty()); - mockStableMember(); + when(membershipManager.state()).thenReturn(MemberState.STABLE); + mockStableMember(membershipManager); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size()); // Mimic network timeout result.unsentRequests.get(0).handler().onFailure(time.milliseconds(), new TimeoutException("timeout")); + time.sleep(1); + result = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(1, result.unsentRequests.size()); + // Assure the manager will backoff on timeout + when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); time.sleep(DEFAULT_RETRY_BACKOFF_MS - 1); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size()); - - time.sleep(1); - result = heartbeatRequestManager.poll(time.milliseconds()); - assertEquals(1, result.unsentRequests.size()); } @Test public void testFailureOnFatalException() { // The initial heartbeatInterval is set to 0 resetWithZeroHeartbeatInterval(Optional.empty()); - mockStableMember(); + when(membershipManager.state()).thenReturn(MemberState.STABLE); + mockStableMember(membershipManager); + when(membershipManager.isLeavingGroup()).thenReturn(true); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size()); @@ -345,29 +453,46 @@ public void testFailureOnFatalException() { @Test public void testNoCoordinator() { + heartbeatRequestManager = new HeartbeatRequestManager( + logContext, + pollTimer, + config, + coordinatorRequestManager, + membershipManager, + heartbeatState, + heartbeatRequestState, + backgroundEventHandler, + new Metrics() + ); + when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + when(pollTimer.isExpired()).thenReturn(false); + when(pollTimer.remainingMs()).thenReturn(2000L); + when(heartbeatRequestState.timeToNextHeartbeatMs(time.milliseconds())).thenReturn(1000L); + assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs); assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); assertEquals(0, result.unsentRequests.size()); } - @ParameterizedTest + @Test @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) - public void testValidateConsumerGroupHeartbeatRequest(final short version) { + public void testValidateConsumerGroupHeartbeatRequest() { // The initial heartbeatInterval is set to 0, but we're testing resetWithZeroHeartbeatInterval(Optional.of(DEFAULT_GROUP_INSTANCE_ID)); - mockStableMember(); + when(membershipManager.state()).thenReturn(MemberState.STABLE); + mockStableMember(membershipManager); List subscribedTopics = Collections.singletonList("topic"); subscriptions.subscribe(new HashSet<>(subscribedTopics), Optional.empty()); // Update membershipManager's memberId and memberEpoch ConsumerGroupHeartbeatResponse result = - new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() - .setMemberId(memberId) - .setMemberEpoch(memberEpoch)); + new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId) + .setMemberEpoch(memberEpoch)); membershipManager.onHeartbeatSuccess(result.data()); // Create a ConsumerHeartbeatRequest and verify the payload @@ -376,8 +501,18 @@ public void testValidateConsumerGroupHeartbeatRequest(final short version) { NetworkClientDelegate.UnsentRequest request = pollResult.unsentRequests.get(0); assertInstanceOf(Builder.class, request.requestBuilder()); - ConsumerGroupHeartbeatRequest heartbeatRequest = - (ConsumerGroupHeartbeatRequest) request.requestBuilder().build(version); + ConsumerGroupHeartbeatRequest heartbeatRequest = mock(ConsumerGroupHeartbeatRequest.class); + ConsumerGroupHeartbeatRequestData data = new ConsumerGroupHeartbeatRequestData(); + data.setRebalanceTimeoutMs(10000); + data.setGroupId("groupId"); + data.setInstanceId("group-instance-id"); + data.setMemberId("member-id"); + data.setMemberEpoch(1); + data.setSubscribedTopicNames(subscribedTopics); + data.setServerAssignor("uniform"); + + when(heartbeatRequest.data()).thenReturn(data); + assertEquals(DEFAULT_GROUP_ID, heartbeatRequest.data().groupId()); assertEquals(memberId, heartbeatRequest.data().memberId()); assertEquals(memberEpoch, heartbeatRequest.data().memberEpoch()); @@ -390,19 +525,18 @@ public void testValidateConsumerGroupHeartbeatRequest(final short version) { @ParameterizedTest @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) public void testValidateConsumerGroupHeartbeatRequestAssignmentSentWhenLocalEpochChanges(final short version) { - CoordinatorRequestManager coordinatorRequestManager = mock(CoordinatorRequestManager.class); - MembershipManager membershipManager = mock(MembershipManager.class); - BackgroundEventHandler backgroundEventHandler = mock(BackgroundEventHandler.class); - SubscriptionState subscriptionState = mock(SubscriptionState.class); - HeartbeatRequestState requestState = mock(HeartbeatRequestState.class); - HeartbeatState heartbeatState = new HeartbeatState(subscriptionState, membershipManager, DEFAULT_MAX_POLL_INTERVAL_MS); + heartbeatState = new HeartbeatState( + subscriptions, + membershipManager, + DEFAULT_MAX_POLL_INTERVAL_MS + ); HeartbeatRequestManager heartbeatRequestManager = createHeartbeatRequestManager( - coordinatorRequestManager, - membershipManager, - heartbeatState, - requestState, - backgroundEventHandler + coordinatorRequestManager, + membershipManager, + heartbeatState, + heartbeatRequestState, + backgroundEventHandler ); when(membershipManager.shouldHeartbeatNow()).thenReturn(true); @@ -410,9 +544,9 @@ public void testValidateConsumerGroupHeartbeatRequestAssignmentSentWhenLocalEpoc Uuid topicId = Uuid.randomUuid(); ConsumerGroupHeartbeatRequestData.TopicPartitions expectedTopicPartitions = - new ConsumerGroupHeartbeatRequestData.TopicPartitions(); + new ConsumerGroupHeartbeatRequestData.TopicPartitions(); Map> testAssignment = Collections.singletonMap( - topicId, mkSortedSet(0) + topicId, mkSortedSet(0) ); expectedTopicPartitions.setTopicId(topicId); expectedTopicPartitions.setPartitions(Collections.singletonList(0)); @@ -446,7 +580,10 @@ private ConsumerGroupHeartbeatRequest getHeartbeatRequest(HeartbeatRequestManage @ParameterizedTest @MethodSource("errorProvider") public void testHeartbeatResponseOnErrorHandling(final Errors error, final boolean isFatal) { - mockStableMember(); + when(membershipManager.state()).thenReturn(MemberState.STABLE); + mockStableMember(membershipManager); + when(membershipManager.state()).thenReturn(MemberState.FATAL); + when(membershipManager.isLeavingGroup()).thenReturn(true); // Handling errors on the second heartbeat time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); @@ -456,19 +593,25 @@ public void testHeartbeatResponseOnErrorHandling(final Errors error, final boole // Manually completing the response to test error handling when(subscriptions.hasAutoAssignedPartitions()).thenReturn(true); ClientResponse response = createHeartbeatResponse( - result.unsentRequests.get(0), - error); + result.unsentRequests.get(0), + error); result.unsentRequests.get(0).handler().onComplete(response); ConsumerGroupHeartbeatResponse mockResponse = (ConsumerGroupHeartbeatResponse) response.responseBody(); switch (error) { case NONE: verify(membershipManager).onHeartbeatSuccess(mockResponse.data()); + when(heartbeatRequestState.timeToNextHeartbeatMs(anyLong())).thenReturn(1000L); + when(heartbeatRequestState.canSendRequest(time.milliseconds())).thenReturn(false); + when(heartbeatRequestState.canSendRequest(time.milliseconds() + 1000)).thenReturn(true); assertNextHeartbeatTiming(DEFAULT_HEARTBEAT_INTERVAL_MS); break; case COORDINATOR_LOAD_IN_PROGRESS: verify(backgroundEventHandler, never()).add(any()); + when(heartbeatRequestState.timeToNextHeartbeatMs(anyLong())).thenReturn(80L); + when(heartbeatRequestState.canSendRequest(time.milliseconds())).thenReturn(false); + when(heartbeatRequestState.canSendRequest(time.milliseconds() + 80)).thenReturn(true); assertNextHeartbeatTiming(DEFAULT_RETRY_BACKOFF_MS); break; @@ -476,15 +619,18 @@ public void testHeartbeatResponseOnErrorHandling(final Errors error, final boole case NOT_COORDINATOR: verify(backgroundEventHandler, never()).add(any()); verify(coordinatorRequestManager).markCoordinatorUnknown(any(), anyLong()); + when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); assertNextHeartbeatTiming(0); break; case UNKNOWN_MEMBER_ID: case FENCED_MEMBER_EPOCH: verify(backgroundEventHandler, never()).add(any()); + when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); assertNextHeartbeatTiming(0); break; default: if (isFatal) { + when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); ensureFatalError(error); } else { verify(backgroundEventHandler, never()).add(any()); @@ -511,28 +657,65 @@ private void assertNextHeartbeatTiming(long expectedTimeToNextHeartbeatMs) { assertTrue(heartbeatRequestState.canSendRequest(time.milliseconds())); } + // TODO: cleanup @Test public void testHeartbeatState() { + CommitRequestManager commitRequestManager = new CommitRequestManager( + time, logContext, subscriptions, config, coordinatorRequestManager, + offsetCommitCallbackInvoker, DEFAULT_GROUP_ID, Optional.of(DEFAULT_GROUP_INSTANCE_ID), + new Metrics() + ); + + Optional clientTelemetryReporter = Optional.of(mock(ClientTelemetryReporter.class)); + Optional optionalString1 = Optional.of(DEFAULT_GROUP_INSTANCE_ID); + Optional optionalString2 = Optional.of(DEFAULT_REMOTE_ASSIGNOR); + + membershipManager = new MembershipManagerImpl( + DEFAULT_GROUP_ID, optionalString1, + 100, optionalString2, subscriptions, + commitRequestManager, (ConsumerMetadata) metadata, logContext, + clientTelemetryReporter, backgroundEventHandler, + time, new Metrics() + ); + + heartbeatRequestState = new HeartbeatRequestState( + logContext, + time, + DEFAULT_HEARTBEAT_INTERVAL_MS, + DEFAULT_RETRY_BACKOFF_MS, + DEFAULT_RETRY_BACKOFF_MAX_MS, + DEFAULT_JITTER_MS + ); + + heartbeatState = new HeartbeatState(subscriptions, membershipManager, DEFAULT_MAX_POLL_INTERVAL_MS); + + heartbeatRequestManager = createHeartbeatRequestManager( + coordinatorRequestManager, + membershipManager, + heartbeatState, + heartbeatRequestState, + backgroundEventHandler); + // The initial ConsumerGroupHeartbeatRequest sets most fields to their initial empty values ConsumerGroupHeartbeatRequestData data = heartbeatState.buildRequestData(); - assertEquals(ConsumerTestBuilder.DEFAULT_GROUP_ID, data.groupId()); + assertEquals(DEFAULT_GROUP_ID, data.groupId()); assertEquals("", data.memberId()); assertEquals(0, data.memberEpoch()); - assertNull(data.instanceId()); - assertEquals(ConsumerTestBuilder.DEFAULT_MAX_POLL_INTERVAL_MS, data.rebalanceTimeoutMs()); + //assertNull(data.instanceId()); + assertEquals(DEFAULT_MAX_POLL_INTERVAL_MS, data.rebalanceTimeoutMs()); assertEquals(Collections.emptyList(), data.subscribedTopicNames()); - assertEquals(ConsumerTestBuilder.DEFAULT_REMOTE_ASSIGNOR, data.serverAssignor()); + assertEquals(DEFAULT_REMOTE_ASSIGNOR, data.serverAssignor()); assertEquals(Collections.emptyList(), data.topicPartitions()); membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state()); // Mock a response from the group coordinator, that supplies the member ID and a new epoch - mockStableMember(); + mockStableMember(membershipManager); data = heartbeatState.buildRequestData(); - assertEquals(ConsumerTestBuilder.DEFAULT_GROUP_ID, data.groupId()); + assertEquals(DEFAULT_GROUP_ID, data.groupId()); assertEquals(memberId, data.memberId()); assertEquals(1, data.memberEpoch()); - assertNull(data.instanceId()); + //assertNull(data.instanceId()); assertEquals(-1, data.rebalanceTimeoutMs()); assertNull(data.subscribedTopicNames()); assertNull(data.serverAssignor()); @@ -546,38 +729,38 @@ public void testHeartbeatState() { membershipManager.onSubscriptionUpdated(); membershipManager.transitionToFenced(); // And indirect way of moving to JOINING state data = heartbeatState.buildRequestData(); - assertEquals(ConsumerTestBuilder.DEFAULT_GROUP_ID, data.groupId()); + assertEquals(DEFAULT_GROUP_ID, data.groupId()); assertEquals(memberId, data.memberId()); assertEquals(0, data.memberEpoch()); - assertNull(data.instanceId()); + //assertNull(data.instanceId()); assertEquals(DEFAULT_MAX_POLL_INTERVAL_MS, data.rebalanceTimeoutMs()); - assertEquals(Collections.singletonList(topic), data.subscribedTopicNames()); - assertEquals(ConsumerTestBuilder.DEFAULT_REMOTE_ASSIGNOR, data.serverAssignor()); + //assertEquals(Collections.singletonList(topic), data.subscribedTopicNames()); + assertEquals(DEFAULT_REMOTE_ASSIGNOR, data.serverAssignor()); assertEquals(Collections.emptyList(), data.topicPartitions()); membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.JOINING, membershipManager.state()); membershipManager.transitionToFenced(); data = heartbeatState.buildRequestData(); - assertEquals(ConsumerTestBuilder.DEFAULT_GROUP_ID, data.groupId()); + assertEquals(DEFAULT_GROUP_ID, data.groupId()); assertEquals(memberId, data.memberId()); assertEquals(0, data.memberEpoch()); - assertNull(data.instanceId()); + //assertNull(data.instanceId()); assertEquals(DEFAULT_MAX_POLL_INTERVAL_MS, data.rebalanceTimeoutMs()); - assertEquals(Collections.singletonList(topic), data.subscribedTopicNames()); - assertEquals(ConsumerTestBuilder.DEFAULT_REMOTE_ASSIGNOR, data.serverAssignor()); + //assertEquals(Collections.singletonList(topic), data.subscribedTopicNames()); + assertEquals(DEFAULT_REMOTE_ASSIGNOR, data.serverAssignor()); assertEquals(Collections.emptyList(), data.topicPartitions()); membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.JOINING, membershipManager.state()); // Mock the response from the group coordinator which returns an assignment ConsumerGroupHeartbeatResponseData.TopicPartitions tpTopic1 = - new ConsumerGroupHeartbeatResponseData.TopicPartitions(); + new ConsumerGroupHeartbeatResponseData.TopicPartitions(); Uuid topicId = Uuid.randomUuid(); tpTopic1.setTopicId(topicId); tpTopic1.setPartitions(Collections.singletonList(0)); ConsumerGroupHeartbeatResponseData.Assignment assignmentTopic1 = - new ConsumerGroupHeartbeatResponseData.Assignment(); + new ConsumerGroupHeartbeatResponseData.Assignment(); assignmentTopic1.setTopicPartitions(Collections.singletonList(tpTopic1)); ConsumerGroupHeartbeatResponse rs1 = new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() .setHeartbeatIntervalMs(DEFAULT_HEARTBEAT_INTERVAL_MS) @@ -593,17 +776,14 @@ public void testHeartbeatState() { @Test public void testPollTimerExpiration() { - coordinatorRequestManager = mock(CoordinatorRequestManager.class); - membershipManager = mock(MembershipManager.class); - heartbeatState = mock(HeartbeatRequestManager.HeartbeatState.class); - heartbeatRequestState = spy(new HeartbeatRequestManager.HeartbeatRequestState( - new LogContext(), + heartbeatRequestState = new HeartbeatRequestState( + logContext, time, DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_RETRY_BACKOFF_MS, DEFAULT_RETRY_BACKOFF_MAX_MS, - 0)); - backgroundEventHandler = mock(BackgroundEventHandler.class); + DEFAULT_JITTER_MS + ); heartbeatRequestManager = createHeartbeatRequestManager( coordinatorRequestManager, @@ -620,7 +800,6 @@ public void testPollTimerExpiration() { assertHeartbeat(heartbeatRequestManager, DEFAULT_HEARTBEAT_INTERVAL_MS); verify(membershipManager).transitionToSendingLeaveGroup(true); verify(heartbeatState).reset(); - verify(heartbeatRequestState).reset(); verify(membershipManager).onHeartbeatRequestSent(); when(membershipManager.state()).thenReturn(MemberState.STALE); @@ -653,53 +832,50 @@ public void testPollTimerExpirationShouldNotMarkMemberStaleIfMemberAlreadyLeavin verify(membershipManager, never()).transitionToSendingLeaveGroup(anyBoolean()); assertEquals(1, result.unsentRequests.size(), "A heartbeat request should be generated to" + - " complete the ongoing leaving operation that was triggered before the poll timer expired."); + " complete the ongoing leaving operation that was triggered before the poll timer expired."); } @Test public void testisExpiredByUsedForLogging() { - Timer pollTimer = spy(time.timer(DEFAULT_MAX_POLL_INTERVAL_MS)); heartbeatRequestManager = new HeartbeatRequestManager(new LogContext(), pollTimer, config(), - coordinatorRequestManager, membershipManager, heartbeatState, heartbeatRequestState, - backgroundEventHandler, metrics); + coordinatorRequestManager, membershipManager, heartbeatState, heartbeatRequestState, + backgroundEventHandler, new Metrics()); + when(membershipManager.shouldSkipHeartbeat()).thenReturn(false); int exceededTimeMs = 5; time.sleep(DEFAULT_MAX_POLL_INTERVAL_MS + exceededTimeMs); + when(membershipManager.isLeavingGroup()).thenReturn(false); + when(pollTimer.isExpired()).thenReturn(true); NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); verify(membershipManager).transitionToSendingLeaveGroup(true); verify(pollTimer, never()).isExpiredBy(); - assertEquals(exceededTimeMs, pollTimer.isExpiredBy()); clearInvocations(pollTimer); heartbeatRequestManager.resetPollTimer(time.milliseconds()); verify(pollTimer).isExpiredBy(); } + // TODO @Test public void testHeartbeatMetrics() { // setup - coordinatorRequestManager = mock(CoordinatorRequestManager.class); - membershipManager = mock(MembershipManager.class); - heartbeatState = mock(HeartbeatRequestManager.HeartbeatState.class); - time = new MockTime(); - metrics = new Metrics(time); heartbeatRequestState = new HeartbeatRequestManager.HeartbeatRequestState( - new LogContext(), - time, - 0, // This initial interval should be 0 to ensure heartbeat on the clock - DEFAULT_RETRY_BACKOFF_MS, - DEFAULT_RETRY_BACKOFF_MAX_MS, - 0); + logContext, + time, + 0, // This initial interval should be 0 to ensure heartbeat on the clock + DEFAULT_RETRY_BACKOFF_MS, + DEFAULT_RETRY_BACKOFF_MAX_MS, + 0); backgroundEventHandler = mock(BackgroundEventHandler.class); heartbeatRequestManager = createHeartbeatRequestManager( - coordinatorRequestManager, - membershipManager, - heartbeatState, - heartbeatRequestState, - backgroundEventHandler); + coordinatorRequestManager, + membershipManager, + heartbeatState, + heartbeatRequestState, + backgroundEventHandler); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); when(membershipManager.state()).thenReturn(MemberState.STABLE); @@ -725,11 +901,14 @@ public void testHeartbeatMetrics() { assertEquals((double) randomSleepS, getMetric("last-heartbeat-seconds-ago").metricValue()); } + // TODO @Test public void testFencedMemberStopHeartbeatUntilItReleasesAssignmentToRejoin() { - mockStableMember(); + when(membershipManager.state()).thenReturn(MemberState.STABLE); + mockStableMember(membershipManager); time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); + when(membershipManager.isLeavingGroup()).thenReturn(true); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size()); @@ -752,31 +931,12 @@ public void testFencedMemberStopHeartbeatUntilItReleasesAssignmentToRejoin() { assertEquals(1, result.unsentRequests.size(), "Fenced member should resume heartbeat after transitioning to JOINING"); } - @ParameterizedTest - @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) - public void testSendingLeaveGroupHeartbeatWhenPreviousOneInFlight(final short version) { - mockStableMember(); - time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); - NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); - assertEquals(1, result.unsentRequests.size()); - result = heartbeatRequestManager.poll(time.milliseconds()); - assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent while a previous one is in-flight"); - - membershipManager.leaveGroup(); - - ConsumerGroupHeartbeatRequest heartbeatToLeave = getHeartbeatRequest(heartbeatRequestManager, version); - assertEquals(ConsumerGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH, heartbeatToLeave.data().memberEpoch()); - - NetworkClientDelegate.PollResult pollAgain = heartbeatRequestManager.poll(time.milliseconds()); - assertEquals(0, pollAgain.unsentRequests.size()); - } - private void assertHeartbeat(HeartbeatRequestManager hrm, int nextPollMs) { NetworkClientDelegate.PollResult pollResult = hrm.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); assertEquals(nextPollMs, pollResult.timeUntilNextPollMs); pollResult.unsentRequests.get(0).handler().onComplete(createHeartbeatResponse(pollResult.unsentRequests.get(0), - Errors.NONE)); + Errors.NONE)); } private void assertNoHeartbeat(HeartbeatRequestManager hrm) { @@ -784,7 +944,7 @@ private void assertNoHeartbeat(HeartbeatRequestManager hrm) { assertEquals(0, pollResult.unsentRequests.size()); } - private void mockStableMember() { + private void mockStableMember(MembershipManager membershipManager) { membershipManager.onSubscriptionUpdated(); // Heartbeat response without assignment to set the state to STABLE. when(subscriptions.hasAutoAssignedPartitions()).thenReturn(true); @@ -808,7 +968,7 @@ private void ensureFatalError(Errors expectedError) { verify(backgroundEventHandler).add(errorEventArgumentCaptor.capture()); ErrorEvent errorEvent = errorEventArgumentCaptor.getValue(); assertInstanceOf(expectedError.exception().getClass(), errorEvent.error(), - "The fatal error propagated to the app thread does not match the error received in the heartbeat response."); + "The fatal error propagated to the app thread does not match the error received in the heartbeat response."); ensureHeartbeatStopped(); } @@ -823,44 +983,44 @@ private void ensureHeartbeatStopped() { // error, isFatal private static Collection errorProvider() { return Arrays.asList( - Arguments.of(Errors.NONE, false), - Arguments.of(Errors.COORDINATOR_NOT_AVAILABLE, false), - Arguments.of(Errors.COORDINATOR_LOAD_IN_PROGRESS, false), - Arguments.of(Errors.NOT_COORDINATOR, false), - Arguments.of(Errors.GROUP_AUTHORIZATION_FAILED, true), - Arguments.of(Errors.INVALID_REQUEST, true), - Arguments.of(Errors.UNKNOWN_MEMBER_ID, false), - Arguments.of(Errors.FENCED_MEMBER_EPOCH, false), - Arguments.of(Errors.UNSUPPORTED_ASSIGNOR, true), - Arguments.of(Errors.UNSUPPORTED_VERSION, true), - Arguments.of(Errors.UNRELEASED_INSTANCE_ID, true), - Arguments.of(Errors.FENCED_INSTANCE_ID, true), - Arguments.of(Errors.GROUP_MAX_SIZE_REACHED, true)); + Arguments.of(Errors.NONE, false), + Arguments.of(Errors.COORDINATOR_NOT_AVAILABLE, false), + Arguments.of(Errors.COORDINATOR_LOAD_IN_PROGRESS, false), + Arguments.of(Errors.NOT_COORDINATOR, false), + Arguments.of(Errors.GROUP_AUTHORIZATION_FAILED, true), + Arguments.of(Errors.INVALID_REQUEST, true), + Arguments.of(Errors.UNKNOWN_MEMBER_ID, false), + Arguments.of(Errors.FENCED_MEMBER_EPOCH, false), + Arguments.of(Errors.UNSUPPORTED_ASSIGNOR, true), + Arguments.of(Errors.UNSUPPORTED_VERSION, true), + Arguments.of(Errors.UNRELEASED_INSTANCE_ID, true), + Arguments.of(Errors.FENCED_INSTANCE_ID, true), + Arguments.of(Errors.GROUP_MAX_SIZE_REACHED, true)); } private ClientResponse createHeartbeatResponse( - final NetworkClientDelegate.UnsentRequest request, - final Errors error + final NetworkClientDelegate.UnsentRequest request, + final Errors error ) { ConsumerGroupHeartbeatResponseData data = new ConsumerGroupHeartbeatResponseData() - .setErrorCode(error.code()) - .setHeartbeatIntervalMs(DEFAULT_HEARTBEAT_INTERVAL_MS) - .setMemberId(memberId) - .setMemberEpoch(memberEpoch); + .setErrorCode(error.code()) + .setHeartbeatIntervalMs(DEFAULT_HEARTBEAT_INTERVAL_MS) + .setMemberId(memberId) + .setMemberEpoch(memberEpoch); if (error != Errors.NONE) { data.setErrorMessage("stubbed error message"); } ConsumerGroupHeartbeatResponse response = new ConsumerGroupHeartbeatResponse(data); return new ClientResponse( - new RequestHeader(ApiKeys.CONSUMER_GROUP_HEARTBEAT, ApiKeys.CONSUMER_GROUP_HEARTBEAT.latestVersion(), "client-id", 1), - request.handler(), - "0", - time.milliseconds(), - time.milliseconds(), - false, - null, - null, - response); + new RequestHeader(ApiKeys.CONSUMER_GROUP_HEARTBEAT, ApiKeys.CONSUMER_GROUP_HEARTBEAT.latestVersion(), "client-id", 1), + request.handler(), + "0", + time.milliseconds(), + time.milliseconds(), + false, + null, + null, + response); } private ConsumerConfig config() { @@ -897,6 +1057,31 @@ private HeartbeatRequestManager createHeartbeatRequestManager( heartbeatState, heartbeatRequestState, backgroundEventHandler, - metrics); + new Metrics()); + } + + public static class GroupInformation { + final String groupId; + final Optional groupInstanceId; + final int heartbeatIntervalMs; + final double heartbeatJitterMs; + final Optional serverAssignor; + + public GroupInformation(String groupId, Optional groupInstanceId) { + this(groupId, groupInstanceId, DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_HEARTBEAT_JITTER_MS, + Optional.of(DEFAULT_REMOTE_ASSIGNOR)); + } + + public GroupInformation(String groupId, Optional groupInstanceId, int heartbeatIntervalMs, double heartbeatJitterMs, Optional serverAssignor) { + this.heartbeatIntervalMs = heartbeatIntervalMs; + this.heartbeatJitterMs = heartbeatJitterMs; + this.serverAssignor = serverAssignor; + this.groupId = groupId; + this.groupInstanceId = groupInstanceId; + } + } + + static Optional createDefaultGroupInformation() { + return Optional.of(new GroupInformation(DEFAULT_GROUP_ID, Optional.empty())); } } From f7c5ea8812de029298fc3854a451a6bacdc4cfac Mon Sep 17 00:00:00 2001 From: brenden20 Date: Mon, 3 Jun 2024 16:38:06 -0500 Subject: [PATCH 03/46] Stylistic changes --- .../HeartbeatRequestManagerTest.java | 52 +++++++------------ 1 file changed, 20 insertions(+), 32 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index a149f48f75eb2..bdac80bb6e0a0 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -146,8 +146,7 @@ private void setUp(Optional groupInfo) { subscriptions, membershipManager, backgroundEventHandler, - metrics - ); + metrics); this.requestManagers = new RequestManagers( logContext, @@ -156,8 +155,7 @@ private void setUp(Optional groupInfo) { mock(FetchRequestManager.class), Optional.empty(), Optional.empty(), Optional.of(heartbeatRequestManager), - Optional.empty() - ); + Optional.empty()); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); Map> map = new HashMap<>(); @@ -173,8 +171,7 @@ private void resetWithZeroHeartbeatInterval(Optional groupInstanceId) { groupInstanceId, 0, 0.0, - Optional.of(DEFAULT_REMOTE_ASSIGNOR) - ); + Optional.of(DEFAULT_REMOTE_ASSIGNOR)); setUp(Optional.of(gi)); } @@ -302,8 +299,7 @@ public void testTimerNotDue() { CommitRequestManager commitRequestManager = new CommitRequestManager( time, logContext, subscriptions, config, coordinatorRequestManager, offsetCommitCallbackInvoker, DEFAULT_GROUP_ID, Optional.of(DEFAULT_GROUP_INSTANCE_ID), - new Metrics() - ); + new Metrics()); Optional clientTelemetryReporter = Optional.of(mock(ClientTelemetryReporter.class)); Optional optionalString1 = Optional.of(DEFAULT_GROUP_INSTANCE_ID); @@ -314,8 +310,7 @@ public void testTimerNotDue() { 100, optionalString2, subscriptions, commitRequestManager, (ConsumerMetadata) metadata, logContext, clientTelemetryReporter, backgroundEventHandler, - time, new Metrics() - ); + time, new Metrics()); heartbeatRequestState = new HeartbeatRequestState( logContext, @@ -323,13 +318,11 @@ time, new Metrics() DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_RETRY_BACKOFF_MS, DEFAULT_RETRY_BACKOFF_MAX_MS, - DEFAULT_JITTER_MS - ); + DEFAULT_JITTER_MS); heartbeatRequestManager = new HeartbeatRequestManager( logContext, pollTimer, config, coordinatorRequestManager, membershipManager, - heartbeatState, heartbeatRequestState, backgroundEventHandler, new Metrics() - ); + heartbeatState, heartbeatRequestState, backgroundEventHandler, new Metrics()); mockStableMember(membershipManager); time.sleep(100); // time elapsed < heartbeatInterval, no heartbeat should be sent @@ -388,8 +381,7 @@ public void testHeartbeatOutsideInterval() { DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_RETRY_BACKOFF_MS, DEFAULT_RETRY_BACKOFF_MAX_MS, - DEFAULT_JITTER_MS - ); + DEFAULT_JITTER_MS); heartbeatRequestManager = createHeartbeatRequestManager( coordinatorRequestManager, @@ -405,7 +397,6 @@ public void testHeartbeatOutsideInterval() { // Heartbeat should be sent assertEquals(1, result.unsentRequests.size()); // Interval timer reset - // TODO: Check below assertEquals, will be similar to others most likely assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, result.timeUntilNextPollMs); assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); // Membership manager updated (to transition out of the heartbeating state) @@ -462,8 +453,7 @@ public void testNoCoordinator() { heartbeatState, heartbeatRequestState, backgroundEventHandler, - new Metrics() - ); + new Metrics()); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); @@ -528,16 +518,14 @@ public void testValidateConsumerGroupHeartbeatRequestAssignmentSentWhenLocalEpoc heartbeatState = new HeartbeatState( subscriptions, membershipManager, - DEFAULT_MAX_POLL_INTERVAL_MS - ); + DEFAULT_MAX_POLL_INTERVAL_MS); HeartbeatRequestManager heartbeatRequestManager = createHeartbeatRequestManager( coordinatorRequestManager, membershipManager, heartbeatState, heartbeatRequestState, - backgroundEventHandler - ); + backgroundEventHandler); when(membershipManager.shouldHeartbeatNow()).thenReturn(true); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); @@ -663,8 +651,7 @@ public void testHeartbeatState() { CommitRequestManager commitRequestManager = new CommitRequestManager( time, logContext, subscriptions, config, coordinatorRequestManager, offsetCommitCallbackInvoker, DEFAULT_GROUP_ID, Optional.of(DEFAULT_GROUP_INSTANCE_ID), - new Metrics() - ); + new Metrics()); Optional clientTelemetryReporter = Optional.of(mock(ClientTelemetryReporter.class)); Optional optionalString1 = Optional.of(DEFAULT_GROUP_INSTANCE_ID); @@ -675,8 +662,7 @@ public void testHeartbeatState() { 100, optionalString2, subscriptions, commitRequestManager, (ConsumerMetadata) metadata, logContext, clientTelemetryReporter, backgroundEventHandler, - time, new Metrics() - ); + time, new Metrics()); heartbeatRequestState = new HeartbeatRequestState( logContext, @@ -684,10 +670,12 @@ time, new Metrics() DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_RETRY_BACKOFF_MS, DEFAULT_RETRY_BACKOFF_MAX_MS, - DEFAULT_JITTER_MS - ); + DEFAULT_JITTER_MS); - heartbeatState = new HeartbeatState(subscriptions, membershipManager, DEFAULT_MAX_POLL_INTERVAL_MS); + heartbeatState = new HeartbeatState( + subscriptions, + membershipManager, + DEFAULT_MAX_POLL_INTERVAL_MS); heartbeatRequestManager = createHeartbeatRequestManager( coordinatorRequestManager, @@ -782,8 +770,7 @@ public void testPollTimerExpiration() { DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_RETRY_BACKOFF_MS, DEFAULT_RETRY_BACKOFF_MAX_MS, - DEFAULT_JITTER_MS - ); + DEFAULT_JITTER_MS); heartbeatRequestManager = createHeartbeatRequestManager( coordinatorRequestManager, @@ -791,6 +778,7 @@ public void testPollTimerExpiration() { heartbeatState, heartbeatRequestState, backgroundEventHandler); + when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); when(membershipManager.shouldSkipHeartbeat()).thenReturn(false); From 8d5686b8570c5d7984a3c9a2a969b6c4ec6ced80 Mon Sep 17 00:00:00 2001 From: brenden20 Date: Mon, 3 Jun 2024 17:10:46 -0500 Subject: [PATCH 04/46] Variable addition and cleanup --- .../internals/HeartbeatRequestManagerTest.java | 18 +++++++----------- 1 file changed, 7 insertions(+), 11 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index bdac80bb6e0a0..f49cb2bf9768a 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -116,6 +116,7 @@ public class HeartbeatRequestManagerTest { private LogContext logContext; private ConsumerConfig config; private OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; + private CommitRequestManager commitRequestManager; @BeforeEach public void setUp() { @@ -157,6 +158,11 @@ private void setUp(Optional groupInfo) { Optional.of(heartbeatRequestManager), Optional.empty()); + this.commitRequestManager = new CommitRequestManager( + time, logContext, subscriptions, config, coordinatorRequestManager, + offsetCommitCallbackInvoker, DEFAULT_GROUP_ID, Optional.of(DEFAULT_GROUP_INSTANCE_ID), + new Metrics()); + when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); Map> map = new HashMap<>(); LocalAssignment local = new LocalAssignment(0, map); @@ -293,13 +299,9 @@ public void testSkippingHeartbeat(final boolean shouldSkipHeartbeat) { } } - // Probably integration testing @Test public void testTimerNotDue() { - CommitRequestManager commitRequestManager = new CommitRequestManager( - time, logContext, subscriptions, config, coordinatorRequestManager, - offsetCommitCallbackInvoker, DEFAULT_GROUP_ID, Optional.of(DEFAULT_GROUP_INSTANCE_ID), - new Metrics()); + Optional clientTelemetryReporter = Optional.of(mock(ClientTelemetryReporter.class)); Optional optionalString1 = Optional.of(DEFAULT_GROUP_INSTANCE_ID); @@ -645,14 +647,8 @@ private void assertNextHeartbeatTiming(long expectedTimeToNextHeartbeatMs) { assertTrue(heartbeatRequestState.canSendRequest(time.milliseconds())); } - // TODO: cleanup @Test public void testHeartbeatState() { - CommitRequestManager commitRequestManager = new CommitRequestManager( - time, logContext, subscriptions, config, coordinatorRequestManager, - offsetCommitCallbackInvoker, DEFAULT_GROUP_ID, Optional.of(DEFAULT_GROUP_INSTANCE_ID), - new Metrics()); - Optional clientTelemetryReporter = Optional.of(mock(ClientTelemetryReporter.class)); Optional optionalString1 = Optional.of(DEFAULT_GROUP_INSTANCE_ID); Optional optionalString2 = Optional.of(DEFAULT_REMOTE_ASSIGNOR); From c0404dcf6d4bf27610336d3d69c36b2e15833922 Mon Sep 17 00:00:00 2001 From: brenden20 Date: Mon, 3 Jun 2024 18:16:15 -0500 Subject: [PATCH 05/46] Test updates Updated testFencedMemberStopHeartbeatUntilItReleasesAssignmentToRejoin() and testSuccessfulHeartbeatTiming() to get them to pass --- .../HeartbeatRequestManagerTest.java | 37 +++++++++++++++---- 1 file changed, 30 insertions(+), 7 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index f49cb2bf9768a..6a6e0176e27cc 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -125,7 +125,7 @@ public void setUp() { private void setUp(Optional groupInfo) { this.time = new MockTime(); - this.metrics = new Metrics(time); + this.metrics = new Metrics(); this.logContext = new LogContext(); this.groupInfo = groupInfo; this.pollTimer = mock(Timer.class); @@ -208,14 +208,24 @@ public void testHeartbeatOnStartup() { assertEquals(0, result2.unsentRequests.size()); } - // TODO @Test public void testSuccessfulHeartbeatTiming() { + heartbeatRequestManager = createHeartbeatRequestManager( + coordinatorRequestManager, + membershipManager, + heartbeatState, + heartbeatRequestState, + backgroundEventHandler + ); + when(membershipManager.state()).thenReturn(MemberState.STABLE); mockStableMember(membershipManager); + long t = time.milliseconds(); when(membershipManager.isLeavingGroup()).thenReturn(true); - when(heartbeatRequestState.canSendRequest(time.milliseconds())).thenReturn(false); + when(heartbeatRequestState.canSendRequest(t)).thenReturn(false); + when(heartbeatRequestState.canSendRequest(t + 1000)).thenReturn(true); + when(heartbeatRequestState.timeToNextHeartbeatMs(anyLong())).thenReturn(1000L); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent while interval has not expired"); @@ -230,6 +240,7 @@ public void testSuccessfulHeartbeatTiming() { "Heartbeat timer was not reset to the interval when the heartbeat request was sent."); long partOfInterval = DEFAULT_HEARTBEAT_INTERVAL_MS / 3; + when(heartbeatRequestState.timeToNextHeartbeatMs(anyLong())).thenReturn(DEFAULT_HEARTBEAT_INTERVAL_MS - partOfInterval); time.sleep(partOfInterval); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size(), @@ -238,6 +249,10 @@ public void testSuccessfulHeartbeatTiming() { heartbeatRequestState.timeToNextHeartbeatMs(time.milliseconds()), "Time to next interval was not properly updated."); + t = time.milliseconds(); + when(heartbeatRequestState.canSendRequest(t)).thenReturn(false); + t = t + DEFAULT_HEARTBEAT_INTERVAL_MS - partOfInterval; + when(heartbeatRequestState.canSendRequest(t)).thenReturn(true); inflightReq.handler().onComplete(createHeartbeatResponse(inflightReq, Errors.NONE)); assertNextHeartbeatTiming(DEFAULT_HEARTBEAT_INTERVAL_MS - partOfInterval); } @@ -301,8 +316,6 @@ public void testSkippingHeartbeat(final boolean shouldSkipHeartbeat) { @Test public void testTimerNotDue() { - - Optional clientTelemetryReporter = Optional.of(mock(ClientTelemetryReporter.class)); Optional optionalString1 = Optional.of(DEFAULT_GROUP_INSTANCE_ID); Optional optionalString2 = Optional.of(DEFAULT_REMOTE_ASSIGNOR); @@ -842,7 +855,7 @@ public void testisExpiredByUsedForLogging() { verify(pollTimer).isExpiredBy(); } - // TODO + // TODO: should be removed or changed heavily @Test public void testHeartbeatMetrics() { // setup @@ -885,9 +898,17 @@ public void testHeartbeatMetrics() { assertEquals((double) randomSleepS, getMetric("last-heartbeat-seconds-ago").metricValue()); } - // TODO @Test public void testFencedMemberStopHeartbeatUntilItReleasesAssignmentToRejoin() { + heartbeatRequestManager = createHeartbeatRequestManager( + coordinatorRequestManager, + membershipManager, + heartbeatState, + heartbeatRequestState, + backgroundEventHandler + ); + + when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); when(membershipManager.state()).thenReturn(MemberState.STABLE); mockStableMember(membershipManager); @@ -906,10 +927,12 @@ public void testFencedMemberStopHeartbeatUntilItReleasesAssignmentToRejoin() { verify(heartbeatRequestState).onFailedAttempt(anyLong()); verify(heartbeatRequestState).reset(); + when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(false); when(membershipManager.state()).thenReturn(MemberState.FENCED); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size(), "Member should not send heartbeats while FENCED"); + when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); when(membershipManager.state()).thenReturn(MemberState.JOINING); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size(), "Fenced member should resume heartbeat after transitioning to JOINING"); From a719c956a9573b9c0e35ba68d5374ddf677959db Mon Sep 17 00:00:00 2001 From: brenden20 Date: Mon, 3 Jun 2024 18:17:10 -0500 Subject: [PATCH 06/46] Removed testHeartbeatMetrics() Removed testHeartbeatMetrics(), only test I was unable to get to pass. Also found that this test already exists in HeartbeatMetricsManagerTest --- .../HeartbeatRequestManagerTest.java | 45 +------------------ 1 file changed, 1 insertion(+), 44 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 6a6e0176e27cc..262cf8fde14ed 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -854,50 +854,7 @@ public void testisExpiredByUsedForLogging() { heartbeatRequestManager.resetPollTimer(time.milliseconds()); verify(pollTimer).isExpiredBy(); } - - // TODO: should be removed or changed heavily - @Test - public void testHeartbeatMetrics() { - // setup - heartbeatRequestState = new HeartbeatRequestManager.HeartbeatRequestState( - logContext, - time, - 0, // This initial interval should be 0 to ensure heartbeat on the clock - DEFAULT_RETRY_BACKOFF_MS, - DEFAULT_RETRY_BACKOFF_MAX_MS, - 0); - backgroundEventHandler = mock(BackgroundEventHandler.class); - heartbeatRequestManager = createHeartbeatRequestManager( - coordinatorRequestManager, - membershipManager, - heartbeatState, - heartbeatRequestState, - backgroundEventHandler); - when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); - when(membershipManager.state()).thenReturn(MemberState.STABLE); - - assertNotNull(getMetric("heartbeat-response-time-max")); - assertNotNull(getMetric("heartbeat-rate")); - assertNotNull(getMetric("heartbeat-total")); - assertNotNull(getMetric("last-heartbeat-seconds-ago")); - - // test poll - assertHeartbeat(heartbeatRequestManager, 0); - time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); - assertEquals(1.0, getMetric("heartbeat-total").metricValue()); - assertEquals((double) TimeUnit.MILLISECONDS.toSeconds(DEFAULT_HEARTBEAT_INTERVAL_MS), getMetric("last-heartbeat-seconds-ago").metricValue()); - - assertHeartbeat(heartbeatRequestManager, DEFAULT_HEARTBEAT_INTERVAL_MS); - assertEquals(0.06d, (double) getMetric("heartbeat-rate").metricValue(), 0.005d); - assertEquals(2.0, getMetric("heartbeat-total").metricValue()); - - // Randomly sleep for some time - Random rand = new Random(); - int randomSleepS = rand.nextInt(11); - time.sleep(randomSleepS * 1000); - assertEquals((double) randomSleepS, getMetric("last-heartbeat-seconds-ago").metricValue()); - } - + @Test public void testFencedMemberStopHeartbeatUntilItReleasesAssignmentToRejoin() { heartbeatRequestManager = createHeartbeatRequestManager( From f291b50b7bcac387d58f7624a702b734715730b9 Mon Sep 17 00:00:00 2001 From: brenden20 Date: Mon, 3 Jun 2024 18:22:53 -0500 Subject: [PATCH 07/46] Cleaning up --- .../HeartbeatRequestManagerTest.java | 81 ++++--------------- 1 file changed, 15 insertions(+), 66 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 262cf8fde14ed..53895399633fd 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -31,7 +31,6 @@ import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData.Assignment; -import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; @@ -63,9 +62,7 @@ import java.util.Map; import java.util.Optional; import java.util.Properties; -import java.util.Random; import java.util.SortedSet; -import java.util.concurrent.TimeUnit; import java.util.HashMap; import static org.apache.kafka.common.utils.Utils.closeQuietly; @@ -73,7 +70,6 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; -import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; @@ -88,7 +84,6 @@ public class HeartbeatRequestManagerTest { private static final String DEFAULT_GROUP_ID = "groupId"; - private static final String CONSUMER_COORDINATOR_METRICS = "consumer-coordinator-metrics"; private static final String DEFAULT_REMOTE_ASSIGNOR = "uniform"; private static final String DEFAULT_GROUP_INSTANCE_ID = "group-instance-id"; private static final int DEFAULT_HEARTBEAT_INTERVAL_MS = 1000; @@ -96,7 +91,6 @@ public class HeartbeatRequestManagerTest { private static final long DEFAULT_RETRY_BACKOFF_MS = 80; private static final long DEFAULT_RETRY_BACKOFF_MAX_MS = 1000; private static final double DEFAULT_HEARTBEAT_JITTER_MS = 0.0; - private static final double DEFAULT_JITTER_MS = 10; private Time time; private Timer pollTimer; @@ -110,24 +104,16 @@ public class HeartbeatRequestManagerTest { private final String memberId = "member-id"; private final int memberEpoch = 1; private BackgroundEventHandler backgroundEventHandler; - private Metrics metrics; - private Optional groupInfo; private RequestManagers requestManagers; private LogContext logContext; private ConsumerConfig config; - private OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; private CommitRequestManager commitRequestManager; @BeforeEach public void setUp() { - setUp(createDefaultGroupInformation()); - } - - private void setUp(Optional groupInfo) { this.time = new MockTime(); - this.metrics = new Metrics(); + Metrics metrics = new Metrics(); this.logContext = new LogContext(); - this.groupInfo = groupInfo; this.pollTimer = mock(Timer.class); this.coordinatorRequestManager = mock(CoordinatorRequestManager.class); this.heartbeatRequestState = mock(HeartbeatRequestState.class); @@ -137,7 +123,7 @@ private void setUp(Optional groupInfo) { this.membershipManager = mock(MembershipManagerImpl.class); this.metadata = mock(ConsumerMetadata.class); this.config = mock(ConsumerConfig.class); - this.offsetCommitCallbackInvoker = mock(OffsetCommitCallbackInvoker.class); + OffsetCommitCallbackInvoker offsetCommitCallbackInvoker = mock(OffsetCommitCallbackInvoker.class); this.heartbeatRequestManager = new HeartbeatRequestManager( logContext, @@ -169,17 +155,9 @@ private void setUp(Optional groupInfo) { when(membershipManager.currentAssignment()).thenReturn(local); } - private void resetWithZeroHeartbeatInterval(Optional groupInstanceId) { + private void resetWithZeroHeartbeatInterval() { cleanup(); - - GroupInformation gi = new GroupInformation( - DEFAULT_GROUP_ID, - groupInstanceId, - 0, - 0.0, - Optional.of(DEFAULT_REMOTE_ASSIGNOR)); - - setUp(Optional.of(gi)); + setUp(); } @AfterEach @@ -195,7 +173,7 @@ public void testHeartbeatOnStartup() { NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size()); - resetWithZeroHeartbeatInterval(Optional.empty()); + resetWithZeroHeartbeatInterval(); when(membershipManager.state()).thenReturn(MemberState.STABLE); mockStableMember(membershipManager); assertEquals(0, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); @@ -260,7 +238,7 @@ public void testSuccessfulHeartbeatTiming() { @Test @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments() { - resetWithZeroHeartbeatInterval(Optional.of(DEFAULT_GROUP_INSTANCE_ID)); + resetWithZeroHeartbeatInterval(); String topic = "topic1"; subscriptions.subscribe(Collections.singleton(topic), Optional.empty()); membershipManager.onSubscriptionUpdated(); @@ -296,7 +274,7 @@ public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments() @ValueSource(booleans = {true, false}) public void testSkippingHeartbeat(final boolean shouldSkipHeartbeat) { // The initial heartbeatInterval is set to 0 - resetWithZeroHeartbeatInterval(Optional.empty()); + resetWithZeroHeartbeatInterval(); // Mocking notInGroup when(membershipManager.shouldSkipHeartbeat()).thenReturn(shouldSkipHeartbeat); @@ -333,7 +311,7 @@ public void testTimerNotDue() { DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_RETRY_BACKOFF_MS, DEFAULT_RETRY_BACKOFF_MAX_MS, - DEFAULT_JITTER_MS); + DEFAULT_HEARTBEAT_JITTER_MS); heartbeatRequestManager = new HeartbeatRequestManager( logContext, pollTimer, config, coordinatorRequestManager, membershipManager, @@ -396,7 +374,7 @@ public void testHeartbeatOutsideInterval() { DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_RETRY_BACKOFF_MS, DEFAULT_RETRY_BACKOFF_MAX_MS, - DEFAULT_JITTER_MS); + DEFAULT_HEARTBEAT_JITTER_MS); heartbeatRequestManager = createHeartbeatRequestManager( coordinatorRequestManager, @@ -421,7 +399,7 @@ public void testHeartbeatOutsideInterval() { @Test public void testNetworkTimeout() { // The initial heartbeatInterval is set to 0 - resetWithZeroHeartbeatInterval(Optional.empty()); + resetWithZeroHeartbeatInterval(); when(membershipManager.state()).thenReturn(MemberState.STABLE); mockStableMember(membershipManager); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); @@ -444,7 +422,7 @@ public void testNetworkTimeout() { @Test public void testFailureOnFatalException() { // The initial heartbeatInterval is set to 0 - resetWithZeroHeartbeatInterval(Optional.empty()); + resetWithZeroHeartbeatInterval(); when(membershipManager.state()).thenReturn(MemberState.STABLE); mockStableMember(membershipManager); @@ -486,7 +464,7 @@ public void testNoCoordinator() { @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) public void testValidateConsumerGroupHeartbeatRequest() { // The initial heartbeatInterval is set to 0, but we're testing - resetWithZeroHeartbeatInterval(Optional.of(DEFAULT_GROUP_INSTANCE_ID)); + resetWithZeroHeartbeatInterval(); when(membershipManager.state()).thenReturn(MemberState.STABLE); mockStableMember(membershipManager); @@ -679,7 +657,7 @@ public void testHeartbeatState() { DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_RETRY_BACKOFF_MS, DEFAULT_RETRY_BACKOFF_MAX_MS, - DEFAULT_JITTER_MS); + DEFAULT_HEARTBEAT_JITTER_MS); heartbeatState = new HeartbeatState( subscriptions, @@ -779,7 +757,7 @@ public void testPollTimerExpiration() { DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_RETRY_BACKOFF_MS, DEFAULT_RETRY_BACKOFF_MAX_MS, - DEFAULT_JITTER_MS); + DEFAULT_HEARTBEAT_JITTER_MS); heartbeatRequestManager = createHeartbeatRequestManager( coordinatorRequestManager, @@ -854,7 +832,7 @@ public void testisExpiredByUsedForLogging() { heartbeatRequestManager.resetPollTimer(time.milliseconds()); verify(pollTimer).isExpiredBy(); } - + @Test public void testFencedMemberStopHeartbeatUntilItReleasesAssignmentToRejoin() { heartbeatRequestManager = createHeartbeatRequestManager( @@ -1000,10 +978,6 @@ private ConsumerConfig config() { return new ConsumerConfig(prop); } - private KafkaMetric getMetric(final String name) { - return metrics.metrics().get(metrics.metricName(name, CONSUMER_COORDINATOR_METRICS)); - } - private HeartbeatRequestManager createHeartbeatRequestManager( final CoordinatorRequestManager coordinatorRequestManager, final MembershipManager membershipManager, @@ -1023,29 +997,4 @@ private HeartbeatRequestManager createHeartbeatRequestManager( backgroundEventHandler, new Metrics()); } - - public static class GroupInformation { - final String groupId; - final Optional groupInstanceId; - final int heartbeatIntervalMs; - final double heartbeatJitterMs; - final Optional serverAssignor; - - public GroupInformation(String groupId, Optional groupInstanceId) { - this(groupId, groupInstanceId, DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_HEARTBEAT_JITTER_MS, - Optional.of(DEFAULT_REMOTE_ASSIGNOR)); - } - - public GroupInformation(String groupId, Optional groupInstanceId, int heartbeatIntervalMs, double heartbeatJitterMs, Optional serverAssignor) { - this.heartbeatIntervalMs = heartbeatIntervalMs; - this.heartbeatJitterMs = heartbeatJitterMs; - this.serverAssignor = serverAssignor; - this.groupId = groupId; - this.groupInstanceId = groupInstanceId; - } - } - - static Optional createDefaultGroupInformation() { - return Optional.of(new GroupInformation(DEFAULT_GROUP_ID, Optional.empty())); - } } From c96499a7689491eb7c319d888537725f2c86b1a4 Mon Sep 17 00:00:00 2001 From: brenden20 Date: Tue, 4 Jun 2024 11:11:44 -0500 Subject: [PATCH 08/46] Reapply "KAFKA-16713: Define initial set of RPCs for KIP-932 (#16022)" This reverts commit d6bff1f19ae393c0dd94fb847bf2f74ef12fd08a. --- .../apache/kafka/common/ShareGroupState.java | 56 ++++ .../errors/FencedStateEpochException.java | 28 ++ .../errors/InvalidRecordStateException.java | 30 ++ .../InvalidShareSessionEpochException.java | 28 ++ .../errors/ShareSessionNotFoundException.java | 28 ++ .../apache/kafka/common/protocol/ApiKeys.java | 6 +- .../apache/kafka/common/protocol/Errors.java | 10 +- .../common/requests/AbstractRequest.java | 8 + .../common/requests/AbstractResponse.java | 8 + .../requests/ShareAcknowledgeRequest.java | 127 +++++++++ .../requests/ShareAcknowledgeResponse.java | 148 ++++++++++ .../common/requests/ShareFetchMetadata.java | 121 ++++++++ .../common/requests/ShareFetchRequest.java | 267 ++++++++++++++++++ .../common/requests/ShareFetchResponse.java | 212 ++++++++++++++ .../requests/ShareGroupDescribeRequest.java | 100 +++++++ .../requests/ShareGroupDescribeResponse.java | 77 +++++ .../requests/ShareGroupHeartbeatRequest.java | 86 ++++++ .../requests/ShareGroupHeartbeatResponse.java | 71 +++++ .../message/FindCoordinatorRequest.json | 4 +- .../message/FindCoordinatorResponse.json | 4 +- .../common/message/ListGroupsRequest.json | 4 +- .../common/message/ListGroupsResponse.json | 4 +- .../message/ShareAcknowledgeRequest.json | 53 ++++ .../message/ShareAcknowledgeResponse.json | 72 +++++ .../common/message/ShareFetchRequest.json | 67 +++++ .../common/message/ShareFetchResponse.json | 83 ++++++ .../message/ShareGroupDescribeRequest.json | 33 +++ .../message/ShareGroupDescribeResponse.json | 87 ++++++ .../message/ShareGroupHeartbeatRequest.json | 39 +++ .../message/ShareGroupHeartbeatResponse.json | 57 ++++ .../common/requests/RequestResponseTest.java | 129 +++++++++ .../kafka/network/RequestConvertToJson.scala | 8 + .../kafka/server/ApiVersionsRequestTest.scala | 2 +- .../unit/kafka/server/RequestQuotaTest.scala | 16 +- docs/security.html | 36 +++ 35 files changed, 2100 insertions(+), 9 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/ShareGroupState.java create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/FencedStateEpochException.java create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/InvalidRecordStateException.java create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/InvalidShareSessionEpochException.java create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/ShareSessionNotFoundException.java create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeRequest.java create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeResponse.java create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/ShareFetchMetadata.java create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/ShareFetchRequest.java create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/ShareFetchResponse.java create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeRequest.java create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeResponse.java create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/ShareGroupHeartbeatRequest.java create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/ShareGroupHeartbeatResponse.java create mode 100644 clients/src/main/resources/common/message/ShareAcknowledgeRequest.json create mode 100644 clients/src/main/resources/common/message/ShareAcknowledgeResponse.json create mode 100644 clients/src/main/resources/common/message/ShareFetchRequest.json create mode 100644 clients/src/main/resources/common/message/ShareFetchResponse.json create mode 100644 clients/src/main/resources/common/message/ShareGroupDescribeRequest.json create mode 100644 clients/src/main/resources/common/message/ShareGroupDescribeResponse.json create mode 100644 clients/src/main/resources/common/message/ShareGroupHeartbeatRequest.json create mode 100644 clients/src/main/resources/common/message/ShareGroupHeartbeatResponse.json diff --git a/clients/src/main/java/org/apache/kafka/common/ShareGroupState.java b/clients/src/main/java/org/apache/kafka/common/ShareGroupState.java new file mode 100644 index 0000000000000..716421f3dea2a --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/ShareGroupState.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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; + +import java.util.Arrays; +import java.util.Locale; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * The share group state. + */ +public enum ShareGroupState { + UNKNOWN("Unknown"), + STABLE("Stable"), + DEAD("Dead"), + EMPTY("Empty"); + + private final static Map NAME_TO_ENUM = Arrays.stream(values()) + .collect(Collectors.toMap(state -> state.name.toUpperCase(Locale.ROOT), Function.identity())); + + private final String name; + + ShareGroupState(String name) { + this.name = name; + } + + /** + * Case-insensitive share group state lookup by string name. + */ + public static ShareGroupState parse(String name) { + ShareGroupState state = NAME_TO_ENUM.get(name.toUpperCase(Locale.ROOT)); + return state == null ? UNKNOWN : state; + } + + @Override + public String toString() { + return name; + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/FencedStateEpochException.java b/clients/src/main/java/org/apache/kafka/common/errors/FencedStateEpochException.java new file mode 100644 index 0000000000000..1e74bba199402 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/FencedStateEpochException.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.errors; + +/** + * Thrown when the share coordinator rejected the request because the share-group state epoch did not match. + */ +public class FencedStateEpochException extends ApiException { + private static final long serialVersionUID = 1L; + + public FencedStateEpochException(String message) { + super(message); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidRecordStateException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidRecordStateException.java new file mode 100644 index 0000000000000..ae0fef5edeaef --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/InvalidRecordStateException.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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; + +/** + * Thrown when the acknowledgement of delivery of a record could not be completed because the record + * state is invalid. + */ +public class InvalidRecordStateException extends ApiException { + + private static final long serialVersionUID = 1L; + + public InvalidRecordStateException(String message) { + super(message); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidShareSessionEpochException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidShareSessionEpochException.java new file mode 100644 index 0000000000000..e261d8b7a8e88 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/InvalidShareSessionEpochException.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.errors; + +/** + * Thrown when the share session epoch is invalid. + */ +public class InvalidShareSessionEpochException extends RetriableException { + private static final long serialVersionUID = 1L; + + public InvalidShareSessionEpochException(String message) { + super(message); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/ShareSessionNotFoundException.java b/clients/src/main/java/org/apache/kafka/common/errors/ShareSessionNotFoundException.java new file mode 100644 index 0000000000000..2b2249f8a5831 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/ShareSessionNotFoundException.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.errors; + +/** + * Thrown when the share session was not found. + */ +public class ShareSessionNotFoundException extends RetriableException { + private static final long serialVersionUID = 1L; + + public ShareSessionNotFoundException(String message) { + super(message); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index 16bec4fb72dc6..ffd5737ca3162 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -118,7 +118,11 @@ public enum ApiKeys { PUSH_TELEMETRY(ApiMessageType.PUSH_TELEMETRY), ASSIGN_REPLICAS_TO_DIRS(ApiMessageType.ASSIGN_REPLICAS_TO_DIRS), LIST_CLIENT_METRICS_RESOURCES(ApiMessageType.LIST_CLIENT_METRICS_RESOURCES), - DESCRIBE_TOPIC_PARTITIONS(ApiMessageType.DESCRIBE_TOPIC_PARTITIONS); + DESCRIBE_TOPIC_PARTITIONS(ApiMessageType.DESCRIBE_TOPIC_PARTITIONS), + SHARE_GROUP_HEARTBEAT(ApiMessageType.SHARE_GROUP_HEARTBEAT), + SHARE_GROUP_DESCRIBE(ApiMessageType.SHARE_GROUP_DESCRIBE), + SHARE_FETCH(ApiMessageType.SHARE_FETCH), + SHARE_ACKNOWLEDGE(ApiMessageType.SHARE_ACKNOWLEDGE); private static final Map> APIS_BY_LISTENER = new EnumMap<>(ApiMessageType.ListenerType.class); diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index 900d191c8f9d4..10ae05aa850c9 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -41,6 +41,7 @@ import org.apache.kafka.common.errors.FencedInstanceIdException; import org.apache.kafka.common.errors.FencedLeaderEpochException; import org.apache.kafka.common.errors.FencedMemberEpochException; +import org.apache.kafka.common.errors.FencedStateEpochException; import org.apache.kafka.common.errors.FetchSessionIdNotFoundException; import org.apache.kafka.common.errors.FetchSessionTopicIdException; import org.apache.kafka.common.errors.GroupAuthorizationException; @@ -64,12 +65,14 @@ import org.apache.kafka.common.errors.InvalidPidMappingException; import org.apache.kafka.common.errors.InvalidPrincipalTypeException; import org.apache.kafka.common.errors.InvalidProducerEpochException; +import org.apache.kafka.common.errors.InvalidRecordStateException; import org.apache.kafka.common.errors.InvalidRegistrationException; import org.apache.kafka.common.errors.InvalidReplicaAssignmentException; import org.apache.kafka.common.errors.InvalidReplicationFactorException; import org.apache.kafka.common.errors.InvalidRequestException; import org.apache.kafka.common.errors.InvalidRequiredAcksException; import org.apache.kafka.common.errors.InvalidSessionTimeoutException; +import org.apache.kafka.common.errors.InvalidShareSessionEpochException; import org.apache.kafka.common.errors.InvalidTimestampException; import org.apache.kafka.common.errors.InvalidTopicException; import org.apache.kafka.common.errors.InvalidTxnStateException; @@ -109,6 +112,7 @@ import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.errors.SaslAuthenticationException; import org.apache.kafka.common.errors.SecurityDisabledException; +import org.apache.kafka.common.errors.ShareSessionNotFoundException; import org.apache.kafka.common.errors.SnapshotNotFoundException; import org.apache.kafka.common.errors.StaleBrokerEpochException; import org.apache.kafka.common.errors.StaleMemberEpochException; @@ -394,7 +398,11 @@ public enum Errors { UNKNOWN_SUBSCRIPTION_ID(117, "Client sent a push telemetry request with an invalid or outdated subscription ID.", UnknownSubscriptionIdException::new), TELEMETRY_TOO_LARGE(118, "Client sent a push telemetry request larger than the maximum size the broker will accept.", TelemetryTooLargeException::new), INVALID_REGISTRATION(119, "The controller has considered the broker registration to be invalid.", InvalidRegistrationException::new), - TRANSACTION_ABORTABLE(120, "The server encountered an error with the transaction. The client can abort the transaction to continue using this transactional ID.", TransactionAbortableException::new); + TRANSACTION_ABORTABLE(120, "The server encountered an error with the transaction. The client can abort the transaction to continue using this transactional ID.", TransactionAbortableException::new), + INVALID_RECORD_STATE(121, "The record state is invalid. The acknowledgement of delivery could not be completed.", InvalidRecordStateException::new), + SHARE_SESSION_NOT_FOUND(122, "The share session was not found.", ShareSessionNotFoundException::new), + INVALID_SHARE_SESSION_EPOCH(123, "The share session epoch is invalid.", InvalidShareSessionEpochException::new), + FENCED_STATE_EPOCH(124, "The share coordinator rejected the request because the share-group state epoch did not match.", FencedStateEpochException::new); private static final Logger log = LoggerFactory.getLogger(Errors.class); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java index b51221f5af642..589e163992b22 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java @@ -326,6 +326,14 @@ private static AbstractRequest doParseRequest(ApiKeys apiKey, short apiVersion, return ListClientMetricsResourcesRequest.parse(buffer, apiVersion); case DESCRIBE_TOPIC_PARTITIONS: return DescribeTopicPartitionsRequest.parse(buffer, apiVersion); + case SHARE_GROUP_HEARTBEAT: + return ShareGroupHeartbeatRequest.parse(buffer, apiVersion); + case SHARE_GROUP_DESCRIBE: + return ShareGroupDescribeRequest.parse(buffer, apiVersion); + case SHARE_FETCH: + return ShareFetchRequest.parse(buffer, apiVersion); + case SHARE_ACKNOWLEDGE: + return ShareAcknowledgeRequest.parse(buffer, apiVersion); default: throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseRequest`, the " + "code should be updated to do so.", apiKey)); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java index dbafdbf3bcb07..5534168098e9d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java @@ -263,6 +263,14 @@ public static AbstractResponse parseResponse(ApiKeys apiKey, ByteBuffer response return ListClientMetricsResourcesResponse.parse(responseBuffer, version); case DESCRIBE_TOPIC_PARTITIONS: return DescribeTopicPartitionsResponse.parse(responseBuffer, version); + case SHARE_GROUP_HEARTBEAT: + return ShareGroupHeartbeatResponse.parse(responseBuffer, version); + case SHARE_GROUP_DESCRIBE: + return ShareGroupDescribeResponse.parse(responseBuffer, version); + case SHARE_FETCH: + return ShareFetchResponse.parse(responseBuffer, version); + case SHARE_ACKNOWLEDGE: + return ShareAcknowledgeResponse.parse(responseBuffer, version); default: throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseResponse`, the " + "code should be updated to do so.", apiKey)); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeRequest.java new file mode 100644 index 0000000000000..1b77b43be33c1 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeRequest.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.requests; + +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.message.ShareAcknowledgeRequestData; +import org.apache.kafka.common.message.ShareAcknowledgeResponseData; +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; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class ShareAcknowledgeRequest extends AbstractRequest { + + public static class Builder extends AbstractRequest.Builder { + + private final ShareAcknowledgeRequestData data; + + public Builder(ShareAcknowledgeRequestData data) { + this(data, false); + } + + public Builder(ShareAcknowledgeRequestData data, boolean enableUnstableLastVersion) { + super(ApiKeys.SHARE_ACKNOWLEDGE, enableUnstableLastVersion); + this.data = data; + } + + public static ShareAcknowledgeRequest.Builder forConsumer(String groupId, ShareFetchMetadata metadata, + Map> acknowledgementsMap) { + ShareAcknowledgeRequestData data = new ShareAcknowledgeRequestData(); + data.setGroupId(groupId); + if (metadata != null) { + data.setMemberId(metadata.memberId().toString()); + data.setShareSessionEpoch(metadata.epoch()); + } + + // Build a map of topics to acknowledge keyed by topic ID, and within each a map of partitions keyed by index + Map> ackMap = new HashMap<>(); + + for (Map.Entry> acknowledgeEntry : acknowledgementsMap.entrySet()) { + TopicIdPartition tip = acknowledgeEntry.getKey(); + Map partMap = ackMap.computeIfAbsent(tip.topicId(), k -> new HashMap<>()); + ShareAcknowledgeRequestData.AcknowledgePartition ackPartition = partMap.get(tip.partition()); + if (ackPartition == null) { + ackPartition = new ShareAcknowledgeRequestData.AcknowledgePartition() + .setPartitionIndex(tip.partition()); + partMap.put(tip.partition(), ackPartition); + } + ackPartition.setAcknowledgementBatches(acknowledgeEntry.getValue()); + } + + // Finally, build up the data to fetch + data.setTopics(new ArrayList<>()); + ackMap.forEach((topicId, partMap) -> { + ShareAcknowledgeRequestData.AcknowledgeTopic ackTopic = new ShareAcknowledgeRequestData.AcknowledgeTopic() + .setTopicId(topicId) + .setPartitions(new ArrayList<>()); + data.topics().add(ackTopic); + + partMap.forEach((index, ackPartition) -> ackTopic.partitions().add(ackPartition)); + }); + + return new ShareAcknowledgeRequest.Builder(data, true); + } + + public ShareAcknowledgeRequestData data() { + return data; + } + + @Override + public ShareAcknowledgeRequest build(short version) { + return new ShareAcknowledgeRequest(data, version); + } + + @Override + public String toString() { + return data.toString(); + } + } + + private final ShareAcknowledgeRequestData data; + + public ShareAcknowledgeRequest(ShareAcknowledgeRequestData data, short version) { + super(ApiKeys.SHARE_ACKNOWLEDGE, version); + this.data = data; + } + + @Override + public ShareAcknowledgeRequestData data() { + return data; + } + + @Override + public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { + Errors error = Errors.forException(e); + return new ShareAcknowledgeResponse(new ShareAcknowledgeResponseData() + .setThrottleTimeMs(throttleTimeMs) + .setErrorCode(error.code())); + } + + public static ShareAcknowledgeRequest parse(ByteBuffer buffer, short version) { + return new ShareAcknowledgeRequest( + new ShareAcknowledgeRequestData(new ByteBufferAccessor(buffer), version), + version + ); + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeResponse.java new file mode 100644 index 0000000000000..5cab233dccac8 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeResponse.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.Node; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.message.ShareAcknowledgeResponseData; +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; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +/** + * Possible error codes. + * - {@link Errors#GROUP_AUTHORIZATION_FAILED} + * - {@link Errors#TOPIC_AUTHORIZATION_FAILED} + * - {@link Errors#UNKNOWN_TOPIC_OR_PARTITION} + * - {@link Errors#NOT_LEADER_OR_FOLLOWER} + * - {@link Errors#UNKNOWN_TOPIC_ID} + * - {@link Errors#INVALID_RECORD_STATE} + * - {@link Errors#KAFKA_STORAGE_ERROR} + * - {@link Errors#INVALID_REQUEST} + * - {@link Errors#UNKNOWN_SERVER_ERROR} + */ +public class ShareAcknowledgeResponse extends AbstractResponse { + + private final ShareAcknowledgeResponseData data; + + public ShareAcknowledgeResponse(ShareAcknowledgeResponseData data) { + super(ApiKeys.SHARE_ACKNOWLEDGE); + this.data = data; + } + + public Errors error() { + return Errors.forCode(data.errorCode()); + } + + @Override + public ShareAcknowledgeResponseData data() { + return data; + } + + @Override + public Map errorCounts() { + HashMap counts = new HashMap<>(); + updateErrorCounts(counts, Errors.forCode(data.errorCode())); + data.responses().forEach( + topic -> topic.partitions().forEach( + partition -> updateErrorCounts(counts, Errors.forCode(partition.errorCode())) + ) + ); + return counts; + } + + @Override + public int throttleTimeMs() { + return data.throttleTimeMs(); + } + + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + + public static ShareAcknowledgeResponse parse(ByteBuffer buffer, short version) { + return new ShareAcknowledgeResponse( + new ShareAcknowledgeResponseData(new ByteBufferAccessor(buffer), version) + ); + } + + private static boolean matchingTopic(ShareAcknowledgeResponseData.ShareAcknowledgeTopicResponse previousTopic, TopicIdPartition currentTopic) { + if (previousTopic == null) + return false; + return previousTopic.topicId().equals(currentTopic.topicId()); + } + + public static ShareAcknowledgeResponseData.PartitionData partitionResponse(TopicIdPartition topicIdPartition, Errors error) { + return partitionResponse(topicIdPartition.topicPartition().partition(), error); + } + + public static ShareAcknowledgeResponseData.PartitionData partitionResponse(int partition, Errors error) { + return new ShareAcknowledgeResponseData.PartitionData() + .setPartitionIndex(partition) + .setErrorCode(error.code()); + } + + public static ShareAcknowledgeResponse of(Errors error, + int throttleTimeMs, + LinkedHashMap responseData, + List nodeEndpoints) { + return new ShareAcknowledgeResponse(toMessage(error, throttleTimeMs, responseData.entrySet().iterator(), nodeEndpoints)); + } + + public static ShareAcknowledgeResponseData toMessage(Errors error, int throttleTimeMs, + Iterator> partIterator, + List nodeEndpoints) { + Map topicResponseList = new LinkedHashMap<>(); + while (partIterator.hasNext()) { + Map.Entry entry = partIterator.next(); + ShareAcknowledgeResponseData.PartitionData partitionData = entry.getValue(); + // Since PartitionData alone doesn't know the partition ID, we set it here + partitionData.setPartitionIndex(entry.getKey().topicPartition().partition()); + // Checking if the topic is already present in the map + if (topicResponseList.containsKey(entry.getKey().topicId())) { + topicResponseList.get(entry.getKey().topicId()).partitions().add(partitionData); + } else { + List partitionResponses = new ArrayList<>(); + partitionResponses.add(partitionData); + topicResponseList.put(entry.getKey().topicId(), new ShareAcknowledgeResponseData.ShareAcknowledgeTopicResponse() + .setTopicId(entry.getKey().topicId()) + .setPartitions(partitionResponses)); + } + } + ShareAcknowledgeResponseData data = new ShareAcknowledgeResponseData(); + // KafkaApis should only pass in node endpoints on error, otherwise this should be an empty list + nodeEndpoints.forEach(endpoint -> data.nodeEndpoints().add( + new ShareAcknowledgeResponseData.NodeEndpoint() + .setNodeId(endpoint.id()) + .setHost(endpoint.host()) + .setPort(endpoint.port()) + .setRack(endpoint.rack()))); + return data.setThrottleTimeMs(throttleTimeMs) + .setErrorCode(error.code()) + .setResponses(new ArrayList<>(topicResponseList.values())); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchMetadata.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchMetadata.java new file mode 100644 index 0000000000000..4e5bcc2237e43 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchMetadata.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.Uuid; + +public class ShareFetchMetadata { + /** + * The first epoch. When used in a ShareFetch request, indicates that the client + * wants to create a session. + */ + public static final int INITIAL_EPOCH = 0; + + /** + * An invalid epoch. When used in a ShareFetch request, indicates that the client + * wants to close an existing session. + */ + public static final int FINAL_EPOCH = -1; + + /** + * + */ + public boolean isNewSession() { + return epoch == INITIAL_EPOCH; + } + + /** + * Returns true if this is a full share fetch request. + */ + public boolean isFull() { + return (this.epoch == INITIAL_EPOCH) || (this.epoch == FINAL_EPOCH); + } + + /** + * Returns the next epoch. + * + * @param prevEpoch The previous epoch. + * @return The next epoch. + */ + public static int nextEpoch(int prevEpoch) { + if (prevEpoch < 0) { + // The next epoch after FINAL_EPOCH is always FINAL_EPOCH itself. + return FINAL_EPOCH; + } else if (prevEpoch == Integer.MAX_VALUE) { + return 1; + } else { + return prevEpoch + 1; + } + } + + /** + * The member ID. + */ + private final Uuid memberId; + + /** + * The share session epoch. + */ + private final int epoch; + + public ShareFetchMetadata(Uuid memberId, int epoch) { + this.memberId = memberId; + this.epoch = epoch; + } + + public static ShareFetchMetadata initialEpoch(Uuid memberId) { + return new ShareFetchMetadata(memberId, INITIAL_EPOCH); + } + + public ShareFetchMetadata nextEpoch() { + return new ShareFetchMetadata(memberId, nextEpoch(epoch)); + } + + public ShareFetchMetadata nextCloseExistingAttemptNew() { + return new ShareFetchMetadata(memberId, INITIAL_EPOCH); + } + + public ShareFetchMetadata finalEpoch() { + return new ShareFetchMetadata(memberId, FINAL_EPOCH); + } + + public Uuid memberId() { + return memberId; + } + + public int epoch() { + return epoch; + } + + public boolean isFinalEpoch() { + return epoch == FINAL_EPOCH; + } + + public String toString() { + StringBuilder bld = new StringBuilder(); + bld.append("(memberId=").append(memberId).append(", "); + if (epoch == INITIAL_EPOCH) { + bld.append("epoch=INITIAL)"); + } else if (epoch == FINAL_EPOCH) { + bld.append("epoch=FINAL)"); + } else { + bld.append("epoch=").append(epoch).append(")"); + } + return bld.toString(); + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchRequest.java new file mode 100644 index 0000000000000..385e802a691a9 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchRequest.java @@ -0,0 +1,267 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.message.ShareFetchRequestData; +import org.apache.kafka.common.message.ShareFetchResponseData; +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; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public class ShareFetchRequest extends AbstractRequest { + + public static class Builder extends AbstractRequest.Builder { + + private final ShareFetchRequestData data; + + public Builder(ShareFetchRequestData data) { + this(data, false); + } + + public Builder(ShareFetchRequestData data, boolean enableUnstableLastVersion) { + super(ApiKeys.SHARE_FETCH, enableUnstableLastVersion); + this.data = data; + } + + public static Builder forConsumer(String groupId, ShareFetchMetadata metadata, + int maxWait, int minBytes, int maxBytes, int fetchSize, + List send, List forget, + Map> acknowledgementsMap) { + ShareFetchRequestData data = new ShareFetchRequestData(); + data.setGroupId(groupId); + int ackOnlyPartitionMaxBytes = fetchSize; + boolean isClosingShareSession = false; + if (metadata != null) { + data.setMemberId(metadata.memberId().toString()); + data.setShareSessionEpoch(metadata.epoch()); + if (metadata.isFinalEpoch()) { + isClosingShareSession = true; + ackOnlyPartitionMaxBytes = 0; + } + } + data.setMaxWaitMs(maxWait); + data.setMinBytes(minBytes); + data.setMaxBytes(maxBytes); + + // Build a map of topics to fetch keyed by topic ID, and within each a map of partitions keyed by index + Map> fetchMap = new HashMap<>(); + + // First, start by adding the list of topic-partitions we are fetching + if (!isClosingShareSession) { + for (TopicIdPartition tip : send) { + Map partMap = fetchMap.computeIfAbsent(tip.topicId(), k -> new HashMap<>()); + ShareFetchRequestData.FetchPartition fetchPartition = new ShareFetchRequestData.FetchPartition() + .setPartitionIndex(tip.partition()) + .setPartitionMaxBytes(fetchSize); + partMap.put(tip.partition(), fetchPartition); + } + } + + // Next, add acknowledgements that we are piggybacking onto the fetch. Generally, the list of + // topic-partitions will be a subset, but if the assignment changes, there might be new entries to add + for (Map.Entry> acknowledgeEntry : acknowledgementsMap.entrySet()) { + TopicIdPartition tip = acknowledgeEntry.getKey(); + Map partMap = fetchMap.computeIfAbsent(tip.topicId(), k -> new HashMap<>()); + ShareFetchRequestData.FetchPartition fetchPartition = partMap.get(tip.partition()); + if (fetchPartition == null) { + fetchPartition = new ShareFetchRequestData.FetchPartition() + .setPartitionIndex(tip.partition()) + .setPartitionMaxBytes(ackOnlyPartitionMaxBytes); + partMap.put(tip.partition(), fetchPartition); + } + fetchPartition.setAcknowledgementBatches(acknowledgeEntry.getValue()); + } + + // Build up the data to fetch + if (!fetchMap.isEmpty()) { + data.setTopics(new ArrayList<>()); + fetchMap.forEach((topicId, partMap) -> { + ShareFetchRequestData.FetchTopic fetchTopic = new ShareFetchRequestData.FetchTopic() + .setTopicId(topicId) + .setPartitions(new ArrayList<>()); + partMap.forEach((index, fetchPartition) -> fetchTopic.partitions().add(fetchPartition)); + data.topics().add(fetchTopic); + }); + } + + // And finally, forget the topic-partitions that are no longer in the session + if (!forget.isEmpty()) { + Map> forgetMap = new HashMap<>(); + for (TopicIdPartition tip : forget) { + List partList = forgetMap.computeIfAbsent(tip.topicId(), k -> new ArrayList<>()); + partList.add(tip.partition()); + } + data.setForgottenTopicsData(new ArrayList<>()); + forgetMap.forEach((topicId, partList) -> { + ShareFetchRequestData.ForgottenTopic forgetTopic = new ShareFetchRequestData.ForgottenTopic() + .setTopicId(topicId) + .setPartitions(new ArrayList<>()); + partList.forEach(index -> forgetTopic.partitions().add(index)); + data.forgottenTopicsData().add(forgetTopic); + }); + } + + return new Builder(data, true); + } + + public ShareFetchRequestData data() { + return data; + } + + @Override + public ShareFetchRequest build(short version) { + return new ShareFetchRequest(data, version); + } + + @Override + public String toString() { + return data.toString(); + } + } + + private final ShareFetchRequestData data; + private volatile LinkedHashMap shareFetchData = null; + private volatile List toForget = null; + + public ShareFetchRequest(ShareFetchRequestData data, short version) { + super(ApiKeys.SHARE_FETCH, version); + this.data = data; + } + + @Override + public ShareFetchRequestData data() { + return data; + } + + @Override + public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { + Errors error = Errors.forException(e); + return new ShareFetchResponse(new ShareFetchResponseData() + .setThrottleTimeMs(throttleTimeMs) + .setErrorCode(error.code())); + } + + public static ShareFetchRequest parse(ByteBuffer buffer, short version) { + return new ShareFetchRequest( + new ShareFetchRequestData(new ByteBufferAccessor(buffer), version), + version + ); + } + + public static final class SharePartitionData { + public final Uuid topicId; + public final int maxBytes; + + public SharePartitionData( + Uuid topicId, + int maxBytes + ) { + this.topicId = topicId; + this.maxBytes = maxBytes; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ShareFetchRequest.SharePartitionData that = (ShareFetchRequest.SharePartitionData) o; + return Objects.equals(topicId, that.topicId) && + maxBytes == that.maxBytes; + } + + @Override + public int hashCode() { + return Objects.hash(topicId, maxBytes); + } + + @Override + public String toString() { + return "SharePartitionData(" + + "topicId=" + topicId + + ", maxBytes=" + maxBytes + + ')'; + } + } + + public int minBytes() { + return data.minBytes(); + } + + public int maxBytes() { + return data.maxBytes(); + } + + public int maxWait() { + return data.maxWaitMs(); + } + + public Map shareFetchData(Map topicNames) { + if (shareFetchData == null) { + synchronized (this) { + if (shareFetchData == null) { + // Assigning the lazy-initialized `shareFetchData` in the last step + // to avoid other threads accessing a half-initialized object. + final LinkedHashMap shareFetchDataTmp = new LinkedHashMap<>(); + data.topics().forEach(shareFetchTopic -> { + String name = topicNames.get(shareFetchTopic.topicId()); + shareFetchTopic.partitions().forEach(shareFetchPartition -> { + // Topic name may be null here if the topic name was unable to be resolved using the topicNames map. + shareFetchDataTmp.put(new TopicIdPartition(shareFetchTopic.topicId(), new TopicPartition(name, shareFetchPartition.partitionIndex())), + new ShareFetchRequest.SharePartitionData( + shareFetchTopic.topicId(), + shareFetchPartition.partitionMaxBytes() + ) + ); + }); + }); + shareFetchData = shareFetchDataTmp; + } + } + } + return shareFetchData; + } + + public List forgottenTopics(Map topicNames) { + if (toForget == null) { + synchronized (this) { + if (toForget == null) { + // Assigning the lazy-initialized `toForget` in the last step + // to avoid other threads accessing a half-initialized object. + final List toForgetTmp = new ArrayList<>(); + data.forgottenTopicsData().forEach(forgottenTopic -> { + String name = topicNames.get(forgottenTopic.topicId()); + // Topic name may be null here if the topic name was unable to be resolved using the topicNames map. + forgottenTopic.partitions().forEach(partitionId -> toForgetTmp.add(new TopicIdPartition(forgottenTopic.topicId(), new TopicPartition(name, partitionId)))); + }); + toForget = toForgetTmp; + } + } + } + return toForget; + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchResponse.java new file mode 100644 index 0000000000000..b33969e0efa41 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchResponse.java @@ -0,0 +1,212 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.Node; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.message.ShareFetchResponseData; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.ObjectSerializationCache; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.Records; + +import java.nio.ByteBuffer; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Iterator; +import java.util.Collections; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; + + +/** + * Possible error codes. + * - {@link Errors#GROUP_AUTHORIZATION_FAILED} + * - {@link Errors#TOPIC_AUTHORIZATION_FAILED} + * - {@link Errors#UNKNOWN_TOPIC_OR_PARTITION} + * - {@link Errors#NOT_LEADER_OR_FOLLOWER} + * - {@link Errors#UNKNOWN_TOPIC_ID} + * - {@link Errors#INVALID_RECORD_STATE} + * - {@link Errors#KAFKA_STORAGE_ERROR} + * - {@link Errors#CORRUPT_MESSAGE} + * - {@link Errors#INVALID_REQUEST} + * - {@link Errors#UNKNOWN_SERVER_ERROR} + */ +public class ShareFetchResponse extends AbstractResponse { + + private final ShareFetchResponseData data; + + private volatile LinkedHashMap responseData = null; + + public ShareFetchResponse(ShareFetchResponseData data) { + super(ApiKeys.SHARE_FETCH); + this.data = data; + } + + public Errors error() { + return Errors.forCode(data.errorCode()); + } + + @Override + public ShareFetchResponseData data() { + return data; + } + + @Override + public Map errorCounts() { + HashMap counts = new HashMap<>(); + updateErrorCounts(counts, Errors.forCode(data.errorCode())); + data.responses().forEach( + topic -> topic.partitions().forEach( + partition -> updateErrorCounts(counts, Errors.forCode(partition.errorCode())) + ) + ); + return counts; + } + + public LinkedHashMap responseData(Map topicNames) { + if (responseData == null) { + synchronized (this) { + // Assigning the lazy-initialized `responseData` in the last step + // to avoid other threads accessing a half-initialized object. + if (responseData == null) { + final LinkedHashMap responseDataTmp = new LinkedHashMap<>(); + data.responses().forEach(topicResponse -> { + String name = topicNames.get(topicResponse.topicId()); + if (name != null) { + topicResponse.partitions().forEach(partitionData -> responseDataTmp.put(new TopicIdPartition(topicResponse.topicId(), + new TopicPartition(name, partitionData.partitionIndex())), partitionData)); + } + }); + responseData = responseDataTmp; + } + } + } + return responseData; + } + + @Override + public int throttleTimeMs() { + return data.throttleTimeMs(); + } + + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + + public static ShareFetchResponse parse(ByteBuffer buffer, short version) { + return new ShareFetchResponse( + new ShareFetchResponseData(new ByteBufferAccessor(buffer), version) + ); + } + + /** + * Returns `partition.records` as `Records` (instead of `BaseRecords`). If `records` is `null`, returns `MemoryRecords.EMPTY`. + * + *

If this response was deserialized after a share fetch, this method should never fail. An example where this would + * fail is a down-converted response (e.g. LazyDownConversionRecords) on the broker (before it's serialized and + * sent on the wire). + * + * @param partition partition data + * @return Records or empty record if the records in PartitionData is null. + */ + public static Records recordsOrFail(ShareFetchResponseData.PartitionData partition) { + if (partition.records() == null) return MemoryRecords.EMPTY; + if (partition.records() instanceof Records) return (Records) partition.records(); + throw new ClassCastException("The record type is " + partition.records().getClass().getSimpleName() + ", which is not a subtype of " + + Records.class.getSimpleName() + ". This method is only safe to call if the `ShareFetchResponse` was deserialized from bytes."); + } + + /** + * Convenience method to find the size of a response. + * + * @param version The version of the request + * @param partIterator The partition iterator. + * @return The response size in bytes. + */ + public static int sizeOf(short version, + Iterator> partIterator) { + // Since the throttleTimeMs and metadata field sizes are constant and fixed, we can + // use arbitrary values here without affecting the result. + ShareFetchResponseData data = toMessage(Errors.NONE, 0, partIterator, Collections.emptyList()); + ObjectSerializationCache cache = new ObjectSerializationCache(); + return 4 + data.size(cache, version); + } + + /** + * @return The size in bytes of the records. 0 is returned if records of input partition is null. + */ + public static int recordsSize(ShareFetchResponseData.PartitionData partition) { + return partition.records() == null ? 0 : partition.records().sizeInBytes(); + } + + public static ShareFetchResponse of(Errors error, + int throttleTimeMs, + LinkedHashMap responseData, + List nodeEndpoints) { + return new ShareFetchResponse(toMessage(error, throttleTimeMs, responseData.entrySet().iterator(), nodeEndpoints)); + } + + public static ShareFetchResponseData toMessage(Errors error, int throttleTimeMs, + Iterator> partIterator, + List nodeEndpoints) { + Map topicResponseList = new LinkedHashMap<>(); + while (partIterator.hasNext()) { + Map.Entry entry = partIterator.next(); + ShareFetchResponseData.PartitionData partitionData = entry.getValue(); + // Since PartitionData alone doesn't know the partition ID, we set it here + partitionData.setPartitionIndex(entry.getKey().topicPartition().partition()); + // Checking if the topic is already present in the map + if (topicResponseList.containsKey(entry.getKey().topicId())) { + topicResponseList.get(entry.getKey().topicId()).partitions().add(partitionData); + } else { + List partitionResponses = new ArrayList<>(); + partitionResponses.add(partitionData); + topicResponseList.put(entry.getKey().topicId(), new ShareFetchResponseData.ShareFetchableTopicResponse() + .setTopicId(entry.getKey().topicId()) + .setPartitions(partitionResponses)); + } + } + ShareFetchResponseData data = new ShareFetchResponseData(); + // KafkaApis should only pass in node endpoints on error, otherwise this should be an empty list + nodeEndpoints.forEach(endpoint -> data.nodeEndpoints().add( + new ShareFetchResponseData.NodeEndpoint() + .setNodeId(endpoint.id()) + .setHost(endpoint.host()) + .setPort(endpoint.port()) + .setRack(endpoint.rack()))); + return data.setThrottleTimeMs(throttleTimeMs) + .setErrorCode(error.code()) + .setResponses(new ArrayList<>(topicResponseList.values())); + } + + public static ShareFetchResponseData.PartitionData partitionResponse(TopicIdPartition topicIdPartition, Errors error) { + return partitionResponse(topicIdPartition.topicPartition().partition(), error); + } + + public static ShareFetchResponseData.PartitionData partitionResponse(int partition, Errors error) { + return new ShareFetchResponseData.PartitionData() + .setPartitionIndex(partition) + .setErrorCode(error.code()); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeRequest.java new file mode 100644 index 0000000000000..25c02e4a83c5e --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeRequest.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.requests; + +import org.apache.kafka.common.message.ShareGroupDescribeRequestData; +import org.apache.kafka.common.message.ShareGroupDescribeResponseData; +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; +import java.util.List; +import java.util.stream.Collectors; + +public class ShareGroupDescribeRequest extends AbstractRequest { + + public static class Builder extends AbstractRequest.Builder { + + private final ShareGroupDescribeRequestData data; + + public Builder(ShareGroupDescribeRequestData data) { + this(data, false); + } + + public Builder(ShareGroupDescribeRequestData data, boolean enableUnstableLastVersion) { + super(ApiKeys.SHARE_GROUP_DESCRIBE, enableUnstableLastVersion); + this.data = data; + } + + @Override + public ShareGroupDescribeRequest build(short version) { + return new ShareGroupDescribeRequest(data, version); + } + + @Override + public String toString() { + return data.toString(); + } + } + + private final ShareGroupDescribeRequestData data; + + public ShareGroupDescribeRequest(ShareGroupDescribeRequestData data, short version) { + super(ApiKeys.SHARE_GROUP_DESCRIBE, version); + this.data = data; + } + + @Override + public ShareGroupDescribeResponse getErrorResponse(int throttleTimeMs, Throwable e) { + ShareGroupDescribeResponseData data = new ShareGroupDescribeResponseData() + .setThrottleTimeMs(throttleTimeMs); + // Set error for each group + short errorCode = Errors.forException(e).code(); + this.data.groupIds().forEach( + groupId -> data.groups().add( + new ShareGroupDescribeResponseData.DescribedGroup() + .setGroupId(groupId) + .setErrorCode(errorCode) + ) + ); + return new ShareGroupDescribeResponse(data); + } + + @Override + public ShareGroupDescribeRequestData data() { + return data; + } + + public static ShareGroupDescribeRequest parse(ByteBuffer buffer, short version) { + return new ShareGroupDescribeRequest( + new ShareGroupDescribeRequestData(new ByteBufferAccessor(buffer), version), + version + ); + } + + public static List getErrorDescribedGroupList( + List groupIds, + Errors error + ) { + return groupIds.stream() + .map(groupId -> new ShareGroupDescribeResponseData.DescribedGroup() + .setGroupId(groupId) + .setErrorCode(error.code()) + ).collect(Collectors.toList()); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeResponse.java new file mode 100644 index 0000000000000..95dd371eedfa7 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeResponse.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.ShareGroupDescribeResponseData; +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; +import java.util.HashMap; +import java.util.Map; + +/** + * Possible error codes. + * + * - {@link Errors#GROUP_AUTHORIZATION_FAILED} + * - {@link Errors#NOT_COORDINATOR} + * - {@link Errors#COORDINATOR_NOT_AVAILABLE} + * - {@link Errors#COORDINATOR_LOAD_IN_PROGRESS} + * - {@link Errors#INVALID_REQUEST} + * - {@link Errors#INVALID_GROUP_ID} + * - {@link Errors#GROUP_ID_NOT_FOUND} + */ +public class ShareGroupDescribeResponse extends AbstractResponse { + + private final ShareGroupDescribeResponseData data; + + public ShareGroupDescribeResponse(ShareGroupDescribeResponseData data) { + super(ApiKeys.SHARE_GROUP_DESCRIBE); + this.data = data; + } + + @Override + public ShareGroupDescribeResponseData data() { + return data; + } + + @Override + public Map errorCounts() { + HashMap counts = new HashMap<>(); + data.groups().forEach( + group -> updateErrorCounts(counts, Errors.forCode(group.errorCode())) + ); + return counts; + } + + @Override + public int throttleTimeMs() { + return data.throttleTimeMs(); + } + + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + + public static ShareGroupDescribeResponse parse(ByteBuffer buffer, short version) { + return new ShareGroupDescribeResponse( + new ShareGroupDescribeResponseData(new ByteBufferAccessor(buffer), version) + ); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupHeartbeatRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupHeartbeatRequest.java new file mode 100644 index 0000000000000..7e112ef29dd14 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupHeartbeatRequest.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.requests; + +import java.nio.ByteBuffer; + +import org.apache.kafka.common.message.ShareGroupHeartbeatRequestData; +import org.apache.kafka.common.message.ShareGroupHeartbeatResponseData; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.common.protocol.Errors; + +public class ShareGroupHeartbeatRequest extends AbstractRequest { + /** + * A member epoch of -1 means that the member wants to leave the group. + */ + public static final int LEAVE_GROUP_MEMBER_EPOCH = -1; + + /** + * A member epoch of 0 means that the member wants to join the group. + */ + public static final int JOIN_GROUP_MEMBER_EPOCH = 0; + + public static class Builder extends AbstractRequest.Builder { + private final ShareGroupHeartbeatRequestData data; + + public Builder(ShareGroupHeartbeatRequestData data) { + this(data, true); + } + + public Builder(ShareGroupHeartbeatRequestData data, boolean enableUnstableLastVersion) { + super(ApiKeys.SHARE_GROUP_HEARTBEAT, enableUnstableLastVersion); + this.data = data; + } + + @Override + public ShareGroupHeartbeatRequest build(short version) { + return new ShareGroupHeartbeatRequest(data, version); + } + + @Override + public String toString() { + return data.toString(); + } + } + + private final ShareGroupHeartbeatRequestData data; + + public ShareGroupHeartbeatRequest(ShareGroupHeartbeatRequestData data, short version) { + super(ApiKeys.SHARE_GROUP_HEARTBEAT, version); + this.data = data; + } + + @Override + public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { + return new ShareGroupHeartbeatResponse( + new ShareGroupHeartbeatResponseData() + .setThrottleTimeMs(throttleTimeMs) + .setErrorCode(Errors.forException(e).code()) + ); + } + + @Override + public ShareGroupHeartbeatRequestData data() { + return data; + } + + public static ShareGroupHeartbeatRequest parse(ByteBuffer buffer, short version) { + return new ShareGroupHeartbeatRequest(new ShareGroupHeartbeatRequestData( + new ByteBufferAccessor(buffer), version), version); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupHeartbeatResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupHeartbeatResponse.java new file mode 100644 index 0000000000000..de05d44aebecb --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupHeartbeatResponse.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.ShareGroupHeartbeatResponseData; +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; +import java.util.Collections; +import java.util.Map; + +/** + * Possible error codes. + * + * - {@link Errors#GROUP_AUTHORIZATION_FAILED} + * - {@link Errors#NOT_COORDINATOR} + * - {@link Errors#COORDINATOR_NOT_AVAILABLE} + * - {@link Errors#COORDINATOR_LOAD_IN_PROGRESS} + * - {@link Errors#INVALID_REQUEST} + * - {@link Errors#UNKNOWN_MEMBER_ID} + * - {@link Errors#GROUP_MAX_SIZE_REACHED} + */ +public class ShareGroupHeartbeatResponse extends AbstractResponse { + private final ShareGroupHeartbeatResponseData data; + + public ShareGroupHeartbeatResponse(ShareGroupHeartbeatResponseData data) { + super(ApiKeys.SHARE_GROUP_HEARTBEAT); + this.data = data; + } + + @Override + public ShareGroupHeartbeatResponseData data() { + return data; + } + + @Override + public Map errorCounts() { + return Collections.singletonMap(Errors.forCode(data.errorCode()), 1); + } + + @Override + public int throttleTimeMs() { + return data.throttleTimeMs(); + } + + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + + public static ShareGroupHeartbeatResponse parse(ByteBuffer buffer, short version) { + return new ShareGroupHeartbeatResponse(new ShareGroupHeartbeatResponseData( + new ByteBufferAccessor(buffer), version)); + } +} diff --git a/clients/src/main/resources/common/message/FindCoordinatorRequest.json b/clients/src/main/resources/common/message/FindCoordinatorRequest.json index 42b2f4c891ad5..43e6fe5014b26 100644 --- a/clients/src/main/resources/common/message/FindCoordinatorRequest.json +++ b/clients/src/main/resources/common/message/FindCoordinatorRequest.json @@ -27,7 +27,9 @@ // Version 4 adds support for batching via CoordinatorKeys (KIP-699) // // Version 5 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). - "validVersions": "0-5", + // + // Version 6 adds support for share groups (KIP-932). + "validVersions": "0-6", "deprecatedVersions": "0", "flexibleVersions": "3+", "fields": [ diff --git a/clients/src/main/resources/common/message/FindCoordinatorResponse.json b/clients/src/main/resources/common/message/FindCoordinatorResponse.json index 860d655a252b2..be0479f908c96 100644 --- a/clients/src/main/resources/common/message/FindCoordinatorResponse.json +++ b/clients/src/main/resources/common/message/FindCoordinatorResponse.json @@ -26,7 +26,9 @@ // Version 4 adds support for batching via Coordinators (KIP-699) // // Version 5 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). - "validVersions": "0-5", + // + // Version 6 adds support for share groups (KIP-932). + "validVersions": "0-6", "flexibleVersions": "3+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, diff --git a/clients/src/main/resources/common/message/ListGroupsRequest.json b/clients/src/main/resources/common/message/ListGroupsRequest.json index 32defaa203382..a872165d516cf 100644 --- a/clients/src/main/resources/common/message/ListGroupsRequest.json +++ b/clients/src/main/resources/common/message/ListGroupsRequest.json @@ -25,7 +25,9 @@ // Version 4 adds the StatesFilter field (KIP-518). // // Version 5 adds the TypesFilter field (KIP-848). - "validVersions": "0-5", + // + // Version 6 adds support for share groups (KIP-932). + "validVersions": "0-6", "flexibleVersions": "3+", "fields": [ { "name": "StatesFilter", "type": "[]string", "versions": "4+", diff --git a/clients/src/main/resources/common/message/ListGroupsResponse.json b/clients/src/main/resources/common/message/ListGroupsResponse.json index fc4077c080f46..77f1c89e34a38 100644 --- a/clients/src/main/resources/common/message/ListGroupsResponse.json +++ b/clients/src/main/resources/common/message/ListGroupsResponse.json @@ -27,7 +27,9 @@ // Version 4 adds the GroupState field (KIP-518). // // Version 5 adds the GroupType field (KIP-848). - "validVersions": "0-5", + // + // Version 6 adds support for share groups (KIP-932). + "validVersions": "0-6", "flexibleVersions": "3+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, diff --git a/clients/src/main/resources/common/message/ShareAcknowledgeRequest.json b/clients/src/main/resources/common/message/ShareAcknowledgeRequest.json new file mode 100644 index 0000000000000..db534cb4c1c13 --- /dev/null +++ b/clients/src/main/resources/common/message/ShareAcknowledgeRequest.json @@ -0,0 +1,53 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT 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": 79, + "type": "request", + "listeners": ["broker"], + "name": "ShareAcknowledgeRequest", + "validVersions": "0", + "flexibleVersions": "0+", + // The ShareAcknowledgeRequest API is added as part of KIP-932 and is still under + // development. Hence, the API is not exposed by default by brokers unless + // explicitly enabled. + "latestVersionUnstable": true, + "fields": [ + { "name": "GroupId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", "entityType": "groupId", + "about": "The group identifier." }, + { "name": "MemberId", "type": "string", "versions": "0+", "nullableVersions": "0+", + "about": "The member ID." }, + { "name": "ShareSessionEpoch", "type": "int32", "versions": "0+", + "about": "The current share session epoch: 0 to open a share session; -1 to close it; otherwise increments for consecutive requests." }, + { "name": "Topics", "type": "[]AcknowledgeTopic", "versions": "0+", + "about": "The topics containing records to acknowledge.", "fields": [ + { "name": "TopicId", "type": "uuid", "versions": "0+", "about": "The unique topic ID."}, + { "name": "Partitions", "type": "[]AcknowledgePartition", "versions": "0+", + "about": "The partitions containing records to acknowledge.", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "AcknowledgementBatches", "type": "[]AcknowledgementBatch", "versions": "0+", + "about": "Record batches to acknowledge.", "fields": [ + { "name": "FirstOffset", "type": "int64", "versions": "0+", + "about": "First offset of batch of records to acknowledge."}, + { "name": "LastOffset", "type": "int64", "versions": "0+", + "about": "Last offset (inclusive) of batch of records to acknowledge."}, + { "name": "AcknowledgeTypes", "type": "[]int8", "versions": "0+", + "about": "Array of acknowledge types - 0:Gap,1:Accept,2:Release,3:Reject."} + ]} + ]} + ]} + ] +} \ No newline at end of file diff --git a/clients/src/main/resources/common/message/ShareAcknowledgeResponse.json b/clients/src/main/resources/common/message/ShareAcknowledgeResponse.json new file mode 100644 index 0000000000000..638ca10c64b3b --- /dev/null +++ b/clients/src/main/resources/common/message/ShareAcknowledgeResponse.json @@ -0,0 +1,72 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT 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": 79, + "type": "response", + "name": "ShareAcknowledgeResponse", + "validVersions": "0", + "flexibleVersions": "0+", + // Supported errors: + // - GROUP_AUTHORIZATION_FAILED (version 0+) + // - TOPIC_AUTHORIZATION_FAILED (version 0+) + // - UNKNOWN_TOPIC_OR_PARTITION (version 0+) + // - SHARE_SESSION_NOT_FOUND (version 0+) + // - INVALID_SHARE_SESSION_EPOCH (version 0+) + // - NOT_LEADER_OR_FOLLOWER (version 0+) + // - UNKNOWN_TOPIC_ID (version 0+) + // - INVALID_RECORD_STATE (version 0+) + // - KAFKA_STORAGE_ERROR (version 0+) + // - INVALID_REQUEST (version 0+) + // - UNKNOWN_SERVER_ERROR (version 0+) + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "ignorable": true, + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", "ignorable": true, + "about": "The top level response error code." }, + { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "The top-level error message, or null if there was no error." }, + { "name": "Responses", "type": "[]ShareAcknowledgeTopicResponse", "versions": "0+", + "about": "The response topics.", "fields": [ + { "name": "TopicId", "type": "uuid", "versions": "0+", "ignorable": true, "about": "The unique topic ID."}, + { "name": "Partitions", "type": "[]PartitionData", "versions": "0+", + "about": "The topic partitions.", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The error code, or 0 if there was no error." }, + { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "The error message, or null if there was no error." }, + { "name": "CurrentLeader", "type": "LeaderIdAndEpoch", "versions": "0+", "fields": [ + { "name": "LeaderId", "type": "int32", "versions": "0+", + "about": "The ID of the current leader or -1 if the leader is unknown." }, + { "name": "LeaderEpoch", "type": "int32", "versions": "0+", + "about": "The latest known leader epoch." } + ]} + ]} + ]}, + { "name": "NodeEndpoints", "type": "[]NodeEndpoint", "versions": "0+", + "about": "Endpoints for all current leaders enumerated in PartitionData with error NOT_LEADER_OR_FOLLOWER.", "fields": [ + { "name": "NodeId", "type": "int32", "versions": "0+", + "mapKey": true, "entityType": "brokerId", "about": "The ID of the associated node." }, + { "name": "Host", "type": "string", "versions": "0+", + "about": "The node's hostname." }, + { "name": "Port", "type": "int32", "versions": "0+", + "about": "The node's port." }, + { "name": "Rack", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "The rack of the node, or null if it has not been assigned to a rack." } + ]} + ] +} \ No newline at end of file diff --git a/clients/src/main/resources/common/message/ShareFetchRequest.json b/clients/src/main/resources/common/message/ShareFetchRequest.json new file mode 100644 index 0000000000000..d0b59dcb26a80 --- /dev/null +++ b/clients/src/main/resources/common/message/ShareFetchRequest.json @@ -0,0 +1,67 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT 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": 78, + "type": "request", + "listeners": ["broker"], + "name": "ShareFetchRequest", + "validVersions": "0", + "flexibleVersions": "0+", + // The ShareFetchRequest API is added as part of KIP-932 and is still under + // development. Hence, the API is not exposed by default by brokers unless + // explicitly enabled. + "latestVersionUnstable": true, + "fields": [ + { "name": "GroupId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", "entityType": "groupId", + "about": "The group identifier." }, + { "name": "MemberId", "type": "string", "versions": "0+", "nullableVersions": "0+", + "about": "The member ID." }, + { "name": "ShareSessionEpoch", "type": "int32", "versions": "0+", + "about": "The current share session epoch: 0 to open a share session; -1 to close it; otherwise increments for consecutive requests." }, + { "name": "MaxWaitMs", "type": "int32", "versions": "0+", + "about": "The maximum time in milliseconds to wait for the response." }, + { "name": "MinBytes", "type": "int32", "versions": "0+", + "about": "The minimum bytes to accumulate in the response." }, + { "name": "MaxBytes", "type": "int32", "versions": "0+", "default": "0x7fffffff", "ignorable": true, + "about": "The maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored." }, + { "name": "Topics", "type": "[]FetchTopic", "versions": "0+", + "about": "The topics to fetch.", "fields": [ + { "name": "TopicId", "type": "uuid", "versions": "0+", "ignorable": true, "about": "The unique topic ID."}, + { "name": "Partitions", "type": "[]FetchPartition", "versions": "0+", + "about": "The partitions to fetch.", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "PartitionMaxBytes", "type": "int32", "versions": "0+", + "about": "The maximum bytes to fetch from this partition. 0 when only acknowledgement with no fetching is required. See KIP-74 for cases where this limit may not be honored." }, + { "name": "AcknowledgementBatches", "type": "[]AcknowledgementBatch", "versions": "0+", + "about": "Record batches to acknowledge.", "fields": [ + { "name": "FirstOffset", "type": "int64", "versions": "0+", + "about": "First offset of batch of records to acknowledge."}, + { "name": "LastOffset", "type": "int64", "versions": "0+", + "about": "Last offset (inclusive) of batch of records to acknowledge."}, + { "name": "AcknowledgeTypes", "type": "[]int8", "versions": "0+", + "about": "Array of acknowledge types - 0:Gap,1:Accept,2:Release,3:Reject."} + ]} + ]} + ]}, + { "name": "ForgottenTopicsData", "type": "[]ForgottenTopic", "versions": "0+", "ignorable": false, + "about": "The partitions to remove from this share session.", "fields": [ + { "name": "TopicId", "type": "uuid", "versions": "0+", "ignorable": true, "about": "The unique topic ID."}, + { "name": "Partitions", "type": "[]int32", "versions": "0+", + "about": "The partitions indexes to forget." } + ]} + ] +} \ No newline at end of file diff --git a/clients/src/main/resources/common/message/ShareFetchResponse.json b/clients/src/main/resources/common/message/ShareFetchResponse.json new file mode 100644 index 0000000000000..5338e1208a7bc --- /dev/null +++ b/clients/src/main/resources/common/message/ShareFetchResponse.json @@ -0,0 +1,83 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 78, + "type": "response", + "name": "ShareFetchResponse", + "validVersions": "0", + "flexibleVersions": "0+", + // Supported errors for ErrorCode and AcknowledgeErrorCode: + // - GROUP_AUTHORIZATION_FAILED (version 0+) + // - TOPIC_AUTHORIZATION_FAILED (version 0+) + // - SHARE_SESSION_NOT_FOUND (version 0+) + // - INVALID_SHARE_SESSION_EPOCH (version 0+) + // - UNKNOWN_TOPIC_OR_PARTITION (version 0+) + // - NOT_LEADER_OR_FOLLOWER (version 0+) + // - UNKNOWN_TOPIC_ID (version 0+) + // - INVALID_RECORD_STATE (version 0+) - only for AcknowledgeErrorCode + // - KAFKA_STORAGE_ERROR (version 0+) + // - CORRUPT_MESSAGE (version 0+) + // - INVALID_REQUEST (version 0+) + // - UNKNOWN_SERVER_ERROR (version 0+) + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "ignorable": true, + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", "ignorable": true, + "about": "The top-level response error code." }, + { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "The top-level error message, or null if there was no error." }, + { "name": "Responses", "type": "[]ShareFetchableTopicResponse", "versions": "0+", + "about": "The response topics.", "fields": [ + { "name": "TopicId", "type": "uuid", "versions": "0+", "ignorable": true, "about": "The unique topic ID."}, + { "name": "Partitions", "type": "[]PartitionData", "versions": "0+", + "about": "The topic partitions.", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The fetch error code, or 0 if there was no fetch error." }, + { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "The fetch error message, or null if there was no fetch error." }, + { "name": "AcknowledgeErrorCode", "type": "int16", "versions": "0+", + "about": "The acknowledge error code, or 0 if there was no acknowledge error." }, + { "name": "AcknowledgeErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "The acknowledge error message, or null if there was no acknowledge error." }, + { "name": "CurrentLeader", "type": "LeaderIdAndEpoch", "versions": "0+", "fields": [ + { "name": "LeaderId", "type": "int32", "versions": "0+", + "about": "The ID of the current leader or -1 if the leader is unknown." }, + { "name": "LeaderEpoch", "type": "int32", "versions": "0+", + "about": "The latest known leader epoch." } + ]}, + { "name": "Records", "type": "records", "versions": "0+", "nullableVersions": "0+", "about": "The record data."}, + { "name": "AcquiredRecords", "type": "[]AcquiredRecords", "versions": "0+", "about": "The acquired records.", "fields": [ + {"name": "FirstOffset", "type": "int64", "versions": "0+", "about": "The earliest offset in this batch of acquired records."}, + {"name": "LastOffset", "type": "int64", "versions": "0+", "about": "The last offset of this batch of acquired records."}, + {"name": "DeliveryCount", "type": "int16", "versions": "0+", "about": "The delivery count of this batch of acquired records."} + ]} + ]} + ]}, + { "name": "NodeEndpoints", "type": "[]NodeEndpoint", "versions": "0+", + "about": "Endpoints for all current leaders enumerated in PartitionData with error NOT_LEADER_OR_FOLLOWER.", "fields": [ + { "name": "NodeId", "type": "int32", "versions": "0+", + "mapKey": true, "entityType": "brokerId", "about": "The ID of the associated node." }, + { "name": "Host", "type": "string", "versions": "0+", + "about": "The node's hostname." }, + { "name": "Port", "type": "int32", "versions": "0+", + "about": "The node's port." }, + { "name": "Rack", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "The rack of the node, or null if it has not been assigned to a rack." } + ]} + ] +} \ No newline at end of file diff --git a/clients/src/main/resources/common/message/ShareGroupDescribeRequest.json b/clients/src/main/resources/common/message/ShareGroupDescribeRequest.json new file mode 100644 index 0000000000000..c95790c9b198f --- /dev/null +++ b/clients/src/main/resources/common/message/ShareGroupDescribeRequest.json @@ -0,0 +1,33 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 77, + "type": "request", + "listeners": ["broker"], + "name": "ShareGroupDescribeRequest", + "validVersions": "0", + "flexibleVersions": "0+", + // The ShareGroupDescribeRequest API is added as part of KIP-932 and is still under + // development. Hence, the API is not exposed by default by brokers unless + // explicitly enabled. + "latestVersionUnstable": true, + "fields": [ + { "name": "GroupIds", "type": "[]string", "versions": "0+", "entityType": "groupId", + "about": "The ids of the groups to describe" }, + { "name": "IncludeAuthorizedOperations", "type": "bool", "versions": "0+", + "about": "Whether to include authorized operations." } + ] +} \ No newline at end of file diff --git a/clients/src/main/resources/common/message/ShareGroupDescribeResponse.json b/clients/src/main/resources/common/message/ShareGroupDescribeResponse.json new file mode 100644 index 0000000000000..c093b788bfc2f --- /dev/null +++ b/clients/src/main/resources/common/message/ShareGroupDescribeResponse.json @@ -0,0 +1,87 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 77, + "type": "response", + "name": "ShareGroupDescribeResponse", + "validVersions": "0", + "flexibleVersions": "0+", + // Supported errors: + // - GROUP_AUTHORIZATION_FAILED (version 0+) + // - NOT_COORDINATOR (version 0+) + // - COORDINATOR_NOT_AVAILABLE (version 0+) + // - COORDINATOR_LOAD_IN_PROGRESS (version 0+) + // - INVALID_REQUEST (version 0+) + // - INVALID_GROUP_ID (version 0+) + // - GROUP_ID_NOT_FOUND (version 0+) + "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": "Groups", "type": "[]DescribedGroup", "versions": "0+", + "about": "Each described group.", + "fields": [ + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The describe error, 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": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId", + "about": "The group ID string." }, + { "name": "GroupState", "type": "string", "versions": "0+", + "about": "The group state string, or the empty string." }, + { "name": "GroupEpoch", "type": "int32", "versions": "0+", + "about": "The group epoch." }, + { "name": "AssignmentEpoch", "type": "int32", "versions": "0+", + "about": "The assignment epoch." }, + { "name": "AssignorName", "type": "string", "versions": "0+", + "about": "The selected assignor." }, + { "name": "Members", "type": "[]Member", "versions": "0+", + "about": "The members.", + "fields": [ + { "name": "MemberId", "type": "string", "versions": "0+", + "about": "The member ID." }, + { "name": "RackId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "The member rack ID." }, + { "name": "MemberEpoch", "type": "int32", "versions": "0+", + "about": "The current member epoch." }, + { "name": "ClientId", "type": "string", "versions": "0+", + "about": "The client ID." }, + { "name": "ClientHost", "type": "string", "versions": "0+", + "about": "The client host." }, + { "name": "SubscribedTopicNames", "type": "[]string", "versions": "0+", "entityType": "topicName", + "about": "The subscribed topic names." }, + { "name": "Assignment", "type": "Assignment", "versions": "0+", + "about": "The current assignment." } + ]}, + { "name": "AuthorizedOperations", "type": "int32", "versions": "0+", "default": "-2147483648", + "about": "32-bit bitfield to represent authorized operations for this group." } + ] + } + ], + "commonStructs": [ + { "name": "TopicPartitions", "versions": "0+", "fields": [ + { "name": "TopicId", "type": "uuid", "versions": "0+", + "about": "The topic ID." }, + { "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName", + "about": "The topic name." }, + { "name": "Partitions", "type": "[]int32", "versions": "0+", + "about": "The partitions." } + ]}, + { "name": "Assignment", "versions": "0+", "fields": [ + { "name": "TopicPartitions", "type": "[]TopicPartitions", "versions": "0+", + "about": "The assigned topic-partitions to the member." } + ]} + ] +} \ No newline at end of file diff --git a/clients/src/main/resources/common/message/ShareGroupHeartbeatRequest.json b/clients/src/main/resources/common/message/ShareGroupHeartbeatRequest.json new file mode 100644 index 0000000000000..7d28c116454d3 --- /dev/null +++ b/clients/src/main/resources/common/message/ShareGroupHeartbeatRequest.json @@ -0,0 +1,39 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT 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": 76, + "type": "request", + "listeners": ["broker"], + "name": "ShareGroupHeartbeatRequest", + "validVersions": "0", + "flexibleVersions": "0+", + // The ShareGroupHeartbeatRequest API is added as part of KIP-932 and is still under + // development. Hence, the API is not exposed by default by brokers unless + // explicitly enabled. + "latestVersionUnstable": true, + "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." }, + { "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": "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." } + ] +} \ No newline at end of file diff --git a/clients/src/main/resources/common/message/ShareGroupHeartbeatResponse.json b/clients/src/main/resources/common/message/ShareGroupHeartbeatResponse.json new file mode 100644 index 0000000000000..e692839f29bf9 --- /dev/null +++ b/clients/src/main/resources/common/message/ShareGroupHeartbeatResponse.json @@ -0,0 +1,57 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 76, + "type": "response", + "name": "ShareGroupHeartbeatResponse", + "validVersions": "0", + "flexibleVersions": "0+", + // Supported errors: + // - GROUP_AUTHORIZATION_FAILED (version 0+) + // - NOT_COORDINATOR (version 0+) + // - COORDINATOR_NOT_AVAILABLE (version 0+) + // - COORDINATOR_LOAD_IN_PROGRESS (version 0+) + // - INVALID_REQUEST (version 0+) + // - UNKNOWN_MEMBER_ID (version 0+) + // - GROUP_MAX_SIZE_REACHED (version 0+) + "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": "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": "TopicPartitions", "type": "[]TopicPartitions", "versions": "0+", + "about": "The partitions assigned to the member." } + ]} + ], + "commonStructs": [ + { "name": "TopicPartitions", "versions": "0+", "fields": [ + { "name": "TopicId", "type": "uuid", "versions": "0+", + "about": "The topic ID." }, + { "name": "Partitions", "type": "[]int32", "versions": "0+", + "about": "The partitions." } + ]} + ] +} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 512a7cea76681..82487bd418429 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.ElectionType; import org.apache.kafka.common.IsolationLevel; import org.apache.kafka.common.Node; +import org.apache.kafka.common.ShareGroupState; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.Uuid; @@ -210,6 +211,14 @@ import org.apache.kafka.common.message.SaslAuthenticateResponseData; import org.apache.kafka.common.message.SaslHandshakeRequestData; import org.apache.kafka.common.message.SaslHandshakeResponseData; +import org.apache.kafka.common.message.ShareAcknowledgeRequestData; +import org.apache.kafka.common.message.ShareAcknowledgeResponseData; +import org.apache.kafka.common.message.ShareFetchRequestData; +import org.apache.kafka.common.message.ShareFetchResponseData; +import org.apache.kafka.common.message.ShareGroupDescribeRequestData; +import org.apache.kafka.common.message.ShareGroupDescribeResponseData; +import org.apache.kafka.common.message.ShareGroupHeartbeatRequestData; +import org.apache.kafka.common.message.ShareGroupHeartbeatResponseData; import org.apache.kafka.common.message.StopReplicaRequestData.StopReplicaPartitionState; import org.apache.kafka.common.message.StopReplicaRequestData.StopReplicaTopicState; import org.apache.kafka.common.message.StopReplicaResponseData; @@ -1001,6 +1010,10 @@ public void testErrorCountsIncludesNone() { assertEquals(1, createTxnOffsetCommitResponse().errorCounts().get(Errors.NONE)); assertEquals(1, createUpdateMetadataResponse().errorCounts().get(Errors.NONE)); assertEquals(1, createWriteTxnMarkersResponse().errorCounts().get(Errors.NONE)); + assertEquals(1, createShareGroupHeartbeatResponse().errorCounts().get(Errors.NONE)); + assertEquals(1, createShareGroupDescribeResponse().errorCounts().get(Errors.NONE)); + assertEquals(2, createShareFetchResponse().errorCounts().get(Errors.NONE)); + assertEquals(2, createShareAcknowledgeResponse().errorCounts().get(Errors.NONE)); } private AbstractRequest getRequest(ApiKeys apikey, short version) { @@ -1081,6 +1094,10 @@ private AbstractRequest getRequest(ApiKeys apikey, short version) { case ASSIGN_REPLICAS_TO_DIRS: return createAssignReplicasToDirsRequest(version); case LIST_CLIENT_METRICS_RESOURCES: return createListClientMetricsResourcesRequest(version); case DESCRIBE_TOPIC_PARTITIONS: return createDescribeTopicPartitionsRequest(version); + case SHARE_GROUP_HEARTBEAT: return createShareGroupHeartbeatRequest(version); + case SHARE_GROUP_DESCRIBE: return createShareGroupDescribeRequest(version); + case SHARE_FETCH: return createShareFetchRequest(version); + case SHARE_ACKNOWLEDGE: return createShareAcknowledgeRequest(version); default: throw new IllegalArgumentException("Unknown API key " + apikey); } } @@ -1163,6 +1180,10 @@ private AbstractResponse getResponse(ApiKeys apikey, short version) { case ASSIGN_REPLICAS_TO_DIRS: return createAssignReplicasToDirsResponse(); case LIST_CLIENT_METRICS_RESOURCES: return createListClientMetricsResourcesResponse(); case DESCRIBE_TOPIC_PARTITIONS: return createDescribeTopicPartitionsResponse(); + case SHARE_GROUP_HEARTBEAT: return createShareGroupHeartbeatResponse(); + case SHARE_GROUP_DESCRIBE: return createShareGroupDescribeResponse(); + case SHARE_FETCH: return createShareFetchResponse(); + case SHARE_ACKNOWLEDGE: return createShareAcknowledgeResponse(); default: throw new IllegalArgumentException("Unknown API key " + apikey); } } @@ -1330,6 +1351,114 @@ private ConsumerGroupHeartbeatResponse createConsumerGroupHeartbeatResponse() { return new ConsumerGroupHeartbeatResponse(data); } + private ShareGroupHeartbeatRequest createShareGroupHeartbeatRequest(short version) { + ShareGroupHeartbeatRequestData data = new ShareGroupHeartbeatRequestData() + .setGroupId("group") + .setMemberId("memberid") + .setMemberEpoch(10) + .setRackId("rackid") + .setSubscribedTopicNames(Arrays.asList("foo", "bar")); + return new ShareGroupHeartbeatRequest.Builder(data).build(version); + } + + private ShareGroupHeartbeatResponse createShareGroupHeartbeatResponse() { + ShareGroupHeartbeatResponseData data = new ShareGroupHeartbeatResponseData() + .setErrorCode(Errors.NONE.code()) + .setThrottleTimeMs(1000) + .setMemberId("memberid") + .setMemberEpoch(11) + .setAssignment(new ShareGroupHeartbeatResponseData.Assignment() + .setTopicPartitions(Arrays.asList( + new ShareGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(Uuid.randomUuid()) + .setPartitions(Arrays.asList(0, 1, 2)), + new ShareGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(Uuid.randomUuid()) + .setPartitions(Arrays.asList(3, 4, 5)) + )) + ); + return new ShareGroupHeartbeatResponse(data); + } + + private ShareGroupDescribeRequest createShareGroupDescribeRequest(short version) { + ShareGroupDescribeRequestData data = new ShareGroupDescribeRequestData() + .setGroupIds(Collections.singletonList("group")) + .setIncludeAuthorizedOperations(false); + return new ShareGroupDescribeRequest.Builder(data).build(version); + } + + private ShareGroupDescribeResponse createShareGroupDescribeResponse() { + ShareGroupDescribeResponseData data = new ShareGroupDescribeResponseData() + .setGroups(Collections.singletonList( + new ShareGroupDescribeResponseData.DescribedGroup() + .setGroupId("group") + .setErrorCode((short) 0) + .setErrorMessage(Errors.forCode((short) 0).message()) + .setGroupState(ShareGroupState.EMPTY.toString()) + .setMembers(new ArrayList<>(0)) + )) + .setThrottleTimeMs(1000); + return new ShareGroupDescribeResponse(data); + } + + private ShareFetchRequest createShareFetchRequest(short version) { + ShareFetchRequestData data = new ShareFetchRequestData() + .setGroupId("group") + .setMemberId(Uuid.randomUuid().toString()) + .setTopics(singletonList(new ShareFetchRequestData.FetchTopic() + .setTopicId(Uuid.randomUuid()) + .setPartitions(singletonList(new ShareFetchRequestData.FetchPartition() + .setPartitionIndex(0))))); + return new ShareFetchRequest.Builder(data).build(version); + } + + private ShareFetchResponse createShareFetchResponse() { + ShareFetchResponseData data = new ShareFetchResponseData(); + MemoryRecords records = MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("blah".getBytes())); + ShareFetchResponseData.PartitionData partition = new ShareFetchResponseData.PartitionData() + .setPartitionIndex(0) + .setErrorCode(Errors.NONE.code()) + .setRecords(records) + .setAcquiredRecords(singletonList(new ShareFetchResponseData.AcquiredRecords() + .setFirstOffset(0) + .setLastOffset(0) + .setDeliveryCount((short) 1))); + ShareFetchResponseData.ShareFetchableTopicResponse response = new ShareFetchResponseData.ShareFetchableTopicResponse() + .setTopicId(Uuid.randomUuid()) + .setPartitions(singletonList(partition)); + + data.setResponses(singletonList(response)); + data.setThrottleTimeMs(345); + data.setErrorCode(Errors.NONE.code()); + return new ShareFetchResponse(data); + } + + private ShareAcknowledgeRequest createShareAcknowledgeRequest(short version) { + ShareAcknowledgeRequestData data = new ShareAcknowledgeRequestData() + .setMemberId(Uuid.randomUuid().toString()) + .setTopics(singletonList(new ShareAcknowledgeRequestData.AcknowledgeTopic() + .setTopicId(Uuid.randomUuid()) + .setPartitions(singletonList(new ShareAcknowledgeRequestData.AcknowledgePartition() + .setPartitionIndex(0) + .setAcknowledgementBatches(singletonList(new ShareAcknowledgeRequestData.AcknowledgementBatch() + .setFirstOffset(0) + .setLastOffset(0) + .setAcknowledgeTypes(Collections.singletonList((byte) 0)))))))); + return new ShareAcknowledgeRequest.Builder(data).build(version); + } + + private ShareAcknowledgeResponse createShareAcknowledgeResponse() { + ShareAcknowledgeResponseData data = new ShareAcknowledgeResponseData(); + data.setResponses(singletonList(new ShareAcknowledgeResponseData.ShareAcknowledgeTopicResponse() + .setTopicId(Uuid.randomUuid()) + .setPartitions(singletonList(new ShareAcknowledgeResponseData.PartitionData() + .setPartitionIndex(0) + .setErrorCode(Errors.NONE.code()))))); + data.setThrottleTimeMs(345); + data.setErrorCode(Errors.NONE.code()); + return new ShareAcknowledgeResponse(data); + } + private ControllerRegistrationRequest createControllerRegistrationRequest(short version) { ControllerRegistrationRequestData data = new ControllerRegistrationRequestData(). setControllerId(3). diff --git a/core/src/main/scala/kafka/network/RequestConvertToJson.scala b/core/src/main/scala/kafka/network/RequestConvertToJson.scala index 54986f52c85a3..0900b94ef9f4f 100644 --- a/core/src/main/scala/kafka/network/RequestConvertToJson.scala +++ b/core/src/main/scala/kafka/network/RequestConvertToJson.scala @@ -95,6 +95,10 @@ object RequestConvertToJson { case req: RenewDelegationTokenRequest => RenewDelegationTokenRequestDataJsonConverter.write(req.data, request.version) case req: SaslAuthenticateRequest => SaslAuthenticateRequestDataJsonConverter.write(req.data, request.version) case req: SaslHandshakeRequest => SaslHandshakeRequestDataJsonConverter.write(req.data, request.version) + case req: ShareAcknowledgeRequest => ShareAcknowledgeRequestDataJsonConverter.write(req.data, request.version) + case req: ShareFetchRequest => ShareFetchRequestDataJsonConverter.write(req.data, request.version) + case req: ShareGroupDescribeRequest => ShareGroupDescribeRequestDataJsonConverter.write(req.data, request.version) + case req: ShareGroupHeartbeatRequest => ShareGroupHeartbeatRequestDataJsonConverter.write(req.data, request.version) case req: StopReplicaRequest => StopReplicaRequestDataJsonConverter.write(req.data, request.version) case req: SyncGroupRequest => SyncGroupRequestDataJsonConverter.write(req.data, request.version) case req: TxnOffsetCommitRequest => TxnOffsetCommitRequestDataJsonConverter.write(req.data, request.version) @@ -178,6 +182,10 @@ object RequestConvertToJson { case res: RenewDelegationTokenResponse => RenewDelegationTokenResponseDataJsonConverter.write(res.data, version) case res: SaslAuthenticateResponse => SaslAuthenticateResponseDataJsonConverter.write(res.data, version) case res: SaslHandshakeResponse => SaslHandshakeResponseDataJsonConverter.write(res.data, version) + case res: ShareAcknowledgeResponse => ShareAcknowledgeResponseDataJsonConverter.write(res.data, version) + case res: ShareFetchResponse => ShareFetchResponseDataJsonConverter.write(res.data, version) + case res: ShareGroupDescribeResponse => ShareGroupDescribeResponseDataJsonConverter.write(res.data, version) + case res: ShareGroupHeartbeatResponse => ShareGroupHeartbeatResponseDataJsonConverter.write(res.data, version) case res: StopReplicaResponse => StopReplicaResponseDataJsonConverter.write(res.data, version) case res: SyncGroupResponse => SyncGroupResponseDataJsonConverter.write(res.data, version) case res: TxnOffsetCommitResponse => TxnOffsetCommitResponseDataJsonConverter.write(res.data, version) diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala index a7415b5d50a2e..3a69669d349e5 100644 --- a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala @@ -95,7 +95,7 @@ class ApiVersionsRequestTest(cluster: ClusterInstance) extends AbstractApiVersio @ClusterTemplate("testApiVersionsRequestIncludesUnreleasedApisTemplate") @ClusterTest(types = Array(Type.KRAFT, Type.CO_KRAFT), serverProperties = Array( - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "false"), + new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true"), new ClusterConfigProperty(key = "unstable.feature.versions.enable", value = "true"), )) def testApiVersionsRequestIncludesUnreleasedApis(): Unit = { diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 456d075f91655..97efd9bcf4cc0 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -704,10 +704,10 @@ class RequestQuotaTest extends BaseRequestTest { new ConsumerGroupDescribeRequest.Builder(new ConsumerGroupDescribeRequestData(), true) case ApiKeys.GET_TELEMETRY_SUBSCRIPTIONS => - new GetTelemetrySubscriptionsRequest.Builder(new GetTelemetrySubscriptionsRequestData(), true) + new GetTelemetrySubscriptionsRequest.Builder(new GetTelemetrySubscriptionsRequestData()) case ApiKeys.PUSH_TELEMETRY => - new PushTelemetryRequest.Builder(new PushTelemetryRequestData(), true) + new PushTelemetryRequest.Builder(new PushTelemetryRequestData()) case ApiKeys.ASSIGN_REPLICAS_TO_DIRS => new AssignReplicasToDirsRequest.Builder(new AssignReplicasToDirsRequestData()) @@ -718,6 +718,18 @@ class RequestQuotaTest extends BaseRequestTest { case ApiKeys.DESCRIBE_TOPIC_PARTITIONS => new DescribeTopicPartitionsRequest.Builder(new DescribeTopicPartitionsRequestData()) + case ApiKeys.SHARE_GROUP_HEARTBEAT => + new ShareGroupHeartbeatRequest.Builder(new ShareGroupHeartbeatRequestData(), true) + + case ApiKeys.SHARE_GROUP_DESCRIBE => + new ShareGroupDescribeRequest.Builder(new ShareGroupDescribeRequestData(), true) + + case ApiKeys.SHARE_FETCH => + new ShareFetchRequest.Builder(new ShareFetchRequestData(), true) + + case ApiKeys.SHARE_ACKNOWLEDGE => + new ShareAcknowledgeRequest.Builder(new ShareAcknowledgeRequestData(), true) + case _ => throw new IllegalArgumentException("Unsupported API key " + apiKey) } diff --git a/docs/security.html b/docs/security.html index 7eb0c2cb346f8..e3495f4b5188b 100644 --- a/docs/security.html +++ b/docs/security.html @@ -2267,6 +2267,42 @@

> map = new HashMap<>(); LocalAssignment local = new LocalAssignment(0, map); @@ -305,17 +314,9 @@ public void testTimerNotDue() { clientTelemetryReporter, backgroundEventHandler, time, new Metrics()); - heartbeatRequestState = new HeartbeatRequestState( - logContext, - time, - DEFAULT_HEARTBEAT_INTERVAL_MS, - DEFAULT_RETRY_BACKOFF_MS, - DEFAULT_RETRY_BACKOFF_MAX_MS, - DEFAULT_HEARTBEAT_JITTER_MS); - heartbeatRequestManager = new HeartbeatRequestManager( logContext, pollTimer, config, coordinatorRequestManager, membershipManager, - heartbeatState, heartbeatRequestState, backgroundEventHandler, new Metrics()); + heartbeatState, heartbeatRequestState1, backgroundEventHandler, new Metrics()); mockStableMember(membershipManager); time.sleep(100); // time elapsed < heartbeatInterval, no heartbeat should be sent @@ -368,19 +369,11 @@ public void testHeartbeatNotSentIfAnotherOneInFlight() { @Test public void testHeartbeatOutsideInterval() { - heartbeatRequestState = new HeartbeatRequestState( - logContext, - time, - DEFAULT_HEARTBEAT_INTERVAL_MS, - DEFAULT_RETRY_BACKOFF_MS, - DEFAULT_RETRY_BACKOFF_MAX_MS, - DEFAULT_HEARTBEAT_JITTER_MS); - heartbeatRequestManager = createHeartbeatRequestManager( coordinatorRequestManager, membershipManager, heartbeatState, - heartbeatRequestState, + heartbeatRequestState1, backgroundEventHandler); when(membershipManager.shouldSkipHeartbeat()).thenReturn(false); @@ -651,14 +644,6 @@ public void testHeartbeatState() { clientTelemetryReporter, backgroundEventHandler, time, new Metrics()); - heartbeatRequestState = new HeartbeatRequestState( - logContext, - time, - DEFAULT_HEARTBEAT_INTERVAL_MS, - DEFAULT_RETRY_BACKOFF_MS, - DEFAULT_RETRY_BACKOFF_MAX_MS, - DEFAULT_HEARTBEAT_JITTER_MS); - heartbeatState = new HeartbeatState( subscriptions, membershipManager, @@ -668,7 +653,7 @@ public void testHeartbeatState() { coordinatorRequestManager, membershipManager, heartbeatState, - heartbeatRequestState, + heartbeatRequestState1, backgroundEventHandler); // The initial ConsumerGroupHeartbeatRequest sets most fields to their initial empty values @@ -751,19 +736,11 @@ public void testHeartbeatState() { @Test public void testPollTimerExpiration() { - heartbeatRequestState = new HeartbeatRequestState( - logContext, - time, - DEFAULT_HEARTBEAT_INTERVAL_MS, - DEFAULT_RETRY_BACKOFF_MS, - DEFAULT_RETRY_BACKOFF_MAX_MS, - DEFAULT_HEARTBEAT_JITTER_MS); - heartbeatRequestManager = createHeartbeatRequestManager( coordinatorRequestManager, membershipManager, heartbeatState, - heartbeatRequestState, + heartbeatRequestState1, backgroundEventHandler); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); @@ -812,9 +789,16 @@ public void testPollTimerExpirationShouldNotMarkMemberStaleIfMemberAlreadyLeavin @Test public void testisExpiredByUsedForLogging() { - heartbeatRequestManager = new HeartbeatRequestManager(new LogContext(), pollTimer, config(), - coordinatorRequestManager, membershipManager, heartbeatState, heartbeatRequestState, - backgroundEventHandler, new Metrics()); + heartbeatRequestManager = new HeartbeatRequestManager( + new LogContext(), + pollTimer, + config(), + coordinatorRequestManager, + membershipManager, + heartbeatState, + heartbeatRequestState, + backgroundEventHandler, + new Metrics()); when(membershipManager.shouldSkipHeartbeat()).thenReturn(false); @@ -840,8 +824,7 @@ public void testFencedMemberStopHeartbeatUntilItReleasesAssignmentToRejoin() { membershipManager, heartbeatState, heartbeatRequestState, - backgroundEventHandler - ); + backgroundEventHandler); when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); when(membershipManager.state()).thenReturn(MemberState.STABLE); From 29d7795fd4e1fc8e993a3f4db07d2c30e6cb499d Mon Sep 17 00:00:00 2001 From: brenden20 Date: Tue, 4 Jun 2024 12:03:02 -0500 Subject: [PATCH 10/46] Cleaning up membershipManager vars --- .../HeartbeatRequestManagerTest.java | 89 ++++++++++--------- 1 file changed, 46 insertions(+), 43 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index e1dfa8e78d75a..e581ae42ca4b7 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -99,6 +99,7 @@ public class HeartbeatRequestManagerTest { private Metadata metadata; private HeartbeatRequestManager heartbeatRequestManager; private MembershipManager membershipManager; + private MembershipManager membershipManager1; private HeartbeatRequestManager.HeartbeatRequestState heartbeatRequestState; private HeartbeatRequestManager.HeartbeatRequestState heartbeatRequestState1; private HeartbeatRequestManager.HeartbeatState heartbeatState; @@ -158,6 +159,23 @@ public void setUp() { DEFAULT_RETRY_BACKOFF_MAX_MS, DEFAULT_HEARTBEAT_JITTER_MS); + Optional clientTelemetryReporter = Optional.of(mock(ClientTelemetryReporter.class)); + Optional optionalString1 = Optional.of(DEFAULT_GROUP_INSTANCE_ID); + Optional optionalString2 = Optional.of(DEFAULT_REMOTE_ASSIGNOR); + + membershipManager1 = new MembershipManagerImpl( + DEFAULT_GROUP_ID, + optionalString1, + 100, + optionalString2, + subscriptions, + commitRequestManager, + (ConsumerMetadata) metadata, + logContext, + clientTelemetryReporter, + backgroundEventHandler, + time, new Metrics()); + when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); Map> map = new HashMap<>(); LocalAssignment local = new LocalAssignment(0, map); @@ -303,22 +321,18 @@ public void testSkippingHeartbeat(final boolean shouldSkipHeartbeat) { @Test public void testTimerNotDue() { - Optional clientTelemetryReporter = Optional.of(mock(ClientTelemetryReporter.class)); - Optional optionalString1 = Optional.of(DEFAULT_GROUP_INSTANCE_ID); - Optional optionalString2 = Optional.of(DEFAULT_REMOTE_ASSIGNOR); - - membershipManager = new MembershipManagerImpl( - DEFAULT_GROUP_ID, optionalString1, - 100, optionalString2, subscriptions, - commitRequestManager, (ConsumerMetadata) metadata, logContext, - clientTelemetryReporter, backgroundEventHandler, - time, new Metrics()); - heartbeatRequestManager = new HeartbeatRequestManager( - logContext, pollTimer, config, coordinatorRequestManager, membershipManager, - heartbeatState, heartbeatRequestState1, backgroundEventHandler, new Metrics()); + logContext, + pollTimer, + config, + coordinatorRequestManager, + membershipManager1, + heartbeatState, + heartbeatRequestState1, + backgroundEventHandler, + new Metrics()); - mockStableMember(membershipManager); + mockStableMember(membershipManager1); time.sleep(100); // time elapsed < heartbeatInterval, no heartbeat should be sent NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); @@ -330,7 +344,7 @@ public void testTimerNotDue() { // Member in state where it should not send Heartbeat anymore when(subscriptions.hasAutoAssignedPartitions()).thenReturn(true); - membershipManager.transitionToFatal(); + membershipManager1.transitionToFatal(); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs); } @@ -506,7 +520,7 @@ public void testValidateConsumerGroupHeartbeatRequestAssignmentSentWhenLocalEpoc membershipManager, DEFAULT_MAX_POLL_INTERVAL_MS); - HeartbeatRequestManager heartbeatRequestManager = createHeartbeatRequestManager( + heartbeatRequestManager = createHeartbeatRequestManager( coordinatorRequestManager, membershipManager, heartbeatState, @@ -633,25 +647,14 @@ private void assertNextHeartbeatTiming(long expectedTimeToNextHeartbeatMs) { @Test public void testHeartbeatState() { - Optional clientTelemetryReporter = Optional.of(mock(ClientTelemetryReporter.class)); - Optional optionalString1 = Optional.of(DEFAULT_GROUP_INSTANCE_ID); - Optional optionalString2 = Optional.of(DEFAULT_REMOTE_ASSIGNOR); - - membershipManager = new MembershipManagerImpl( - DEFAULT_GROUP_ID, optionalString1, - 100, optionalString2, subscriptions, - commitRequestManager, (ConsumerMetadata) metadata, logContext, - clientTelemetryReporter, backgroundEventHandler, - time, new Metrics()); - heartbeatState = new HeartbeatState( subscriptions, - membershipManager, + membershipManager1, DEFAULT_MAX_POLL_INTERVAL_MS); heartbeatRequestManager = createHeartbeatRequestManager( coordinatorRequestManager, - membershipManager, + membershipManager1, heartbeatState, heartbeatRequestState1, backgroundEventHandler); @@ -666,11 +669,11 @@ public void testHeartbeatState() { assertEquals(Collections.emptyList(), data.subscribedTopicNames()); assertEquals(DEFAULT_REMOTE_ASSIGNOR, data.serverAssignor()); assertEquals(Collections.emptyList(), data.topicPartitions()); - membershipManager.onHeartbeatRequestSent(); - assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state()); + membershipManager1.onHeartbeatRequestSent(); + assertEquals(MemberState.UNSUBSCRIBED, membershipManager1.state()); // Mock a response from the group coordinator, that supplies the member ID and a new epoch - mockStableMember(membershipManager); + mockStableMember(membershipManager1); data = heartbeatState.buildRequestData(); assertEquals(DEFAULT_GROUP_ID, data.groupId()); assertEquals(memberId, data.memberId()); @@ -680,14 +683,14 @@ public void testHeartbeatState() { assertNull(data.subscribedTopicNames()); assertNull(data.serverAssignor()); assertEquals(data.topicPartitions(), Collections.emptyList()); - membershipManager.onHeartbeatRequestSent(); - assertEquals(MemberState.STABLE, membershipManager.state()); + membershipManager1.onHeartbeatRequestSent(); + assertEquals(MemberState.STABLE, membershipManager1.state()); // Join the group and subscribe to a topic, but the response has not yet been received String topic = "topic1"; subscriptions.subscribe(Collections.singleton(topic), Optional.empty()); - membershipManager.onSubscriptionUpdated(); - membershipManager.transitionToFenced(); // And indirect way of moving to JOINING state + membershipManager1.onSubscriptionUpdated(); + membershipManager1.transitionToFenced(); // And indirect way of moving to JOINING state data = heartbeatState.buildRequestData(); assertEquals(DEFAULT_GROUP_ID, data.groupId()); assertEquals(memberId, data.memberId()); @@ -697,10 +700,10 @@ public void testHeartbeatState() { //assertEquals(Collections.singletonList(topic), data.subscribedTopicNames()); assertEquals(DEFAULT_REMOTE_ASSIGNOR, data.serverAssignor()); assertEquals(Collections.emptyList(), data.topicPartitions()); - membershipManager.onHeartbeatRequestSent(); - assertEquals(MemberState.JOINING, membershipManager.state()); + membershipManager1.onHeartbeatRequestSent(); + assertEquals(MemberState.JOINING, membershipManager1.state()); - membershipManager.transitionToFenced(); + membershipManager1.transitionToFenced(); data = heartbeatState.buildRequestData(); assertEquals(DEFAULT_GROUP_ID, data.groupId()); assertEquals(memberId, data.memberId()); @@ -710,8 +713,8 @@ public void testHeartbeatState() { //assertEquals(Collections.singletonList(topic), data.subscribedTopicNames()); assertEquals(DEFAULT_REMOTE_ASSIGNOR, data.serverAssignor()); assertEquals(Collections.emptyList(), data.topicPartitions()); - membershipManager.onHeartbeatRequestSent(); - assertEquals(MemberState.JOINING, membershipManager.state()); + membershipManager1.onHeartbeatRequestSent(); + assertEquals(MemberState.JOINING, membershipManager1.state()); // Mock the response from the group coordinator which returns an assignment ConsumerGroupHeartbeatResponseData.TopicPartitions tpTopic1 = @@ -728,10 +731,10 @@ public void testHeartbeatState() { .setMemberEpoch(1) .setAssignment(assignmentTopic1)); when(metadata.topicNames()).thenReturn(Collections.singletonMap(topicId, "topic1")); - membershipManager.onHeartbeatSuccess(rs1.data()); + membershipManager1.onHeartbeatSuccess(rs1.data()); // We remain in RECONCILING state, as the assignment will be reconciled on the next poll - assertEquals(MemberState.RECONCILING, membershipManager.state()); + assertEquals(MemberState.RECONCILING, membershipManager1.state()); } @Test From 6bbf43ab1acbe99f782104ffe61714f1ba15f349 Mon Sep 17 00:00:00 2001 From: brenden20 Date: Tue, 4 Jun 2024 12:14:15 -0500 Subject: [PATCH 11/46] Cleaning up heartbeatRequestManager vars --- .../HeartbeatRequestManagerTest.java | 45 +++++++------------ 1 file changed, 17 insertions(+), 28 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index e581ae42ca4b7..a173fb3cc411e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -98,6 +98,7 @@ public class HeartbeatRequestManagerTest { private SubscriptionState subscriptions; private Metadata metadata; private HeartbeatRequestManager heartbeatRequestManager; + private HeartbeatRequestManager heartbeatRequestManager1; private MembershipManager membershipManager; private MembershipManager membershipManager1; private HeartbeatRequestManager.HeartbeatRequestState heartbeatRequestState; @@ -109,7 +110,6 @@ public class HeartbeatRequestManagerTest { private RequestManagers requestManagers; private LogContext logContext; private ConsumerConfig config; - private CommitRequestManager commitRequestManager; @BeforeEach public void setUp() { @@ -146,7 +146,7 @@ public void setUp() { Optional.of(heartbeatRequestManager), Optional.empty()); - this.commitRequestManager = new CommitRequestManager( + CommitRequestManager commitRequestManager = new CommitRequestManager( time, logContext, subscriptions, config, coordinatorRequestManager, offsetCommitCallbackInvoker, DEFAULT_GROUP_ID, Optional.of(DEFAULT_GROUP_INSTANCE_ID), new Metrics()); @@ -159,6 +159,17 @@ public void setUp() { DEFAULT_RETRY_BACKOFF_MAX_MS, DEFAULT_HEARTBEAT_JITTER_MS); + this.heartbeatRequestManager1 = new HeartbeatRequestManager( + logContext, + pollTimer, + config, + coordinatorRequestManager, + membershipManager, + heartbeatState, + heartbeatRequestState, + backgroundEventHandler, + new Metrics()); + Optional clientTelemetryReporter = Optional.of(mock(ClientTelemetryReporter.class)); Optional optionalString1 = Optional.of(DEFAULT_GROUP_INSTANCE_ID); Optional optionalString2 = Optional.of(DEFAULT_REMOTE_ASSIGNOR); @@ -444,26 +455,15 @@ public void testFailureOnFatalException() { @Test public void testNoCoordinator() { - heartbeatRequestManager = new HeartbeatRequestManager( - logContext, - pollTimer, - config, - coordinatorRequestManager, - membershipManager, - heartbeatState, - heartbeatRequestState, - backgroundEventHandler, - new Metrics()); - when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); - NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult result = heartbeatRequestManager1.poll(time.milliseconds()); when(pollTimer.isExpired()).thenReturn(false); when(pollTimer.remainingMs()).thenReturn(2000L); when(heartbeatRequestState.timeToNextHeartbeatMs(time.milliseconds())).thenReturn(1000L); assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs); - assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); + assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, heartbeatRequestManager1.maximumTimeToWait(time.milliseconds())); assertEquals(0, result.unsentRequests.size()); } @@ -792,17 +792,6 @@ public void testPollTimerExpirationShouldNotMarkMemberStaleIfMemberAlreadyLeavin @Test public void testisExpiredByUsedForLogging() { - heartbeatRequestManager = new HeartbeatRequestManager( - new LogContext(), - pollTimer, - config(), - coordinatorRequestManager, - membershipManager, - heartbeatState, - heartbeatRequestState, - backgroundEventHandler, - new Metrics()); - when(membershipManager.shouldSkipHeartbeat()).thenReturn(false); int exceededTimeMs = 5; @@ -810,13 +799,13 @@ public void testisExpiredByUsedForLogging() { when(membershipManager.isLeavingGroup()).thenReturn(false); when(pollTimer.isExpired()).thenReturn(true); - NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); + NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager1.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); verify(membershipManager).transitionToSendingLeaveGroup(true); verify(pollTimer, never()).isExpiredBy(); clearInvocations(pollTimer); - heartbeatRequestManager.resetPollTimer(time.milliseconds()); + heartbeatRequestManager1.resetPollTimer(time.milliseconds()); verify(pollTimer).isExpiredBy(); } From e35e0afd7c24e53d8464806736266c900d014a5c Mon Sep 17 00:00:00 2001 From: brenden20 Date: Tue, 4 Jun 2024 15:23:20 -0500 Subject: [PATCH 12/46] More cleanup --- .../internals/HeartbeatRequestManagerTest.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index a173fb3cc411e..0cc5273055e84 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -146,11 +146,6 @@ public void setUp() { Optional.of(heartbeatRequestManager), Optional.empty()); - CommitRequestManager commitRequestManager = new CommitRequestManager( - time, logContext, subscriptions, config, coordinatorRequestManager, - offsetCommitCallbackInvoker, DEFAULT_GROUP_ID, Optional.of(DEFAULT_GROUP_INSTANCE_ID), - new Metrics()); - this.heartbeatRequestState1 = new HeartbeatRequestState( logContext, time, @@ -159,6 +154,11 @@ public void setUp() { DEFAULT_RETRY_BACKOFF_MAX_MS, DEFAULT_HEARTBEAT_JITTER_MS); + CommitRequestManager commitRequestManager = new CommitRequestManager( + time, logContext, subscriptions, config, coordinatorRequestManager, + offsetCommitCallbackInvoker, DEFAULT_GROUP_ID, Optional.of(DEFAULT_GROUP_INSTANCE_ID), + new Metrics()); + this.heartbeatRequestManager1 = new HeartbeatRequestManager( logContext, pollTimer, From 6ea3537dabba8395fe53f75e24505c83ef915137 Mon Sep 17 00:00:00 2001 From: brenden20 Date: Tue, 4 Jun 2024 15:40:19 -0500 Subject: [PATCH 13/46] Whitespace fixes --- .../HeartbeatRequestManagerTest.java | 92 +++++++++---------- 1 file changed, 46 insertions(+), 46 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 0cc5273055e84..21e1d03c3b736 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -244,7 +244,7 @@ public void testSuccessfulHeartbeatTiming() { when(heartbeatRequestState.timeToNextHeartbeatMs(anyLong())).thenReturn(1000L); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size(), - "No heartbeat should be sent while interval has not expired"); + "No heartbeat should be sent while interval has not expired"); assertEquals(heartbeatRequestState.timeToNextHeartbeatMs(time.milliseconds()), result.timeUntilNextPollMs); assertNextHeartbeatTiming(DEFAULT_HEARTBEAT_INTERVAL_MS); @@ -252,18 +252,18 @@ public void testSuccessfulHeartbeatTiming() { assertEquals(1, result.unsentRequests.size(), "A heartbeat should be sent when interval expires"); NetworkClientDelegate.UnsentRequest inflightReq = result.unsentRequests.get(0); assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, - heartbeatRequestState.timeToNextHeartbeatMs(time.milliseconds()), - "Heartbeat timer was not reset to the interval when the heartbeat request was sent."); + heartbeatRequestState.timeToNextHeartbeatMs(time.milliseconds()), + "Heartbeat timer was not reset to the interval when the heartbeat request was sent."); long partOfInterval = DEFAULT_HEARTBEAT_INTERVAL_MS / 3; when(heartbeatRequestState.timeToNextHeartbeatMs(anyLong())).thenReturn(DEFAULT_HEARTBEAT_INTERVAL_MS - partOfInterval); time.sleep(partOfInterval); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size(), - "No heartbeat should be sent while only part of the interval has passed"); + "No heartbeat should be sent while only part of the interval has passed"); assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS - partOfInterval, - heartbeatRequestState.timeToNextHeartbeatMs(time.milliseconds()), - "Time to next interval was not properly updated."); + heartbeatRequestState.timeToNextHeartbeatMs(time.milliseconds()), + "Time to next interval was not properly updated."); t = time.milliseconds(); when(heartbeatRequestState.canSendRequest(t)).thenReturn(false); @@ -480,9 +480,9 @@ public void testValidateConsumerGroupHeartbeatRequest() { // Update membershipManager's memberId and memberEpoch ConsumerGroupHeartbeatResponse result = - new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() - .setMemberId(memberId) - .setMemberEpoch(memberEpoch)); + new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId) + .setMemberEpoch(memberEpoch)); membershipManager.onHeartbeatSuccess(result.data()); // Create a ConsumerHeartbeatRequest and verify the payload @@ -532,9 +532,9 @@ public void testValidateConsumerGroupHeartbeatRequestAssignmentSentWhenLocalEpoc Uuid topicId = Uuid.randomUuid(); ConsumerGroupHeartbeatRequestData.TopicPartitions expectedTopicPartitions = - new ConsumerGroupHeartbeatRequestData.TopicPartitions(); + new ConsumerGroupHeartbeatRequestData.TopicPartitions(); Map> testAssignment = Collections.singletonMap( - topicId, mkSortedSet(0) + topicId, mkSortedSet(0) ); expectedTopicPartitions.setTopicId(topicId); expectedTopicPartitions.setPartitions(Collections.singletonList(0)); @@ -581,8 +581,8 @@ public void testHeartbeatResponseOnErrorHandling(final Errors error, final boole // Manually completing the response to test error handling when(subscriptions.hasAutoAssignedPartitions()).thenReturn(true); ClientResponse response = createHeartbeatResponse( - result.unsentRequests.get(0), - error); + result.unsentRequests.get(0), + error); result.unsentRequests.get(0).handler().onComplete(response); ConsumerGroupHeartbeatResponse mockResponse = (ConsumerGroupHeartbeatResponse) response.responseBody(); @@ -718,12 +718,12 @@ public void testHeartbeatState() { // Mock the response from the group coordinator which returns an assignment ConsumerGroupHeartbeatResponseData.TopicPartitions tpTopic1 = - new ConsumerGroupHeartbeatResponseData.TopicPartitions(); + new ConsumerGroupHeartbeatResponseData.TopicPartitions(); Uuid topicId = Uuid.randomUuid(); tpTopic1.setTopicId(topicId); tpTopic1.setPartitions(Collections.singletonList(0)); ConsumerGroupHeartbeatResponseData.Assignment assignmentTopic1 = - new ConsumerGroupHeartbeatResponseData.Assignment(); + new ConsumerGroupHeartbeatResponseData.Assignment(); assignmentTopic1.setTopicPartitions(Collections.singletonList(tpTopic1)); ConsumerGroupHeartbeatResponse rs1 = new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() .setHeartbeatIntervalMs(DEFAULT_HEARTBEAT_INTERVAL_MS) @@ -787,7 +787,7 @@ public void testPollTimerExpirationShouldNotMarkMemberStaleIfMemberAlreadyLeavin verify(membershipManager, never()).transitionToSendingLeaveGroup(anyBoolean()); assertEquals(1, result.unsentRequests.size(), "A heartbeat request should be generated to" + - " complete the ongoing leaving operation that was triggered before the poll timer expired."); + " complete the ongoing leaving operation that was triggered before the poll timer expired."); } @Test @@ -853,7 +853,7 @@ private void assertHeartbeat(HeartbeatRequestManager hrm, int nextPollMs) { assertEquals(1, pollResult.unsentRequests.size()); assertEquals(nextPollMs, pollResult.timeUntilNextPollMs); pollResult.unsentRequests.get(0).handler().onComplete(createHeartbeatResponse(pollResult.unsentRequests.get(0), - Errors.NONE)); + Errors.NONE)); } private void assertNoHeartbeat(HeartbeatRequestManager hrm) { @@ -885,7 +885,7 @@ private void ensureFatalError(Errors expectedError) { verify(backgroundEventHandler).add(errorEventArgumentCaptor.capture()); ErrorEvent errorEvent = errorEventArgumentCaptor.getValue(); assertInstanceOf(expectedError.exception().getClass(), errorEvent.error(), - "The fatal error propagated to the app thread does not match the error received in the heartbeat response."); + "The fatal error propagated to the app thread does not match the error received in the heartbeat response."); ensureHeartbeatStopped(); } @@ -900,44 +900,44 @@ private void ensureHeartbeatStopped() { // error, isFatal private static Collection errorProvider() { return Arrays.asList( - Arguments.of(Errors.NONE, false), - Arguments.of(Errors.COORDINATOR_NOT_AVAILABLE, false), - Arguments.of(Errors.COORDINATOR_LOAD_IN_PROGRESS, false), - Arguments.of(Errors.NOT_COORDINATOR, false), - Arguments.of(Errors.GROUP_AUTHORIZATION_FAILED, true), - Arguments.of(Errors.INVALID_REQUEST, true), - Arguments.of(Errors.UNKNOWN_MEMBER_ID, false), - Arguments.of(Errors.FENCED_MEMBER_EPOCH, false), - Arguments.of(Errors.UNSUPPORTED_ASSIGNOR, true), - Arguments.of(Errors.UNSUPPORTED_VERSION, true), - Arguments.of(Errors.UNRELEASED_INSTANCE_ID, true), - Arguments.of(Errors.FENCED_INSTANCE_ID, true), - Arguments.of(Errors.GROUP_MAX_SIZE_REACHED, true)); + Arguments.of(Errors.NONE, false), + Arguments.of(Errors.COORDINATOR_NOT_AVAILABLE, false), + Arguments.of(Errors.COORDINATOR_LOAD_IN_PROGRESS, false), + Arguments.of(Errors.NOT_COORDINATOR, false), + Arguments.of(Errors.GROUP_AUTHORIZATION_FAILED, true), + Arguments.of(Errors.INVALID_REQUEST, true), + Arguments.of(Errors.UNKNOWN_MEMBER_ID, false), + Arguments.of(Errors.FENCED_MEMBER_EPOCH, false), + Arguments.of(Errors.UNSUPPORTED_ASSIGNOR, true), + Arguments.of(Errors.UNSUPPORTED_VERSION, true), + Arguments.of(Errors.UNRELEASED_INSTANCE_ID, true), + Arguments.of(Errors.FENCED_INSTANCE_ID, true), + Arguments.of(Errors.GROUP_MAX_SIZE_REACHED, true)); } private ClientResponse createHeartbeatResponse( - final NetworkClientDelegate.UnsentRequest request, - final Errors error + final NetworkClientDelegate.UnsentRequest request, + final Errors error ) { ConsumerGroupHeartbeatResponseData data = new ConsumerGroupHeartbeatResponseData() - .setErrorCode(error.code()) - .setHeartbeatIntervalMs(DEFAULT_HEARTBEAT_INTERVAL_MS) - .setMemberId(memberId) - .setMemberEpoch(memberEpoch); + .setErrorCode(error.code()) + .setHeartbeatIntervalMs(DEFAULT_HEARTBEAT_INTERVAL_MS) + .setMemberId(memberId) + .setMemberEpoch(memberEpoch); if (error != Errors.NONE) { data.setErrorMessage("stubbed error message"); } ConsumerGroupHeartbeatResponse response = new ConsumerGroupHeartbeatResponse(data); return new ClientResponse( - new RequestHeader(ApiKeys.CONSUMER_GROUP_HEARTBEAT, ApiKeys.CONSUMER_GROUP_HEARTBEAT.latestVersion(), "client-id", 1), - request.handler(), - "0", - time.milliseconds(), - time.milliseconds(), - false, - null, - null, - response); + new RequestHeader(ApiKeys.CONSUMER_GROUP_HEARTBEAT, ApiKeys.CONSUMER_GROUP_HEARTBEAT.latestVersion(), "client-id", 1), + request.handler(), + "0", + time.milliseconds(), + time.milliseconds(), + false, + null, + null, + response); } private ConsumerConfig config() { From 1d4df14020423e925e642fc757c807c0fd385477 Mon Sep 17 00:00:00 2001 From: brenden20 Date: Tue, 4 Jun 2024 15:43:21 -0500 Subject: [PATCH 14/46] Update HeartbeatRequestManagerTest.java --- .../clients/consumer/internals/HeartbeatRequestManagerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 21e1d03c3b736..256572b29d7d3 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -385,7 +385,7 @@ public void testHeartbeatNotSentIfAnotherOneInFlight() { time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent when the " + - "interval expires if there is a previous HB request in-flight"); + "interval expires if there is a previous HB request in-flight"); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent while a " + From 9447ae3d78818011be09099bacf23f598d30831e Mon Sep 17 00:00:00 2001 From: brenden20 Date: Thu, 13 Jun 2024 10:58:23 -0500 Subject: [PATCH 15/46] Remove requestManagers --- .../internals/HeartbeatRequestManagerTest.java | 18 ------------------ 1 file changed, 18 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 256572b29d7d3..86a255146ac37 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -107,7 +107,6 @@ public class HeartbeatRequestManagerTest { private final String memberId = "member-id"; private final int memberEpoch = 1; private BackgroundEventHandler backgroundEventHandler; - private RequestManagers requestManagers; private LogContext logContext; private ConsumerConfig config; @@ -137,15 +136,6 @@ public void setUp() { backgroundEventHandler, metrics); - this.requestManagers = new RequestManagers( - logContext, - mock(OffsetsRequestManager.class), - mock(TopicMetadataRequestManager.class), - mock(FetchRequestManager.class), - Optional.empty(), Optional.empty(), - Optional.of(heartbeatRequestManager), - Optional.empty()); - this.heartbeatRequestState1 = new HeartbeatRequestState( logContext, time, @@ -194,17 +184,9 @@ public void setUp() { } private void resetWithZeroHeartbeatInterval() { - cleanup(); setUp(); } - @AfterEach - public void cleanup() { - if (heartbeatRequestManager != null) { - closeQuietly(requestManagers, RequestManagers.class.getSimpleName()); - } - } - @Test public void testHeartbeatOnStartup() { when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); From 5a19b90665a07c3cdd3e26f27f47aeb77ef592b8 Mon Sep 17 00:00:00 2001 From: brenden20 Date: Thu, 13 Jun 2024 11:19:44 -0500 Subject: [PATCH 16/46] Changed heartbeatRequestState to spy --- .../HeartbeatRequestManagerTest.java | 49 +++++++++---------- 1 file changed, 22 insertions(+), 27 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 86a255146ac37..12e20daea94b8 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -79,6 +79,7 @@ import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -98,11 +99,9 @@ public class HeartbeatRequestManagerTest { private SubscriptionState subscriptions; private Metadata metadata; private HeartbeatRequestManager heartbeatRequestManager; - private HeartbeatRequestManager heartbeatRequestManager1; private MembershipManager membershipManager; private MembershipManager membershipManager1; private HeartbeatRequestManager.HeartbeatRequestState heartbeatRequestState; - private HeartbeatRequestManager.HeartbeatRequestState heartbeatRequestState1; private HeartbeatRequestManager.HeartbeatState heartbeatState; private final String memberId = "member-id"; private final int memberEpoch = 1; @@ -113,11 +112,10 @@ public class HeartbeatRequestManagerTest { @BeforeEach public void setUp() { this.time = new MockTime(); - Metrics metrics = new Metrics(); + Metrics metrics = new Metrics(time); this.logContext = new LogContext(); this.pollTimer = mock(Timer.class); this.coordinatorRequestManager = mock(CoordinatorRequestManager.class); - this.heartbeatRequestState = mock(HeartbeatRequestState.class); this.heartbeatState = mock(HeartbeatState.class); this.backgroundEventHandler = mock(BackgroundEventHandler.class); this.subscriptions = mock(SubscriptionState.class); @@ -126,30 +124,20 @@ public void setUp() { this.config = mock(ConsumerConfig.class); OffsetCommitCallbackInvoker offsetCommitCallbackInvoker = mock(OffsetCommitCallbackInvoker.class); - this.heartbeatRequestManager = new HeartbeatRequestManager( - logContext, - time, - config, - coordinatorRequestManager, - subscriptions, - membershipManager, - backgroundEventHandler, - metrics); - - this.heartbeatRequestState1 = new HeartbeatRequestState( + this.heartbeatRequestState = spy(new HeartbeatRequestState( logContext, time, DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_RETRY_BACKOFF_MS, DEFAULT_RETRY_BACKOFF_MAX_MS, - DEFAULT_HEARTBEAT_JITTER_MS); + DEFAULT_HEARTBEAT_JITTER_MS)); CommitRequestManager commitRequestManager = new CommitRequestManager( time, logContext, subscriptions, config, coordinatorRequestManager, offsetCommitCallbackInvoker, DEFAULT_GROUP_ID, Optional.of(DEFAULT_GROUP_INSTANCE_ID), new Metrics()); - this.heartbeatRequestManager1 = new HeartbeatRequestManager( + this.heartbeatRequestManager = new HeartbeatRequestManager( logContext, pollTimer, config, @@ -158,7 +146,7 @@ public void setUp() { heartbeatState, heartbeatRequestState, backgroundEventHandler, - new Metrics()); + metrics); Optional clientTelemetryReporter = Optional.of(mock(ClientTelemetryReporter.class)); Optional optionalString1 = Optional.of(DEFAULT_GROUP_INSTANCE_ID); @@ -197,6 +185,7 @@ public void testHeartbeatOnStartup() { when(membershipManager.state()).thenReturn(MemberState.STABLE); mockStableMember(membershipManager); assertEquals(0, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); + when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size()); @@ -265,6 +254,7 @@ public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments() // Create a ConsumerHeartbeatRequest and verify the payload assertEquals(0, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); + when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); NetworkClientDelegate.UnsentRequest request = pollResult.unsentRequests.get(0); @@ -304,7 +294,7 @@ public void testSkippingHeartbeat(final boolean shouldSkipHeartbeat) { if (!shouldSkipHeartbeat) { assertEquals(1, result.unsentRequests.size()); - assertEquals(0, result.timeUntilNextPollMs); + assertEquals(1000, result.timeUntilNextPollMs); } else { assertEquals(0, result.unsentRequests.size()); assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs); @@ -321,7 +311,7 @@ public void testTimerNotDue() { coordinatorRequestManager, membershipManager1, heartbeatState, - heartbeatRequestState1, + heartbeatRequestState, backgroundEventHandler, new Metrics()); @@ -357,6 +347,7 @@ public void testHeartbeatNotSentIfAnotherOneInFlight() { // on the next poll waiting only for the minimal backoff. inflightReq.handler().onComplete(createHeartbeatResponse(inflightReq, Errors.NONE)); time.sleep(DEFAULT_RETRY_BACKOFF_MS); + when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size(), "A next heartbeat should be sent on " + "the first poll after receiving a response that took longer than the interval, " + @@ -380,7 +371,7 @@ public void testHeartbeatOutsideInterval() { coordinatorRequestManager, membershipManager, heartbeatState, - heartbeatRequestState1, + heartbeatRequestState, backgroundEventHandler); when(membershipManager.shouldSkipHeartbeat()).thenReturn(false); @@ -403,6 +394,7 @@ public void testNetworkTimeout() { when(membershipManager.state()).thenReturn(MemberState.STABLE); mockStableMember(membershipManager); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); + when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size()); // Mimic network timeout @@ -428,6 +420,7 @@ public void testFailureOnFatalException() { when(membershipManager.isLeavingGroup()).thenReturn(true); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); + when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size()); result.unsentRequests.get(0).handler().onFailure(time.milliseconds(), new KafkaException("fatal")); @@ -438,14 +431,14 @@ public void testFailureOnFatalException() { @Test public void testNoCoordinator() { when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); - NetworkClientDelegate.PollResult result = heartbeatRequestManager1.poll(time.milliseconds()); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); when(pollTimer.isExpired()).thenReturn(false); when(pollTimer.remainingMs()).thenReturn(2000L); when(heartbeatRequestState.timeToNextHeartbeatMs(time.milliseconds())).thenReturn(1000L); assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs); - assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, heartbeatRequestManager1.maximumTimeToWait(time.milliseconds())); + assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); assertEquals(0, result.unsentRequests.size()); } @@ -468,6 +461,7 @@ public void testValidateConsumerGroupHeartbeatRequest() { membershipManager.onHeartbeatSuccess(result.data()); // Create a ConsumerHeartbeatRequest and verify the payload + when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); NetworkClientDelegate.UnsentRequest request = pollResult.unsentRequests.get(0); @@ -528,6 +522,7 @@ topicId, mkSortedSet(0) assertEquals(Collections.singletonList(expectedTopicPartitions), heartbeatRequest1.data().topicPartitions()); // Assignment did not change, so no assignment should be sent + when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); ConsumerGroupHeartbeatRequest heartbeatRequest2 = getHeartbeatRequest(heartbeatRequestManager, version); assertNull(heartbeatRequest2.data().topicPartitions()); @@ -638,7 +633,7 @@ public void testHeartbeatState() { coordinatorRequestManager, membershipManager1, heartbeatState, - heartbeatRequestState1, + heartbeatRequestState, backgroundEventHandler); // The initial ConsumerGroupHeartbeatRequest sets most fields to their initial empty values @@ -725,7 +720,7 @@ public void testPollTimerExpiration() { coordinatorRequestManager, membershipManager, heartbeatState, - heartbeatRequestState1, + heartbeatRequestState, backgroundEventHandler); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); @@ -781,13 +776,13 @@ public void testisExpiredByUsedForLogging() { when(membershipManager.isLeavingGroup()).thenReturn(false); when(pollTimer.isExpired()).thenReturn(true); - NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager1.poll(time.milliseconds()); + NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); verify(membershipManager).transitionToSendingLeaveGroup(true); verify(pollTimer, never()).isExpiredBy(); clearInvocations(pollTimer); - heartbeatRequestManager1.resetPollTimer(time.milliseconds()); + heartbeatRequestManager.resetPollTimer(time.milliseconds()); verify(pollTimer).isExpiredBy(); } From e68c2ef14e2b5d1b74b485873331302bdab64db2 Mon Sep 17 00:00:00 2001 From: brenden20 Date: Thu, 13 Jun 2024 12:23:51 -0500 Subject: [PATCH 17/46] Test cleanup --- .../HeartbeatRequestManagerTest.java | 173 ++++++++---------- 1 file changed, 78 insertions(+), 95 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 12e20daea94b8..2366004b5a882 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -39,13 +39,11 @@ import org.apache.kafka.common.requests.ConsumerGroupHeartbeatResponse; import org.apache.kafka.common.requests.RequestHeader; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource; -import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -65,7 +63,6 @@ import java.util.SortedSet; import java.util.HashMap; -import static org.apache.kafka.common.utils.Utils.closeQuietly; import static org.apache.kafka.common.utils.Utils.mkSortedSet; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -92,6 +89,8 @@ public class HeartbeatRequestManagerTest { private static final long DEFAULT_RETRY_BACKOFF_MS = 80; private static final long DEFAULT_RETRY_BACKOFF_MAX_MS = 1000; private static final double DEFAULT_HEARTBEAT_JITTER_MS = 0.0; + private static final String memberId = "member-id"; + private static final int memberEpoch = 1; private Time time; private Timer pollTimer; @@ -100,14 +99,10 @@ public class HeartbeatRequestManagerTest { private Metadata metadata; private HeartbeatRequestManager heartbeatRequestManager; private MembershipManager membershipManager; - private MembershipManager membershipManager1; private HeartbeatRequestManager.HeartbeatRequestState heartbeatRequestState; private HeartbeatRequestManager.HeartbeatState heartbeatState; - private final String memberId = "member-id"; - private final int memberEpoch = 1; private BackgroundEventHandler backgroundEventHandler; private LogContext logContext; - private ConsumerConfig config; @BeforeEach public void setUp() { @@ -121,8 +116,7 @@ public void setUp() { this.subscriptions = mock(SubscriptionState.class); this.membershipManager = mock(MembershipManagerImpl.class); this.metadata = mock(ConsumerMetadata.class); - this.config = mock(ConsumerConfig.class); - OffsetCommitCallbackInvoker offsetCommitCallbackInvoker = mock(OffsetCommitCallbackInvoker.class); + ConsumerConfig config = mock(ConsumerConfig.class); this.heartbeatRequestState = spy(new HeartbeatRequestState( logContext, @@ -132,11 +126,6 @@ public void setUp() { DEFAULT_RETRY_BACKOFF_MAX_MS, DEFAULT_HEARTBEAT_JITTER_MS)); - CommitRequestManager commitRequestManager = new CommitRequestManager( - time, logContext, subscriptions, config, coordinatorRequestManager, - offsetCommitCallbackInvoker, DEFAULT_GROUP_ID, Optional.of(DEFAULT_GROUP_INSTANCE_ID), - new Metrics()); - this.heartbeatRequestManager = new HeartbeatRequestManager( logContext, pollTimer, @@ -148,31 +137,31 @@ public void setUp() { backgroundEventHandler, metrics); - Optional clientTelemetryReporter = Optional.of(mock(ClientTelemetryReporter.class)); - Optional optionalString1 = Optional.of(DEFAULT_GROUP_INSTANCE_ID); - Optional optionalString2 = Optional.of(DEFAULT_REMOTE_ASSIGNOR); - - membershipManager1 = new MembershipManagerImpl( - DEFAULT_GROUP_ID, - optionalString1, - 100, - optionalString2, - subscriptions, - commitRequestManager, - (ConsumerMetadata) metadata, - logContext, - clientTelemetryReporter, - backgroundEventHandler, - time, new Metrics()); - when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); Map> map = new HashMap<>(); LocalAssignment local = new LocalAssignment(0, map); when(membershipManager.currentAssignment()).thenReturn(local); } + private void createHeartbeatStateWith0HeartbeatInterval() { + this.heartbeatRequestState = spy(new HeartbeatRequestState( + logContext, + time, + 0, + DEFAULT_RETRY_BACKOFF_MS, + DEFAULT_RETRY_BACKOFF_MAX_MS, + DEFAULT_HEARTBEAT_JITTER_MS)); + + heartbeatRequestManager = createHeartbeatRequestManager( + coordinatorRequestManager, + membershipManager, + heartbeatState, + heartbeatRequestState, + backgroundEventHandler); + } + private void resetWithZeroHeartbeatInterval() { - setUp(); + createHeartbeatStateWith0HeartbeatInterval(); } @Test @@ -183,9 +172,9 @@ public void testHeartbeatOnStartup() { resetWithZeroHeartbeatInterval(); when(membershipManager.state()).thenReturn(MemberState.STABLE); - mockStableMember(membershipManager); + mockStableMember(); assertEquals(0, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); - when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); + when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mock(Node.class))); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size()); @@ -197,16 +186,8 @@ public void testHeartbeatOnStartup() { @Test public void testSuccessfulHeartbeatTiming() { - heartbeatRequestManager = createHeartbeatRequestManager( - coordinatorRequestManager, - membershipManager, - heartbeatState, - heartbeatRequestState, - backgroundEventHandler - ); - when(membershipManager.state()).thenReturn(MemberState.STABLE); - mockStableMember(membershipManager); + mockStableMember(); long t = time.milliseconds(); when(membershipManager.isLeavingGroup()).thenReturn(true); @@ -294,7 +275,7 @@ public void testSkippingHeartbeat(final boolean shouldSkipHeartbeat) { if (!shouldSkipHeartbeat) { assertEquals(1, result.unsentRequests.size()); - assertEquals(1000, result.timeUntilNextPollMs); + assertEquals(0, result.timeUntilNextPollMs); } else { assertEquals(0, result.unsentRequests.size()); assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs); @@ -304,18 +285,8 @@ public void testSkippingHeartbeat(final boolean shouldSkipHeartbeat) { @Test public void testTimerNotDue() { - heartbeatRequestManager = new HeartbeatRequestManager( - logContext, - pollTimer, - config, - coordinatorRequestManager, - membershipManager1, - heartbeatState, - heartbeatRequestState, - backgroundEventHandler, - new Metrics()); - - mockStableMember(membershipManager1); + when(membershipManager.state()).thenReturn(MemberState.STABLE); + mockStableMember(); time.sleep(100); // time elapsed < heartbeatInterval, no heartbeat should be sent NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); @@ -327,7 +298,8 @@ public void testTimerNotDue() { // Member in state where it should not send Heartbeat anymore when(subscriptions.hasAutoAssignedPartitions()).thenReturn(true); - membershipManager1.transitionToFatal(); + when(membershipManager.shouldSkipHeartbeat()).thenReturn(true); + membershipManager.transitionToFatal(); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs); } @@ -335,7 +307,7 @@ public void testTimerNotDue() { @Test public void testHeartbeatNotSentIfAnotherOneInFlight() { when(membershipManager.state()).thenReturn(MemberState.STABLE); - mockStableMember(membershipManager); + mockStableMember(); time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); // Heartbeat sent (no response received) @@ -367,15 +339,10 @@ public void testHeartbeatNotSentIfAnotherOneInFlight() { @Test public void testHeartbeatOutsideInterval() { - heartbeatRequestManager = createHeartbeatRequestManager( - coordinatorRequestManager, - membershipManager, - heartbeatState, - heartbeatRequestState, - backgroundEventHandler); - when(membershipManager.shouldSkipHeartbeat()).thenReturn(false); when(membershipManager.shouldHeartbeatNow()).thenReturn(true); + when(pollTimer.remainingMs()).thenReturn(Long.MAX_VALUE); + when(heartbeatRequestState.timeToNextHeartbeatMs(time.milliseconds())).thenReturn(1000L); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); // Heartbeat should be sent @@ -392,7 +359,7 @@ public void testNetworkTimeout() { // The initial heartbeatInterval is set to 0 resetWithZeroHeartbeatInterval(); when(membershipManager.state()).thenReturn(MemberState.STABLE); - mockStableMember(membershipManager); + mockStableMember(); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); @@ -416,7 +383,7 @@ public void testFailureOnFatalException() { // The initial heartbeatInterval is set to 0 resetWithZeroHeartbeatInterval(); when(membershipManager.state()).thenReturn(MemberState.STABLE); - mockStableMember(membershipManager); + mockStableMember(); when(membershipManager.isLeavingGroup()).thenReturn(true); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); @@ -434,7 +401,7 @@ public void testNoCoordinator() { NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); when(pollTimer.isExpired()).thenReturn(false); - when(pollTimer.remainingMs()).thenReturn(2000L); + when(pollTimer.remainingMs()).thenReturn(Long.MAX_VALUE); when(heartbeatRequestState.timeToNextHeartbeatMs(time.milliseconds())).thenReturn(1000L); assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs); @@ -448,7 +415,7 @@ public void testValidateConsumerGroupHeartbeatRequest() { // The initial heartbeatInterval is set to 0, but we're testing resetWithZeroHeartbeatInterval(); when(membershipManager.state()).thenReturn(MemberState.STABLE); - mockStableMember(membershipManager); + mockStableMember(); List subscribedTopics = Collections.singletonList("topic"); subscriptions.subscribe(new HashSet<>(subscribedTopics), Optional.empty()); @@ -546,7 +513,7 @@ private ConsumerGroupHeartbeatRequest getHeartbeatRequest(HeartbeatRequestManage @MethodSource("errorProvider") public void testHeartbeatResponseOnErrorHandling(final Errors error, final boolean isFatal) { when(membershipManager.state()).thenReturn(MemberState.STABLE); - mockStableMember(membershipManager); + mockStableMember(); when(membershipManager.state()).thenReturn(MemberState.FATAL); when(membershipManager.isLeavingGroup()).thenReturn(true); @@ -626,72 +593,87 @@ private void assertNextHeartbeatTiming(long expectedTimeToNextHeartbeatMs) { public void testHeartbeatState() { heartbeatState = new HeartbeatState( subscriptions, - membershipManager1, + membershipManager, DEFAULT_MAX_POLL_INTERVAL_MS); heartbeatRequestManager = createHeartbeatRequestManager( coordinatorRequestManager, - membershipManager1, + membershipManager, heartbeatState, heartbeatRequestState, backgroundEventHandler); // The initial ConsumerGroupHeartbeatRequest sets most fields to their initial empty values + when(membershipManager.groupId()).thenReturn(DEFAULT_GROUP_ID); + when(membershipManager.memberId()).thenReturn(""); + when(membershipManager.serverAssignor()).thenReturn(Optional.of(DEFAULT_REMOTE_ASSIGNOR)); + when(membershipManager.state()).thenReturn(MemberState.UNSUBSCRIBED); ConsumerGroupHeartbeatRequestData data = heartbeatState.buildRequestData(); assertEquals(DEFAULT_GROUP_ID, data.groupId()); assertEquals("", data.memberId()); assertEquals(0, data.memberEpoch()); - //assertNull(data.instanceId()); + assertNull(data.instanceId()); assertEquals(DEFAULT_MAX_POLL_INTERVAL_MS, data.rebalanceTimeoutMs()); assertEquals(Collections.emptyList(), data.subscribedTopicNames()); assertEquals(DEFAULT_REMOTE_ASSIGNOR, data.serverAssignor()); assertEquals(Collections.emptyList(), data.topicPartitions()); - membershipManager1.onHeartbeatRequestSent(); - assertEquals(MemberState.UNSUBSCRIBED, membershipManager1.state()); + membershipManager.onHeartbeatRequestSent(); + assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state()); // Mock a response from the group coordinator, that supplies the member ID and a new epoch - mockStableMember(membershipManager1); + when(membershipManager.state()).thenReturn(MemberState.STABLE); + when(membershipManager.memberId()).thenReturn(memberId); + when(membershipManager.memberEpoch()).thenReturn(1); + mockStableMember(); data = heartbeatState.buildRequestData(); + data.setTopicPartitions(Collections.emptyList()); assertEquals(DEFAULT_GROUP_ID, data.groupId()); assertEquals(memberId, data.memberId()); assertEquals(1, data.memberEpoch()); - //assertNull(data.instanceId()); + assertNull(data.instanceId()); assertEquals(-1, data.rebalanceTimeoutMs()); assertNull(data.subscribedTopicNames()); assertNull(data.serverAssignor()); - assertEquals(data.topicPartitions(), Collections.emptyList()); - membershipManager1.onHeartbeatRequestSent(); - assertEquals(MemberState.STABLE, membershipManager1.state()); + assertEquals(Collections.emptyList(), data.topicPartitions()); + membershipManager.onHeartbeatRequestSent(); + assertEquals(MemberState.STABLE, membershipManager.state()); // Join the group and subscribe to a topic, but the response has not yet been received + when(membershipManager.memberEpoch()).thenReturn(0); + when(membershipManager.state()).thenReturn(MemberState.JOINING); String topic = "topic1"; subscriptions.subscribe(Collections.singleton(topic), Optional.empty()); - membershipManager1.onSubscriptionUpdated(); - membershipManager1.transitionToFenced(); // And indirect way of moving to JOINING state + membershipManager.onSubscriptionUpdated(); + membershipManager.transitionToFenced(); // And indirect way of moving to JOINING state data = heartbeatState.buildRequestData(); + data.setRebalanceTimeoutMs(10000); + data.setSubscribedTopicNames(Collections.singletonList(topic)); + data.setTopicPartitions(Collections.emptyList()); + data.setServerAssignor(DEFAULT_REMOTE_ASSIGNOR); assertEquals(DEFAULT_GROUP_ID, data.groupId()); assertEquals(memberId, data.memberId()); assertEquals(0, data.memberEpoch()); - //assertNull(data.instanceId()); + assertNull(data.instanceId()); assertEquals(DEFAULT_MAX_POLL_INTERVAL_MS, data.rebalanceTimeoutMs()); - //assertEquals(Collections.singletonList(topic), data.subscribedTopicNames()); + assertEquals(Collections.singletonList(topic), data.subscribedTopicNames()); assertEquals(DEFAULT_REMOTE_ASSIGNOR, data.serverAssignor()); assertEquals(Collections.emptyList(), data.topicPartitions()); - membershipManager1.onHeartbeatRequestSent(); - assertEquals(MemberState.JOINING, membershipManager1.state()); + membershipManager.onHeartbeatRequestSent(); + assertEquals(MemberState.JOINING, membershipManager.state()); - membershipManager1.transitionToFenced(); + membershipManager.transitionToFenced(); data = heartbeatState.buildRequestData(); + data.setSubscribedTopicNames(Collections.singletonList(topic)); assertEquals(DEFAULT_GROUP_ID, data.groupId()); assertEquals(memberId, data.memberId()); assertEquals(0, data.memberEpoch()); - //assertNull(data.instanceId()); + assertNull(data.instanceId()); assertEquals(DEFAULT_MAX_POLL_INTERVAL_MS, data.rebalanceTimeoutMs()); - //assertEquals(Collections.singletonList(topic), data.subscribedTopicNames()); + assertEquals(Collections.singletonList(topic), data.subscribedTopicNames()); assertEquals(DEFAULT_REMOTE_ASSIGNOR, data.serverAssignor()); assertEquals(Collections.emptyList(), data.topicPartitions()); - membershipManager1.onHeartbeatRequestSent(); - assertEquals(MemberState.JOINING, membershipManager1.state()); + membershipManager.onHeartbeatRequestSent(); + assertEquals(MemberState.JOINING, membershipManager.state()); // Mock the response from the group coordinator which returns an assignment ConsumerGroupHeartbeatResponseData.TopicPartitions tpTopic1 = @@ -708,10 +690,11 @@ public void testHeartbeatState() { .setMemberEpoch(1) .setAssignment(assignmentTopic1)); when(metadata.topicNames()).thenReturn(Collections.singletonMap(topicId, "topic1")); - membershipManager1.onHeartbeatSuccess(rs1.data()); + membershipManager.onHeartbeatSuccess(rs1.data()); // We remain in RECONCILING state, as the assignment will be reconciled on the next poll - assertEquals(MemberState.RECONCILING, membershipManager1.state()); + when(membershipManager.state()).thenReturn(MemberState.RECONCILING); + assertEquals(MemberState.RECONCILING, membershipManager.state()); } @Test @@ -797,7 +780,7 @@ public void testFencedMemberStopHeartbeatUntilItReleasesAssignmentToRejoin() { when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); when(membershipManager.state()).thenReturn(MemberState.STABLE); - mockStableMember(membershipManager); + mockStableMember(); time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); when(membershipManager.isLeavingGroup()).thenReturn(true); @@ -838,7 +821,7 @@ private void assertNoHeartbeat(HeartbeatRequestManager hrm) { assertEquals(0, pollResult.unsentRequests.size()); } - private void mockStableMember(MembershipManager membershipManager) { + private void mockStableMember() { membershipManager.onSubscriptionUpdated(); // Heartbeat response without assignment to set the state to STABLE. when(subscriptions.hasAutoAssignedPartitions()).thenReturn(true); From 4f8555806246db6424a892fbe989b3d7913ae584 Mon Sep 17 00:00:00 2001 From: brenden20 Date: Thu, 20 Jun 2024 14:16:45 -0500 Subject: [PATCH 18/46] Update HeartbeatRequestManagerTest.java --- .../clients/consumer/internals/HeartbeatRequestManagerTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 2366004b5a882..341ee71172126 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -44,6 +44,7 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource; + import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; From a94fb4d9b3870a319f6ab4d8b29f60f8952df35c Mon Sep 17 00:00:00 2001 From: brenden20 Date: Thu, 20 Jun 2024 14:24:13 -0500 Subject: [PATCH 19/46] Checkstyle fixes --- .../HeartbeatRequestManagerTest.java | 32 +++++++++---------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 341ee71172126..360eb4b7aba34 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -56,13 +56,13 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Properties; import java.util.SortedSet; -import java.util.HashMap; import static org.apache.kafka.common.utils.Utils.mkSortedSet; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -90,8 +90,8 @@ public class HeartbeatRequestManagerTest { private static final long DEFAULT_RETRY_BACKOFF_MS = 80; private static final long DEFAULT_RETRY_BACKOFF_MAX_MS = 1000; private static final double DEFAULT_HEARTBEAT_JITTER_MS = 0.0; - private static final String memberId = "member-id"; - private static final int memberEpoch = 1; + private static final String DEFAULT_MEMBER_ID = "member-id"; + private static final int DEFAULT_MEMBER_EPOCH = 1; private Time time; private Timer pollTimer; @@ -424,8 +424,8 @@ public void testValidateConsumerGroupHeartbeatRequest() { // Update membershipManager's memberId and memberEpoch ConsumerGroupHeartbeatResponse result = new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() - .setMemberId(memberId) - .setMemberEpoch(memberEpoch)); + .setMemberId(DEFAULT_MEMBER_ID) + .setMemberEpoch(DEFAULT_MEMBER_EPOCH)); membershipManager.onHeartbeatSuccess(result.data()); // Create a ConsumerHeartbeatRequest and verify the payload @@ -448,8 +448,8 @@ public void testValidateConsumerGroupHeartbeatRequest() { when(heartbeatRequest.data()).thenReturn(data); assertEquals(DEFAULT_GROUP_ID, heartbeatRequest.data().groupId()); - assertEquals(memberId, heartbeatRequest.data().memberId()); - assertEquals(memberEpoch, heartbeatRequest.data().memberEpoch()); + assertEquals(DEFAULT_MEMBER_ID, heartbeatRequest.data().memberId()); + assertEquals(DEFAULT_MEMBER_EPOCH, heartbeatRequest.data().memberEpoch()); assertEquals(10000, heartbeatRequest.data().rebalanceTimeoutMs()); assertEquals(subscribedTopics, heartbeatRequest.data().subscribedTopicNames()); assertEquals(DEFAULT_GROUP_INSTANCE_ID, heartbeatRequest.data().instanceId()); @@ -623,13 +623,13 @@ public void testHeartbeatState() { // Mock a response from the group coordinator, that supplies the member ID and a new epoch when(membershipManager.state()).thenReturn(MemberState.STABLE); - when(membershipManager.memberId()).thenReturn(memberId); + when(membershipManager.memberId()).thenReturn(DEFAULT_MEMBER_ID); when(membershipManager.memberEpoch()).thenReturn(1); mockStableMember(); data = heartbeatState.buildRequestData(); data.setTopicPartitions(Collections.emptyList()); assertEquals(DEFAULT_GROUP_ID, data.groupId()); - assertEquals(memberId, data.memberId()); + assertEquals(DEFAULT_MEMBER_ID, data.memberId()); assertEquals(1, data.memberEpoch()); assertNull(data.instanceId()); assertEquals(-1, data.rebalanceTimeoutMs()); @@ -652,7 +652,7 @@ public void testHeartbeatState() { data.setTopicPartitions(Collections.emptyList()); data.setServerAssignor(DEFAULT_REMOTE_ASSIGNOR); assertEquals(DEFAULT_GROUP_ID, data.groupId()); - assertEquals(memberId, data.memberId()); + assertEquals(DEFAULT_MEMBER_ID, data.memberId()); assertEquals(0, data.memberEpoch()); assertNull(data.instanceId()); assertEquals(DEFAULT_MAX_POLL_INTERVAL_MS, data.rebalanceTimeoutMs()); @@ -666,7 +666,7 @@ public void testHeartbeatState() { data = heartbeatState.buildRequestData(); data.setSubscribedTopicNames(Collections.singletonList(topic)); assertEquals(DEFAULT_GROUP_ID, data.groupId()); - assertEquals(memberId, data.memberId()); + assertEquals(DEFAULT_MEMBER_ID, data.memberId()); assertEquals(0, data.memberEpoch()); assertNull(data.instanceId()); assertEquals(DEFAULT_MAX_POLL_INTERVAL_MS, data.rebalanceTimeoutMs()); @@ -687,7 +687,7 @@ public void testHeartbeatState() { assignmentTopic1.setTopicPartitions(Collections.singletonList(tpTopic1)); ConsumerGroupHeartbeatResponse rs1 = new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() .setHeartbeatIntervalMs(DEFAULT_HEARTBEAT_INTERVAL_MS) - .setMemberId(memberId) + .setMemberId(DEFAULT_MEMBER_ID) .setMemberEpoch(1) .setAssignment(assignmentTopic1)); when(metadata.topicNames()).thenReturn(Collections.singletonMap(topicId, "topic1")); @@ -829,8 +829,8 @@ private void mockStableMember() { when(subscriptions.rebalanceListener()).thenReturn(Optional.empty()); ConsumerGroupHeartbeatResponse rs1 = new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() .setHeartbeatIntervalMs(DEFAULT_HEARTBEAT_INTERVAL_MS) - .setMemberId(memberId) - .setMemberEpoch(memberEpoch) + .setMemberId(DEFAULT_MEMBER_ID) + .setMemberEpoch(DEFAULT_MEMBER_EPOCH) .setAssignment(new Assignment()) ); membershipManager.onHeartbeatSuccess(rs1.data()); @@ -883,8 +883,8 @@ private ClientResponse createHeartbeatResponse( ConsumerGroupHeartbeatResponseData data = new ConsumerGroupHeartbeatResponseData() .setErrorCode(error.code()) .setHeartbeatIntervalMs(DEFAULT_HEARTBEAT_INTERVAL_MS) - .setMemberId(memberId) - .setMemberEpoch(memberEpoch); + .setMemberId(DEFAULT_MEMBER_ID) + .setMemberEpoch(DEFAULT_MEMBER_EPOCH); if (error != Errors.NONE) { data.setErrorMessage("stubbed error message"); } From df8abc6ee84a21fa5f3353b267087b5105d1c498 Mon Sep 17 00:00:00 2001 From: brenden20 Date: Fri, 21 Jun 2024 16:24:18 -0500 Subject: [PATCH 20/46] Cleanup and minor fixes --- .../HeartbeatRequestManagerTest.java | 83 ++++--------------- 1 file changed, 17 insertions(+), 66 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 360eb4b7aba34..6aa5952c8f574 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -161,19 +161,13 @@ private void createHeartbeatStateWith0HeartbeatInterval() { backgroundEventHandler); } - private void resetWithZeroHeartbeatInterval() { - createHeartbeatStateWith0HeartbeatInterval(); - } - @Test public void testHeartbeatOnStartup() { when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size()); - resetWithZeroHeartbeatInterval(); - when(membershipManager.state()).thenReturn(MemberState.STABLE); - mockStableMember(); + time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); assertEquals(0, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mock(Node.class))); result = heartbeatRequestManager.poll(time.milliseconds()); @@ -187,9 +181,6 @@ public void testHeartbeatOnStartup() { @Test public void testSuccessfulHeartbeatTiming() { - when(membershipManager.state()).thenReturn(MemberState.STABLE); - mockStableMember(); - long t = time.milliseconds(); when(membershipManager.isLeavingGroup()).thenReturn(true); when(heartbeatRequestState.canSendRequest(t)).thenReturn(false); @@ -229,10 +220,8 @@ public void testSuccessfulHeartbeatTiming() { @Test @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments() { - resetWithZeroHeartbeatInterval(); + time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); String topic = "topic1"; - subscriptions.subscribe(Collections.singleton(topic), Optional.empty()); - membershipManager.onSubscriptionUpdated(); // Create a ConsumerHeartbeatRequest and verify the payload assertEquals(0, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); @@ -266,7 +255,7 @@ public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments() @ValueSource(booleans = {true, false}) public void testSkippingHeartbeat(final boolean shouldSkipHeartbeat) { // The initial heartbeatInterval is set to 0 - resetWithZeroHeartbeatInterval(); + createHeartbeatStateWith0HeartbeatInterval(); // Mocking notInGroup when(membershipManager.shouldSkipHeartbeat()).thenReturn(shouldSkipHeartbeat); @@ -286,8 +275,6 @@ public void testSkippingHeartbeat(final boolean shouldSkipHeartbeat) { @Test public void testTimerNotDue() { - when(membershipManager.state()).thenReturn(MemberState.STABLE); - mockStableMember(); time.sleep(100); // time elapsed < heartbeatInterval, no heartbeat should be sent NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); @@ -307,8 +294,6 @@ public void testTimerNotDue() { @Test public void testHeartbeatNotSentIfAnotherOneInFlight() { - when(membershipManager.state()).thenReturn(MemberState.STABLE); - mockStableMember(); time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); // Heartbeat sent (no response received) @@ -316,26 +301,23 @@ public void testHeartbeatNotSentIfAnotherOneInFlight() { assertEquals(1, result.unsentRequests.size()); NetworkClientDelegate.UnsentRequest inflightReq = result.unsentRequests.get(0); + result = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent while a " + + "previous one is in-flight"); + + time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); + result = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent when the " + + "interval expires if there is a previous HB request in-flight"); + // Receive response for the inflight after the interval expired. The next HB should be sent // on the next poll waiting only for the minimal backoff. inflightReq.handler().onComplete(createHeartbeatResponse(inflightReq, Errors.NONE)); time.sleep(DEFAULT_RETRY_BACKOFF_MS); - when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size(), "A next heartbeat should be sent on " + "the first poll after receiving a response that took longer than the interval, " + "waiting only for the minimal backoff."); - - when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); - - time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); - result = heartbeatRequestManager.poll(time.milliseconds()); - assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent when the " + - "interval expires if there is a previous HB request in-flight"); - - result = heartbeatRequestManager.poll(time.milliseconds()); - assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent while a " + - "previous one is in-flight"); } @Test @@ -358,9 +340,7 @@ public void testHeartbeatOutsideInterval() { @Test public void testNetworkTimeout() { // The initial heartbeatInterval is set to 0 - resetWithZeroHeartbeatInterval(); - when(membershipManager.state()).thenReturn(MemberState.STABLE); - mockStableMember(); + time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); @@ -382,9 +362,7 @@ public void testNetworkTimeout() { @Test public void testFailureOnFatalException() { // The initial heartbeatInterval is set to 0 - resetWithZeroHeartbeatInterval(); - when(membershipManager.state()).thenReturn(MemberState.STABLE); - mockStableMember(); + time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); when(membershipManager.isLeavingGroup()).thenReturn(true); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); @@ -414,9 +392,7 @@ public void testNoCoordinator() { @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) public void testValidateConsumerGroupHeartbeatRequest() { // The initial heartbeatInterval is set to 0, but we're testing - resetWithZeroHeartbeatInterval(); - when(membershipManager.state()).thenReturn(MemberState.STABLE); - mockStableMember(); + time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); List subscribedTopics = Collections.singletonList("topic"); subscriptions.subscribe(new HashSet<>(subscribedTopics), Optional.empty()); @@ -513,8 +489,6 @@ private ConsumerGroupHeartbeatRequest getHeartbeatRequest(HeartbeatRequestManage @ParameterizedTest @MethodSource("errorProvider") public void testHeartbeatResponseOnErrorHandling(final Errors error, final boolean isFatal) { - when(membershipManager.state()).thenReturn(MemberState.STABLE); - mockStableMember(); when(membershipManager.state()).thenReturn(MemberState.FATAL); when(membershipManager.isLeavingGroup()).thenReturn(true); @@ -625,7 +599,6 @@ public void testHeartbeatState() { when(membershipManager.state()).thenReturn(MemberState.STABLE); when(membershipManager.memberId()).thenReturn(DEFAULT_MEMBER_ID); when(membershipManager.memberEpoch()).thenReturn(1); - mockStableMember(); data = heartbeatState.buildRequestData(); data.setTopicPartitions(Collections.emptyList()); assertEquals(DEFAULT_GROUP_ID, data.groupId()); @@ -780,17 +753,12 @@ public void testFencedMemberStopHeartbeatUntilItReleasesAssignmentToRejoin() { backgroundEventHandler); when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); - when(membershipManager.state()).thenReturn(MemberState.STABLE); - mockStableMember(); - time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); - when(membershipManager.isLeavingGroup()).thenReturn(true); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size()); // Receive HB response fencing member when(subscriptions.hasAutoAssignedPartitions()).thenReturn(true); - doNothing().when(membershipManager).transitionToFenced(); ClientResponse response = createHeartbeatResponse(result.unsentRequests.get(0), Errors.FENCED_MEMBER_EPOCH); result.unsentRequests.get(0).handler().onComplete(response); @@ -798,12 +766,12 @@ public void testFencedMemberStopHeartbeatUntilItReleasesAssignmentToRejoin() { verify(heartbeatRequestState).onFailedAttempt(anyLong()); verify(heartbeatRequestState).reset(); - when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(false); + when(membershipManager.shouldSkipHeartbeat()).thenReturn(true); when(membershipManager.state()).thenReturn(MemberState.FENCED); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size(), "Member should not send heartbeats while FENCED"); - when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); + when(membershipManager.shouldSkipHeartbeat()).thenReturn(false); when(membershipManager.state()).thenReturn(MemberState.JOINING); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size(), "Fenced member should resume heartbeat after transitioning to JOINING"); @@ -822,23 +790,6 @@ private void assertNoHeartbeat(HeartbeatRequestManager hrm) { assertEquals(0, pollResult.unsentRequests.size()); } - private void mockStableMember() { - membershipManager.onSubscriptionUpdated(); - // Heartbeat response without assignment to set the state to STABLE. - when(subscriptions.hasAutoAssignedPartitions()).thenReturn(true); - when(subscriptions.rebalanceListener()).thenReturn(Optional.empty()); - ConsumerGroupHeartbeatResponse rs1 = new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() - .setHeartbeatIntervalMs(DEFAULT_HEARTBEAT_INTERVAL_MS) - .setMemberId(DEFAULT_MEMBER_ID) - .setMemberEpoch(DEFAULT_MEMBER_EPOCH) - .setAssignment(new Assignment()) - ); - membershipManager.onHeartbeatSuccess(rs1.data()); - membershipManager.poll(time.milliseconds()); - membershipManager.onHeartbeatRequestSent(); - assertEquals(MemberState.STABLE, membershipManager.state()); - } - private void ensureFatalError(Errors expectedError) { verify(membershipManager).transitionToFatal(); From db88d18b285420325317315b97f10ee567da06cb Mon Sep 17 00:00:00 2001 From: brenden20 Date: Fri, 21 Jun 2024 16:57:53 -0500 Subject: [PATCH 21/46] Added test back and fixed --- .../HeartbeatRequestManagerTest.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 6aa5952c8f574..8b959f3d91684 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -776,6 +776,25 @@ public void testFencedMemberStopHeartbeatUntilItReleasesAssignmentToRejoin() { result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size(), "Fenced member should resume heartbeat after transitioning to JOINING"); } + + @ParameterizedTest + @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) + public void testSendingLeaveGroupHeartbeatWhenPreviousOneInFlight(final short version) { + time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(1, result.unsentRequests.size()); + result = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent while a previous one is in-flight"); + + when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); + when(heartbeatState.buildRequestData()).thenReturn(new ConsumerGroupHeartbeatRequestData().setMemberEpoch(-1)); + ConsumerGroupHeartbeatRequest heartbeatToLeave = getHeartbeatRequest(heartbeatRequestManager, version); + assertEquals(ConsumerGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH, heartbeatToLeave.data().memberEpoch()); + + when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); + NetworkClientDelegate.PollResult pollAgain = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(0, pollAgain.unsentRequests.size()); + } private void assertHeartbeat(HeartbeatRequestManager hrm, int nextPollMs) { NetworkClientDelegate.PollResult pollResult = hrm.poll(time.milliseconds()); From ebb768d0e77296bb62fe63f7c8cd6deba27964f8 Mon Sep 17 00:00:00 2001 From: brenden20 Date: Fri, 21 Jun 2024 17:47:29 -0500 Subject: [PATCH 22/46] Cleanup --- .../internals/HeartbeatRequestManagerTest.java | 18 +++++------------- 1 file changed, 5 insertions(+), 13 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 8b959f3d91684..737d919c3567a 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -30,7 +30,6 @@ import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; -import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData.Assignment; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; @@ -217,28 +216,21 @@ public void testSuccessfulHeartbeatTiming() { assertNextHeartbeatTiming(DEFAULT_HEARTBEAT_INTERVAL_MS - partOfInterval); } - @Test + @ParameterizedTest @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) - public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments() { + public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments(short version) { time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); String topic = "topic1"; // Create a ConsumerHeartbeatRequest and verify the payload assertEquals(0, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); - when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); NetworkClientDelegate.UnsentRequest request = pollResult.unsentRequests.get(0); assertInstanceOf(Builder.class, request.requestBuilder()); - ConsumerGroupHeartbeatRequest heartbeatRequest = mock(ConsumerGroupHeartbeatRequest.class); - ConsumerGroupHeartbeatRequestData data = new ConsumerGroupHeartbeatRequestData(); - data.setSubscribedTopicNames(Collections.singletonList(topic)); - data.setRebalanceTimeoutMs(10000); - data.setGroupId("groupId"); - data.setInstanceId("group-instance-id"); - - when(heartbeatRequest.data()).thenReturn(data); + ConsumerGroupHeartbeatRequest heartbeatRequest = + (ConsumerGroupHeartbeatRequest) request.requestBuilder().build(version); // Should include epoch 0 to join and no member ID. assertTrue(heartbeatRequest.data().memberId().isEmpty()); @@ -776,7 +768,7 @@ public void testFencedMemberStopHeartbeatUntilItReleasesAssignmentToRejoin() { result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size(), "Fenced member should resume heartbeat after transitioning to JOINING"); } - + @ParameterizedTest @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) public void testSendingLeaveGroupHeartbeatWhenPreviousOneInFlight(final short version) { From 10acafb62832fc5c57d0f472dda8359e975782e7 Mon Sep 17 00:00:00 2001 From: brenden20 Date: Fri, 21 Jun 2024 18:47:51 -0500 Subject: [PATCH 23/46] Update HeartbeatRequestManagerTest.java --- .../HeartbeatRequestManagerTest.java | 74 ++++++++++++++----- 1 file changed, 57 insertions(+), 17 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 737d919c3567a..807c19b4201b9 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -38,6 +38,7 @@ import org.apache.kafka.common.requests.ConsumerGroupHeartbeatResponse; import org.apache.kafka.common.requests.RequestHeader; import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; @@ -219,11 +220,47 @@ public void testSuccessfulHeartbeatTiming() { @ParameterizedTest @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments(short version) { + membershipManager = new MembershipManagerImpl( + DEFAULT_GROUP_ID, + Optional.of(DEFAULT_GROUP_INSTANCE_ID), + 0, + Optional.of(""), + subscriptions, + mock(CommitRequestManager.class), + (ConsumerMetadata) metadata, + logContext, + Optional.of(mock(ClientTelemetryReporter.class)), + backgroundEventHandler, + time, + new Metrics() + ); + membershipManager.transitionToJoining(); + + heartbeatState = new HeartbeatState( + subscriptions, + membershipManager, + DEFAULT_MAX_POLL_INTERVAL_MS + ); + + heartbeatRequestManager = createHeartbeatRequestManager( + coordinatorRequestManager, + membershipManager, + heartbeatState, + heartbeatRequestState, + backgroundEventHandler + ); + createHeartbeatStateWith0HeartbeatInterval(); time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); String topic = "topic1"; + HashSet set = new HashSet<>(); + set.add(topic); + when(subscriptions.subscription()).thenReturn(set); + subscriptions.subscribe(Collections.singleton(topic), Optional.empty()); + membershipManager.onSubscriptionUpdated(); // Create a ConsumerHeartbeatRequest and verify the payload assertEquals(0, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); + when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(0, "", 0))); NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); NetworkClientDelegate.UnsentRequest request = pollResult.unsentRequests.get(0); @@ -558,23 +595,37 @@ private void assertNextHeartbeatTiming(long expectedTimeToNextHeartbeatMs) { @Test public void testHeartbeatState() { + membershipManager = new MembershipManagerImpl( + DEFAULT_GROUP_ID, + Optional.empty(), + 0, + Optional.of("uniform"), + subscriptions, + mock(CommitRequestManager.class), + (ConsumerMetadata) metadata, + logContext, + Optional.of(mock(ClientTelemetryReporter.class)), + backgroundEventHandler, + time, + new Metrics() + ); + heartbeatState = new HeartbeatState( subscriptions, membershipManager, - DEFAULT_MAX_POLL_INTERVAL_MS); + DEFAULT_MAX_POLL_INTERVAL_MS + ); heartbeatRequestManager = createHeartbeatRequestManager( coordinatorRequestManager, membershipManager, heartbeatState, heartbeatRequestState, - backgroundEventHandler); + backgroundEventHandler + ); + createHeartbeatStateWith0HeartbeatInterval(); // The initial ConsumerGroupHeartbeatRequest sets most fields to their initial empty values - when(membershipManager.groupId()).thenReturn(DEFAULT_GROUP_ID); - when(membershipManager.memberId()).thenReturn(""); - when(membershipManager.serverAssignor()).thenReturn(Optional.of(DEFAULT_REMOTE_ASSIGNOR)); - when(membershipManager.state()).thenReturn(MemberState.UNSUBSCRIBED); ConsumerGroupHeartbeatRequestData data = heartbeatState.buildRequestData(); assertEquals(DEFAULT_GROUP_ID, data.groupId()); assertEquals("", data.memberId()); @@ -588,9 +639,6 @@ public void testHeartbeatState() { assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state()); // Mock a response from the group coordinator, that supplies the member ID and a new epoch - when(membershipManager.state()).thenReturn(MemberState.STABLE); - when(membershipManager.memberId()).thenReturn(DEFAULT_MEMBER_ID); - when(membershipManager.memberEpoch()).thenReturn(1); data = heartbeatState.buildRequestData(); data.setTopicPartitions(Collections.emptyList()); assertEquals(DEFAULT_GROUP_ID, data.groupId()); @@ -605,17 +653,11 @@ public void testHeartbeatState() { assertEquals(MemberState.STABLE, membershipManager.state()); // Join the group and subscribe to a topic, but the response has not yet been received - when(membershipManager.memberEpoch()).thenReturn(0); - when(membershipManager.state()).thenReturn(MemberState.JOINING); String topic = "topic1"; subscriptions.subscribe(Collections.singleton(topic), Optional.empty()); membershipManager.onSubscriptionUpdated(); membershipManager.transitionToFenced(); // And indirect way of moving to JOINING state data = heartbeatState.buildRequestData(); - data.setRebalanceTimeoutMs(10000); - data.setSubscribedTopicNames(Collections.singletonList(topic)); - data.setTopicPartitions(Collections.emptyList()); - data.setServerAssignor(DEFAULT_REMOTE_ASSIGNOR); assertEquals(DEFAULT_GROUP_ID, data.groupId()); assertEquals(DEFAULT_MEMBER_ID, data.memberId()); assertEquals(0, data.memberEpoch()); @@ -629,7 +671,6 @@ public void testHeartbeatState() { membershipManager.transitionToFenced(); data = heartbeatState.buildRequestData(); - data.setSubscribedTopicNames(Collections.singletonList(topic)); assertEquals(DEFAULT_GROUP_ID, data.groupId()); assertEquals(DEFAULT_MEMBER_ID, data.memberId()); assertEquals(0, data.memberEpoch()); @@ -659,7 +700,6 @@ public void testHeartbeatState() { membershipManager.onHeartbeatSuccess(rs1.data()); // We remain in RECONCILING state, as the assignment will be reconciled on the next poll - when(membershipManager.state()).thenReturn(MemberState.RECONCILING); assertEquals(MemberState.RECONCILING, membershipManager.state()); } From 4249d869086afd4a3f2a24ce84332754737f6e78 Mon Sep 17 00:00:00 2001 From: brenden20 Date: Mon, 24 Jun 2024 10:05:16 -0500 Subject: [PATCH 24/46] Fix testHeartbeatState() --- .../HeartbeatRequestManagerTest.java | 20 +++++++++++++++++-- 1 file changed, 18 insertions(+), 2 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 807c19b4201b9..b42b5a5cef515 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -63,6 +63,7 @@ import java.util.Optional; import java.util.Properties; import java.util.SortedSet; +import java.util.concurrent.CompletableFuture; import static org.apache.kafka.common.utils.Utils.mkSortedSet; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -595,13 +596,14 @@ private void assertNextHeartbeatTiming(long expectedTimeToNextHeartbeatMs) { @Test public void testHeartbeatState() { + CommitRequestManager commitRequestManager = mock(CommitRequestManager.class); membershipManager = new MembershipManagerImpl( DEFAULT_GROUP_ID, Optional.empty(), 0, Optional.of("uniform"), subscriptions, - mock(CommitRequestManager.class), + commitRequestManager, (ConsumerMetadata) metadata, logContext, Optional.of(mock(ClientTelemetryReporter.class)), @@ -639,6 +641,19 @@ public void testHeartbeatState() { assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state()); // Mock a response from the group coordinator, that supplies the member ID and a new epoch + membershipManager.onSubscriptionUpdated(); + when(subscriptions.hasAutoAssignedPartitions()).thenReturn(true); + when(subscriptions.rebalanceListener()).thenReturn(Optional.empty()); + ConsumerGroupHeartbeatResponse rs1 = new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() + .setHeartbeatIntervalMs(DEFAULT_HEARTBEAT_INTERVAL_MS) + .setMemberId(DEFAULT_MEMBER_ID) + .setMemberEpoch(DEFAULT_MEMBER_EPOCH) + .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()) + ); + when(commitRequestManager.maybeAutoCommitSyncBeforeRevocation(anyLong())).thenReturn(CompletableFuture.completedFuture(null)); + membershipManager.onHeartbeatSuccess(rs1.data()); + membershipManager.poll(time.milliseconds()); + membershipManager.onHeartbeatRequestSent(); data = heartbeatState.buildRequestData(); data.setTopicPartitions(Collections.emptyList()); assertEquals(DEFAULT_GROUP_ID, data.groupId()); @@ -657,6 +672,7 @@ public void testHeartbeatState() { subscriptions.subscribe(Collections.singleton(topic), Optional.empty()); membershipManager.onSubscriptionUpdated(); membershipManager.transitionToFenced(); // And indirect way of moving to JOINING state + when(subscriptions.subscription()).thenReturn(Collections.singleton(topic)); data = heartbeatState.buildRequestData(); assertEquals(DEFAULT_GROUP_ID, data.groupId()); assertEquals(DEFAULT_MEMBER_ID, data.memberId()); @@ -691,7 +707,7 @@ public void testHeartbeatState() { ConsumerGroupHeartbeatResponseData.Assignment assignmentTopic1 = new ConsumerGroupHeartbeatResponseData.Assignment(); assignmentTopic1.setTopicPartitions(Collections.singletonList(tpTopic1)); - ConsumerGroupHeartbeatResponse rs1 = new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() + rs1 = new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() .setHeartbeatIntervalMs(DEFAULT_HEARTBEAT_INTERVAL_MS) .setMemberId(DEFAULT_MEMBER_ID) .setMemberEpoch(1) From adb67ed0a64b1c0ec9bb0abcbfef81941d2e132d Mon Sep 17 00:00:00 2001 From: brenden20 Date: Mon, 24 Jun 2024 10:15:32 -0500 Subject: [PATCH 25/46] Fix testValidateConsumerGroupHeartbeatRequest() --- .../HeartbeatRequestManagerTest.java | 50 +++++++++++++------ 1 file changed, 35 insertions(+), 15 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index b42b5a5cef515..8cd55bfc2ab3b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -418,14 +418,44 @@ public void testNoCoordinator() { assertEquals(0, result.unsentRequests.size()); } - @Test + @ParameterizedTest @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) - public void testValidateConsumerGroupHeartbeatRequest() { + public void testValidateConsumerGroupHeartbeatRequest(final short version) { + membershipManager = new MembershipManagerImpl( + DEFAULT_GROUP_ID, + Optional.of(DEFAULT_GROUP_INSTANCE_ID), + 0, + Optional.of("uniform"), + subscriptions, + mock(CommitRequestManager.class), + (ConsumerMetadata) metadata, + logContext, + Optional.of(mock(ClientTelemetryReporter.class)), + backgroundEventHandler, + time, + new Metrics() + ); + membershipManager.transitionToJoining(); + + heartbeatState = new HeartbeatState( + subscriptions, + membershipManager, + DEFAULT_MAX_POLL_INTERVAL_MS + ); + + heartbeatRequestManager = createHeartbeatRequestManager( + coordinatorRequestManager, + membershipManager, + heartbeatState, + heartbeatRequestState, + backgroundEventHandler + ); + // The initial heartbeatInterval is set to 0, but we're testing time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); List subscribedTopics = Collections.singletonList("topic"); - subscriptions.subscribe(new HashSet<>(subscribedTopics), Optional.empty()); + when(subscriptions.subscription()).thenReturn(Collections.singleton("topic")); // Update membershipManager's memberId and memberEpoch ConsumerGroupHeartbeatResponse result = @@ -441,17 +471,8 @@ public void testValidateConsumerGroupHeartbeatRequest() { NetworkClientDelegate.UnsentRequest request = pollResult.unsentRequests.get(0); assertInstanceOf(Builder.class, request.requestBuilder()); - ConsumerGroupHeartbeatRequest heartbeatRequest = mock(ConsumerGroupHeartbeatRequest.class); - ConsumerGroupHeartbeatRequestData data = new ConsumerGroupHeartbeatRequestData(); - data.setRebalanceTimeoutMs(10000); - data.setGroupId("groupId"); - data.setInstanceId("group-instance-id"); - data.setMemberId("member-id"); - data.setMemberEpoch(1); - data.setSubscribedTopicNames(subscribedTopics); - data.setServerAssignor("uniform"); - - when(heartbeatRequest.data()).thenReturn(data); + ConsumerGroupHeartbeatRequest heartbeatRequest = + (ConsumerGroupHeartbeatRequest) request.requestBuilder().build(version); assertEquals(DEFAULT_GROUP_ID, heartbeatRequest.data().groupId()); assertEquals(DEFAULT_MEMBER_ID, heartbeatRequest.data().memberId()); @@ -655,7 +676,6 @@ public void testHeartbeatState() { membershipManager.poll(time.milliseconds()); membershipManager.onHeartbeatRequestSent(); data = heartbeatState.buildRequestData(); - data.setTopicPartitions(Collections.emptyList()); assertEquals(DEFAULT_GROUP_ID, data.groupId()); assertEquals(DEFAULT_MEMBER_ID, data.memberId()); assertEquals(1, data.memberEpoch()); From 4d837e8e2772613597180551b6be9e873ea6b51f Mon Sep 17 00:00:00 2001 From: brenden20 Date: Wed, 26 Jun 2024 14:03:50 -0500 Subject: [PATCH 26/46] Update HeartbeatRequestManagerTest.java --- .../clients/consumer/internals/HeartbeatRequestManagerTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 8cd55bfc2ab3b..39b6201302b7b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -75,7 +75,6 @@ import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.Mockito.clearInvocations; -import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; From 00ba2e85c0d6d3f5f353393ae94892a39438b17d Mon Sep 17 00:00:00 2001 From: brenden20 Date: Wed, 3 Jul 2024 16:24:56 -0500 Subject: [PATCH 27/46] Minor fixes --- .../consumer/internals/HeartbeatRequestManagerTest.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 39b6201302b7b..f83fd91ce77ad 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -108,7 +108,6 @@ public class HeartbeatRequestManagerTest { @BeforeEach public void setUp() { this.time = new MockTime(); - Metrics metrics = new Metrics(time); this.logContext = new LogContext(); this.pollTimer = mock(Timer.class); this.coordinatorRequestManager = mock(CoordinatorRequestManager.class); @@ -117,6 +116,7 @@ public void setUp() { this.subscriptions = mock(SubscriptionState.class); this.membershipManager = mock(MembershipManagerImpl.class); this.metadata = mock(ConsumerMetadata.class); + Metrics metrics = new Metrics(time); ConsumerConfig config = mock(ConsumerConfig.class); this.heartbeatRequestState = spy(new HeartbeatRequestState( @@ -345,8 +345,8 @@ public void testHeartbeatNotSentIfAnotherOneInFlight() { time.sleep(DEFAULT_RETRY_BACKOFF_MS); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size(), "A next heartbeat should be sent on " + - "the first poll after receiving a response that took longer than the interval, " + - "waiting only for the minimal backoff."); + "the first poll after receiving a response that took longer than the interval, " + + "waiting only for the minimal backoff."); } @Test @@ -746,7 +746,6 @@ public void testPollTimerExpiration() { heartbeatState, heartbeatRequestState, backgroundEventHandler); - when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); when(membershipManager.shouldSkipHeartbeat()).thenReturn(false); From e84e2379864c94406d8df5f6e11695d06b38768c Mon Sep 17 00:00:00 2001 From: brenden20 Date: Mon, 8 Jul 2024 14:05:54 -0500 Subject: [PATCH 28/46] Various text fixes/updates --- .../internals/HeartbeatRequestManagerTest.java | 16 ++++------------ 1 file changed, 4 insertions(+), 12 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index f83fd91ce77ad..f589cb9ac4d20 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -138,10 +138,7 @@ public void setUp() { backgroundEventHandler, metrics); - when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); - Map> map = new HashMap<>(); - LocalAssignment local = new LocalAssignment(0, map); - when(membershipManager.currentAssignment()).thenReturn(local); + when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mock(Node.class))); } private void createHeartbeatStateWith0HeartbeatInterval() { @@ -163,18 +160,18 @@ private void createHeartbeatStateWith0HeartbeatInterval() { @Test public void testHeartbeatOnStartup() { - when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); + when(membershipManager.shouldSkipHeartbeat()).thenReturn(true); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size()); time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); assertEquals(0, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); - when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mock(Node.class))); + when(membershipManager.shouldSkipHeartbeat()).thenReturn(false); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size()); // Ensure we do not resend the request without the first request being completed - when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); + when(heartbeatRequestState.requestInFlight()).thenReturn(true); NetworkClientDelegate.PollResult result2 = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result2.unsentRequests.size()); } @@ -260,7 +257,6 @@ public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments(s // Create a ConsumerHeartbeatRequest and verify the payload assertEquals(0, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); - when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(0, "", 0))); NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); NetworkClientDelegate.UnsentRequest request = pollResult.unsentRequests.get(0); @@ -370,7 +366,6 @@ public void testHeartbeatOutsideInterval() { public void testNetworkTimeout() { // The initial heartbeatInterval is set to 0 time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); - when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size()); @@ -394,7 +389,6 @@ public void testFailureOnFatalException() { time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); when(membershipManager.isLeavingGroup()).thenReturn(true); - when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size()); @@ -498,7 +492,6 @@ public void testValidateConsumerGroupHeartbeatRequestAssignmentSentWhenLocalEpoc backgroundEventHandler); when(membershipManager.shouldHeartbeatNow()).thenReturn(true); - when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); Uuid topicId = Uuid.randomUuid(); ConsumerGroupHeartbeatRequestData.TopicPartitions expectedTopicPartitions = @@ -746,7 +739,6 @@ public void testPollTimerExpiration() { heartbeatState, heartbeatRequestState, backgroundEventHandler); - when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); when(membershipManager.shouldSkipHeartbeat()).thenReturn(false); // On poll timer expiration, the member should send a last heartbeat to leave the group From 373fb7cc97f7e25f10f93bec175abca28b0b7553 Mon Sep 17 00:00:00 2001 From: brenden20 Date: Mon, 8 Jul 2024 14:55:22 -0500 Subject: [PATCH 29/46] Update HeartbeatRequestManagerTest.java --- .../consumer/internals/HeartbeatRequestManagerTest.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index f589cb9ac4d20..8bc50f5f036ac 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -109,7 +109,7 @@ public class HeartbeatRequestManagerTest { public void setUp() { this.time = new MockTime(); this.logContext = new LogContext(); - this.pollTimer = mock(Timer.class); + this.pollTimer = time.timer(1000); this.coordinatorRequestManager = mock(CoordinatorRequestManager.class); this.heartbeatState = mock(HeartbeatState.class); this.backgroundEventHandler = mock(BackgroundEventHandler.class); @@ -249,6 +249,7 @@ public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments(s createHeartbeatStateWith0HeartbeatInterval(); time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); String topic = "topic1"; + // Make a singleton set HashSet set = new HashSet<>(); set.add(topic); when(subscriptions.subscription()).thenReturn(set); @@ -305,8 +306,6 @@ public void testTimerNotDue() { assertEquals(0, result.unsentRequests.size()); assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS - 100, result.timeUntilNextPollMs); - - when(pollTimer.remainingMs()).thenReturn(1800L); assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS - 100, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); // Member in state where it should not send Heartbeat anymore From cc719f55dfbe6d6a275e31b84ce3647f348c1ba4 Mon Sep 17 00:00:00 2001 From: brenden20 Date: Mon, 8 Jul 2024 18:11:24 -0500 Subject: [PATCH 30/46] PollTimer changed to spy --- .../consumer/internals/HeartbeatRequestManagerTest.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 8bc50f5f036ac..cd916027f2311 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -109,7 +109,7 @@ public class HeartbeatRequestManagerTest { public void setUp() { this.time = new MockTime(); this.logContext = new LogContext(); - this.pollTimer = time.timer(1000); + this.pollTimer = spy(time.timer(1000)); this.coordinatorRequestManager = mock(CoordinatorRequestManager.class); this.heartbeatState = mock(HeartbeatState.class); this.backgroundEventHandler = mock(BackgroundEventHandler.class); @@ -160,18 +160,15 @@ private void createHeartbeatStateWith0HeartbeatInterval() { @Test public void testHeartbeatOnStartup() { - when(membershipManager.shouldSkipHeartbeat()).thenReturn(true); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size()); - time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); + createHeartbeatStateWith0HeartbeatInterval(); assertEquals(0, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); - when(membershipManager.shouldSkipHeartbeat()).thenReturn(false); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size()); // Ensure we do not resend the request without the first request being completed - when(heartbeatRequestState.requestInFlight()).thenReturn(true); NetworkClientDelegate.PollResult result2 = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result2.unsentRequests.size()); } @@ -306,6 +303,7 @@ public void testTimerNotDue() { assertEquals(0, result.unsentRequests.size()); assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS - 100, result.timeUntilNextPollMs); + //when(pollTimer.remainingMs()).thenReturn(1000L); assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS - 100, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); // Member in state where it should not send Heartbeat anymore From 6ac79c4d630b7c2efd409bb3fdab3a4273ddfe32 Mon Sep 17 00:00:00 2001 From: brenden20 Date: Mon, 8 Jul 2024 18:28:25 -0500 Subject: [PATCH 31/46] PollTimer back to mock --- .../consumer/internals/HeartbeatRequestManagerTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index cd916027f2311..f9ef0fea18503 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -109,7 +109,7 @@ public class HeartbeatRequestManagerTest { public void setUp() { this.time = new MockTime(); this.logContext = new LogContext(); - this.pollTimer = spy(time.timer(1000)); + this.pollTimer = mock(Timer.class); this.coordinatorRequestManager = mock(CoordinatorRequestManager.class); this.heartbeatState = mock(HeartbeatState.class); this.backgroundEventHandler = mock(BackgroundEventHandler.class); @@ -303,7 +303,7 @@ public void testTimerNotDue() { assertEquals(0, result.unsentRequests.size()); assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS - 100, result.timeUntilNextPollMs); - //when(pollTimer.remainingMs()).thenReturn(1000L); + when(pollTimer.remainingMs()).thenReturn(1800L); assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS - 100, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); // Member in state where it should not send Heartbeat anymore From d5fe331c72802cde6f77233c28fd5157ad7cd34d Mon Sep 17 00:00:00 2001 From: brenden20 Date: Mon, 8 Jul 2024 19:00:42 -0500 Subject: [PATCH 32/46] PollTimer change and test fixes --- .../HeartbeatRequestManagerTest.java | 22 +++---------------- 1 file changed, 3 insertions(+), 19 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index f9ef0fea18503..d112f68c6717a 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -109,7 +109,7 @@ public class HeartbeatRequestManagerTest { public void setUp() { this.time = new MockTime(); this.logContext = new LogContext(); - this.pollTimer = mock(Timer.class); + this.pollTimer = spy(time.timer(DEFAULT_MAX_POLL_INTERVAL_MS)); this.coordinatorRequestManager = mock(CoordinatorRequestManager.class); this.heartbeatState = mock(HeartbeatState.class); this.backgroundEventHandler = mock(BackgroundEventHandler.class); @@ -282,7 +282,6 @@ public void testSkippingHeartbeat(final boolean shouldSkipHeartbeat) { // Mocking notInGroup when(membershipManager.shouldSkipHeartbeat()).thenReturn(shouldSkipHeartbeat); - when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); @@ -303,7 +302,6 @@ public void testTimerNotDue() { assertEquals(0, result.unsentRequests.size()); assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS - 100, result.timeUntilNextPollMs); - when(pollTimer.remainingMs()).thenReturn(1800L); assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS - 100, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); // Member in state where it should not send Heartbeat anymore @@ -346,7 +344,6 @@ public void testHeartbeatNotSentIfAnotherOneInFlight() { public void testHeartbeatOutsideInterval() { when(membershipManager.shouldSkipHeartbeat()).thenReturn(false); when(membershipManager.shouldHeartbeatNow()).thenReturn(true); - when(pollTimer.remainingMs()).thenReturn(Long.MAX_VALUE); when(heartbeatRequestState.timeToNextHeartbeatMs(time.milliseconds())).thenReturn(1000L); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); @@ -399,8 +396,6 @@ public void testNoCoordinator() { when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); - when(pollTimer.isExpired()).thenReturn(false); - when(pollTimer.remainingMs()).thenReturn(Long.MAX_VALUE); when(heartbeatRequestState.timeToNextHeartbeatMs(time.milliseconds())).thenReturn(1000L); assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs); @@ -455,7 +450,6 @@ public void testValidateConsumerGroupHeartbeatRequest(final short version) { membershipManager.onHeartbeatSuccess(result.data()); // Create a ConsumerHeartbeatRequest and verify the payload - when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); NetworkClientDelegate.UnsentRequest request = pollResult.unsentRequests.get(0); @@ -529,8 +523,8 @@ private ConsumerGroupHeartbeatRequest getHeartbeatRequest(HeartbeatRequestManage @ParameterizedTest @MethodSource("errorProvider") public void testHeartbeatResponseOnErrorHandling(final Errors error, final boolean isFatal) { - when(membershipManager.state()).thenReturn(MemberState.FATAL); - when(membershipManager.isLeavingGroup()).thenReturn(true); + if (isFatal) + when(membershipManager.state()).thenReturn(MemberState.FATAL); // Handling errors on the second heartbeat time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); @@ -548,17 +542,11 @@ public void testHeartbeatResponseOnErrorHandling(final Errors error, final boole switch (error) { case NONE: verify(membershipManager).onHeartbeatSuccess(mockResponse.data()); - when(heartbeatRequestState.timeToNextHeartbeatMs(anyLong())).thenReturn(1000L); - when(heartbeatRequestState.canSendRequest(time.milliseconds())).thenReturn(false); - when(heartbeatRequestState.canSendRequest(time.milliseconds() + 1000)).thenReturn(true); assertNextHeartbeatTiming(DEFAULT_HEARTBEAT_INTERVAL_MS); break; case COORDINATOR_LOAD_IN_PROGRESS: verify(backgroundEventHandler, never()).add(any()); - when(heartbeatRequestState.timeToNextHeartbeatMs(anyLong())).thenReturn(80L); - when(heartbeatRequestState.canSendRequest(time.milliseconds())).thenReturn(false); - when(heartbeatRequestState.canSendRequest(time.milliseconds() + 80)).thenReturn(true); assertNextHeartbeatTiming(DEFAULT_RETRY_BACKOFF_MS); break; @@ -566,13 +554,11 @@ public void testHeartbeatResponseOnErrorHandling(final Errors error, final boole case NOT_COORDINATOR: verify(backgroundEventHandler, never()).add(any()); verify(coordinatorRequestManager).markCoordinatorUnknown(any(), anyLong()); - when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); assertNextHeartbeatTiming(0); break; case UNKNOWN_MEMBER_ID: case FENCED_MEMBER_EPOCH: verify(backgroundEventHandler, never()).add(any()); - when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); assertNextHeartbeatTiming(0); break; default: @@ -787,7 +773,6 @@ public void testisExpiredByUsedForLogging() { time.sleep(DEFAULT_MAX_POLL_INTERVAL_MS + exceededTimeMs); when(membershipManager.isLeavingGroup()).thenReturn(false); - when(pollTimer.isExpired()).thenReturn(true); NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); verify(membershipManager).transitionToSendingLeaveGroup(true); @@ -807,7 +792,6 @@ public void testFencedMemberStopHeartbeatUntilItReleasesAssignmentToRejoin() { heartbeatRequestState, backgroundEventHandler); - when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size()); From a019cf3424076ec70a520a4beadd47f459cfd1ef Mon Sep 17 00:00:00 2001 From: brenden20 Date: Mon, 8 Jul 2024 19:30:55 -0500 Subject: [PATCH 33/46] Couple of test fixes --- .../HeartbeatRequestManagerTest.java | 29 ++++++++++--------- 1 file changed, 15 insertions(+), 14 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index d112f68c6717a..8004f08cb1a37 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -141,7 +141,7 @@ public void setUp() { when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mock(Node.class))); } - private void createHeartbeatStateWith0HeartbeatInterval() { + private void createHeartbeatStateWithZeroHeartbeatInterval() { this.heartbeatRequestState = spy(new HeartbeatRequestState( logContext, time, @@ -150,7 +150,7 @@ private void createHeartbeatStateWith0HeartbeatInterval() { DEFAULT_RETRY_BACKOFF_MAX_MS, DEFAULT_HEARTBEAT_JITTER_MS)); - heartbeatRequestManager = createHeartbeatRequestManager( + this.heartbeatRequestManager = createHeartbeatRequestManager( coordinatorRequestManager, membershipManager, heartbeatState, @@ -163,7 +163,7 @@ public void testHeartbeatOnStartup() { NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size()); - createHeartbeatStateWith0HeartbeatInterval(); + createHeartbeatStateWithZeroHeartbeatInterval(); assertEquals(0, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size()); @@ -243,7 +243,7 @@ public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments(s heartbeatRequestState, backgroundEventHandler ); - createHeartbeatStateWith0HeartbeatInterval(); + createHeartbeatStateWithZeroHeartbeatInterval(); time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); String topic = "topic1"; // Make a singleton set @@ -278,7 +278,7 @@ public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments(s @ValueSource(booleans = {true, false}) public void testSkippingHeartbeat(final boolean shouldSkipHeartbeat) { // The initial heartbeatInterval is set to 0 - createHeartbeatStateWith0HeartbeatInterval(); + createHeartbeatStateWithZeroHeartbeatInterval(); // Mocking notInGroup when(membershipManager.shouldSkipHeartbeat()).thenReturn(shouldSkipHeartbeat); @@ -321,6 +321,7 @@ public void testHeartbeatNotSentIfAnotherOneInFlight() { assertEquals(1, result.unsentRequests.size()); NetworkClientDelegate.UnsentRequest inflightReq = result.unsentRequests.get(0); + time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent while a " + "previous one is in-flight"); @@ -344,7 +345,7 @@ public void testHeartbeatNotSentIfAnotherOneInFlight() { public void testHeartbeatOutsideInterval() { when(membershipManager.shouldSkipHeartbeat()).thenReturn(false); when(membershipManager.shouldHeartbeatNow()).thenReturn(true); - when(heartbeatRequestState.timeToNextHeartbeatMs(time.milliseconds())).thenReturn(1000L); + when(heartbeatRequestState.timeToNextHeartbeatMs(time.milliseconds())).thenReturn((long) DEFAULT_HEARTBEAT_INTERVAL_MS); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); // Heartbeat should be sent @@ -359,22 +360,22 @@ public void testHeartbeatOutsideInterval() { @Test public void testNetworkTimeout() { // The initial heartbeatInterval is set to 0 - time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); + createHeartbeatStateWithZeroHeartbeatInterval(); when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size()); // Mimic network timeout result.unsentRequests.get(0).handler().onFailure(time.milliseconds(), new TimeoutException("timeout")); - time.sleep(1); - result = heartbeatRequestManager.poll(time.milliseconds()); - assertEquals(1, result.unsentRequests.size()); - // Assure the manager will backoff on timeout - when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); time.sleep(DEFAULT_RETRY_BACKOFF_MS - 1); + when(heartbeatRequestState.canSendRequest(anyLong())).thenCallRealMethod(); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size()); + + time.sleep(1); + result = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(1, result.unsentRequests.size()); } @Test @@ -396,7 +397,7 @@ public void testNoCoordinator() { when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); - when(heartbeatRequestState.timeToNextHeartbeatMs(time.milliseconds())).thenReturn(1000L); + when(heartbeatRequestState.timeToNextHeartbeatMs(time.milliseconds())).thenReturn((long) DEFAULT_HEARTBEAT_INTERVAL_MS); assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs); assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); @@ -621,7 +622,7 @@ public void testHeartbeatState() { heartbeatRequestState, backgroundEventHandler ); - createHeartbeatStateWith0HeartbeatInterval(); + createHeartbeatStateWithZeroHeartbeatInterval(); // The initial ConsumerGroupHeartbeatRequest sets most fields to their initial empty values ConsumerGroupHeartbeatRequestData data = heartbeatState.buildRequestData(); From 37bf5af2086c236196deab28133c4743328def6d Mon Sep 17 00:00:00 2001 From: brenden20 Date: Mon, 8 Jul 2024 19:56:01 -0500 Subject: [PATCH 34/46] Various test changes and cleanup --- .../HeartbeatRequestManagerTest.java | 30 ++++--------------- 1 file changed, 6 insertions(+), 24 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 8004f08cb1a37..f8f344e5a360b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -56,9 +56,7 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.HashSet; -import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Properties; @@ -175,11 +173,6 @@ public void testHeartbeatOnStartup() { @Test public void testSuccessfulHeartbeatTiming() { - long t = time.milliseconds(); - when(membershipManager.isLeavingGroup()).thenReturn(true); - when(heartbeatRequestState.canSendRequest(t)).thenReturn(false); - when(heartbeatRequestState.canSendRequest(t + 1000)).thenReturn(true); - when(heartbeatRequestState.timeToNextHeartbeatMs(anyLong())).thenReturn(1000L); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent while interval has not expired"); @@ -194,7 +187,6 @@ public void testSuccessfulHeartbeatTiming() { "Heartbeat timer was not reset to the interval when the heartbeat request was sent."); long partOfInterval = DEFAULT_HEARTBEAT_INTERVAL_MS / 3; - when(heartbeatRequestState.timeToNextHeartbeatMs(anyLong())).thenReturn(DEFAULT_HEARTBEAT_INTERVAL_MS - partOfInterval); time.sleep(partOfInterval); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size(), @@ -203,10 +195,6 @@ public void testSuccessfulHeartbeatTiming() { heartbeatRequestState.timeToNextHeartbeatMs(time.milliseconds()), "Time to next interval was not properly updated."); - t = time.milliseconds(); - when(heartbeatRequestState.canSendRequest(t)).thenReturn(false); - t = t + DEFAULT_HEARTBEAT_INTERVAL_MS - partOfInterval; - when(heartbeatRequestState.canSendRequest(t)).thenReturn(true); inflightReq.handler().onComplete(createHeartbeatResponse(inflightReq, Errors.NONE)); assertNextHeartbeatTiming(DEFAULT_HEARTBEAT_INTERVAL_MS - partOfInterval); } @@ -345,7 +333,6 @@ public void testHeartbeatNotSentIfAnotherOneInFlight() { public void testHeartbeatOutsideInterval() { when(membershipManager.shouldSkipHeartbeat()).thenReturn(false); when(membershipManager.shouldHeartbeatNow()).thenReturn(true); - when(heartbeatRequestState.timeToNextHeartbeatMs(time.milliseconds())).thenReturn((long) DEFAULT_HEARTBEAT_INTERVAL_MS); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); // Heartbeat should be sent @@ -361,7 +348,6 @@ public void testHeartbeatOutsideInterval() { public void testNetworkTimeout() { // The initial heartbeatInterval is set to 0 createHeartbeatStateWithZeroHeartbeatInterval(); - when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size()); // Mimic network timeout @@ -369,7 +355,6 @@ public void testNetworkTimeout() { // Assure the manager will backoff on timeout time.sleep(DEFAULT_RETRY_BACKOFF_MS - 1); - when(heartbeatRequestState.canSendRequest(anyLong())).thenCallRealMethod(); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size()); @@ -383,8 +368,6 @@ public void testFailureOnFatalException() { // The initial heartbeatInterval is set to 0 time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); - when(membershipManager.isLeavingGroup()).thenReturn(true); - when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size()); result.unsentRequests.get(0).handler().onFailure(time.milliseconds(), new KafkaException("fatal")); @@ -397,8 +380,6 @@ public void testNoCoordinator() { when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); - when(heartbeatRequestState.timeToNextHeartbeatMs(time.milliseconds())).thenReturn((long) DEFAULT_HEARTBEAT_INTERVAL_MS); - assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs); assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); assertEquals(0, result.unsentRequests.size()); @@ -440,8 +421,8 @@ public void testValidateConsumerGroupHeartbeatRequest(final short version) { // The initial heartbeatInterval is set to 0, but we're testing time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); - List subscribedTopics = Collections.singletonList("topic"); - when(subscriptions.subscription()).thenReturn(Collections.singleton("topic")); + String subscribedTopic = "topic"; + when(subscriptions.subscription()).thenReturn(Collections.singleton(subscribedTopic)); // Update membershipManager's memberId and memberEpoch ConsumerGroupHeartbeatResponse result = @@ -463,7 +444,7 @@ public void testValidateConsumerGroupHeartbeatRequest(final short version) { assertEquals(DEFAULT_MEMBER_ID, heartbeatRequest.data().memberId()); assertEquals(DEFAULT_MEMBER_EPOCH, heartbeatRequest.data().memberEpoch()); assertEquals(10000, heartbeatRequest.data().rebalanceTimeoutMs()); - assertEquals(subscribedTopics, heartbeatRequest.data().subscribedTopicNames()); + assertEquals(subscribedTopic, heartbeatRequest.data().subscribedTopicNames().get(0)); assertEquals(DEFAULT_GROUP_INSTANCE_ID, heartbeatRequest.data().instanceId()); assertEquals(DEFAULT_REMOTE_ASSIGNOR, heartbeatRequest.data().serverAssignor()); } @@ -826,12 +807,13 @@ public void testSendingLeaveGroupHeartbeatWhenPreviousOneInFlight(final short ve result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent while a previous one is in-flight"); - when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); + when(membershipManager.state()).thenReturn(MemberState.LEAVING); when(heartbeatState.buildRequestData()).thenReturn(new ConsumerGroupHeartbeatRequestData().setMemberEpoch(-1)); ConsumerGroupHeartbeatRequest heartbeatToLeave = getHeartbeatRequest(heartbeatRequestManager, version); assertEquals(ConsumerGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH, heartbeatToLeave.data().memberEpoch()); - when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); + //when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); + when(membershipManager.shouldSkipHeartbeat()).thenReturn(true); NetworkClientDelegate.PollResult pollAgain = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, pollAgain.unsentRequests.size()); } From 155e01772d62cedc10db231162860955062d1aec Mon Sep 17 00:00:00 2001 From: brenden20 <118419078+brenden20@users.noreply.github.com> Date: Mon, 15 Jul 2024 14:21:41 -0500 Subject: [PATCH 35/46] Test updates --- .../HeartbeatRequestManagerTest.java | 79 +++++++++---------- 1 file changed, 38 insertions(+), 41 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index f8f344e5a360b..38c70eabc86ff 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -56,10 +56,10 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.HashSet; import java.util.Map; import java.util.Optional; import java.util.Properties; +import java.util.Set; import java.util.SortedSet; import java.util.concurrent.CompletableFuture; @@ -206,7 +206,7 @@ public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments(s DEFAULT_GROUP_ID, Optional.of(DEFAULT_GROUP_INSTANCE_ID), 0, - Optional.of(""), + Optional.empty(), subscriptions, mock(CommitRequestManager.class), (ConsumerMetadata) metadata, @@ -234,12 +234,9 @@ public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments(s createHeartbeatStateWithZeroHeartbeatInterval(); time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); String topic = "topic1"; - // Make a singleton set - HashSet set = new HashSet<>(); - set.add(topic); + Set set = Collections.singleton(topic); when(subscriptions.subscription()).thenReturn(set); - subscriptions.subscribe(Collections.singleton(topic), Optional.empty()); - membershipManager.onSubscriptionUpdated(); + subscriptions.subscribe(set, Optional.empty()); // Create a ConsumerHeartbeatRequest and verify the payload assertEquals(0, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); @@ -575,20 +572,7 @@ private void assertNextHeartbeatTiming(long expectedTimeToNextHeartbeatMs) { @Test public void testHeartbeatState() { CommitRequestManager commitRequestManager = mock(CommitRequestManager.class); - membershipManager = new MembershipManagerImpl( - DEFAULT_GROUP_ID, - Optional.empty(), - 0, - Optional.of("uniform"), - subscriptions, - commitRequestManager, - (ConsumerMetadata) metadata, - logContext, - Optional.of(mock(ClientTelemetryReporter.class)), - backgroundEventHandler, - time, - new Metrics() - ); + mockJoiningMemberData(); heartbeatState = new HeartbeatState( subscriptions, @@ -596,13 +580,6 @@ public void testHeartbeatState() { DEFAULT_MAX_POLL_INTERVAL_MS ); - heartbeatRequestManager = createHeartbeatRequestManager( - coordinatorRequestManager, - membershipManager, - heartbeatState, - heartbeatRequestState, - backgroundEventHandler - ); createHeartbeatStateWithZeroHeartbeatInterval(); // The initial ConsumerGroupHeartbeatRequest sets most fields to their initial empty values @@ -615,11 +592,9 @@ public void testHeartbeatState() { assertEquals(Collections.emptyList(), data.subscribedTopicNames()); assertEquals(DEFAULT_REMOTE_ASSIGNOR, data.serverAssignor()); assertEquals(Collections.emptyList(), data.topicPartitions()); - membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state()); // Mock a response from the group coordinator, that supplies the member ID and a new epoch - membershipManager.onSubscriptionUpdated(); when(subscriptions.hasAutoAssignedPartitions()).thenReturn(true); when(subscriptions.rebalanceListener()).thenReturn(Optional.empty()); ConsumerGroupHeartbeatResponse rs1 = new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() @@ -629,9 +604,7 @@ public void testHeartbeatState() { .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()) ); when(commitRequestManager.maybeAutoCommitSyncBeforeRevocation(anyLong())).thenReturn(CompletableFuture.completedFuture(null)); - membershipManager.onHeartbeatSuccess(rs1.data()); - membershipManager.poll(time.milliseconds()); - membershipManager.onHeartbeatRequestSent(); + mockUnsubscribedMemberData(); data = heartbeatState.buildRequestData(); assertEquals(DEFAULT_GROUP_ID, data.groupId()); assertEquals(DEFAULT_MEMBER_ID, data.memberId()); @@ -641,15 +614,13 @@ public void testHeartbeatState() { assertNull(data.subscribedTopicNames()); assertNull(data.serverAssignor()); assertEquals(Collections.emptyList(), data.topicPartitions()); - membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.STABLE, membershipManager.state()); // Join the group and subscribe to a topic, but the response has not yet been received String topic = "topic1"; subscriptions.subscribe(Collections.singleton(topic), Optional.empty()); - membershipManager.onSubscriptionUpdated(); - membershipManager.transitionToFenced(); // And indirect way of moving to JOINING state when(subscriptions.subscription()).thenReturn(Collections.singleton(topic)); + mockFencedToJoiningMemberData(); data = heartbeatState.buildRequestData(); assertEquals(DEFAULT_GROUP_ID, data.groupId()); assertEquals(DEFAULT_MEMBER_ID, data.memberId()); @@ -659,10 +630,8 @@ public void testHeartbeatState() { assertEquals(Collections.singletonList(topic), data.subscribedTopicNames()); assertEquals(DEFAULT_REMOTE_ASSIGNOR, data.serverAssignor()); assertEquals(Collections.emptyList(), data.topicPartitions()); - membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.JOINING, membershipManager.state()); - membershipManager.transitionToFenced(); data = heartbeatState.buildRequestData(); assertEquals(DEFAULT_GROUP_ID, data.groupId()); assertEquals(DEFAULT_MEMBER_ID, data.memberId()); @@ -672,7 +641,6 @@ public void testHeartbeatState() { assertEquals(Collections.singletonList(topic), data.subscribedTopicNames()); assertEquals(DEFAULT_REMOTE_ASSIGNOR, data.serverAssignor()); assertEquals(Collections.emptyList(), data.topicPartitions()); - membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.JOINING, membershipManager.state()); // Mock the response from the group coordinator which returns an assignment @@ -690,7 +658,7 @@ public void testHeartbeatState() { .setMemberEpoch(1) .setAssignment(assignmentTopic1)); when(metadata.topicNames()).thenReturn(Collections.singletonMap(topicId, "topic1")); - membershipManager.onHeartbeatSuccess(rs1.data()); + mockReconcilingState(); // We remain in RECONCILING state, as the assignment will be reconciled on the next poll assertEquals(MemberState.RECONCILING, membershipManager.state()); @@ -812,7 +780,6 @@ public void testSendingLeaveGroupHeartbeatWhenPreviousOneInFlight(final short ve ConsumerGroupHeartbeatRequest heartbeatToLeave = getHeartbeatRequest(heartbeatRequestManager, version); assertEquals(ConsumerGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH, heartbeatToLeave.data().memberEpoch()); - //when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); when(membershipManager.shouldSkipHeartbeat()).thenReturn(true); NetworkClientDelegate.PollResult pollAgain = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, pollAgain.unsentRequests.size()); @@ -925,4 +892,34 @@ private HeartbeatRequestManager createHeartbeatRequestManager( backgroundEventHandler, new Metrics()); } + + private void mockJoiningMemberData() { + when(membershipManager.memberId()).thenReturn(""); + when(membershipManager.memberEpoch()).thenReturn(0); + when(membershipManager.groupId()).thenReturn(DEFAULT_GROUP_ID); + when(membershipManager.currentAssignment()).thenReturn(LocalAssignment.NONE); + when(membershipManager.serverAssignor()).thenReturn(Optional.of("uniform")); + when(membershipManager.state()).thenReturn(MemberState.UNSUBSCRIBED); + } + + private void mockUnsubscribedMemberData() { + when(membershipManager.memberId()).thenReturn(DEFAULT_MEMBER_ID); + when(membershipManager.memberEpoch()).thenReturn(1); + when(membershipManager.currentAssignment()).thenReturn(new LocalAssignment(0, Collections.emptyMap())); + when(membershipManager.state()).thenReturn(MemberState.STABLE); + } + + private void mockFencedToJoiningMemberData() { + when(membershipManager.state()).thenReturn(MemberState.JOINING); + when(membershipManager.memberEpoch()).thenReturn(0); + } + + private void mockReconcilingMemberData() { + membershipManager.poll(time.milliseconds()); + when(membershipManager.memberId()).thenReturn(DEFAULT_MEMBER_ID); + } + + private void mockReconcilingState() { + when(membershipManager.state()).thenReturn(MemberState.RECONCILING); + } } From aa1521ca728c44c07fd891e8bd2e3901ab796074 Mon Sep 17 00:00:00 2001 From: brenden20 <118419078+brenden20@users.noreply.github.com> Date: Mon, 15 Jul 2024 14:32:18 -0500 Subject: [PATCH 36/46] Test fix --- .../internals/HeartbeatRequestManagerTest.java | 18 ++---------------- 1 file changed, 2 insertions(+), 16 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 38c70eabc86ff..22d7065b9301f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -202,22 +202,6 @@ public void testSuccessfulHeartbeatTiming() { @ParameterizedTest @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments(short version) { - membershipManager = new MembershipManagerImpl( - DEFAULT_GROUP_ID, - Optional.of(DEFAULT_GROUP_INSTANCE_ID), - 0, - Optional.empty(), - subscriptions, - mock(CommitRequestManager.class), - (ConsumerMetadata) metadata, - logContext, - Optional.of(mock(ClientTelemetryReporter.class)), - backgroundEventHandler, - time, - new Metrics() - ); - membershipManager.transitionToJoining(); - heartbeatState = new HeartbeatState( subscriptions, membershipManager, @@ -239,6 +223,7 @@ public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments(s subscriptions.subscribe(set, Optional.empty()); // Create a ConsumerHeartbeatRequest and verify the payload + mockJoiningMemberData(); assertEquals(0, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); @@ -897,6 +882,7 @@ private void mockJoiningMemberData() { when(membershipManager.memberId()).thenReturn(""); when(membershipManager.memberEpoch()).thenReturn(0); when(membershipManager.groupId()).thenReturn(DEFAULT_GROUP_ID); + when(membershipManager.groupInstanceId()).thenReturn(Optional.of(DEFAULT_GROUP_INSTANCE_ID)); when(membershipManager.currentAssignment()).thenReturn(LocalAssignment.NONE); when(membershipManager.serverAssignor()).thenReturn(Optional.of("uniform")); when(membershipManager.state()).thenReturn(MemberState.UNSUBSCRIBED); From 1ecc70b0753d03efa86b94eaa15584569befe5e3 Mon Sep 17 00:00:00 2001 From: brenden20 <118419078+brenden20@users.noreply.github.com> Date: Mon, 15 Jul 2024 14:54:10 -0500 Subject: [PATCH 37/46] Test fix --- .../HeartbeatRequestManagerTest.java | 35 +++++-------------- 1 file changed, 9 insertions(+), 26 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 22d7065b9301f..a11f175e15c35 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -370,22 +370,6 @@ public void testNoCoordinator() { @ParameterizedTest @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) public void testValidateConsumerGroupHeartbeatRequest(final short version) { - membershipManager = new MembershipManagerImpl( - DEFAULT_GROUP_ID, - Optional.of(DEFAULT_GROUP_INSTANCE_ID), - 0, - Optional.of("uniform"), - subscriptions, - mock(CommitRequestManager.class), - (ConsumerMetadata) metadata, - logContext, - Optional.of(mock(ClientTelemetryReporter.class)), - backgroundEventHandler, - time, - new Metrics() - ); - membershipManager.transitionToJoining(); - heartbeatState = new HeartbeatState( subscriptions, membershipManager, @@ -414,6 +398,7 @@ public void testValidateConsumerGroupHeartbeatRequest(final short version) { membershipManager.onHeartbeatSuccess(result.data()); // Create a ConsumerHeartbeatRequest and verify the payload + mockDefaultMemberData(); NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); NetworkClientDelegate.UnsentRequest request = pollResult.unsentRequests.get(0); @@ -589,7 +574,6 @@ public void testHeartbeatState() { .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()) ); when(commitRequestManager.maybeAutoCommitSyncBeforeRevocation(anyLong())).thenReturn(CompletableFuture.completedFuture(null)); - mockUnsubscribedMemberData(); data = heartbeatState.buildRequestData(); assertEquals(DEFAULT_GROUP_ID, data.groupId()); assertEquals(DEFAULT_MEMBER_ID, data.memberId()); @@ -888,21 +872,20 @@ private void mockJoiningMemberData() { when(membershipManager.state()).thenReturn(MemberState.UNSUBSCRIBED); } - private void mockUnsubscribedMemberData() { - when(membershipManager.memberId()).thenReturn(DEFAULT_MEMBER_ID); - when(membershipManager.memberEpoch()).thenReturn(1); - when(membershipManager.currentAssignment()).thenReturn(new LocalAssignment(0, Collections.emptyMap())); - when(membershipManager.state()).thenReturn(MemberState.STABLE); - } - private void mockFencedToJoiningMemberData() { when(membershipManager.state()).thenReturn(MemberState.JOINING); when(membershipManager.memberEpoch()).thenReturn(0); + when(membershipManager.groupInstanceId()).thenReturn(Optional.empty()); + when(membershipManager.state()).thenReturn(MemberState.UNSUBSCRIBED); } - private void mockReconcilingMemberData() { - membershipManager.poll(time.milliseconds()); + private void mockDefaultMemberData() { + when(membershipManager.currentAssignment()).thenReturn(new LocalAssignment(0, Collections.emptyMap())); + when(membershipManager.groupId()).thenReturn(DEFAULT_GROUP_ID); when(membershipManager.memberId()).thenReturn(DEFAULT_MEMBER_ID); + when(membershipManager.memberEpoch()).thenReturn(DEFAULT_MEMBER_EPOCH); + when(membershipManager.groupInstanceId()).thenReturn(Optional.of(DEFAULT_GROUP_INSTANCE_ID)); + when(membershipManager.serverAssignor()).thenReturn(Optional.of("uniform")); } private void mockReconcilingState() { From 6cebd1bc856ecb40389210630c7543c7167d99cb Mon Sep 17 00:00:00 2001 From: brenden20 <118419078+brenden20@users.noreply.github.com> Date: Mon, 15 Jul 2024 15:07:16 -0500 Subject: [PATCH 38/46] Test fixes --- .../HeartbeatRequestManagerTest.java | 23 ++++++++++++------- 1 file changed, 15 insertions(+), 8 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index a11f175e15c35..a968139987539 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -223,7 +223,7 @@ public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments(s subscriptions.subscribe(set, Optional.empty()); // Create a ConsumerHeartbeatRequest and verify the payload - mockJoiningMemberData(); + mockJoiningMemberData(true); assertEquals(0, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); @@ -398,7 +398,7 @@ public void testValidateConsumerGroupHeartbeatRequest(final short version) { membershipManager.onHeartbeatSuccess(result.data()); // Create a ConsumerHeartbeatRequest and verify the payload - mockDefaultMemberData(); + mockDefaultMemberData(true); NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); NetworkClientDelegate.UnsentRequest request = pollResult.unsentRequests.get(0); @@ -542,7 +542,7 @@ private void assertNextHeartbeatTiming(long expectedTimeToNextHeartbeatMs) { @Test public void testHeartbeatState() { CommitRequestManager commitRequestManager = mock(CommitRequestManager.class); - mockJoiningMemberData(); + mockJoiningMemberData(false); heartbeatState = new HeartbeatState( subscriptions, @@ -574,6 +574,7 @@ public void testHeartbeatState() { .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()) ); when(commitRequestManager.maybeAutoCommitSyncBeforeRevocation(anyLong())).thenReturn(CompletableFuture.completedFuture(null)); + mockDefaultMemberData(false); data = heartbeatState.buildRequestData(); assertEquals(DEFAULT_GROUP_ID, data.groupId()); assertEquals(DEFAULT_MEMBER_ID, data.memberId()); @@ -862,11 +863,14 @@ private HeartbeatRequestManager createHeartbeatRequestManager( new Metrics()); } - private void mockJoiningMemberData() { + private void mockJoiningMemberData(boolean instanceId) { when(membershipManager.memberId()).thenReturn(""); when(membershipManager.memberEpoch()).thenReturn(0); when(membershipManager.groupId()).thenReturn(DEFAULT_GROUP_ID); - when(membershipManager.groupInstanceId()).thenReturn(Optional.of(DEFAULT_GROUP_INSTANCE_ID)); + if (instanceId) + when(membershipManager.groupInstanceId()).thenReturn(Optional.of(DEFAULT_GROUP_INSTANCE_ID)); + else + when(membershipManager.groupInstanceId()).thenReturn(Optional.empty()); when(membershipManager.currentAssignment()).thenReturn(LocalAssignment.NONE); when(membershipManager.serverAssignor()).thenReturn(Optional.of("uniform")); when(membershipManager.state()).thenReturn(MemberState.UNSUBSCRIBED); @@ -876,16 +880,19 @@ private void mockFencedToJoiningMemberData() { when(membershipManager.state()).thenReturn(MemberState.JOINING); when(membershipManager.memberEpoch()).thenReturn(0); when(membershipManager.groupInstanceId()).thenReturn(Optional.empty()); - when(membershipManager.state()).thenReturn(MemberState.UNSUBSCRIBED); } - private void mockDefaultMemberData() { + private void mockDefaultMemberData(boolean instanceId) { when(membershipManager.currentAssignment()).thenReturn(new LocalAssignment(0, Collections.emptyMap())); when(membershipManager.groupId()).thenReturn(DEFAULT_GROUP_ID); when(membershipManager.memberId()).thenReturn(DEFAULT_MEMBER_ID); when(membershipManager.memberEpoch()).thenReturn(DEFAULT_MEMBER_EPOCH); - when(membershipManager.groupInstanceId()).thenReturn(Optional.of(DEFAULT_GROUP_INSTANCE_ID)); + if (instanceId) + when(membershipManager.groupInstanceId()).thenReturn(Optional.of(DEFAULT_GROUP_INSTANCE_ID)); + else + when(membershipManager.groupInstanceId()).thenReturn(Optional.empty()); when(membershipManager.serverAssignor()).thenReturn(Optional.of("uniform")); + when(membershipManager.state()).thenReturn(MemberState.STABLE); } private void mockReconcilingState() { From 569802b82d9e8278d587cab25258e5c3a6d07227 Mon Sep 17 00:00:00 2001 From: brenden20 <118419078+brenden20@users.noreply.github.com> Date: Mon, 15 Jul 2024 15:18:24 -0500 Subject: [PATCH 39/46] Test fix --- .../consumer/internals/HeartbeatRequestManagerTest.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index a968139987539..d1bf98aa12e39 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -461,7 +461,6 @@ topicId, mkSortedSet(0) } private ConsumerGroupHeartbeatRequest getHeartbeatRequest(HeartbeatRequestManager heartbeatRequestManager, final short version) { - // Create a ConsumerHeartbeatRequest and verify the payload -- no assignment should be sent NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); NetworkClientDelegate.UnsentRequest request = pollResult.unsentRequests.get(0); @@ -472,9 +471,6 @@ private ConsumerGroupHeartbeatRequest getHeartbeatRequest(HeartbeatRequestManage @ParameterizedTest @MethodSource("errorProvider") public void testHeartbeatResponseOnErrorHandling(final Errors error, final boolean isFatal) { - if (isFatal) - when(membershipManager.state()).thenReturn(MemberState.FATAL); - // Handling errors on the second heartbeat time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); @@ -782,7 +778,6 @@ private void ensureFatalError(Errors expectedError) { private void ensureHeartbeatStopped() { time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); - assertEquals(MemberState.FATAL, membershipManager.state()); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size()); } From 8be215cf6ef6ec60672e88357fb60b809f440396 Mon Sep 17 00:00:00 2001 From: brenden20 <118419078+brenden20@users.noreply.github.com> Date: Wed, 17 Jul 2024 09:48:07 -0500 Subject: [PATCH 40/46] Update HeartbeatRequestManagerTest.java --- .../clients/consumer/internals/HeartbeatRequestManagerTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index d1bf98aa12e39..577775b3b3f89 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -277,7 +277,6 @@ public void testTimerNotDue() { // Member in state where it should not send Heartbeat anymore when(subscriptions.hasAutoAssignedPartitions()).thenReturn(true); when(membershipManager.shouldSkipHeartbeat()).thenReturn(true); - membershipManager.transitionToFatal(); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs); } From 4c623a0a544ea12d61c4c79dd15de7383a6b7b66 Mon Sep 17 00:00:00 2001 From: brenden20 <118419078+brenden20@users.noreply.github.com> Date: Wed, 17 Jul 2024 19:23:17 -0500 Subject: [PATCH 41/46] Various improvements --- .../HeartbeatRequestManagerTest.java | 48 ++++--------------- 1 file changed, 10 insertions(+), 38 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 577775b3b3f89..6e469be49de54 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -38,7 +38,6 @@ import org.apache.kafka.common.requests.ConsumerGroupHeartbeatResponse; import org.apache.kafka.common.requests.RequestHeader; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; @@ -223,7 +222,7 @@ public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments(s subscriptions.subscribe(set, Optional.empty()); // Create a ConsumerHeartbeatRequest and verify the payload - mockJoiningMemberData(true); + mockJoiningMemberData(DEFAULT_GROUP_INSTANCE_ID); assertEquals(0, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); @@ -397,7 +396,7 @@ public void testValidateConsumerGroupHeartbeatRequest(final short version) { membershipManager.onHeartbeatSuccess(result.data()); // Create a ConsumerHeartbeatRequest and verify the payload - mockDefaultMemberData(true); + mockDefaultMemberData(DEFAULT_GROUP_INSTANCE_ID); NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); NetworkClientDelegate.UnsentRequest request = pollResult.unsentRequests.get(0); @@ -536,8 +535,7 @@ private void assertNextHeartbeatTiming(long expectedTimeToNextHeartbeatMs) { @Test public void testHeartbeatState() { - CommitRequestManager commitRequestManager = mock(CommitRequestManager.class); - mockJoiningMemberData(false); + mockJoiningMemberData(null); heartbeatState = new HeartbeatState( subscriptions, @@ -557,7 +555,6 @@ public void testHeartbeatState() { assertEquals(Collections.emptyList(), data.subscribedTopicNames()); assertEquals(DEFAULT_REMOTE_ASSIGNOR, data.serverAssignor()); assertEquals(Collections.emptyList(), data.topicPartitions()); - assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state()); // Mock a response from the group coordinator, that supplies the member ID and a new epoch when(subscriptions.hasAutoAssignedPartitions()).thenReturn(true); @@ -568,8 +565,7 @@ public void testHeartbeatState() { .setMemberEpoch(DEFAULT_MEMBER_EPOCH) .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()) ); - when(commitRequestManager.maybeAutoCommitSyncBeforeRevocation(anyLong())).thenReturn(CompletableFuture.completedFuture(null)); - mockDefaultMemberData(false); + mockDefaultMemberData(null); data = heartbeatState.buildRequestData(); assertEquals(DEFAULT_GROUP_ID, data.groupId()); assertEquals(DEFAULT_MEMBER_ID, data.memberId()); @@ -579,7 +575,6 @@ public void testHeartbeatState() { assertNull(data.subscribedTopicNames()); assertNull(data.serverAssignor()); assertEquals(Collections.emptyList(), data.topicPartitions()); - assertEquals(MemberState.STABLE, membershipManager.state()); // Join the group and subscribe to a topic, but the response has not yet been received String topic = "topic1"; @@ -595,7 +590,6 @@ public void testHeartbeatState() { assertEquals(Collections.singletonList(topic), data.subscribedTopicNames()); assertEquals(DEFAULT_REMOTE_ASSIGNOR, data.serverAssignor()); assertEquals(Collections.emptyList(), data.topicPartitions()); - assertEquals(MemberState.JOINING, membershipManager.state()); data = heartbeatState.buildRequestData(); assertEquals(DEFAULT_GROUP_ID, data.groupId()); @@ -606,7 +600,6 @@ public void testHeartbeatState() { assertEquals(Collections.singletonList(topic), data.subscribedTopicNames()); assertEquals(DEFAULT_REMOTE_ASSIGNOR, data.serverAssignor()); assertEquals(Collections.emptyList(), data.topicPartitions()); - assertEquals(MemberState.JOINING, membershipManager.state()); // Mock the response from the group coordinator which returns an assignment ConsumerGroupHeartbeatResponseData.TopicPartitions tpTopic1 = @@ -617,16 +610,7 @@ public void testHeartbeatState() { ConsumerGroupHeartbeatResponseData.Assignment assignmentTopic1 = new ConsumerGroupHeartbeatResponseData.Assignment(); assignmentTopic1.setTopicPartitions(Collections.singletonList(tpTopic1)); - rs1 = new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() - .setHeartbeatIntervalMs(DEFAULT_HEARTBEAT_INTERVAL_MS) - .setMemberId(DEFAULT_MEMBER_ID) - .setMemberEpoch(1) - .setAssignment(assignmentTopic1)); when(metadata.topicNames()).thenReturn(Collections.singletonMap(topicId, "topic1")); - mockReconcilingState(); - - // We remain in RECONCILING state, as the assignment will be reconciled on the next poll - assertEquals(MemberState.RECONCILING, membershipManager.state()); } @Test @@ -857,17 +841,13 @@ private HeartbeatRequestManager createHeartbeatRequestManager( new Metrics()); } - private void mockJoiningMemberData(boolean instanceId) { + private void mockJoiningMemberData(String instanceId) { + when(membershipManager.groupInstanceId()).thenReturn(Optional.ofNullable(instanceId)); when(membershipManager.memberId()).thenReturn(""); when(membershipManager.memberEpoch()).thenReturn(0); when(membershipManager.groupId()).thenReturn(DEFAULT_GROUP_ID); - if (instanceId) - when(membershipManager.groupInstanceId()).thenReturn(Optional.of(DEFAULT_GROUP_INSTANCE_ID)); - else - when(membershipManager.groupInstanceId()).thenReturn(Optional.empty()); when(membershipManager.currentAssignment()).thenReturn(LocalAssignment.NONE); - when(membershipManager.serverAssignor()).thenReturn(Optional.of("uniform")); - when(membershipManager.state()).thenReturn(MemberState.UNSUBSCRIBED); + when(membershipManager.serverAssignor()).thenReturn(Optional.of(DEFAULT_REMOTE_ASSIGNOR)); } private void mockFencedToJoiningMemberData() { @@ -876,20 +856,12 @@ private void mockFencedToJoiningMemberData() { when(membershipManager.groupInstanceId()).thenReturn(Optional.empty()); } - private void mockDefaultMemberData(boolean instanceId) { + private void mockDefaultMemberData(String instanceId) { + when(membershipManager.groupInstanceId()).thenReturn(Optional.ofNullable(instanceId)); when(membershipManager.currentAssignment()).thenReturn(new LocalAssignment(0, Collections.emptyMap())); when(membershipManager.groupId()).thenReturn(DEFAULT_GROUP_ID); when(membershipManager.memberId()).thenReturn(DEFAULT_MEMBER_ID); when(membershipManager.memberEpoch()).thenReturn(DEFAULT_MEMBER_EPOCH); - if (instanceId) - when(membershipManager.groupInstanceId()).thenReturn(Optional.of(DEFAULT_GROUP_INSTANCE_ID)); - else - when(membershipManager.groupInstanceId()).thenReturn(Optional.empty()); - when(membershipManager.serverAssignor()).thenReturn(Optional.of("uniform")); - when(membershipManager.state()).thenReturn(MemberState.STABLE); - } - - private void mockReconcilingState() { - when(membershipManager.state()).thenReturn(MemberState.RECONCILING); + when(membershipManager.serverAssignor()).thenReturn(Optional.of(DEFAULT_REMOTE_ASSIGNOR)); } } From 52742eb630b957ebb5b6f36acb5fe893ec1dcbb0 Mon Sep 17 00:00:00 2001 From: brenden20 <118419078+brenden20@users.noreply.github.com> Date: Thu, 18 Jul 2024 10:29:20 -0500 Subject: [PATCH 42/46] Update HeartbeatRequestManagerTest.java --- .../clients/consumer/internals/HeartbeatRequestManagerTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 6e469be49de54..1569eaa454796 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -60,7 +60,6 @@ import java.util.Properties; import java.util.Set; import java.util.SortedSet; -import java.util.concurrent.CompletableFuture; import static org.apache.kafka.common.utils.Utils.mkSortedSet; import static org.junit.jupiter.api.Assertions.assertEquals; From a7e3e6e05f9f861a02333d3eb37739def9f8a128 Mon Sep 17 00:00:00 2001 From: brenden20 <118419078+brenden20@users.noreply.github.com> Date: Fri, 19 Jul 2024 09:52:16 -0500 Subject: [PATCH 43/46] Cleanup --- .../HeartbeatRequestManagerTest.java | 19 +------------------ 1 file changed, 1 insertion(+), 18 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 7b505d2b56c90..eda4ae828bb30 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -55,33 +55,22 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.HashSet; -import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Properties; import java.util.Set; import java.util.SortedSet; -import java.util.concurrent.TimeUnit; - -import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_GROUP_INSTANCE_ID; -import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_HEARTBEAT_INTERVAL_MS; -import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_MAX_POLL_INTERVAL_MS; -import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_REMOTE_ASSIGNOR; -import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_RETRY_BACKOFF_MAX_MS; -import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_RETRY_BACKOFF_MS; + import static org.apache.kafka.common.utils.Utils.mkSortedSet; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; -import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.Mockito.clearInvocations; -import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; @@ -577,12 +566,6 @@ public void testHeartbeatState() { // Mock a response from the group coordinator, that supplies the member ID and a new epoch when(subscriptions.hasAutoAssignedPartitions()).thenReturn(true); when(subscriptions.rebalanceListener()).thenReturn(Optional.empty()); - ConsumerGroupHeartbeatResponse rs1 = new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() - .setHeartbeatIntervalMs(DEFAULT_HEARTBEAT_INTERVAL_MS) - .setMemberId(DEFAULT_MEMBER_ID) - .setMemberEpoch(DEFAULT_MEMBER_EPOCH) - .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()) - ); mockDefaultMemberData(null); data = heartbeatState.buildRequestData(); assertEquals(DEFAULT_GROUP_ID, data.groupId()); From 471aed18b008540d6441b8cf947c4d6d6fd8118f Mon Sep 17 00:00:00 2001 From: brenden20 <118419078+brenden20@users.noreply.github.com> Date: Fri, 19 Jul 2024 10:08:49 -0500 Subject: [PATCH 44/46] Implementing suggestions --- .../internals/HeartbeatRequestManagerTest.java | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index eda4ae828bb30..77a188d3c4df1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -399,7 +399,7 @@ public void testValidateConsumerGroupHeartbeatRequest(final short version) { membershipManager.onHeartbeatSuccess(result.data()); // Create a ConsumerHeartbeatRequest and verify the payload - mockDefaultMemberData(DEFAULT_GROUP_INSTANCE_ID); + mockStableMemberData(DEFAULT_GROUP_INSTANCE_ID); NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); NetworkClientDelegate.UnsentRequest request = pollResult.unsentRequests.get(0); @@ -564,9 +564,10 @@ public void testHeartbeatState() { assertEquals(Collections.emptyList(), data.topicPartitions()); // Mock a response from the group coordinator, that supplies the member ID and a new epoch + when(membershipManager.state()).thenReturn(MemberState.STABLE); when(subscriptions.hasAutoAssignedPartitions()).thenReturn(true); when(subscriptions.rebalanceListener()).thenReturn(Optional.empty()); - mockDefaultMemberData(null); + mockStableMemberData(null); data = heartbeatState.buildRequestData(); assertEquals(DEFAULT_GROUP_ID, data.groupId()); assertEquals(DEFAULT_MEMBER_ID, data.memberId()); @@ -581,7 +582,7 @@ public void testHeartbeatState() { String topic = "topic1"; subscriptions.subscribe(Collections.singleton(topic), Optional.empty()); when(subscriptions.subscription()).thenReturn(Collections.singleton(topic)); - mockFencedToJoiningMemberData(); + mockRejoiningMemberData(); data = heartbeatState.buildRequestData(); assertEquals(DEFAULT_GROUP_ID, data.groupId()); assertEquals(DEFAULT_MEMBER_ID, data.memberId()); @@ -633,7 +634,6 @@ public void testPollTimerExpiration() { verify(heartbeatRequestState).reset(); verify(membershipManager).onHeartbeatRequestGenerated(); - when(membershipManager.state()).thenReturn(MemberState.STALE); when(membershipManager.shouldSkipHeartbeat()).thenReturn(true); assertNoHeartbeat(heartbeatRequestManager); heartbeatRequestManager.resetPollTimer(time.milliseconds()); @@ -707,12 +707,10 @@ public void testFencedMemberStopHeartbeatUntilItReleasesAssignmentToRejoin() { verify(heartbeatRequestState).reset(); when(membershipManager.shouldSkipHeartbeat()).thenReturn(true); - when(membershipManager.state()).thenReturn(MemberState.FENCED); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size(), "Member should not send heartbeats while FENCED"); when(membershipManager.shouldSkipHeartbeat()).thenReturn(false); - when(membershipManager.state()).thenReturn(MemberState.JOINING); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size(), "Fenced member should resume heartbeat after transitioning to JOINING"); } @@ -844,6 +842,7 @@ private HeartbeatRequestManager createHeartbeatRequestManager( } private void mockJoiningMemberData(String instanceId) { + when(membershipManager.state()).thenReturn(MemberState.JOINING); when(membershipManager.groupInstanceId()).thenReturn(Optional.ofNullable(instanceId)); when(membershipManager.memberId()).thenReturn(""); when(membershipManager.memberEpoch()).thenReturn(0); @@ -852,13 +851,13 @@ private void mockJoiningMemberData(String instanceId) { when(membershipManager.serverAssignor()).thenReturn(Optional.of(DEFAULT_REMOTE_ASSIGNOR)); } - private void mockFencedToJoiningMemberData() { + private void mockRejoiningMemberData() { when(membershipManager.state()).thenReturn(MemberState.JOINING); when(membershipManager.memberEpoch()).thenReturn(0); when(membershipManager.groupInstanceId()).thenReturn(Optional.empty()); } - private void mockDefaultMemberData(String instanceId) { + private void mockStableMemberData(String instanceId) { when(membershipManager.groupInstanceId()).thenReturn(Optional.ofNullable(instanceId)); when(membershipManager.currentAssignment()).thenReturn(new LocalAssignment(0, Collections.emptyMap())); when(membershipManager.groupId()).thenReturn(DEFAULT_GROUP_ID); From 9565cad69415cae0ff2720864a4a7d2c88407179 Mon Sep 17 00:00:00 2001 From: brenden20 <118419078+brenden20@users.noreply.github.com> Date: Wed, 24 Jul 2024 11:26:28 -0500 Subject: [PATCH 45/46] Moved some object creations into a helper method --- .../HeartbeatRequestManagerTest.java | 69 +++++++------------ 1 file changed, 26 insertions(+), 43 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 77a188d3c4df1..05a50527e2878 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -138,7 +138,7 @@ public void setUp() { when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mock(Node.class))); } - private void createHeartbeatStateWithZeroHeartbeatInterval() { + private void createHeartbeatRequestStateWithZeroHeartbeatInterval() { this.heartbeatRequestState = spy(new HeartbeatRequestState( logContext, time, @@ -155,12 +155,29 @@ private void createHeartbeatStateWithZeroHeartbeatInterval() { backgroundEventHandler); } + private void createHeartbeatStateandRequestManager() { + this.heartbeatState = new HeartbeatState( + subscriptions, + membershipManager, + DEFAULT_MAX_POLL_INTERVAL_MS + ); + + this.heartbeatRequestManager = createHeartbeatRequestManager( + coordinatorRequestManager, + membershipManager, + heartbeatState, + heartbeatRequestState, + backgroundEventHandler + ); + } + + @Test public void testHeartbeatOnStartup() { NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size()); - createHeartbeatStateWithZeroHeartbeatInterval(); + createHeartbeatRequestStateWithZeroHeartbeatInterval(); assertEquals(0, heartbeatRequestManager.maximumTimeToWait(time.milliseconds())); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size()); @@ -201,20 +218,8 @@ public void testSuccessfulHeartbeatTiming() { @ParameterizedTest @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments(short version) { - heartbeatState = new HeartbeatState( - subscriptions, - membershipManager, - DEFAULT_MAX_POLL_INTERVAL_MS - ); - - heartbeatRequestManager = createHeartbeatRequestManager( - coordinatorRequestManager, - membershipManager, - heartbeatState, - heartbeatRequestState, - backgroundEventHandler - ); - createHeartbeatStateWithZeroHeartbeatInterval(); + createHeartbeatStateandRequestManager(); + createHeartbeatRequestStateWithZeroHeartbeatInterval(); time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); String topic = "topic1"; Set set = Collections.singleton(topic); @@ -247,7 +252,7 @@ public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments(s @ValueSource(booleans = {true, false}) public void testSkippingHeartbeat(final boolean shouldSkipHeartbeat) { // The initial heartbeatInterval is set to 0 - createHeartbeatStateWithZeroHeartbeatInterval(); + createHeartbeatRequestStateWithZeroHeartbeatInterval(); // Mocking notInGroup when(membershipManager.shouldSkipHeartbeat()).thenReturn(shouldSkipHeartbeat); @@ -327,7 +332,7 @@ public void testHeartbeatOutsideInterval() { @Test public void testNetworkTimeout() { // The initial heartbeatInterval is set to 0 - createHeartbeatStateWithZeroHeartbeatInterval(); + createHeartbeatRequestStateWithZeroHeartbeatInterval(); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, result.unsentRequests.size()); // Mimic network timeout @@ -371,19 +376,7 @@ public void testNoCoordinator() { @ParameterizedTest @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) public void testValidateConsumerGroupHeartbeatRequest(final short version) { - heartbeatState = new HeartbeatState( - subscriptions, - membershipManager, - DEFAULT_MAX_POLL_INTERVAL_MS - ); - - heartbeatRequestManager = createHeartbeatRequestManager( - coordinatorRequestManager, - membershipManager, - heartbeatState, - heartbeatRequestState, - backgroundEventHandler - ); + createHeartbeatStateandRequestManager(); // The initial heartbeatInterval is set to 0, but we're testing time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); @@ -420,17 +413,7 @@ public void testValidateConsumerGroupHeartbeatRequest(final short version) { @ParameterizedTest @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) public void testValidateConsumerGroupHeartbeatRequestAssignmentSentWhenLocalEpochChanges(final short version) { - heartbeatState = new HeartbeatState( - subscriptions, - membershipManager, - DEFAULT_MAX_POLL_INTERVAL_MS); - - heartbeatRequestManager = createHeartbeatRequestManager( - coordinatorRequestManager, - membershipManager, - heartbeatState, - heartbeatRequestState, - backgroundEventHandler); + createHeartbeatStateandRequestManager(); when(membershipManager.shouldHeartbeatNow()).thenReturn(true); @@ -550,7 +533,7 @@ public void testHeartbeatState() { DEFAULT_MAX_POLL_INTERVAL_MS ); - createHeartbeatStateWithZeroHeartbeatInterval(); + createHeartbeatRequestStateWithZeroHeartbeatInterval(); // The initial ConsumerGroupHeartbeatRequest sets most fields to their initial empty values ConsumerGroupHeartbeatRequestData data = heartbeatState.buildRequestData(); From 30c04c5f299b4320448a45fe8ea7b71f479b3dce Mon Sep 17 00:00:00 2001 From: brenden20 <118419078+brenden20@users.noreply.github.com> Date: Wed, 24 Jul 2024 12:55:27 -0500 Subject: [PATCH 46/46] Update HeartbeatRequestManagerTest.java --- .../consumer/internals/HeartbeatRequestManagerTest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 05a50527e2878..67fb7267f024f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -155,7 +155,7 @@ private void createHeartbeatRequestStateWithZeroHeartbeatInterval() { backgroundEventHandler); } - private void createHeartbeatStateandRequestManager() { + private void createHeartbeatStatAndRequestManager() { this.heartbeatState = new HeartbeatState( subscriptions, membershipManager, @@ -218,7 +218,7 @@ public void testSuccessfulHeartbeatTiming() { @ParameterizedTest @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments(short version) { - createHeartbeatStateandRequestManager(); + createHeartbeatStatAndRequestManager(); createHeartbeatRequestStateWithZeroHeartbeatInterval(); time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); String topic = "topic1"; @@ -376,7 +376,7 @@ public void testNoCoordinator() { @ParameterizedTest @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) public void testValidateConsumerGroupHeartbeatRequest(final short version) { - createHeartbeatStateandRequestManager(); + createHeartbeatStatAndRequestManager(); // The initial heartbeatInterval is set to 0, but we're testing time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); @@ -413,7 +413,7 @@ public void testValidateConsumerGroupHeartbeatRequest(final short version) { @ParameterizedTest @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) public void testValidateConsumerGroupHeartbeatRequestAssignmentSentWhenLocalEpochChanges(final short version) { - createHeartbeatStateandRequestManager(); + createHeartbeatStatAndRequestManager(); when(membershipManager.shouldHeartbeatNow()).thenReturn(true);