MINOR: Simplify OffsetFetchResponse (#19642)

While working on https://github.com/apache/kafka/pull/19515, I came to
the conclusion that the OffsetFetchResponse is quite messy and overall
too complicated. This patch rationalize the constructors.
OffsetFetchResponse has a single constructor accepting the
OffsetFetchResponseData. A builder is introduced to handle the down
conversion. This will also simplify adding the topic ids. All the
changes are mechanical, replacing data structures by others.

Reviewers: Lianet Magrans <lmagrans@confluent.io>
This commit is contained in:
David Jacot 2025-05-08 14:57:45 +02:00 committed by GitHub
parent 2dd6126b5d
commit 98e535b524
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
14 changed files with 1331 additions and 1155 deletions

View File

@ -26,6 +26,7 @@ import org.apache.kafka.common.requests.AbstractResponse;
import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType;
import org.apache.kafka.common.requests.OffsetFetchRequest;
import org.apache.kafka.common.requests.OffsetFetchResponse;
import org.apache.kafka.common.requests.RequestUtils;
import org.apache.kafka.common.utils.LogContext;
import org.slf4j.Logger;
@ -36,7 +37,6 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.stream.Collectors;
@ -139,40 +139,52 @@ public class ListConsumerGroupOffsetsHandler implements AdminApiHandler<Coordina
) {
validateKeys(groupIds);
final OffsetFetchResponse response = (OffsetFetchResponse) abstractResponse;
var response = (OffsetFetchResponse) abstractResponse;
var completed = new HashMap<CoordinatorKey, Map<TopicPartition, OffsetAndMetadata>>();
var failed = new HashMap<CoordinatorKey, Throwable>();
var unmapped = new ArrayList<CoordinatorKey>();
Map<CoordinatorKey, Map<TopicPartition, OffsetAndMetadata>> completed = new HashMap<>();
Map<CoordinatorKey, Throwable> failed = new HashMap<>();
List<CoordinatorKey> unmapped = new ArrayList<>();
for (CoordinatorKey coordinatorKey : groupIds) {
String group = coordinatorKey.idValue;
if (response.groupHasError(group)) {
handleGroupError(CoordinatorKey.byGroupId(group), response.groupLevelError(group), failed, unmapped);
} else {
final Map<TopicPartition, OffsetAndMetadata> groupOffsetsListing = new HashMap<>();
Map<TopicPartition, OffsetFetchResponse.PartitionData> responseData = response.partitionDataMap(group);
for (Map.Entry<TopicPartition, OffsetFetchResponse.PartitionData> partitionEntry : responseData.entrySet()) {
final TopicPartition topicPartition = partitionEntry.getKey();
OffsetFetchResponse.PartitionData partitionData = partitionEntry.getValue();
final Errors error = partitionData.error;
var groupId = coordinatorKey.idValue;
var group = response.group(groupId);
var error = Errors.forCode(group.errorCode());
if (error == Errors.NONE) {
final long offset = partitionData.offset;
final String metadata = partitionData.metadata;
final Optional<Integer> leaderEpoch = partitionData.leaderEpoch;
// Negative offset indicates that the group has no committed offset for this partition
if (offset < 0) {
groupOffsetsListing.put(topicPartition, null);
if (error != Errors.NONE) {
handleGroupError(
coordinatorKey,
error,
failed,
unmapped
);
} else {
var offsets = new HashMap<TopicPartition, OffsetAndMetadata>();
group.topics().forEach(topic ->
topic.partitions().forEach(partition -> {
var tp = new TopicPartition(topic.name(), partition.partitionIndex());
var partitionError = Errors.forCode(partition.errorCode());
if (partitionError == Errors.NONE) {
// Negative offset indicates that the group has no committed offset for this partition.
if (partition.committedOffset() < 0) {
offsets.put(tp, null);
} else {
offsets.put(tp, new OffsetAndMetadata(
partition.committedOffset(),
RequestUtils.getLeaderEpoch(partition.committedLeaderEpoch()),
partition.metadata()
));
}
} else {
groupOffsetsListing.put(topicPartition, new OffsetAndMetadata(offset, leaderEpoch, metadata));
log.warn("Skipping return offset for {} due to error {}.", tp, partitionError);
}
} else {
log.warn("Skipping return offset for {} due to error {}.", topicPartition, error);
}
}
completed.put(CoordinatorKey.byGroupId(group), groupOffsetsListing);
})
);
completed.put(coordinatorKey, offsets);
}
}
return new ApiResult<>(completed, failed, unmapped);
}

View File

@ -35,6 +35,7 @@ import org.apache.kafka.common.errors.UnstableOffsetCommitException;
import org.apache.kafka.common.message.OffsetCommitRequestData;
import org.apache.kafka.common.message.OffsetCommitResponseData;
import org.apache.kafka.common.message.OffsetFetchRequestData;
import org.apache.kafka.common.message.OffsetFetchResponseData;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.RecordBatch;
@ -43,6 +44,7 @@ import org.apache.kafka.common.requests.OffsetCommitRequest;
import org.apache.kafka.common.requests.OffsetCommitResponse;
import org.apache.kafka.common.requests.OffsetFetchRequest;
import org.apache.kafka.common.requests.OffsetFetchResponse;
import org.apache.kafka.common.requests.RequestUtils;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Timer;
@ -1012,13 +1014,14 @@ public class CommitRequestManager implements RequestManager, MemberStateListener
@Override
void onResponse(final ClientResponse response) {
long currentTimeMs = response.receivedTimeMs();
OffsetFetchResponse fetchResponse = (OffsetFetchResponse) response.responseBody();
Errors responseError = fetchResponse.groupLevelError(groupId);
if (responseError != Errors.NONE) {
onFailure(currentTimeMs, responseError);
var fetchResponse = (OffsetFetchResponse) response.responseBody();
var groupResponse = fetchResponse.group(groupId);
var error = Errors.forCode(groupResponse.errorCode());
if (error != Errors.NONE) {
onFailure(currentTimeMs, error);
return;
}
onSuccess(currentTimeMs, fetchResponse);
onSuccess(currentTimeMs, groupResponse);
}
/**
@ -1083,53 +1086,58 @@ public class CommitRequestManager implements RequestManager, MemberStateListener
* offsets contained in the response, and record a successful request attempt.
*/
private void onSuccess(final long currentTimeMs,
final OffsetFetchResponse response) {
Set<String> unauthorizedTopics = null;
Map<TopicPartition, OffsetFetchResponse.PartitionData> responseData =
response.partitionDataMap(groupId);
Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>(responseData.size());
Set<TopicPartition> unstableTxnOffsetTopicPartitions = new HashSet<>();
boolean failedRequestRegistered = false;
for (Map.Entry<TopicPartition, OffsetFetchResponse.PartitionData> entry : responseData.entrySet()) {
TopicPartition tp = entry.getKey();
OffsetFetchResponse.PartitionData partitionData = entry.getValue();
if (partitionData.hasError()) {
Errors error = partitionData.error;
log.debug("Failed to fetch offset for partition {}: {}", tp, error.message());
final OffsetFetchResponseData.OffsetFetchResponseGroup response) {
var offsets = new HashMap<TopicPartition, OffsetAndMetadata>();
var unstableTxnOffsetTopicPartitions = new HashSet<TopicPartition>();
var unauthorizedTopics = new HashSet<String>();
var failedRequestRegistered = false;
if (!failedRequestRegistered) {
onFailedAttempt(currentTimeMs);
failedRequestRegistered = true;
}
for (var topic : response.topics()) {
for (var partition : topic.partitions()) {
var tp = new TopicPartition(
topic.name(),
partition.partitionIndex()
);
var error = Errors.forCode(partition.errorCode());
if (error != Errors.NONE) {
log.debug("Failed to fetch offset for partition {}: {}", tp, error.message());
if (error == Errors.UNKNOWN_TOPIC_OR_PARTITION) {
future.completeExceptionally(new KafkaException("Topic or Partition " + tp + " does not exist"));
return;
} else if (error == Errors.TOPIC_AUTHORIZATION_FAILED) {
if (unauthorizedTopics == null) {
unauthorizedTopics = new HashSet<>();
if (!failedRequestRegistered) {
onFailedAttempt(currentTimeMs);
failedRequestRegistered = true;
}
unauthorizedTopics.add(tp.topic());
} else if (error == Errors.UNSTABLE_OFFSET_COMMIT) {
unstableTxnOffsetTopicPartitions.add(tp);
} else {
// Fail with a non-retriable KafkaException for all unexpected partition
// errors (even if they are retriable)
future.completeExceptionally(new KafkaException("Unexpected error in fetch offset " +
if (error == Errors.UNKNOWN_TOPIC_OR_PARTITION) {
future.completeExceptionally(new KafkaException("Topic or Partition " + tp + " does not exist"));
return;
} else if (error == Errors.TOPIC_AUTHORIZATION_FAILED) {
unauthorizedTopics.add(tp.topic());
} else if (error == Errors.UNSTABLE_OFFSET_COMMIT) {
unstableTxnOffsetTopicPartitions.add(tp);
} else {
// Fail with a non-retriable KafkaException for all unexpected partition
// errors (even if they are retriable)
future.completeExceptionally(new KafkaException("Unexpected error in fetch offset " +
"response for partition " + tp + ": " + error.message()));
return;
return;
}
} else if (partition.committedOffset() >= 0) {
// record the position with the offset (-1 indicates no committed offset to fetch);
// if there's no committed offset, record as null
offsets.put(tp, new OffsetAndMetadata(
partition.committedOffset(),
RequestUtils.getLeaderEpoch(partition.committedLeaderEpoch()),
partition.metadata()
));
} else {
log.info("Found no committed offset for partition {}", tp);
offsets.put(tp, null);
}
} else if (partitionData.offset >= 0) {
// record the position with the offset (-1 indicates no committed offset to fetch);
// if there's no committed offset, record as null
offsets.put(tp, new OffsetAndMetadata(partitionData.offset, partitionData.leaderEpoch, partitionData.metadata));
} else {
log.info("Found no committed offset for partition {}", tp);
offsets.put(tp, null);
}
}
if (unauthorizedTopics != null) {
if (!unauthorizedTopics.isEmpty()) {
future.completeExceptionally(new TopicAuthorizationException(unauthorizedTopics));
} else if (!unstableTxnOffsetTopicPartitions.isEmpty()) {
// TODO: Optimization question: Do we need to retry all partitions upon a single partition error?

View File

@ -63,6 +63,7 @@ import org.apache.kafka.common.requests.OffsetCommitRequest;
import org.apache.kafka.common.requests.OffsetCommitResponse;
import org.apache.kafka.common.requests.OffsetFetchRequest;
import org.apache.kafka.common.requests.OffsetFetchResponse;
import org.apache.kafka.common.requests.RequestUtils;
import org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time;
@ -1512,64 +1513,71 @@ public final class ConsumerCoordinator extends AbstractCoordinator {
@Override
public void handle(OffsetFetchResponse response, RequestFuture<Map<TopicPartition, OffsetAndMetadata>> future) {
Errors responseError = response.groupLevelError(rebalanceConfig.groupId);
if (responseError != Errors.NONE) {
log.debug("Offset fetch failed: {}", responseError.message());
var group = response.group(rebalanceConfig.groupId);
var groupError = Errors.forCode(group.errorCode());
if (responseError == Errors.COORDINATOR_NOT_AVAILABLE ||
responseError == Errors.NOT_COORDINATOR) {
if (groupError != Errors.NONE) {
log.debug("Offset fetch failed: {}", groupError.message());
if (groupError == Errors.COORDINATOR_NOT_AVAILABLE ||
groupError == Errors.NOT_COORDINATOR) {
// re-discover the coordinator and retry
markCoordinatorUnknown(responseError);
future.raise(responseError);
} else if (responseError == Errors.GROUP_AUTHORIZATION_FAILED) {
markCoordinatorUnknown(groupError);
future.raise(groupError);
} else if (groupError == Errors.GROUP_AUTHORIZATION_FAILED) {
future.raise(GroupAuthorizationException.forGroupId(rebalanceConfig.groupId));
} else if (responseError.exception() instanceof RetriableException) {
} else if (groupError.exception() instanceof RetriableException) {
// retry
future.raise(responseError);
future.raise(groupError);
} else {
future.raise(new KafkaException("Unexpected error in fetch offset response: " + responseError.message()));
future.raise(new KafkaException("Unexpected error in fetch offset response: " + groupError.message()));
}
return;
}
Set<String> unauthorizedTopics = null;
Map<TopicPartition, OffsetFetchResponse.PartitionData> responseData =
response.partitionDataMap(rebalanceConfig.groupId);
Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>(responseData.size());
Set<TopicPartition> unstableTxnOffsetTopicPartitions = new HashSet<>();
for (Map.Entry<TopicPartition, OffsetFetchResponse.PartitionData> entry : responseData.entrySet()) {
TopicPartition tp = entry.getKey();
OffsetFetchResponse.PartitionData partitionData = entry.getValue();
if (partitionData.hasError()) {
Errors error = partitionData.error;
log.debug("Failed to fetch offset for partition {}: {}", tp, error.message());
var offsets = new HashMap<TopicPartition, OffsetAndMetadata>();
var unstableTxnOffsetTopicPartitions = new HashSet<TopicPartition>();
var unauthorizedTopics = new HashSet<String>();
if (error == Errors.UNKNOWN_TOPIC_OR_PARTITION) {
future.raise(new KafkaException("Topic or Partition " + tp + " does not exist"));
return;
} else if (error == Errors.TOPIC_AUTHORIZATION_FAILED) {
if (unauthorizedTopics == null) {
unauthorizedTopics = new HashSet<>();
for (var topic : group.topics()) {
for (var partition : topic.partitions()) {
var tp = new TopicPartition(
topic.name(),
partition.partitionIndex()
);
var error = Errors.forCode(partition.errorCode());
if (error != Errors.NONE) {
log.debug("Failed to fetch offset for partition {}: {}", tp, error.message());
if (error == Errors.UNKNOWN_TOPIC_OR_PARTITION) {
future.raise(new KafkaException("Topic or Partition " + tp + " does not exist"));
return;
} else if (error == Errors.TOPIC_AUTHORIZATION_FAILED) {
unauthorizedTopics.add(tp.topic());
} else if (error == Errors.UNSTABLE_OFFSET_COMMIT) {
unstableTxnOffsetTopicPartitions.add(tp);
} else {
future.raise(new KafkaException("Unexpected error in fetch offset response for partition " +
tp + ": " + error.message()));
return;
}
unauthorizedTopics.add(tp.topic());
} else if (error == Errors.UNSTABLE_OFFSET_COMMIT) {
unstableTxnOffsetTopicPartitions.add(tp);
} else if (partition.committedOffset() >= 0) {
// record the position with the offset (-1 indicates no committed offset to fetch);
// if there's no committed offset, record as null
offsets.put(tp, new OffsetAndMetadata(
partition.committedOffset(),
RequestUtils.getLeaderEpoch(partition.committedLeaderEpoch()),
partition.metadata()
));
} else {
future.raise(new KafkaException("Unexpected error in fetch offset response for partition " +
tp + ": " + error.message()));
return;
log.info("Found no committed offset for partition {}", tp);
offsets.put(tp, null);
}
} else if (partitionData.offset >= 0) {
// record the position with the offset (-1 indicates no committed offset to fetch);
// if there's no committed offset, record as null
offsets.put(tp, new OffsetAndMetadata(partitionData.offset, partitionData.leaderEpoch, partitionData.metadata));
} else {
log.info("Found no committed offset for partition {}", tp);
offsets.put(tp, null);
}
}
if (unauthorizedTopics != null) {
if (!unauthorizedTopics.isEmpty()) {
future.raise(new TopicAuthorizationException(unauthorizedTopics));
} else if (!unstableTxnOffsetTopicPartitions.isEmpty()) {
// just retry

View File

@ -41,9 +41,9 @@ import java.util.stream.Collectors;
public class OffsetFetchRequest extends AbstractRequest {
private static final Logger log = LoggerFactory.getLogger(OffsetFetchRequest.class);
private static final short TOP_LEVEL_ERROR_AND_NULL_TOPICS_MIN_VERSION = 2;
private static final short REQUIRE_STABLE_OFFSET_MIN_VERSION = 7;
private static final short BATCH_MIN_VERSION = 8;
public static final short TOP_LEVEL_ERROR_AND_NULL_TOPICS_MIN_VERSION = 2;
public static final short REQUIRE_STABLE_OFFSET_MIN_VERSION = 7;
public static final short BATCH_MIN_VERSION = 8;
private final OffsetFetchRequestData data;

View File

@ -16,30 +16,25 @@
*/
package org.apache.kafka.common.requests;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.message.OffsetFetchResponseData;
import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponseGroup;
import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponsePartition;
import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponsePartitions;
import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponseTopic;
import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponseTopics;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.Readable;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.EnumMap;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Objects;
import java.util.Optional;
import java.util.function.Function;
import java.util.stream.Collectors;
import static org.apache.kafka.common.record.RecordBatch.NO_PARTITION_LEADER_EPOCH;
import static org.apache.kafka.common.requests.OffsetFetchRequest.BATCH_MIN_VERSION;
import static org.apache.kafka.common.requests.OffsetFetchRequest.TOP_LEVEL_ERROR_AND_NULL_TOPICS_MIN_VERSION;
/**
* Possible error codes:
@ -60,221 +55,127 @@ import static org.apache.kafka.common.record.RecordBatch.NO_PARTITION_LEADER_EPO
public class OffsetFetchResponse extends AbstractResponse {
public static final long INVALID_OFFSET = -1L;
public static final String NO_METADATA = "";
public static final PartitionData UNKNOWN_PARTITION = new PartitionData(INVALID_OFFSET,
Optional.empty(),
NO_METADATA,
Errors.UNKNOWN_TOPIC_OR_PARTITION);
public static final PartitionData UNAUTHORIZED_PARTITION = new PartitionData(INVALID_OFFSET,
Optional.empty(),
NO_METADATA,
Errors.TOPIC_AUTHORIZATION_FAILED);
// We only need to track the partition errors returned in version 1. This
// is used to identify group level errors when the response is normalized.
private static final List<Errors> PARTITION_ERRORS = Arrays.asList(
Errors.UNKNOWN_TOPIC_OR_PARTITION, Errors.TOPIC_AUTHORIZATION_FAILED);
Errors.UNKNOWN_TOPIC_OR_PARTITION,
Errors.TOPIC_AUTHORIZATION_FAILED
);
private final short version;
private final OffsetFetchResponseData data;
private final Errors error;
private final Map<String, Errors> groupLevelErrors = new HashMap<>();
// Lazily initialized when OffsetFetchResponse#group is called.
private Map<String, OffsetFetchResponseData.OffsetFetchResponseGroup> groups = null;
public static final class PartitionData {
public final long offset;
public final String metadata;
public final Errors error;
public final Optional<Integer> leaderEpoch;
public static class Builder {
private final List<OffsetFetchResponseGroup> groups;
public PartitionData(long offset,
Optional<Integer> leaderEpoch,
String metadata,
Errors error) {
this.offset = offset;
this.leaderEpoch = leaderEpoch;
this.metadata = metadata;
this.error = error;
public Builder(OffsetFetchResponseGroup group) {
this(List.of(group));
}
public boolean hasError() {
return this.error != Errors.NONE;
public Builder(List<OffsetFetchResponseGroup> groups) {
this.groups = groups;
}
@Override
public boolean equals(Object other) {
if (!(other instanceof PartitionData))
return false;
PartitionData otherPartition = (PartitionData) other;
return Objects.equals(this.offset, otherPartition.offset)
&& Objects.equals(this.leaderEpoch, otherPartition.leaderEpoch)
&& Objects.equals(this.metadata, otherPartition.metadata)
&& Objects.equals(this.error, otherPartition.error);
}
public OffsetFetchResponse build(short version) {
var data = new OffsetFetchResponseData();
@Override
public String toString() {
return "PartitionData("
+ "offset=" + offset
+ ", leaderEpoch=" + leaderEpoch.orElse(NO_PARTITION_LEADER_EPOCH)
+ ", metadata=" + metadata
+ ", error='" + error.toString()
+ ")";
}
if (version >= BATCH_MIN_VERSION) {
data.setGroups(groups);
} else {
if (groups.size() != 1) {
throw new UnsupportedVersionException(
"Version " + version + " of OffsetFetchResponse only supports one group."
);
}
@Override
public int hashCode() {
return Objects.hash(offset, leaderEpoch, metadata, error);
}
}
OffsetFetchResponseGroup group = groups.get(0);
data.setErrorCode(group.errorCode());
/**
* Constructor without throttle time.
* @param error Potential coordinator or group level error code (for api version 2 and later)
* @param responseData Fetched offset information grouped by topic-partition
*/
public OffsetFetchResponse(Errors error, Map<TopicPartition, PartitionData> responseData) {
this(DEFAULT_THROTTLE_TIME, error, responseData);
}
group.topics().forEach(topic -> {
OffsetFetchResponseTopic newTopic = new OffsetFetchResponseTopic().setName(topic.name());
data.topics().add(newTopic);
/**
* Constructor with throttle time for version 0 to 7
* @param throttleTimeMs The time in milliseconds that this response was throttled
* @param error Potential coordinator or group level error code (for api version 2 and later)
* @param responseData Fetched offset information grouped by topic-partition
*/
public OffsetFetchResponse(int throttleTimeMs, Errors error, Map<TopicPartition, PartitionData> responseData) {
super(ApiKeys.OFFSET_FETCH);
Map<String, OffsetFetchResponseTopic> offsetFetchResponseTopicMap = new HashMap<>();
for (Map.Entry<TopicPartition, PartitionData> entry : responseData.entrySet()) {
String topicName = entry.getKey().topic();
OffsetFetchResponseTopic topic = offsetFetchResponseTopicMap.getOrDefault(
topicName, new OffsetFetchResponseTopic().setName(topicName));
PartitionData partitionData = entry.getValue();
topic.partitions().add(new OffsetFetchResponsePartition()
.setPartitionIndex(entry.getKey().partition())
.setErrorCode(partitionData.error.code())
.setCommittedOffset(partitionData.offset)
.setCommittedLeaderEpoch(
partitionData.leaderEpoch.orElse(NO_PARTITION_LEADER_EPOCH))
.setMetadata(partitionData.metadata)
);
offsetFetchResponseTopicMap.put(topicName, topic);
}
topic.partitions().forEach(partition -> {
OffsetFetchResponsePartition newPartition;
this.data = new OffsetFetchResponseData()
.setTopics(new ArrayList<>(offsetFetchResponseTopicMap.values()))
.setErrorCode(error.code())
.setThrottleTimeMs(throttleTimeMs);
this.error = error;
}
if (version < TOP_LEVEL_ERROR_AND_NULL_TOPICS_MIN_VERSION && group.errorCode() != Errors.NONE.code()) {
// Versions prior to version 2 do not support a top level error. Therefore,
// we put it at the partition level.
newPartition = new OffsetFetchResponsePartition()
.setPartitionIndex(partition.partitionIndex())
.setErrorCode(group.errorCode())
.setCommittedOffset(INVALID_OFFSET)
.setMetadata(NO_METADATA)
.setCommittedLeaderEpoch(NO_PARTITION_LEADER_EPOCH);
} else {
newPartition = new OffsetFetchResponsePartition()
.setPartitionIndex(partition.partitionIndex())
.setErrorCode(partition.errorCode())
.setCommittedOffset(partition.committedOffset())
.setMetadata(partition.metadata())
.setCommittedLeaderEpoch(partition.committedLeaderEpoch());
}
/**
* Constructor with throttle time for version 8 and above.
* @param throttleTimeMs The time in milliseconds that this response was throttled
* @param errors Potential coordinator or group level error code
* @param responseData Fetched offset information grouped by topic-partition and by group
*/
public OffsetFetchResponse(int throttleTimeMs,
Map<String, Errors> errors,
Map<String, Map<TopicPartition, PartitionData>> responseData) {
super(ApiKeys.OFFSET_FETCH);
List<OffsetFetchResponseGroup> groupList = new ArrayList<>();
for (Entry<String, Map<TopicPartition, PartitionData>> entry : responseData.entrySet()) {
String groupName = entry.getKey();
Map<TopicPartition, PartitionData> partitionDataMap = entry.getValue();
Map<String, OffsetFetchResponseTopics> offsetFetchResponseTopicsMap = new HashMap<>();
for (Entry<TopicPartition, PartitionData> partitionEntry : partitionDataMap.entrySet()) {
String topicName = partitionEntry.getKey().topic();
OffsetFetchResponseTopics topic =
offsetFetchResponseTopicsMap.getOrDefault(topicName,
new OffsetFetchResponseTopics().setName(topicName));
PartitionData partitionData = partitionEntry.getValue();
topic.partitions().add(new OffsetFetchResponsePartitions()
.setPartitionIndex(partitionEntry.getKey().partition())
.setErrorCode(partitionData.error.code())
.setCommittedOffset(partitionData.offset)
.setCommittedLeaderEpoch(
partitionData.leaderEpoch.orElse(NO_PARTITION_LEADER_EPOCH))
.setMetadata(partitionData.metadata));
offsetFetchResponseTopicsMap.put(topicName, topic);
}
groupList.add(new OffsetFetchResponseGroup()
.setGroupId(groupName)
.setTopics(new ArrayList<>(offsetFetchResponseTopicsMap.values()))
.setErrorCode(errors.get(groupName).code()));
groupLevelErrors.put(groupName, errors.get(groupName));
}
this.data = new OffsetFetchResponseData()
.setGroups(groupList)
.setThrottleTimeMs(throttleTimeMs);
this.error = null;
}
public OffsetFetchResponse(List<OffsetFetchResponseGroup> groups, short version) {
super(ApiKeys.OFFSET_FETCH);
data = new OffsetFetchResponseData();
if (version >= 8) {
data.setGroups(groups);
error = null;
for (OffsetFetchResponseGroup group : data.groups()) {
this.groupLevelErrors.put(group.groupId(), Errors.forCode(group.errorCode()));
}
} else {
if (groups.size() != 1) {
throw new UnsupportedVersionException(
"Version " + version + " of OffsetFetchResponse only supports one group."
);
}
OffsetFetchResponseGroup group = groups.get(0);
data.setErrorCode(group.errorCode());
error = Errors.forCode(group.errorCode());
group.topics().forEach(topic -> {
OffsetFetchResponseTopic newTopic = new OffsetFetchResponseTopic().setName(topic.name());
data.topics().add(newTopic);
topic.partitions().forEach(partition -> {
OffsetFetchResponsePartition newPartition;
if (version < 2 && group.errorCode() != Errors.NONE.code()) {
// Versions prior to version 2 do not support a top level error. Therefore,
// we put it at the partition level.
newPartition = new OffsetFetchResponsePartition()
.setPartitionIndex(partition.partitionIndex())
.setErrorCode(group.errorCode())
.setCommittedOffset(INVALID_OFFSET)
.setMetadata(NO_METADATA)
.setCommittedLeaderEpoch(NO_PARTITION_LEADER_EPOCH);
} else {
newPartition = new OffsetFetchResponsePartition()
.setPartitionIndex(partition.partitionIndex())
.setErrorCode(partition.errorCode())
.setCommittedOffset(partition.committedOffset())
.setMetadata(partition.metadata())
.setCommittedLeaderEpoch(partition.committedLeaderEpoch());
}
newTopic.partitions().add(newPartition);
newTopic.partitions().add(newPartition);
});
});
});
}
return new OffsetFetchResponse(data, version);
}
}
public OffsetFetchResponse(OffsetFetchResponseData data, short version) {
super(ApiKeys.OFFSET_FETCH);
this.data = data;
// for version 2 and later use the top-level error code (in ERROR_CODE_KEY_NAME) from the response.
// for older versions there is no top-level error in the response and all errors are partition errors,
// so if there is a group or coordinator error at the partition level use that as the top-level error.
// this way clients can depend on the top-level error regardless of the offset fetch version.
// we return the error differently starting with version 8, so we will only populate the
// error field if we are between version 2 and 7. if we are in version 8 or greater, then
// we will populate the map of group id to error codes.
if (version < 8) {
this.error = version >= 2 ? Errors.forCode(data.errorCode()) : topLevelError(data);
} else {
for (OffsetFetchResponseGroup group : data.groups()) {
this.groupLevelErrors.put(group.groupId(), Errors.forCode(group.errorCode()));
this.version = version;
}
public OffsetFetchResponseData.OffsetFetchResponseGroup group(String groupId) {
if (version < BATCH_MIN_VERSION) {
// for version 2 and later use the top-level error code from the response.
// for older versions there is no top-level error in the response and all errors are partition errors,
// so if there is a group or coordinator error at the partition level use that as the top-level error.
// this way clients can depend on the top-level error regardless of the offset fetch version.
// we return the error differently starting with version 8, so we will only populate the
// error field if we are between version 2 and 7. if we are in version 8 or greater, then
// we will populate the map of group id to error codes.
short topLevelError = version < TOP_LEVEL_ERROR_AND_NULL_TOPICS_MIN_VERSION ? topLevelError(data).code() : data.errorCode();
if (topLevelError != Errors.NONE.code()) {
return new OffsetFetchResponseGroup()
.setGroupId(groupId)
.setErrorCode(topLevelError);
} else {
return new OffsetFetchResponseGroup()
.setGroupId(groupId)
.setTopics(data.topics().stream().map(topic ->
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(topic.name())
.setPartitions(topic.partitions().stream().map(partition ->
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(partition.partitionIndex())
.setErrorCode(partition.errorCode())
.setCommittedOffset(partition.committedOffset())
.setMetadata(partition.metadata())
.setCommittedLeaderEpoch(partition.committedLeaderEpoch())
).collect(Collectors.toList()))
).collect(Collectors.toList()));
}
this.error = null;
} else {
if (groups == null) {
groups = data.groups().stream().collect(Collectors.toMap(
OffsetFetchResponseData.OffsetFetchResponseGroup::groupId,
Function.identity()
));
}
var group = groups.get(groupId);
if (group == null) {
throw new IllegalArgumentException("Group " + groupId + " not found in the response");
}
return group;
}
}
@ -300,96 +201,31 @@ public class OffsetFetchResponse extends AbstractResponse {
data.setThrottleTimeMs(throttleTimeMs);
}
public boolean hasError() {
return error != Errors.NONE;
}
public boolean groupHasError(String groupId) {
Errors error = groupLevelErrors.get(groupId);
if (error == null) {
return this.error != null && this.error != Errors.NONE;
}
return error != Errors.NONE;
}
public Errors error() {
return error;
}
public Errors groupLevelError(String groupId) {
if (error != null) {
return error;
}
return groupLevelErrors.get(groupId);
}
@Override
public Map<Errors, Integer> errorCounts() {
Map<Errors, Integer> counts = new EnumMap<>(Errors.class);
if (!groupLevelErrors.isEmpty()) {
// built response with v8 or above
for (Map.Entry<String, Errors> entry : groupLevelErrors.entrySet()) {
updateErrorCounts(counts, entry.getValue());
if (version < BATCH_MIN_VERSION) {
if (version >= TOP_LEVEL_ERROR_AND_NULL_TOPICS_MIN_VERSION) {
updateErrorCounts(counts, Errors.forCode(data.errorCode()));
}
for (OffsetFetchResponseGroup group : data.groups()) {
group.topics().forEach(topic ->
topic.partitions().forEach(partition ->
updateErrorCounts(counts, Errors.forCode(partition.errorCode()))));
}
} else {
// built response with v0-v7
updateErrorCounts(counts, error);
data.topics().forEach(topic ->
topic.partitions().forEach(partition ->
updateErrorCounts(counts, Errors.forCode(partition.errorCode()))));
updateErrorCounts(counts, Errors.forCode(partition.errorCode()))
)
);
} else {
data.groups().forEach(group -> {
updateErrorCounts(counts, Errors.forCode(group.errorCode()));
group.topics().forEach(topic ->
topic.partitions().forEach(partition ->
updateErrorCounts(counts, Errors.forCode(partition.errorCode()))
)
);
});
}
return counts;
}
// package-private for testing purposes
Map<TopicPartition, PartitionData> responseDataV0ToV7() {
Map<TopicPartition, PartitionData> responseData = new HashMap<>();
for (OffsetFetchResponseTopic topic : data.topics()) {
for (OffsetFetchResponsePartition partition : topic.partitions()) {
responseData.put(new TopicPartition(topic.name(), partition.partitionIndex()),
new PartitionData(partition.committedOffset(),
RequestUtils.getLeaderEpoch(partition.committedLeaderEpoch()),
partition.metadata(),
Errors.forCode(partition.errorCode()))
);
}
}
return responseData;
}
private Map<TopicPartition, PartitionData> buildResponseData(String groupId) {
Map<TopicPartition, PartitionData> responseData = new HashMap<>();
OffsetFetchResponseGroup group = data
.groups()
.stream()
.filter(g -> g.groupId().equals(groupId))
.collect(Collectors.toList())
.get(0);
for (OffsetFetchResponseTopics topic : group.topics()) {
for (OffsetFetchResponsePartitions partition : topic.partitions()) {
responseData.put(new TopicPartition(topic.name(), partition.partitionIndex()),
new PartitionData(partition.committedOffset(),
RequestUtils.getLeaderEpoch(partition.committedLeaderEpoch()),
partition.metadata(),
Errors.forCode(partition.errorCode()))
);
}
}
return responseData;
}
public Map<TopicPartition, PartitionData> partitionDataMap(String groupId) {
if (groupLevelErrors.isEmpty()) {
return responseDataV0ToV7();
}
return buildResponseData(groupId);
}
public static OffsetFetchResponse parse(Readable readable, short version) {
return new OffsetFetchResponse(new OffsetFetchResponseData(readable, version), version);
}

View File

@ -161,6 +161,7 @@ import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResp
import org.apache.kafka.common.message.OffsetFetchRequestData;
import org.apache.kafka.common.message.OffsetFetchRequestData.OffsetFetchRequestGroup;
import org.apache.kafka.common.message.OffsetFetchRequestData.OffsetFetchRequestTopics;
import org.apache.kafka.common.message.OffsetFetchResponseData;
import org.apache.kafka.common.message.RemoveRaftVoterRequestData;
import org.apache.kafka.common.message.RemoveRaftVoterResponseData;
import org.apache.kafka.common.message.ShareGroupDescribeResponseData;
@ -239,7 +240,6 @@ import org.apache.kafka.common.requests.OffsetCommitResponse;
import org.apache.kafka.common.requests.OffsetDeleteResponse;
import org.apache.kafka.common.requests.OffsetFetchRequest;
import org.apache.kafka.common.requests.OffsetFetchResponse;
import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData;
import org.apache.kafka.common.requests.RemoveRaftVoterRequest;
import org.apache.kafka.common.requests.RemoveRaftVoterResponse;
import org.apache.kafka.common.requests.RequestTestUtils;
@ -337,7 +337,6 @@ import static org.mockito.Mockito.when;
public class KafkaAdminClientTest {
private static final Logger log = LoggerFactory.getLogger(KafkaAdminClientTest.class);
private static final String GROUP_ID = "group-0";
private static final int THROTTLE = 10;
public static final Uuid REPLICA_DIRECTORY_ID = Uuid.randomUuid();
@Test
@ -4417,7 +4416,7 @@ public class KafkaAdminClientTest {
env.kafkaClient().setNodeApiVersions(NodeApiVersions.create());
env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller()));
env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap()));
env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.NOT_COORDINATOR));
env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller()));
final ListConsumerGroupOffsetsResult result = env.adminClient().listConsumerGroupOffsets(GROUP_ID);
@ -4445,14 +4444,14 @@ public class KafkaAdminClientTest {
mockClient.prepareResponse(body -> {
firstAttemptTime.set(time.milliseconds());
return true;
}, offsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap()));
}, offsetFetchResponse(Errors.NOT_COORDINATOR));
mockClient.prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller()));
mockClient.prepareResponse(body -> {
secondAttemptTime.set(time.milliseconds());
return true;
}, offsetFetchResponse(Errors.NONE, Collections.emptyMap()));
}, offsetFetchResponse(Errors.NONE));
final KafkaFuture<Map<TopicPartition, OffsetAndMetadata>> future = env.adminClient().listConsumerGroupOffsets(GROUP_ID).partitionsToOffsetAndMetadata();
@ -4481,7 +4480,7 @@ public class KafkaAdminClientTest {
prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller()));
env.kafkaClient().prepareResponse(
offsetFetchResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Collections.emptyMap()));
offsetFetchResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS));
/*
* We need to return two responses here, one for NOT_COORDINATOR call when calling list consumer offsets
@ -4491,19 +4490,19 @@ public class KafkaAdminClientTest {
* And the same reason for the following COORDINATOR_NOT_AVAILABLE error response
*/
env.kafkaClient().prepareResponse(
offsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap()));
offsetFetchResponse(Errors.NOT_COORDINATOR));
env.kafkaClient().prepareResponse(
prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller()));
env.kafkaClient().prepareResponse(
offsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE, Collections.emptyMap()));
offsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE));
env.kafkaClient().prepareResponse(
prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller()));
env.kafkaClient().prepareResponse(
offsetFetchResponse(Errors.NONE, Collections.emptyMap()));
offsetFetchResponse(Errors.NONE));
final ListConsumerGroupOffsetsResult errorResult1 = env.adminClient().listConsumerGroupOffsets(GROUP_ID);
@ -4525,7 +4524,7 @@ public class KafkaAdminClientTest {
env.kafkaClient().prepareResponse(
prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller()));
env.kafkaClient().prepareResponse(offsetFetchResponse(error, Collections.emptyMap()));
env.kafkaClient().prepareResponse(offsetFetchResponse(error));
ListConsumerGroupOffsetsResult errorResult = env.adminClient().listConsumerGroupOffsets(GROUP_ID);
@ -4545,7 +4544,7 @@ public class KafkaAdminClientTest {
env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller()));
// Retriable errors should be retried
env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Collections.emptyMap()));
env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS));
/*
* We need to return two responses here, one for NOT_COORDINATOR error when calling list consumer group offsets
@ -4554,10 +4553,10 @@ public class KafkaAdminClientTest {
*
* And the same reason for the following COORDINATOR_NOT_AVAILABLE error response
*/
env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap()));
env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.NOT_COORDINATOR));
env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller()));
env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE, Collections.emptyMap()));
env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE));
env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller()));
TopicPartition myTopicPartition0 = new TopicPartition("my_topic", 0);
@ -4565,16 +4564,31 @@ public class KafkaAdminClientTest {
TopicPartition myTopicPartition2 = new TopicPartition("my_topic", 2);
TopicPartition myTopicPartition3 = new TopicPartition("my_topic", 3);
final Map<TopicPartition, OffsetFetchResponse.PartitionData> responseData = new HashMap<>();
responseData.put(myTopicPartition0, new OffsetFetchResponse.PartitionData(10,
Optional.empty(), "", Errors.NONE));
responseData.put(myTopicPartition1, new OffsetFetchResponse.PartitionData(0,
Optional.empty(), "", Errors.NONE));
responseData.put(myTopicPartition2, new OffsetFetchResponse.PartitionData(20,
Optional.empty(), "", Errors.NONE));
responseData.put(myTopicPartition3, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET,
Optional.empty(), "", Errors.NONE));
env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.NONE, responseData));
final OffsetFetchResponseData response = new OffsetFetchResponseData()
.setGroups(List.of(
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(GROUP_ID)
.setTopics(List.of(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName("my_topic")
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(myTopicPartition0.partition())
.setCommittedOffset(10),
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(myTopicPartition1.partition())
.setCommittedOffset(0),
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(myTopicPartition2.partition())
.setCommittedOffset(20),
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(myTopicPartition3.partition())
.setCommittedOffset(OffsetFetchResponse.INVALID_OFFSET)
))
))
));
env.kafkaClient().prepareResponse(new OffsetFetchResponse(response, ApiKeys.OFFSET_FETCH.latestVersion()));
final ListConsumerGroupOffsetsResult result = env.adminClient().listConsumerGroupOffsets(GROUP_ID);
final Map<TopicPartition, OffsetAndMetadata> partitionToOffsetAndMetadata = result.partitionsToOffsetAndMetadata().get();
@ -4704,7 +4718,7 @@ public class KafkaAdminClientTest {
env.kafkaClient().prepareResponse(prepareBatchedFindCoordinatorResponse(Errors.NONE, env.cluster().controller(), groupSpecs.keySet()));
// Prepare a response to force client to attempt batched request creation that throws
// NoBatchedOffsetFetchRequestException. This triggers creation of non-batched requests.
env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE, Collections.emptyMap()));
env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE));
ListConsumerGroupOffsetsResult result = env.adminClient().listConsumerGroupOffsets(groupSpecs);
@ -4737,7 +4751,7 @@ public class KafkaAdminClientTest {
env.kafkaClient().prepareResponse(prepareBatchedFindCoordinatorResponse(Errors.NONE, env.cluster().controller(), groupSpecs.keySet()));
// Prepare a response to force client to attempt batched request creation that throws
// NoBatchedOffsetFetchRequestException. This triggers creation of non-batched requests.
env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE, Collections.emptyMap()));
env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE));
ListStreamsGroupOffsetsResult result = env.adminClient().listStreamsGroupOffsets(groupSpecs);
@ -4792,21 +4806,29 @@ public class KafkaAdminClientTest {
ClientRequest clientRequest = mockClient.requests().peek();
OffsetFetchRequestData data = ((OffsetFetchRequest.Builder) clientRequest.requestBuilder()).build().data();
Map<String, Map<TopicPartition, PartitionData>> results = new HashMap<>();
Map<String, Errors> errors = new HashMap<>();
data.groups().forEach(group -> {
Map<TopicPartition, PartitionData> partitionResults = new HashMap<>();
for (TopicPartition tp : groupSpecs.get(group.groupId()).topicPartitions()) {
partitionResults.put(tp, new PartitionData(10, Optional.empty(), "", Errors.NONE));
}
results.put(group.groupId(), partitionResults);
errors.put(group.groupId(), error);
});
if (!batched) {
assertEquals(1, data.groups().size());
mockClient.respond(new OffsetFetchResponse(THROTTLE, error, results.values().iterator().next()));
} else
mockClient.respond(new OffsetFetchResponse(THROTTLE, errors, results));
}
OffsetFetchResponseData response = new OffsetFetchResponseData()
.setGroups(data.groups().stream().map(group ->
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(group.groupId())
.setErrorCode(error.code())
.setTopics(groupSpecs.get(group.groupId()).topicPartitions().stream()
.collect(Collectors.groupingBy(TopicPartition::topic)).entrySet().stream().map(entry ->
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(entry.getKey())
.setPartitions(entry.getValue().stream().map(partition ->
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(partition.partition())
.setCommittedOffset(10)
).collect(Collectors.toList()))
).collect(Collectors.toList()))
).collect(Collectors.toList()));
mockClient.respond(new OffsetFetchResponse(response, ApiKeys.OFFSET_FETCH.latestVersion()));
}
private void sendStreamsOffsetFetchResponse(MockClient mockClient, Map<String, ListStreamsGroupOffsetsSpec> groupSpecs, boolean batched, Errors error) throws Exception {
@ -4814,21 +4836,29 @@ public class KafkaAdminClientTest {
ClientRequest clientRequest = mockClient.requests().peek();
OffsetFetchRequestData data = ((OffsetFetchRequest.Builder) clientRequest.requestBuilder()).build().data();
Map<String, Map<TopicPartition, PartitionData>> results = new HashMap<>();
Map<String, Errors> errors = new HashMap<>();
data.groups().forEach(group -> {
Map<TopicPartition, PartitionData> partitionResults = new HashMap<>();
for (TopicPartition tp : groupSpecs.get(group.groupId()).topicPartitions()) {
partitionResults.put(tp, new PartitionData(10, Optional.empty(), "", Errors.NONE));
}
results.put(group.groupId(), partitionResults);
errors.put(group.groupId(), error);
});
if (!batched) {
assertEquals(1, data.groups().size());
mockClient.respond(new OffsetFetchResponse(THROTTLE, error, results.values().iterator().next()));
} else
mockClient.respond(new OffsetFetchResponse(THROTTLE, errors, results));
}
OffsetFetchResponseData response = new OffsetFetchResponseData()
.setGroups(data.groups().stream().map(group ->
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(group.groupId())
.setErrorCode(error.code())
.setTopics(groupSpecs.get(group.groupId()).topicPartitions().stream()
.collect(Collectors.groupingBy(TopicPartition::topic)).entrySet().stream().map(entry ->
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(entry.getKey())
.setPartitions(entry.getValue().stream().map(partition ->
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(partition.partition())
.setCommittedOffset(10)
).collect(Collectors.toList()))
).collect(Collectors.toList()))
).collect(Collectors.toList()));
mockClient.respond(new OffsetFetchResponse(response, ApiKeys.OFFSET_FETCH.latestVersion()));
}
private void verifyListOffsetsForMultipleGroups(Map<String, ListConsumerGroupOffsetsSpec> groupSpecs,
@ -10158,10 +10188,16 @@ public class KafkaAdminClientTest {
.setLogDir(logDir))));
}
private OffsetFetchResponse offsetFetchResponse(Errors error, Map<TopicPartition, PartitionData> responseData) {
return new OffsetFetchResponse(THROTTLE,
Collections.singletonMap(GROUP_ID, error),
Collections.singletonMap(GROUP_ID, responseData));
private static OffsetFetchResponse offsetFetchResponse(Errors error) {
return new OffsetFetchResponse(
new OffsetFetchResponseData()
.setGroups(List.of(
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(GROUP_ID)
.setErrorCode(error.code())
)),
ApiKeys.OFFSET_FETCH.latestVersion()
);
}
private static MemberDescription convertToMemberDescriptions(DescribedGroupMember member,

View File

@ -24,12 +24,14 @@ import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.GroupAuthorizationException;
import org.apache.kafka.common.errors.GroupIdNotFoundException;
import org.apache.kafka.common.errors.InvalidGroupIdException;
import org.apache.kafka.common.message.OffsetFetchRequestData;
import org.apache.kafka.common.message.OffsetFetchRequestData.OffsetFetchRequestGroup;
import org.apache.kafka.common.message.OffsetFetchRequestData.OffsetFetchRequestTopics;
import org.apache.kafka.common.message.OffsetFetchResponseData;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.common.requests.OffsetFetchRequest;
import org.apache.kafka.common.requests.OffsetFetchResponse;
import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData;
import org.apache.kafka.common.utils.LogContext;
import org.junit.jupiter.api.Test;
@ -41,7 +43,6 @@ import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.function.Function;
import java.util.stream.Collectors;
@ -58,11 +59,11 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
public class ListConsumerGroupOffsetsHandlerTest {
private final LogContext logContext = new LogContext();
private final int throttleMs = 10;
private final String groupZero = "group0";
private final String groupOne = "group1";
private final String groupTwo = "group2";
private final List<String> groups = Arrays.asList(groupZero, groupOne, groupTwo);
private final String group0 = "group0";
private final String group1 = "group1";
private final String group2 = "group2";
private final String group3 = "group3";
private final List<String> groups = List.of(group0, group1, group2);
private final TopicPartition t0p0 = new TopicPartition("t0", 0);
private final TopicPartition t0p1 = new TopicPartition("t0", 1);
private final TopicPartition t1p0 = new TopicPartition("t1", 0);
@ -70,84 +71,129 @@ public class ListConsumerGroupOffsetsHandlerTest {
private final TopicPartition t2p0 = new TopicPartition("t2", 0);
private final TopicPartition t2p1 = new TopicPartition("t2", 1);
private final TopicPartition t2p2 = new TopicPartition("t2", 2);
private final Map<String, ListConsumerGroupOffsetsSpec> singleRequestMap = Collections.singletonMap(groupZero,
new ListConsumerGroupOffsetsSpec().topicPartitions(Arrays.asList(t0p0, t0p1, t1p0, t1p1)));
private final Map<String, ListConsumerGroupOffsetsSpec> batchedRequestMap =
new HashMap<>() {{
put(groupZero, new ListConsumerGroupOffsetsSpec().topicPartitions(singletonList(t0p0)));
put(groupOne, new ListConsumerGroupOffsetsSpec().topicPartitions(Arrays.asList(t0p0, t1p0, t1p1)));
put(groupTwo, new ListConsumerGroupOffsetsSpec().topicPartitions(Arrays.asList(t0p0, t1p0, t1p1, t2p0, t2p1, t2p2)));
}};
private final TopicPartition t3p0 = new TopicPartition("t3", 0);
private final TopicPartition t3p1 = new TopicPartition("t3", 1);
private final Map<String, ListConsumerGroupOffsetsSpec> singleGroupSpec = Map.of(
group0, new ListConsumerGroupOffsetsSpec().topicPartitions(Arrays.asList(t0p0, t0p1, t1p0, t1p1))
);
private final Map<String, ListConsumerGroupOffsetsSpec> multiGroupSpecs = Map.of(
group0, new ListConsumerGroupOffsetsSpec().topicPartitions(singletonList(t0p0)),
group1, new ListConsumerGroupOffsetsSpec().topicPartitions(Arrays.asList(t0p0, t1p0, t1p1)),
group2, new ListConsumerGroupOffsetsSpec().topicPartitions(Arrays.asList(t0p0, t1p0, t1p1, t2p0, t2p1, t2p2))
);
@Test
public void testBuildRequest() {
ListConsumerGroupOffsetsHandler handler =
new ListConsumerGroupOffsetsHandler(singleRequestMap, false, logContext);
OffsetFetchRequest request = handler.buildBatchedRequest(coordinatorKeys(groupZero)).build();
assertEquals(groupZero, request.data().groups().get(0).groupId());
assertEquals(2, request.data().groups().get(0).topics().size());
assertEquals(2, request.data().groups().get(0).topics().get(0).partitionIndexes().size());
assertEquals(2, request.data().groups().get(0).topics().get(1).partitionIndexes().size());
var handler = new ListConsumerGroupOffsetsHandler(
singleGroupSpec,
false,
logContext
);
assertEquals(
new OffsetFetchRequestData()
.setGroups(List.of(
new OffsetFetchRequestData.OffsetFetchRequestGroup()
.setGroupId(group0)
.setTopics(List.of(
new OffsetFetchRequestData.OffsetFetchRequestTopics()
.setName(t0p0.topic())
.setPartitionIndexes(List.of(t0p0.partition(), t0p1.partition())),
new OffsetFetchRequestData.OffsetFetchRequestTopics()
.setName(t1p0.topic())
.setPartitionIndexes(List.of(t1p0.partition(), t1p1.partition()))
))
)),
handler.buildBatchedRequest(coordinatorKeys(group0)).build().data()
);
}
@Test
public void testBuildRequestWithMultipleGroups() {
Map<String, ListConsumerGroupOffsetsSpec> requestMap = new HashMap<>(this.batchedRequestMap);
String groupThree = "group3";
requestMap.put(groupThree, new ListConsumerGroupOffsetsSpec()
.topicPartitions(Arrays.asList(new TopicPartition("t3", 0), new TopicPartition("t3", 1))));
var groupSpecs = new HashMap<>(multiGroupSpecs);
groupSpecs.put(
group3,
new ListConsumerGroupOffsetsSpec().topicPartitions(List.of(t3p0, t3p1))
);
ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(requestMap, false, logContext);
OffsetFetchRequest request1 = handler.buildBatchedRequest(coordinatorKeys(groupZero, groupOne, groupTwo)).build();
assertEquals(Set.of(groupZero, groupOne, groupTwo), requestGroups(request1));
var handler = new ListConsumerGroupOffsetsHandler(
groupSpecs,
false,
logContext
);
OffsetFetchRequest request2 = handler.buildBatchedRequest(coordinatorKeys(groupThree)).build();
assertEquals(Set.of(groupThree), requestGroups(request2));
var request1 = handler.buildBatchedRequest(coordinatorKeys(group0, group1, group2)).build();
Map<String, ListConsumerGroupOffsetsSpec> builtRequests = new HashMap<>();
request1.groupIdsToPartitions().forEach((group, partitions) ->
builtRequests.put(group, new ListConsumerGroupOffsetsSpec().topicPartitions(partitions)));
request2.groupIdsToPartitions().forEach((group, partitions) ->
builtRequests.put(group, new ListConsumerGroupOffsetsSpec().topicPartitions(partitions)));
assertEquals(
Set.of(
new OffsetFetchRequestData.OffsetFetchRequestGroup()
.setGroupId(group0)
.setTopics(List.of(
new OffsetFetchRequestData.OffsetFetchRequestTopics()
.setName(t0p0.topic())
.setPartitionIndexes(List.of(t0p0.partition()))
)),
new OffsetFetchRequestData.OffsetFetchRequestGroup()
.setGroupId(group1)
.setTopics(List.of(
new OffsetFetchRequestData.OffsetFetchRequestTopics()
.setName(t0p0.topic())
.setPartitionIndexes(List.of(t0p0.partition())),
new OffsetFetchRequestData.OffsetFetchRequestTopics()
.setName(t1p0.topic())
.setPartitionIndexes(List.of(t1p0.partition(), t1p1.partition()))
)),
new OffsetFetchRequestData.OffsetFetchRequestGroup()
.setGroupId(group2)
.setTopics(List.of(
new OffsetFetchRequestData.OffsetFetchRequestTopics()
.setName(t0p0.topic())
.setPartitionIndexes(List.of(t0p0.partition())),
new OffsetFetchRequestData.OffsetFetchRequestTopics()
.setName(t1p0.topic())
.setPartitionIndexes(List.of(t1p0.partition(), t1p1.partition())),
new OffsetFetchRequestData.OffsetFetchRequestTopics()
.setName(t2p0.topic())
.setPartitionIndexes(List.of(t2p0.partition(), t2p1.partition(), t2p2.partition()))
))
),
new HashSet(request1.data().groups())
);
assertEquals(requestMap, builtRequests);
Map<String, List<OffsetFetchRequestTopics>> groupIdsToTopics = request1.groupIdsToTopics();
var request2 = handler.buildBatchedRequest(coordinatorKeys(group3)).build();
assertEquals(3, groupIdsToTopics.size());
assertEquals(1, groupIdsToTopics.get(groupZero).size());
assertEquals(2, groupIdsToTopics.get(groupOne).size());
assertEquals(3, groupIdsToTopics.get(groupTwo).size());
assertEquals(1, groupIdsToTopics.get(groupZero).get(0).partitionIndexes().size());
assertEquals(1, groupIdsToTopics.get(groupOne).get(0).partitionIndexes().size());
assertEquals(2, groupIdsToTopics.get(groupOne).get(1).partitionIndexes().size());
assertEquals(1, groupIdsToTopics.get(groupTwo).get(0).partitionIndexes().size());
assertEquals(2, groupIdsToTopics.get(groupTwo).get(1).partitionIndexes().size());
assertEquals(3, groupIdsToTopics.get(groupTwo).get(2).partitionIndexes().size());
groupIdsToTopics = request2.groupIdsToTopics();
assertEquals(1, groupIdsToTopics.size());
assertEquals(1, groupIdsToTopics.get(groupThree).size());
assertEquals(2, groupIdsToTopics.get(groupThree).get(0).partitionIndexes().size());
assertEquals(
Set.of(
new OffsetFetchRequestData.OffsetFetchRequestGroup()
.setGroupId(group3)
.setTopics(List.of(
new OffsetFetchRequestData.OffsetFetchRequestTopics()
.setName(t3p0.topic())
.setPartitionIndexes(List.of(t3p0.partition(), t3p1.partition()))
))
),
new HashSet(request2.data().groups())
);
}
@Test
public void testBuildRequestBatchGroups() {
ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(batchedRequestMap, false, logContext);
Collection<RequestAndKeys<CoordinatorKey>> requests = handler.buildRequest(1, coordinatorKeys(groupZero, groupOne, groupTwo));
ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(multiGroupSpecs, false, logContext);
Collection<RequestAndKeys<CoordinatorKey>> requests = handler.buildRequest(1, coordinatorKeys(group0, group1, group2));
assertEquals(1, requests.size());
assertEquals(Set.of(groupZero, groupOne, groupTwo), requestGroups((OffsetFetchRequest) requests.iterator().next().request.build()));
assertEquals(Set.of(group0, group1, group2), requestGroups((OffsetFetchRequest) requests.iterator().next().request.build()));
}
@Test
public void testBuildRequestDoesNotBatchGroup() {
ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(batchedRequestMap, false, logContext);
ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(multiGroupSpecs, false, logContext);
// Disable batching.
((CoordinatorStrategy) handler.lookupStrategy()).disableBatch();
Collection<RequestAndKeys<CoordinatorKey>> requests = handler.buildRequest(1, coordinatorKeys(groupZero, groupOne, groupTwo));
Collection<RequestAndKeys<CoordinatorKey>> requests = handler.buildRequest(1, coordinatorKeys(group0, group1, group2));
assertEquals(3, requests.size());
assertEquals(
Set.of(Set.of(groupZero), Set.of(groupOne), Set.of(groupTwo)),
Set.of(Set.of(group0), Set.of(group1), Set.of(group2)),
requests.stream().map(requestAndKey -> requestGroups((OffsetFetchRequest) requestAndKey.request.build())).collect(Collectors.toSet())
);
}
@ -170,32 +216,31 @@ public class ListConsumerGroupOffsetsHandlerTest {
@Test
public void testSuccessfulHandleResponseWithOnePartitionErrorWithMultipleGroups() {
Map<TopicPartition, OffsetAndMetadata> offsetAndMetadataMapZero =
Collections.singletonMap(t0p0, new OffsetAndMetadata(10L));
Map<TopicPartition, OffsetAndMetadata> offsetAndMetadataMapOne =
Collections.singletonMap(t1p1, new OffsetAndMetadata(10L));
Map<TopicPartition, OffsetAndMetadata> offsetAndMetadataMapTwo =
Collections.singletonMap(t2p2, new OffsetAndMetadata(10L));
Map<String, Map<TopicPartition, OffsetAndMetadata>> expectedResult =
new HashMap<>() {{
put(groupZero, offsetAndMetadataMapZero);
put(groupOne, offsetAndMetadataMapOne);
put(groupTwo, offsetAndMetadataMapTwo);
}};
var expectedResult = Map.of(
group0, Map.of(t0p0, new OffsetAndMetadata(10L)),
group1, Map.of(t1p1, new OffsetAndMetadata(10L)),
group2, Map.of(t2p2, new OffsetAndMetadata(10L))
);
assertCompletedForMultipleGroups(
handleWithPartitionErrorMultipleGroups(Errors.UNKNOWN_TOPIC_OR_PARTITION), expectedResult);
handleWithPartitionErrorMultipleGroups(Errors.UNKNOWN_TOPIC_OR_PARTITION),
expectedResult
);
assertCompletedForMultipleGroups(
handleWithPartitionErrorMultipleGroups(Errors.TOPIC_AUTHORIZATION_FAILED), expectedResult);
handleWithPartitionErrorMultipleGroups(Errors.TOPIC_AUTHORIZATION_FAILED),
expectedResult
);
assertCompletedForMultipleGroups(
handleWithPartitionErrorMultipleGroups(Errors.UNSTABLE_OFFSET_COMMIT), expectedResult);
handleWithPartitionErrorMultipleGroups(Errors.UNSTABLE_OFFSET_COMMIT),
expectedResult
);
}
@Test
public void testSuccessfulHandleResponseWithMultipleGroups() {
Map<String, Map<TopicPartition, OffsetAndMetadata>> expected = new HashMap<>();
Map<String, Errors> errorMap = errorMap(groups, Errors.NONE);
assertCompletedForMultipleGroups(handleWithErrorWithMultipleGroups(errorMap, batchedRequestMap), expected);
assertCompletedForMultipleGroups(handleWithErrorWithMultipleGroups(errorMap, multiGroupSpecs), expected);
}
@Test
@ -206,11 +251,12 @@ public class ListConsumerGroupOffsetsHandlerTest {
@Test
public void testUnmappedHandleResponseWithMultipleGroups() {
Map<String, Errors> errorMap = new HashMap<>();
errorMap.put(groupZero, Errors.NOT_COORDINATOR);
errorMap.put(groupOne, Errors.COORDINATOR_NOT_AVAILABLE);
errorMap.put(groupTwo, Errors.NOT_COORDINATOR);
assertUnmappedWithMultipleGroups(handleWithErrorWithMultipleGroups(errorMap, batchedRequestMap));
var errorMap = Map.of(
group0, Errors.NOT_COORDINATOR,
group1, Errors.COORDINATOR_NOT_AVAILABLE,
group2, Errors.NOT_COORDINATOR
);
assertUnmappedWithMultipleGroups(handleWithErrorWithMultipleGroups(errorMap, multiGroupSpecs));
}
@Test
@ -221,7 +267,7 @@ public class ListConsumerGroupOffsetsHandlerTest {
@Test
public void testRetriableHandleResponseWithMultipleGroups() {
Map<String, Errors> errorMap = errorMap(groups, Errors.COORDINATOR_LOAD_IN_PROGRESS);
assertRetriable(handleWithErrorWithMultipleGroups(errorMap, batchedRequestMap));
assertRetriable(handleWithErrorWithMultipleGroups(errorMap, multiGroupSpecs));
}
@Test
@ -233,117 +279,226 @@ public class ListConsumerGroupOffsetsHandlerTest {
@Test
public void testFailedHandleResponseWithMultipleGroups() {
Map<String, Errors> errorMap = new HashMap<>();
errorMap.put(groupZero, Errors.GROUP_AUTHORIZATION_FAILED);
errorMap.put(groupOne, Errors.GROUP_ID_NOT_FOUND);
errorMap.put(groupTwo, Errors.INVALID_GROUP_ID);
Map<String, Class<? extends Throwable>> groupToExceptionMap = new HashMap<>();
groupToExceptionMap.put(groupZero, GroupAuthorizationException.class);
groupToExceptionMap.put(groupOne, GroupIdNotFoundException.class);
groupToExceptionMap.put(groupTwo, InvalidGroupIdException.class);
assertFailedForMultipleGroups(groupToExceptionMap,
handleWithErrorWithMultipleGroups(errorMap, batchedRequestMap));
var errorMap = Map.of(
group0, Errors.GROUP_AUTHORIZATION_FAILED,
group1, Errors.GROUP_ID_NOT_FOUND,
group2, Errors.INVALID_GROUP_ID
);
var groupToExceptionMap = Map.of(
group0, (Class<? extends Throwable>) GroupAuthorizationException.class,
group1, (Class<? extends Throwable>) GroupIdNotFoundException.class,
group2, (Class<? extends Throwable>) InvalidGroupIdException.class
);
assertFailedForMultipleGroups(
groupToExceptionMap,
handleWithErrorWithMultipleGroups(errorMap, multiGroupSpecs)
);
}
private OffsetFetchResponse buildResponse(Errors error) {
return new OffsetFetchResponse(
throttleMs,
Collections.singletonMap(groupZero, error),
Collections.singletonMap(groupZero, new HashMap<>()));
}
private OffsetFetchResponse buildResponseWithMultipleGroups(
Map<String, Errors> errorMap,
Map<String, Map<TopicPartition, PartitionData>> responseData
) {
return new OffsetFetchResponse(throttleMs, errorMap, responseData);
new OffsetFetchResponseData()
.setGroups(List.of(
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(group0)
.setErrorCode(error.code())
)),
ApiKeys.OFFSET_FETCH.latestVersion()
);
}
private AdminApiHandler.ApiResult<CoordinatorKey, Map<TopicPartition, OffsetAndMetadata>> handleWithErrorWithMultipleGroups(
Map<String, Errors> errorMap,
Map<String, ListConsumerGroupOffsetsSpec> groupSpecs
) {
ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(groupSpecs, false, logContext);
Map<String, Map<TopicPartition, PartitionData>> responseData = new HashMap<>();
for (String group : errorMap.keySet()) {
responseData.put(group, new HashMap<>());
}
OffsetFetchResponse response = buildResponseWithMultipleGroups(errorMap, responseData);
var handler = new ListConsumerGroupOffsetsHandler(
groupSpecs,
false,
logContext
);
var response = new OffsetFetchResponse(
new OffsetFetchResponseData()
.setGroups(errorMap.entrySet().stream().map(entry ->
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(entry.getKey())
.setErrorCode(entry.getValue().code())
).collect(Collectors.toList())),
ApiKeys.OFFSET_FETCH.latestVersion()
);
return handler.handleResponse(new Node(1, "host", 1234),
errorMap.keySet()
.stream()
.map(CoordinatorKey::byGroupId)
.collect(Collectors.toSet()),
response);
errorMap.keySet()
.stream()
.map(CoordinatorKey::byGroupId)
.collect(Collectors.toSet()),
response
);
}
private OffsetFetchResponse buildResponseWithPartitionError(Errors error) {
Map<TopicPartition, PartitionData> responseData = new HashMap<>();
responseData.put(t0p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", Errors.NONE));
responseData.put(t0p1, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error));
return new OffsetFetchResponse(Errors.NONE, responseData);
return new OffsetFetchResponse(
new OffsetFetchResponseData()
.setGroups(List.of(
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(group0)
.setTopics(List.of(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(t0p0.topic())
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(t0p0.partition())
.setCommittedOffset(10),
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(t0p1.partition())
.setCommittedOffset(10)
.setErrorCode(error.code())
))
))
)),
ApiKeys.OFFSET_FETCH.latestVersion()
);
}
private OffsetFetchResponse buildResponseWithPartitionErrorWithMultipleGroups(Errors error) {
Map<TopicPartition, PartitionData> responseDataZero = new HashMap<>();
responseDataZero.put(t0p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", Errors.NONE));
var data = new OffsetFetchResponseData()
.setGroups(List.of(
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(group0)
.setTopics(List.of(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(t0p0.topic())
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(t0p0.partition())
.setCommittedOffset(10)
.setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
.setMetadata(OffsetFetchResponse.NO_METADATA)
.setErrorCode(Errors.NONE.code())
))
)),
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(group1)
.setTopics(List.of(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(t0p0.topic())
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(t0p0.partition())
.setCommittedOffset(10)
.setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
.setMetadata(OffsetFetchResponse.NO_METADATA)
.setErrorCode(error.code())
)),
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(t1p0.topic())
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(t1p0.partition())
.setCommittedOffset(10)
.setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
.setMetadata(OffsetFetchResponse.NO_METADATA)
.setErrorCode(error.code()),
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(t1p1.partition())
.setCommittedOffset(10)
.setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
.setMetadata(OffsetFetchResponse.NO_METADATA)
.setErrorCode(Errors.NONE.code())
))
)),
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(group2)
.setTopics(List.of(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(t0p0.topic())
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(t0p0.partition())
.setCommittedOffset(10)
.setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
.setMetadata(OffsetFetchResponse.NO_METADATA)
.setErrorCode(error.code())
)),
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(t1p0.topic())
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(t1p0.partition())
.setCommittedOffset(10)
.setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
.setMetadata(OffsetFetchResponse.NO_METADATA)
.setErrorCode(error.code()),
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(t1p1.partition())
.setCommittedOffset(10)
.setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
.setMetadata(OffsetFetchResponse.NO_METADATA)
.setErrorCode(error.code())
)),
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(t2p0.topic())
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(t2p0.partition())
.setCommittedOffset(10)
.setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
.setMetadata(OffsetFetchResponse.NO_METADATA)
.setErrorCode(error.code()),
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(t2p1.partition())
.setCommittedOffset(10)
.setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
.setMetadata(OffsetFetchResponse.NO_METADATA)
.setErrorCode(error.code()),
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(t2p2.partition())
.setCommittedOffset(10)
.setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
.setMetadata(OffsetFetchResponse.NO_METADATA)
.setErrorCode(Errors.NONE.code())
))
))
));
Map<TopicPartition, PartitionData> responseDataOne = new HashMap<>();
responseDataOne.put(t0p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error));
responseDataOne.put(t1p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error));
responseDataOne.put(t1p1, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", Errors.NONE));
Map<TopicPartition, PartitionData> responseDataTwo = new HashMap<>();
responseDataTwo.put(t0p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error));
responseDataTwo.put(t1p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error));
responseDataTwo.put(t1p1, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error));
responseDataTwo.put(t2p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error));
responseDataTwo.put(t2p1, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error));
responseDataTwo.put(t2p2, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", Errors.NONE));
Map<String, Map<TopicPartition, PartitionData>> responseData =
new HashMap<>() {{
put(groupZero, responseDataZero);
put(groupOne, responseDataOne);
put(groupTwo, responseDataTwo);
}};
Map<String, Errors> errorMap = errorMap(groups, Errors.NONE);
return new OffsetFetchResponse(0, errorMap, responseData);
return new OffsetFetchResponse(data, ApiKeys.OFFSET_FETCH.latestVersion());
}
private AdminApiHandler.ApiResult<CoordinatorKey, Map<TopicPartition, OffsetAndMetadata>> handleWithPartitionError(
Errors error
) {
ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(singleRequestMap,
false, logContext);
ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(
singleGroupSpec,
false,
logContext
);
OffsetFetchResponse response = buildResponseWithPartitionError(error);
return handler.handleResponse(new Node(1, "host", 1234),
singleton(CoordinatorKey.byGroupId(groupZero)), response);
singleton(CoordinatorKey.byGroupId(group0)), response);
}
private AdminApiHandler.ApiResult<CoordinatorKey, Map<TopicPartition, OffsetAndMetadata>> handleWithPartitionErrorMultipleGroups(
Errors error
) {
ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(
batchedRequestMap, false, logContext);
multiGroupSpecs,
false,
logContext
);
OffsetFetchResponse response = buildResponseWithPartitionErrorWithMultipleGroups(error);
return handler.handleResponse(
new Node(1, "host", 1234),
coordinatorKeys(groupZero, groupOne, groupTwo),
response);
coordinatorKeys(group0, group1, group2),
response
);
}
private AdminApiHandler.ApiResult<CoordinatorKey, Map<TopicPartition, OffsetAndMetadata>> handleWithError(
Errors error
) {
ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(
singleRequestMap, false, logContext);
singleGroupSpec, false, logContext);
OffsetFetchResponse response = buildResponse(error);
return handler.handleResponse(new Node(1, "host", 1234),
singleton(CoordinatorKey.byGroupId(groupZero)),
singleton(CoordinatorKey.byGroupId(group0)),
response);
}
@ -352,7 +507,7 @@ public class ListConsumerGroupOffsetsHandlerTest {
) {
assertEquals(emptySet(), result.completedKeys.keySet());
assertEquals(emptySet(), result.failedKeys.keySet());
assertEquals(singletonList(CoordinatorKey.byGroupId(groupZero)), result.unmappedKeys);
assertEquals(singletonList(CoordinatorKey.byGroupId(group0)), result.unmappedKeys);
}
private void assertUnmappedWithMultipleGroups(
@ -360,7 +515,7 @@ public class ListConsumerGroupOffsetsHandlerTest {
) {
assertEquals(emptySet(), result.completedKeys.keySet());
assertEquals(emptySet(), result.failedKeys.keySet());
assertEquals(coordinatorKeys(groupZero, groupOne, groupTwo), new HashSet<>(result.unmappedKeys));
assertEquals(coordinatorKeys(group0, group1, group2), new HashSet<>(result.unmappedKeys));
}
private void assertRetriable(
@ -375,7 +530,7 @@ public class ListConsumerGroupOffsetsHandlerTest {
AdminApiHandler.ApiResult<CoordinatorKey, Map<TopicPartition, OffsetAndMetadata>> result,
Map<TopicPartition, OffsetAndMetadata> expected
) {
CoordinatorKey key = CoordinatorKey.byGroupId(groupZero);
CoordinatorKey key = CoordinatorKey.byGroupId(group0);
assertEquals(emptySet(), result.failedKeys.keySet());
assertEquals(emptyList(), result.unmappedKeys);
assertEquals(singleton(key), result.completedKeys.keySet());
@ -399,7 +554,7 @@ public class ListConsumerGroupOffsetsHandlerTest {
Class<? extends Throwable> expectedExceptionType,
AdminApiHandler.ApiResult<CoordinatorKey, Map<TopicPartition, OffsetAndMetadata>> result
) {
CoordinatorKey key = CoordinatorKey.byGroupId(groupZero);
CoordinatorKey key = CoordinatorKey.byGroupId(group0);
assertEquals(emptySet(), result.completedKeys.keySet());
assertEquals(emptyList(), result.unmappedKeys);
assertEquals(singleton(key), result.failedKeys.keySet());

View File

@ -60,6 +60,7 @@ import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartiti
import org.apache.kafka.common.message.ListOffsetsResponseData;
import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
import org.apache.kafka.common.message.OffsetFetchResponseData;
import org.apache.kafka.common.message.SyncGroupResponseData;
import org.apache.kafka.common.metrics.JmxReporter;
import org.apache.kafka.common.metrics.KafkaMetric;
@ -2870,16 +2871,26 @@ public class KafkaConsumerTest {
}
private OffsetFetchResponse offsetResponse(Map<TopicPartition, Long> offsets, Errors error) {
Map<TopicPartition, OffsetFetchResponse.PartitionData> partitionData = new HashMap<>();
for (Map.Entry<TopicPartition, Long> entry : offsets.entrySet()) {
partitionData.put(entry.getKey(), new OffsetFetchResponse.PartitionData(entry.getValue(),
Optional.empty(), "", error));
}
int throttleMs = 10;
var grouped = offsets.entrySet().stream().collect(Collectors.groupingBy(e -> e.getKey().topic()));
return new OffsetFetchResponse(
throttleMs,
Collections.singletonMap(groupId, Errors.NONE),
Collections.singletonMap(groupId, partitionData));
new OffsetFetchResponseData()
.setGroups(List.of(
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(groupId)
.setTopics(grouped.entrySet().stream().map(entry ->
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(entry.getKey())
.setPartitions(entry.getValue().stream().map(partition ->
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(partition.getKey().partition())
.setErrorCode(error.code())
.setCommittedOffset(partition.getValue())
).collect(Collectors.toList()))
).collect(Collectors.toList()))
)),
ApiKeys.OFFSET_FETCH.latestVersion()
);
}
private ListOffsetsResponse listOffsetsResponse(Map<TopicPartition, Long> offsets) {

View File

@ -36,6 +36,7 @@ import org.apache.kafka.common.errors.UnknownMemberIdException;
import org.apache.kafka.common.message.OffsetCommitRequestData;
import org.apache.kafka.common.message.OffsetCommitResponseData;
import org.apache.kafka.common.message.OffsetFetchRequestData;
import org.apache.kafka.common.message.OffsetFetchResponseData;
import org.apache.kafka.common.metrics.KafkaMetric;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.protocol.ApiKeys;
@ -752,10 +753,10 @@ public class CommitRequestManagerTest {
CommitRequestManager commitManager = create(false, 100);
when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode));
TopicPartition tp = new TopicPartition("topic1", 0);
long deadlineMs = time.milliseconds() + defaultApiTimeoutMs;
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> fetchResult =
commitManager.fetchOffsets(Collections.singleton(new TopicPartition("test", 0)),
deadlineMs);
commitManager.fetchOffsets(Collections.singleton(tp), deadlineMs);
// Send fetch request
NetworkClientDelegate.PollResult result = commitManager.poll(time.milliseconds());
@ -764,14 +765,21 @@ public class CommitRequestManagerTest {
assertFalse(fetchResult.isDone());
// Complete request with a response
TopicPartition tp = new TopicPartition("topic1", 0);
long expectedOffset = 100;
NetworkClientDelegate.UnsentRequest req = result.unsentRequests.get(0);
Map<TopicPartition, OffsetFetchResponse.PartitionData> topicPartitionData =
Collections.singletonMap(
tp,
new OffsetFetchResponse.PartitionData(expectedOffset, Optional.of(1), "", Errors.NONE));
req.handler().onComplete(buildOffsetFetchClientResponse(req, topicPartitionData, Errors.NONE, false));
OffsetFetchResponseData.OffsetFetchResponseGroup groupResponse = new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(DEFAULT_GROUP_ID)
.setTopics(List.of(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(tp.topic())
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(tp.partition())
.setCommittedOffset(expectedOffset)
.setCommittedLeaderEpoch(1)
))
));
req.handler().onComplete(buildOffsetFetchClientResponse(req, groupResponse, false));
// Validate request future completes with the response received
assertTrue(fetchResult.isDone());
@ -1400,15 +1408,43 @@ public class CommitRequestManagerTest {
assertEquals(1, res.unsentRequests.size());
// Setting 1 partition with error
HashMap<TopicPartition, OffsetFetchResponse.PartitionData> topicPartitionData = new HashMap<>();
topicPartitionData.put(tp1, new OffsetFetchResponse.PartitionData(100L, Optional.of(1), "metadata", error));
topicPartitionData.put(tp2, new OffsetFetchResponse.PartitionData(100L, Optional.of(1), "metadata", Errors.NONE));
topicPartitionData.put(tp3, new OffsetFetchResponse.PartitionData(100L, Optional.of(1), "metadata", error));
OffsetFetchResponseData.OffsetFetchResponseGroup groupResponse = new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(DEFAULT_GROUP_ID)
.setTopics(List.of(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(tp1.topic())
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(tp1.partition())
.setCommittedOffset(100L)
.setCommittedLeaderEpoch(1)
.setMetadata("metadata")
.setErrorCode(error.code())
)),
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(tp2.topic())
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(tp2.partition())
.setCommittedOffset(100L)
.setCommittedLeaderEpoch(1)
.setMetadata("metadata")
)),
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(tp3.topic())
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(tp3.partition())
.setCommittedOffset(100L)
.setCommittedLeaderEpoch(1)
.setMetadata("metadata")
.setErrorCode(error.code())
))
));
res.unsentRequests.get(0).handler().onComplete(buildOffsetFetchClientResponse(
res.unsentRequests.get(0),
topicPartitionData,
Errors.NONE,
groupResponse,
false));
if (isRetriable)
testRetriable(commitRequestManager, Collections.singletonList(future), error);
@ -1549,18 +1585,26 @@ public class CommitRequestManagerTest {
final NetworkClientDelegate.UnsentRequest request,
final Set<TopicPartition> topicPartitions,
final Errors error) {
HashMap<TopicPartition, OffsetFetchResponse.PartitionData> topicPartitionData = new HashMap<>();
topicPartitions.forEach(tp -> topicPartitionData.put(tp, new OffsetFetchResponse.PartitionData(
100L,
Optional.of(1),
"metadata",
Errors.NONE)));
return buildOffsetFetchClientResponse(request, topicPartitionData, error, false);
OffsetFetchResponseData.OffsetFetchResponseGroup group = new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(DEFAULT_GROUP_ID)
.setErrorCode(error.code())
.setTopics(topicPartitions.stream().collect(Collectors.groupingBy(TopicPartition::topic)).entrySet().stream().map(entry ->
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(entry.getKey())
.setPartitions(entry.getValue().stream().map(partition ->
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(partition.partition())
.setCommittedOffset(100L)
.setCommittedLeaderEpoch(1)
.setMetadata("metadata")
).collect(Collectors.toList()))
).collect(Collectors.toList()));
return buildOffsetFetchClientResponse(request, group, false);
}
private ClientResponse buildOffsetFetchClientResponseDisconnected(
final NetworkClientDelegate.UnsentRequest request) {
return buildOffsetFetchClientResponse(request, Collections.emptyMap(), Errors.NONE, true);
return buildOffsetFetchClientResponse(request, new OffsetFetchResponseData.OffsetFetchResponseGroup(), true);
}
private ClientResponse buildOffsetCommitClientResponse(final OffsetCommitResponse commitResponse) {
@ -1676,14 +1720,12 @@ public class CommitRequestManagerTest {
private ClientResponse buildOffsetFetchClientResponse(
final NetworkClientDelegate.UnsentRequest request,
final Map<TopicPartition, OffsetFetchResponse.PartitionData> topicPartitionData,
final Errors error,
final OffsetFetchResponseData.OffsetFetchResponseGroup groupResponse,
final boolean disconnected) {
AbstractRequest abstractRequest = request.requestBuilder().build();
assertInstanceOf(OffsetFetchRequest.class, abstractRequest);
OffsetFetchRequest offsetFetchRequest = (OffsetFetchRequest) abstractRequest;
OffsetFetchResponse response =
new OffsetFetchResponse(error, topicPartitionData);
OffsetFetchResponse response = new OffsetFetchResponse.Builder(groupResponse).build(ApiKeys.OFFSET_FETCH.latestVersion());
return new ClientResponse(
new RequestHeader(ApiKeys.OFFSET_FETCH, offsetFetchRequest.version(), "", 1),
request.handler(),

View File

@ -52,6 +52,7 @@ import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity;
import org.apache.kafka.common.message.LeaveGroupResponseData;
import org.apache.kafka.common.message.OffsetCommitRequestData;
import org.apache.kafka.common.message.OffsetCommitResponseData;
import org.apache.kafka.common.message.OffsetFetchResponseData;
import org.apache.kafka.common.message.SyncGroupResponseData;
import org.apache.kafka.common.metrics.KafkaMetric;
import org.apache.kafka.common.metrics.Metrics;
@ -73,7 +74,6 @@ import org.apache.kafka.common.requests.MetadataResponse.PartitionMetadata;
import org.apache.kafka.common.requests.OffsetCommitRequest;
import org.apache.kafka.common.requests.OffsetCommitResponse;
import org.apache.kafka.common.requests.OffsetFetchResponse;
import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData;
import org.apache.kafka.common.requests.RequestTestUtils;
import org.apache.kafka.common.requests.SyncGroupRequest;
import org.apache.kafka.common.requests.SyncGroupResponse;
@ -3111,10 +3111,19 @@ public abstract class ConsumerCoordinatorTest {
long offset = 500L;
String metadata = "blahblah";
Optional<Integer> leaderEpoch = Optional.of(15);
OffsetFetchResponse.PartitionData data = new OffsetFetchResponse.PartitionData(offset, leaderEpoch,
metadata, Errors.NONE);
client.prepareResponse(offsetFetchResponse(Errors.NONE, singletonMap(t1p, data)));
client.prepareResponse(offsetFetchResponse(Errors.NONE, List.of(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(t1p.topic())
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(t1p.partition())
.setCommittedOffset(offset)
.setCommittedLeaderEpoch(leaderEpoch.get())
.setMetadata(metadata)
))
)));
Map<TopicPartition, OffsetAndMetadata> fetchedOffsets = coordinator.fetchCommittedOffsets(singleton(t1p),
time.timer(Long.MAX_VALUE));
@ -3127,10 +3136,17 @@ public abstract class ConsumerCoordinatorTest {
client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
OffsetFetchResponse.PartitionData data = new OffsetFetchResponse.PartitionData(-1, Optional.empty(),
"", Errors.TOPIC_AUTHORIZATION_FAILED);
client.prepareResponse(offsetFetchResponse(Errors.NONE, List.of(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(t1p.topic())
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(t1p.partition())
.setCommittedOffset(-1)
.setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code())
))
)));
client.prepareResponse(offsetFetchResponse(Errors.NONE, singletonMap(t1p, data)));
TopicAuthorizationException exception = assertThrows(TopicAuthorizationException.class, () ->
coordinator.fetchCommittedOffsets(singleton(t1p), time.timer(Long.MAX_VALUE)));
@ -3143,7 +3159,7 @@ public abstract class ConsumerCoordinatorTest {
coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
subscriptions.assignFromUser(singleton(t1p));
client.prepareResponse(offsetFetchResponse(Errors.GROUP_AUTHORIZATION_FAILED, Collections.emptyMap()));
client.prepareResponse(offsetFetchResponse(Errors.GROUP_AUTHORIZATION_FAILED, List.of()));
try {
coordinator.initWithCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE));
fail("Expected group authorization error");
@ -3192,7 +3208,7 @@ public abstract class ConsumerCoordinatorTest {
coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
subscriptions.assignFromUser(singleton(t1p));
client.prepareResponse(offsetFetchResponse(error, Collections.emptyMap()));
client.prepareResponse(offsetFetchResponse(error, List.of()));
if (expectCoordinatorRelookup) {
client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
}
@ -3692,14 +3708,19 @@ public abstract class ConsumerCoordinatorTest {
long offset = 500L;
String metadata = "blahblah";
Optional<Integer> leaderEpoch = Optional.of(15);
OffsetFetchResponse.PartitionData data = new OffsetFetchResponse.PartitionData(offset, leaderEpoch,
metadata, Errors.NONE);
if (upperVersion < 8) {
client.prepareResponse(new OffsetFetchResponse(Errors.NONE, singletonMap(t1p, data)));
} else {
client.prepareResponse(offsetFetchResponse(Errors.NONE, singletonMap(t1p, data)));
}
client.prepareResponse(offsetFetchResponse(Errors.NONE, List.of(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(t1p.topic())
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(t1p.partition())
.setCommittedOffset(offset)
.setCommittedLeaderEpoch(leaderEpoch.get())
.setMetadata(metadata)
))
)));
if (expectThrows) {
assertThrows(UnsupportedVersionException.class,
() -> coordinator.fetchCommittedOffsets(singleton(t1p), time.timer(Long.MAX_VALUE)));
@ -3964,10 +3985,20 @@ public abstract class ConsumerCoordinatorTest {
return new OffsetCommitResponse(responseData);
}
private OffsetFetchResponse offsetFetchResponse(Errors error, Map<TopicPartition, PartitionData> responseData) {
return new OffsetFetchResponse(throttleMs,
singletonMap(groupId, error),
singletonMap(groupId, responseData));
private OffsetFetchResponse offsetFetchResponse(
Errors errors,
List<OffsetFetchResponseData.OffsetFetchResponseTopics> topics
) {
return new OffsetFetchResponse(
new OffsetFetchResponseData()
.setGroups(List.of(
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(groupId)
.setErrorCode(errors.code())
.setTopics(topics)
)),
ApiKeys.OFFSET_FETCH.latestVersion()
);
}
private OffsetFetchResponse offsetFetchResponse(TopicPartition tp, Errors partitionLevelError, String metadata, long offset) {
@ -3975,9 +4006,18 @@ public abstract class ConsumerCoordinatorTest {
}
private OffsetFetchResponse offsetFetchResponse(TopicPartition tp, Errors partitionLevelError, String metadata, long offset, Optional<Integer> epoch) {
OffsetFetchResponse.PartitionData data = new OffsetFetchResponse.PartitionData(offset,
epoch, metadata, partitionLevelError);
return offsetFetchResponse(Errors.NONE, singletonMap(tp, data));
return offsetFetchResponse(Errors.NONE, List.of(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(tp.topic())
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(tp.partition())
.setCommittedOffset(offset)
.setCommittedLeaderEpoch(epoch.orElse(-1))
.setMetadata(metadata)
.setErrorCode(partitionLevelError.code())
))
));
}
private OffsetCommitCallback callback(final AtomicBoolean success) {

View File

@ -16,426 +16,231 @@
*/
package org.apache.kafka.common.requests;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.message.OffsetFetchResponseData;
import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponseGroup;
import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponsePartition;
import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponsePartitions;
import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponseTopic;
import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponseTopics;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData;
import org.apache.kafka.common.utils.Utils;
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;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import java.util.List;
import static org.apache.kafka.common.requests.AbstractResponse.DEFAULT_THROTTLE_TIME;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.assertThrows;
public class OffsetFetchResponseTest {
private final int throttleTimeMs = 10;
private final int offset = 100;
private final String metadata = "metadata";
@ParameterizedTest
@ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_FETCH)
public void testBuilderWithSingleGroup(short version) {
var group = new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId("group")
.setTopics(List.of(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName("foo")
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(0)
.setCommittedOffset(10)
.setCommittedLeaderEpoch(5)
.setMetadata("metadata")
))
));
private final String groupOne = "group1";
private final String groupTwo = "group2";
private final String groupThree = "group3";
private final String topicOne = "topic1";
private final int partitionOne = 1;
private final Optional<Integer> leaderEpochOne = Optional.of(1);
private final String topicTwo = "topic2";
private final int partitionTwo = 2;
private final Optional<Integer> leaderEpochTwo = Optional.of(2);
private final String topicThree = "topic3";
private final int partitionThree = 3;
private final Optional<Integer> leaderEpochThree = Optional.of(3);
private Map<TopicPartition, PartitionData> partitionDataMap;
@BeforeEach
public void setUp() {
partitionDataMap = new HashMap<>();
partitionDataMap.put(new TopicPartition(topicOne, partitionOne), new PartitionData(
offset,
leaderEpochOne,
metadata,
Errors.TOPIC_AUTHORIZATION_FAILED
));
partitionDataMap.put(new TopicPartition(topicTwo, partitionTwo), new PartitionData(
offset,
leaderEpochTwo,
metadata,
Errors.UNKNOWN_TOPIC_OR_PARTITION
));
}
@Test
public void testConstructor() {
for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
if (version < 8) {
OffsetFetchResponse response = new OffsetFetchResponse(throttleTimeMs, Errors.NOT_COORDINATOR, partitionDataMap);
assertEquals(Errors.NOT_COORDINATOR, response.error());
assertEquals(3, response.errorCounts().size());
assertEquals(Utils.mkMap(Utils.mkEntry(Errors.NOT_COORDINATOR, 1),
Utils.mkEntry(Errors.TOPIC_AUTHORIZATION_FAILED, 1),
Utils.mkEntry(Errors.UNKNOWN_TOPIC_OR_PARTITION, 1)),
response.errorCounts());
assertEquals(throttleTimeMs, response.throttleTimeMs());
Map<TopicPartition, PartitionData> responseData = response.responseDataV0ToV7();
assertEquals(partitionDataMap, responseData);
responseData.forEach((tp, data) -> assertTrue(data.hasError()));
} else {
OffsetFetchResponse response = new OffsetFetchResponse(
throttleTimeMs,
Collections.singletonMap(groupOne, Errors.NOT_COORDINATOR),
Collections.singletonMap(groupOne, partitionDataMap));
assertEquals(Errors.NOT_COORDINATOR, response.groupLevelError(groupOne));
assertEquals(3, response.errorCounts().size());
assertEquals(Utils.mkMap(Utils.mkEntry(Errors.NOT_COORDINATOR, 1),
Utils.mkEntry(Errors.TOPIC_AUTHORIZATION_FAILED, 1),
Utils.mkEntry(Errors.UNKNOWN_TOPIC_OR_PARTITION, 1)),
response.errorCounts());
assertEquals(throttleTimeMs, response.throttleTimeMs());
Map<TopicPartition, PartitionData> responseData = response.partitionDataMap(groupOne);
assertEquals(partitionDataMap, responseData);
responseData.forEach((tp, data) -> assertTrue(data.hasError()));
}
if (version < 8) {
assertEquals(
new OffsetFetchResponseData()
.setTopics(List.of(
new OffsetFetchResponseData.OffsetFetchResponseTopic()
.setName("foo")
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartition()
.setPartitionIndex(0)
.setCommittedOffset(10)
.setCommittedLeaderEpoch(5)
.setMetadata("metadata")
))
)),
new OffsetFetchResponse.Builder(group).build(version).data()
);
} else {
assertEquals(
new OffsetFetchResponseData()
.setGroups(List.of(group)),
new OffsetFetchResponse.Builder(group).build(version).data()
);
}
}
@Test
public void testConstructorWithMultipleGroups() {
Map<String, Map<TopicPartition, PartitionData>> responseData = new HashMap<>();
Map<String, Errors> errorMap = new HashMap<>();
Map<TopicPartition, PartitionData> pd1 = new HashMap<>();
Map<TopicPartition, PartitionData> pd2 = new HashMap<>();
Map<TopicPartition, PartitionData> pd3 = new HashMap<>();
pd1.put(new TopicPartition(topicOne, partitionOne), new PartitionData(
offset,
leaderEpochOne,
metadata,
Errors.TOPIC_AUTHORIZATION_FAILED));
pd2.put(new TopicPartition(topicTwo, partitionTwo), new PartitionData(
offset,
leaderEpochTwo,
metadata,
Errors.UNKNOWN_TOPIC_OR_PARTITION));
pd3.put(new TopicPartition(topicThree, partitionThree), new PartitionData(
offset,
leaderEpochThree,
metadata,
Errors.NONE));
responseData.put(groupOne, pd1);
responseData.put(groupTwo, pd2);
responseData.put(groupThree, pd3);
errorMap.put(groupOne, Errors.NOT_COORDINATOR);
errorMap.put(groupTwo, Errors.COORDINATOR_LOAD_IN_PROGRESS);
errorMap.put(groupThree, Errors.NONE);
for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
if (version >= 8) {
OffsetFetchResponse response = new OffsetFetchResponse(
throttleTimeMs, errorMap, responseData);
assertEquals(Errors.NOT_COORDINATOR, response.groupLevelError(groupOne));
assertEquals(Errors.COORDINATOR_LOAD_IN_PROGRESS, response.groupLevelError(groupTwo));
assertEquals(Errors.NONE, response.groupLevelError(groupThree));
assertTrue(response.groupHasError(groupOne));
assertTrue(response.groupHasError(groupTwo));
assertFalse(response.groupHasError(groupThree));
assertEquals(5, response.errorCounts().size());
assertEquals(Utils.mkMap(Utils.mkEntry(Errors.NOT_COORDINATOR, 1),
Utils.mkEntry(Errors.TOPIC_AUTHORIZATION_FAILED, 1),
Utils.mkEntry(Errors.UNKNOWN_TOPIC_OR_PARTITION, 1),
Utils.mkEntry(Errors.COORDINATOR_LOAD_IN_PROGRESS, 1),
Utils.mkEntry(Errors.NONE, 2)),
response.errorCounts());
assertEquals(throttleTimeMs, response.throttleTimeMs());
Map<TopicPartition, PartitionData> responseData1 = response.partitionDataMap(groupOne);
assertEquals(pd1, responseData1);
responseData1.forEach((tp, data) -> assertTrue(data.hasError()));
Map<TopicPartition, PartitionData> responseData2 = response.partitionDataMap(groupTwo);
assertEquals(pd2, responseData2);
responseData2.forEach((tp, data) -> assertTrue(data.hasError()));
Map<TopicPartition, PartitionData> responseData3 = response.partitionDataMap(groupThree);
assertEquals(pd3, responseData3);
responseData3.forEach((tp, data) -> assertFalse(data.hasError()));
}
}
}
/**
* Test behavior changes over the versions. Refer to resources.common.messages.OffsetFetchResponse.json
*/
@Test
public void testStructBuild() {
for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
if (version < 8) {
partitionDataMap.put(new TopicPartition(topicTwo, partitionTwo), new PartitionData(
offset,
leaderEpochTwo,
metadata,
Errors.GROUP_AUTHORIZATION_FAILED
));
OffsetFetchResponse latestResponse = new OffsetFetchResponse(throttleTimeMs, Errors.NONE, partitionDataMap);
OffsetFetchResponseData data = new OffsetFetchResponseData(
latestResponse.serialize(version), version);
OffsetFetchResponse oldResponse = new OffsetFetchResponse(data, version);
if (version <= 1) {
assertEquals(Errors.NONE.code(), data.errorCode());
// Partition level error populated in older versions.
assertEquals(Errors.GROUP_AUTHORIZATION_FAILED, oldResponse.error());
assertEquals(Utils.mkMap(Utils.mkEntry(Errors.GROUP_AUTHORIZATION_FAILED, 2),
Utils.mkEntry(Errors.TOPIC_AUTHORIZATION_FAILED, 1)),
oldResponse.errorCounts());
} else {
assertEquals(Errors.NONE.code(), data.errorCode());
assertEquals(Errors.NONE, oldResponse.error());
assertEquals(Utils.mkMap(
Utils.mkEntry(Errors.NONE, 1),
Utils.mkEntry(Errors.GROUP_AUTHORIZATION_FAILED, 1),
Utils.mkEntry(Errors.TOPIC_AUTHORIZATION_FAILED, 1)),
oldResponse.errorCounts());
}
if (version <= 2) {
assertEquals(DEFAULT_THROTTLE_TIME, oldResponse.throttleTimeMs());
} else {
assertEquals(throttleTimeMs, oldResponse.throttleTimeMs());
}
Map<TopicPartition, PartitionData> expectedDataMap = new HashMap<>();
for (Map.Entry<TopicPartition, PartitionData> entry : partitionDataMap.entrySet()) {
PartitionData partitionData = entry.getValue();
expectedDataMap.put(entry.getKey(), new PartitionData(
partitionData.offset,
version <= 4 ? Optional.empty() : partitionData.leaderEpoch,
partitionData.metadata,
partitionData.error
));
}
Map<TopicPartition, PartitionData> responseData = oldResponse.responseDataV0ToV7();
assertEquals(expectedDataMap, responseData);
responseData.forEach((tp, rdata) -> assertTrue(rdata.hasError()));
} else {
partitionDataMap.put(new TopicPartition(topicTwo, partitionTwo), new PartitionData(
offset,
leaderEpochTwo,
metadata,
Errors.GROUP_AUTHORIZATION_FAILED));
OffsetFetchResponse latestResponse = new OffsetFetchResponse(
throttleTimeMs,
Collections.singletonMap(groupOne, Errors.NONE),
Collections.singletonMap(groupOne, partitionDataMap));
OffsetFetchResponseData data = new OffsetFetchResponseData(
latestResponse.serialize(version), version);
OffsetFetchResponse oldResponse = new OffsetFetchResponse(data, version);
assertEquals(Errors.NONE.code(), data.groups().get(0).errorCode());
assertEquals(Errors.NONE, oldResponse.groupLevelError(groupOne));
assertEquals(Utils.mkMap(
Utils.mkEntry(Errors.NONE, 1),
Utils.mkEntry(Errors.GROUP_AUTHORIZATION_FAILED, 1),
Utils.mkEntry(Errors.TOPIC_AUTHORIZATION_FAILED, 1)),
oldResponse.errorCounts());
assertEquals(throttleTimeMs, oldResponse.throttleTimeMs());
Map<TopicPartition, PartitionData> expectedDataMap = new HashMap<>();
for (Map.Entry<TopicPartition, PartitionData> entry : partitionDataMap.entrySet()) {
PartitionData partitionData = entry.getValue();
expectedDataMap.put(entry.getKey(), new PartitionData(
partitionData.offset,
partitionData.leaderEpoch,
partitionData.metadata,
partitionData.error
));
}
Map<TopicPartition, PartitionData> responseData = oldResponse.partitionDataMap(groupOne);
assertEquals(expectedDataMap, responseData);
responseData.forEach((tp, rdata) -> assertTrue(rdata.hasError()));
}
}
}
@Test
public void testShouldThrottle() {
for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
if (version < 8) {
OffsetFetchResponse response = new OffsetFetchResponse(throttleTimeMs, Errors.NONE, partitionDataMap);
if (version >= 4) {
assertTrue(response.shouldClientThrottle(version));
} else {
assertFalse(response.shouldClientThrottle(version));
}
} else {
OffsetFetchResponse response = new OffsetFetchResponse(
throttleTimeMs,
Collections.singletonMap(groupOne, Errors.NOT_COORDINATOR),
Collections.singletonMap(groupOne, partitionDataMap));
assertTrue(response.shouldClientThrottle(version));
}
}
}
@Test
public void testNullableMetadataV0ToV7() {
PartitionData pd = new PartitionData(
offset,
leaderEpochOne,
null,
Errors.UNKNOWN_TOPIC_OR_PARTITION);
// test PartitionData.equals with null metadata
assertEquals(pd, pd);
partitionDataMap.clear();
partitionDataMap.put(new TopicPartition(topicOne, partitionOne), pd);
OffsetFetchResponse response = new OffsetFetchResponse(throttleTimeMs, Errors.GROUP_AUTHORIZATION_FAILED, partitionDataMap);
OffsetFetchResponseData expectedData =
new OffsetFetchResponseData()
.setErrorCode(Errors.GROUP_AUTHORIZATION_FAILED.code())
.setThrottleTimeMs(throttleTimeMs)
.setTopics(Collections.singletonList(
new OffsetFetchResponseTopic()
.setName(topicOne)
.setPartitions(Collections.singletonList(
new OffsetFetchResponsePartition()
.setPartitionIndex(partitionOne)
.setCommittedOffset(offset)
.setCommittedLeaderEpoch(leaderEpochOne.orElse(-1))
.setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code())
.setMetadata(null))
@ParameterizedTest
@ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_FETCH)
public void testBuilderWithMultipleGroups(short version) {
var groups = List.of(
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId("group1")
.setTopics(List.of(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName("foo")
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(0)
.setCommittedOffset(10)
.setCommittedLeaderEpoch(5)
.setMetadata("metadata")
))
);
assertEquals(expectedData, response.data());
}
@Test
public void testNullableMetadataV8AndAbove() {
PartitionData pd = new PartitionData(
offset,
leaderEpochOne,
null,
Errors.UNKNOWN_TOPIC_OR_PARTITION);
// test PartitionData.equals with null metadata
assertEquals(pd, pd);
partitionDataMap.clear();
partitionDataMap.put(new TopicPartition(topicOne, partitionOne), pd);
OffsetFetchResponse response = new OffsetFetchResponse(
throttleTimeMs,
Collections.singletonMap(groupOne, Errors.GROUP_AUTHORIZATION_FAILED),
Collections.singletonMap(groupOne, partitionDataMap));
OffsetFetchResponseData expectedData =
new OffsetFetchResponseData()
.setGroups(Collections.singletonList(
new OffsetFetchResponseGroup()
.setGroupId(groupOne)
.setTopics(Collections.singletonList(
new OffsetFetchResponseTopics()
.setName(topicOne)
.setPartitions(Collections.singletonList(
new OffsetFetchResponsePartitions()
.setPartitionIndex(partitionOne)
.setCommittedOffset(offset)
.setCommittedLeaderEpoch(leaderEpochOne.orElse(-1))
.setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code())
.setMetadata(null)))))
.setErrorCode(Errors.GROUP_AUTHORIZATION_FAILED.code())))
.setThrottleTimeMs(throttleTimeMs);
assertEquals(expectedData, response.data());
}
@Test
public void testUseDefaultLeaderEpochV0ToV7() {
final Optional<Integer> emptyLeaderEpoch = Optional.empty();
partitionDataMap.clear();
partitionDataMap.put(new TopicPartition(topicOne, partitionOne),
new PartitionData(
offset,
emptyLeaderEpoch,
metadata,
Errors.UNKNOWN_TOPIC_OR_PARTITION)
)),
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId("group2")
.setTopics(List.of(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName("bar")
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(0)
.setCommittedOffset(10)
.setCommittedLeaderEpoch(5)
.setMetadata("metadata")
))
))
);
OffsetFetchResponse response = new OffsetFetchResponse(throttleTimeMs, Errors.NOT_COORDINATOR, partitionDataMap);
OffsetFetchResponseData expectedData =
new OffsetFetchResponseData()
.setErrorCode(Errors.NOT_COORDINATOR.code())
.setThrottleTimeMs(throttleTimeMs)
.setTopics(Collections.singletonList(
new OffsetFetchResponseTopic()
.setName(topicOne)
.setPartitions(Collections.singletonList(
new OffsetFetchResponsePartition()
.setPartitionIndex(partitionOne)
.setCommittedOffset(offset)
.setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
.setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code())
.setMetadata(metadata))
))
);
assertEquals(expectedData, response.data());
if (version < 8) {
assertThrows(UnsupportedVersionException.class,
() -> new OffsetFetchResponse.Builder(groups).build(version));
} else {
assertEquals(
new OffsetFetchResponseData()
.setGroups(groups),
new OffsetFetchResponse.Builder(groups).build(version).data()
);
}
}
@Test
public void testUseDefaultLeaderEpochV8() {
final Optional<Integer> emptyLeaderEpoch = Optional.empty();
partitionDataMap.clear();
@ParameterizedTest
@ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_FETCH)
public void testGroupWithSingleGroup(short version) {
var data = new OffsetFetchResponseData();
partitionDataMap.put(new TopicPartition(topicOne, partitionOne),
new PartitionData(
offset,
emptyLeaderEpoch,
metadata,
Errors.UNKNOWN_TOPIC_OR_PARTITION)
if (version < 8) {
data.setTopics(List.of(
new OffsetFetchResponseData.OffsetFetchResponseTopic()
.setName("foo")
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartition()
.setPartitionIndex(0)
.setCommittedOffset(10)
.setCommittedLeaderEpoch(5)
.setMetadata("metadata")
))
));
} else {
data.setGroups(List.of(
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId("foo")
.setTopics(List.of(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName("foo")
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(0)
.setCommittedOffset(10)
.setCommittedLeaderEpoch(5)
.setMetadata("metadata")
))
))
));
}
assertEquals(
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId("foo")
.setTopics(List.of(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName("foo")
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(0)
.setCommittedOffset(10)
.setCommittedLeaderEpoch(5)
.setMetadata("metadata")
))
)),
new OffsetFetchResponse(data, version).group("foo")
);
}
@ParameterizedTest
@ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_FETCH, fromVersion = 8)
public void testGroupWithMultipleGroups(short version) {
var groups = List.of(
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId("group1")
.setTopics(List.of(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName("foo")
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(0)
.setCommittedOffset(10)
.setCommittedLeaderEpoch(5)
.setMetadata("metadata")
))
)),
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId("group2")
.setTopics(List.of(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName("bar")
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(0)
.setCommittedOffset(10)
.setCommittedLeaderEpoch(5)
.setMetadata("metadata")
))
))
);
var response = new OffsetFetchResponse(
new OffsetFetchResponseData().setGroups(groups),
version
);
groups.forEach(group ->
assertEquals(group, response.group(group.groupId()))
);
}
@ParameterizedTest
@ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_FETCH)
public void testGroupWithSingleGroupWithTopLevelError(short version) {
var data = new OffsetFetchResponseData();
if (version < 2) {
data.setTopics(List.of(
new OffsetFetchResponseData.OffsetFetchResponseTopic()
.setName("foo")
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartition()
.setPartitionIndex(0)
.setErrorCode(Errors.INVALID_GROUP_ID.code())
))
));
} else if (version < 8) {
data.setErrorCode(Errors.INVALID_GROUP_ID.code());
} else {
data.setGroups(List.of(
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId("foo")
.setErrorCode(Errors.INVALID_GROUP_ID.code())
));
}
assertEquals(
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId("foo")
.setErrorCode(Errors.INVALID_GROUP_ID.code()),
new OffsetFetchResponse(data, version).group("foo")
);
OffsetFetchResponse response = new OffsetFetchResponse(
throttleTimeMs,
Collections.singletonMap(groupOne, Errors.NOT_COORDINATOR),
Collections.singletonMap(groupOne, partitionDataMap));
OffsetFetchResponseData expectedData =
new OffsetFetchResponseData()
.setGroups(Collections.singletonList(
new OffsetFetchResponseGroup()
.setGroupId(groupOne)
.setTopics(Collections.singletonList(
new OffsetFetchResponseTopics()
.setName(topicOne)
.setPartitions(Collections.singletonList(
new OffsetFetchResponsePartitions()
.setPartitionIndex(partitionOne)
.setCommittedOffset(offset)
.setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
.setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code())
.setMetadata(metadata)))))
.setErrorCode(Errors.NOT_COORDINATOR.code())))
.setThrottleTimeMs(throttleTimeMs);
assertEquals(expectedData, response.data());
}
}

View File

@ -200,6 +200,7 @@ import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResp
import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopic;
import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection;
import org.apache.kafka.common.message.OffsetFetchRequestData;
import org.apache.kafka.common.message.OffsetFetchResponseData;
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition;
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderTopic;
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderTopicCollection;
@ -2483,17 +2484,23 @@ public class RequestResponseTest {
}
private OffsetFetchResponse createOffsetFetchResponse(short version) {
Map<TopicPartition, OffsetFetchResponse.PartitionData> responseData = new HashMap<>();
responseData.put(new TopicPartition("test", 0), new OffsetFetchResponse.PartitionData(
100L, Optional.empty(), "", Errors.NONE));
responseData.put(new TopicPartition("test", 1), new OffsetFetchResponse.PartitionData(
100L, Optional.of(10), null, Errors.NONE));
if (version < 8) {
return new OffsetFetchResponse(Errors.NONE, responseData);
}
int throttleMs = 10;
return new OffsetFetchResponse(throttleMs, Collections.singletonMap("group1", Errors.NONE),
Collections.singletonMap("group1", responseData));
var group = new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId("group1")
.setTopics(List.of(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName("test")
.setPartitions(List.of(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(0)
.setCommittedOffset(100),
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(1)
.setCommittedOffset(100)
.setCommittedLeaderEpoch(10)
.setMetadata(null)
))
));
return new OffsetFetchResponse.Builder(group).build(version);
}
private ProduceRequest createProduceRequest(short version) {

View File

@ -1035,7 +1035,7 @@ class KafkaApis(val requestChannel: RequestChannel,
CompletableFuture.allOf(futures.toArray: _*).handle[Unit] { (_, _) =>
val groupResponses = new ArrayBuffer[OffsetFetchResponseData.OffsetFetchResponseGroup](futures.size)
futures.foreach(future => groupResponses += future.get())
requestHelper.sendMaybeThrottle(request, new OffsetFetchResponse(groupResponses.asJava, request.context.apiVersion))
requestHelper.sendMaybeThrottle(request, new OffsetFetchResponse.Builder(groupResponses.asJava).build(request.context.apiVersion))
}
}

View File

@ -17,7 +17,7 @@ import java.time.Duration
import java.util
import java.util.concurrent.{ExecutionException, Semaphore}
import java.util.regex.Pattern
import java.util.{Collections, Optional, Properties}
import java.util.{Collections, Comparator, Optional, Properties}
import kafka.utils.{TestInfoUtils, TestUtils}
import kafka.utils.TestUtils.waitUntilTrue
import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, ListGroupsOptions, NewTopic}
@ -37,11 +37,10 @@ import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProt
import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity
import org.apache.kafka.common.message.ListOffsetsRequestData.{ListOffsetsPartition, ListOffsetsTopic}
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.{OffsetForLeaderPartition, OffsetForLeaderTopic, OffsetForLeaderTopicCollection}
import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AlterPartitionReassignmentsRequestData, AlterReplicaLogDirsRequestData, ConsumerGroupDescribeRequestData, ConsumerGroupHeartbeatRequestData, ConsumerGroupHeartbeatResponseData, CreateAclsRequestData, CreatePartitionsRequestData, CreateTopicsRequestData, DeleteAclsRequestData, DeleteGroupsRequestData, DeleteRecordsRequestData, DeleteShareGroupOffsetsRequestData, DeleteShareGroupStateRequestData, DeleteTopicsRequestData, DescribeClusterRequestData, DescribeConfigsRequestData, DescribeGroupsRequestData, DescribeLogDirsRequestData, DescribeProducersRequestData, DescribeShareGroupOffsetsRequestData, DescribeTransactionsRequestData, FetchResponseData, FindCoordinatorRequestData, HeartbeatRequestData, IncrementalAlterConfigsRequestData, InitializeShareGroupStateRequestData, JoinGroupRequestData, ListPartitionReassignmentsRequestData, ListTransactionsRequestData, MetadataRequestData, OffsetCommitRequestData, OffsetFetchRequestData, ProduceRequestData, ReadShareGroupStateRequestData, ReadShareGroupStateSummaryRequestData, ShareAcknowledgeRequestData, ShareFetchRequestData, ShareGroupDescribeRequestData, ShareGroupHeartbeatRequestData, SyncGroupRequestData, WriteShareGroupStateRequestData, WriteTxnMarkersRequestData}
import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AlterPartitionReassignmentsRequestData, AlterReplicaLogDirsRequestData, ConsumerGroupDescribeRequestData, ConsumerGroupHeartbeatRequestData, ConsumerGroupHeartbeatResponseData, CreateAclsRequestData, CreatePartitionsRequestData, CreateTopicsRequestData, DeleteAclsRequestData, DeleteGroupsRequestData, DeleteRecordsRequestData, DeleteShareGroupOffsetsRequestData, DeleteShareGroupStateRequestData, DeleteTopicsRequestData, DescribeClusterRequestData, DescribeConfigsRequestData, DescribeGroupsRequestData, DescribeLogDirsRequestData, DescribeProducersRequestData, DescribeShareGroupOffsetsRequestData, DescribeTransactionsRequestData, FetchResponseData, FindCoordinatorRequestData, HeartbeatRequestData, IncrementalAlterConfigsRequestData, InitializeShareGroupStateRequestData, JoinGroupRequestData, ListPartitionReassignmentsRequestData, ListTransactionsRequestData, MetadataRequestData, OffsetCommitRequestData, OffsetFetchRequestData, OffsetFetchResponseData, ProduceRequestData, ReadShareGroupStateRequestData, ReadShareGroupStateSummaryRequestData, ShareAcknowledgeRequestData, ShareFetchRequestData, ShareGroupDescribeRequestData, ShareGroupHeartbeatRequestData, SyncGroupRequestData, WriteShareGroupStateRequestData, WriteTxnMarkersRequestData}
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, SimpleRecord}
import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData
import org.apache.kafka.common.requests._
import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED}
import org.apache.kafka.common.resource.ResourceType._
@ -123,7 +122,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
}),
ApiKeys.OFFSET_COMMIT -> ((resp: requests.OffsetCommitResponse) => Errors.forCode(
resp.data.topics().get(0).partitions().get(0).errorCode)),
ApiKeys.OFFSET_FETCH -> ((resp: requests.OffsetFetchResponse) => resp.groupLevelError(group)),
ApiKeys.OFFSET_FETCH -> ((resp: requests.OffsetFetchResponse) => Errors.forCode(resp.group(group).errorCode())),
ApiKeys.FIND_COORDINATOR -> ((resp: FindCoordinatorResponse) => {
Errors.forCode(resp.data.coordinators.asScala.find(g => group == g.key).head.errorCode)
}),
@ -1609,15 +1608,20 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
// without describe permission on the topic, we shouldn't be able to fetch offsets
val offsetFetchRequest = createOffsetFetchRequestAllPartitions
var offsetFetchResponse = connectAndReceive[OffsetFetchResponse](offsetFetchRequest)
assertEquals(Errors.NONE, offsetFetchResponse.groupLevelError(group))
assertTrue(offsetFetchResponse.partitionDataMap(group).isEmpty)
assertEquals(Errors.NONE, Errors.forCode(offsetFetchResponse.group(group).errorCode()))
assertTrue(offsetFetchResponse.group(group).topics.isEmpty)
// now add describe permission on the topic and verify that the offset can be fetched
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResource)
offsetFetchResponse = connectAndReceive[OffsetFetchResponse](offsetFetchRequest)
assertEquals(Errors.NONE, offsetFetchResponse.groupLevelError(group))
assertTrue(offsetFetchResponse.partitionDataMap(group).containsKey(tp))
assertEquals(offset, offsetFetchResponse.partitionDataMap(group).get(tp).offset)
assertEquals(Errors.NONE, Errors.forCode(offsetFetchResponse.group(group).errorCode()))
assertEquals(
offset,
offsetFetchResponse.group(group).topics.asScala
.find(_.name == tp.topic)
.flatMap(_.partitions.asScala.find(_.partitionIndex == tp.partition).map(_.committedOffset))
.getOrElse(-1L)
)
}
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames)
@ -1652,21 +1656,33 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
createTopicWithBrokerPrincipal(topics(0))
createTopicWithBrokerPrincipal(topics(1), numPartitions = 2)
createTopicWithBrokerPrincipal(topics(2), numPartitions = 3)
groupResources.foreach(r => {
groupResources.foreach { r =>
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), r)
})
topicResources.foreach(t => {
}
topicResources.foreach { t =>
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), t)
})
}
val offset = 15L
val leaderEpoch: Optional[Integer] = Optional.of(1)
val metadata = "metadata"
def assertResponse(
expected: OffsetFetchResponseData.OffsetFetchResponseGroup,
actual: OffsetFetchResponseData.OffsetFetchResponseGroup
): Unit = {
actual.topics.sort((t1, t2) => t1.name.compareTo(t2.name))
actual.topics.asScala.foreach { topic =>
topic.partitions.sort(Comparator.comparingInt[OffsetFetchResponseData.OffsetFetchResponsePartitions](_.partitionIndex))
}
assertEquals(expected, actual)
}
def commitOffsets(tpList: util.List[TopicPartition]): Unit = {
val consumer = createConsumer()
consumer.assign(tpList)
val offsets = tpList.asScala.map{
val offsets = tpList.asScala.map {
tp => (tp, new OffsetAndMetadata(offset, leaderEpoch, metadata))
}.toMap.asJava
consumer.commitSync(offsets)
@ -1682,98 +1698,298 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
removeAllClientAcls()
def verifyPartitionData(partitionData: OffsetFetchResponse.PartitionData): Unit = {
assertTrue(!partitionData.hasError)
assertEquals(offset, partitionData.offset)
assertEquals(metadata, partitionData.metadata)
assertEquals(leaderEpoch.get(), partitionData.leaderEpoch.get())
}
def verifyResponse(groupLevelResponse: Errors,
partitionData: util.Map[TopicPartition, PartitionData],
topicList: util.List[TopicPartition]): Unit = {
assertEquals(Errors.NONE, groupLevelResponse)
assertTrue(partitionData.size() == topicList.size())
topicList.forEach(t => verifyPartitionData(partitionData.get(t)))
}
// test handling partial errors, where one group is fully authorized, some groups don't have
// the right topic authorizations, and some groups have no authorization
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResources(0))
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResources(1))
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResources(3))
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResources(0))
val offsetFetchRequest = createOffsetFetchRequest(groupToPartitionMap)
var offsetFetchResponse = connectAndReceive[OffsetFetchResponse](offsetFetchRequest)
offsetFetchResponse.data().groups().forEach(g =>
g.groupId() match {
offsetFetchResponse.data.groups.forEach { g =>
g.groupId match {
case "group1" =>
verifyResponse(offsetFetchResponse.groupLevelError(groups(0)), offsetFetchResponse
.partitionDataMap(groups(0)), topic1List)
assertResponse(
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(g.groupId)
.setTopics(List(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(topics(0))
.setPartitions(List(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(0)
.setCommittedOffset(offset)
.setCommittedLeaderEpoch(leaderEpoch.get)
.setMetadata(metadata)
).asJava)
).asJava),
offsetFetchResponse.group(g.groupId)
)
case "group2" =>
assertEquals(Errors.NONE, offsetFetchResponse.groupLevelError(groups(1)))
val group2Response = offsetFetchResponse.partitionDataMap(groups(1))
assertTrue(group2Response.size() == 3)
assertTrue(group2Response.keySet().containsAll(topic1And2List))
verifyPartitionData(group2Response.get(topic1And2List.get(0)))
assertTrue(group2Response.get(topic1And2List.get(1)).hasError)
assertTrue(group2Response.get(topic1And2List.get(2)).hasError)
assertEquals(OffsetFetchResponse.UNAUTHORIZED_PARTITION, group2Response.get(topic1And2List.get(1)))
assertEquals(OffsetFetchResponse.UNAUTHORIZED_PARTITION, group2Response.get(topic1And2List.get(2)))
assertResponse(
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(g.groupId)
.setTopics(List(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(topics(0))
.setPartitions(List(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(0)
.setCommittedOffset(offset)
.setCommittedLeaderEpoch(leaderEpoch.get)
.setMetadata(metadata)
).asJava),
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(topics(1))
.setPartitions(List(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(0)
.setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code)
.setCommittedOffset(OffsetFetchResponse.INVALID_OFFSET)
.setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
.setMetadata(OffsetFetchResponse.NO_METADATA),
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(1)
.setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code)
.setCommittedOffset(OffsetFetchResponse.INVALID_OFFSET)
.setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
.setMetadata(OffsetFetchResponse.NO_METADATA)
).asJava)
).asJava),
offsetFetchResponse.group(g.groupId)
)
case "group3" =>
assertEquals(Errors.GROUP_AUTHORIZATION_FAILED, offsetFetchResponse.groupLevelError(groups(2)))
assertTrue(offsetFetchResponse.partitionDataMap(groups(2)).size() == 0)
assertResponse(
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(g.groupId)
.setErrorCode(Errors.GROUP_AUTHORIZATION_FAILED.code),
offsetFetchResponse.group(g.groupId)
)
case "group4" =>
verifyResponse(offsetFetchResponse.groupLevelError(groups(3)), offsetFetchResponse
.partitionDataMap(groups(3)), topic1List)
assertResponse(
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(g.groupId)
.setTopics(List(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(topics(0))
.setPartitions(List(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(0)
.setCommittedOffset(offset)
.setCommittedLeaderEpoch(leaderEpoch.get)
.setMetadata(metadata)
).asJava)
).asJava),
offsetFetchResponse.group(g.groupId)
)
case "group5" =>
assertEquals(Errors.GROUP_AUTHORIZATION_FAILED, offsetFetchResponse.groupLevelError(groups(4)))
assertTrue(offsetFetchResponse.partitionDataMap(groups(4)).size() == 0)
})
assertResponse(
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(g.groupId)
.setErrorCode(Errors.GROUP_AUTHORIZATION_FAILED.code),
offsetFetchResponse.group(g.groupId)
)
}
}
// test that after adding some of the ACLs, we get no group level authorization errors, but
// still get topic level authorization errors for topics we don't have ACLs for
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResources(2))
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResources(4))
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResources(1))
offsetFetchResponse = connectAndReceive[OffsetFetchResponse](offsetFetchRequest)
offsetFetchResponse.data().groups().forEach(g =>
g.groupId() match {
offsetFetchResponse.data.groups.forEach { g =>
g.groupId match {
case "group1" =>
verifyResponse(offsetFetchResponse.groupLevelError(groups(0)), offsetFetchResponse
.partitionDataMap(groups(0)), topic1List)
assertResponse(
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(g.groupId)
.setTopics(List(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(topics(0))
.setPartitions(List(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(0)
.setCommittedOffset(offset)
.setCommittedLeaderEpoch(leaderEpoch.get)
.setMetadata(metadata)
).asJava)
).asJava),
offsetFetchResponse.group(g.groupId)
)
case "group2" =>
verifyResponse(offsetFetchResponse.groupLevelError(groups(1)), offsetFetchResponse
.partitionDataMap(groups(1)), topic1And2List)
assertResponse(
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(g.groupId)
.setTopics(List(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(topics(0))
.setPartitions(List(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(0)
.setCommittedOffset(offset)
.setCommittedLeaderEpoch(leaderEpoch.get)
.setMetadata(metadata)
).asJava),
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(topics(1))
.setPartitions(List(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(0)
.setCommittedOffset(offset)
.setCommittedLeaderEpoch(leaderEpoch.get)
.setMetadata(metadata),
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(1)
.setCommittedOffset(offset)
.setCommittedLeaderEpoch(leaderEpoch.get)
.setMetadata(metadata)
).asJava)
).asJava),
offsetFetchResponse.group(g.groupId)
)
case "group3" =>
assertEquals(Errors.NONE, offsetFetchResponse.groupLevelError(groups(2)))
val group3Response = offsetFetchResponse.partitionDataMap(groups(2))
assertTrue(group3Response.size() == 6)
assertTrue(group3Response.keySet().containsAll(allTopicsList))
verifyPartitionData(group3Response.get(allTopicsList.get(0)))
verifyPartitionData(group3Response.get(allTopicsList.get(1)))
verifyPartitionData(group3Response.get(allTopicsList.get(2)))
assertTrue(group3Response.get(allTopicsList.get(3)).hasError)
assertTrue(group3Response.get(allTopicsList.get(4)).hasError)
assertTrue(group3Response.get(allTopicsList.get(5)).hasError)
assertEquals(OffsetFetchResponse.UNAUTHORIZED_PARTITION, group3Response.get(allTopicsList.get(3)))
assertEquals(OffsetFetchResponse.UNAUTHORIZED_PARTITION, group3Response.get(allTopicsList.get(4)))
assertEquals(OffsetFetchResponse.UNAUTHORIZED_PARTITION, group3Response.get(allTopicsList.get(5)))
assertResponse(
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(g.groupId)
.setTopics(List(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(topics(0))
.setPartitions(List(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(0)
.setCommittedOffset(offset)
.setCommittedLeaderEpoch(leaderEpoch.get)
.setMetadata(metadata)
).asJava),
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(topics(1))
.setPartitions(List(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(0)
.setCommittedOffset(offset)
.setCommittedLeaderEpoch(leaderEpoch.get)
.setMetadata(metadata),
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(1)
.setCommittedOffset(offset)
.setCommittedLeaderEpoch(leaderEpoch.get)
.setMetadata(metadata)
).asJava),
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(topics(2))
.setPartitions(List(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(0)
.setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code)
.setCommittedOffset(OffsetFetchResponse.INVALID_OFFSET)
.setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
.setMetadata(OffsetFetchResponse.NO_METADATA),
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(1)
.setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code)
.setCommittedOffset(OffsetFetchResponse.INVALID_OFFSET)
.setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
.setMetadata(OffsetFetchResponse.NO_METADATA),
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(2)
.setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code)
.setCommittedOffset(OffsetFetchResponse.INVALID_OFFSET)
.setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
.setMetadata(OffsetFetchResponse.NO_METADATA)
).asJava)
).asJava),
offsetFetchResponse.group(g.groupId)
)
case "group4" =>
verifyResponse(offsetFetchResponse.groupLevelError(groups(3)), offsetFetchResponse
.partitionDataMap(groups(3)), topic1And2List)
assertResponse(
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(g.groupId)
.setTopics(List(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(topics(0))
.setPartitions(List(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(0)
.setCommittedOffset(offset)
.setCommittedLeaderEpoch(leaderEpoch.get)
.setMetadata(metadata)
).asJava),
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(topics(1))
.setPartitions(List(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(0)
.setCommittedOffset(offset)
.setCommittedLeaderEpoch(leaderEpoch.get)
.setMetadata(metadata),
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(1)
.setCommittedOffset(offset)
.setCommittedLeaderEpoch(leaderEpoch.get)
.setMetadata(metadata)
).asJava)
).asJava),
offsetFetchResponse.group(g.groupId)
)
case "group5" =>
verifyResponse(offsetFetchResponse.groupLevelError(groups(4)), offsetFetchResponse
.partitionDataMap(groups(4)), topic1And2List)
})
assertResponse(
new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(g.groupId)
.setTopics(List(
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(topics(0))
.setPartitions(List(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(0)
.setCommittedOffset(offset)
.setCommittedLeaderEpoch(leaderEpoch.get)
.setMetadata(metadata)
).asJava),
new OffsetFetchResponseData.OffsetFetchResponseTopics()
.setName(topics(1))
.setPartitions(List(
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(0)
.setCommittedOffset(offset)
.setCommittedLeaderEpoch(leaderEpoch.get)
.setMetadata(metadata),
new OffsetFetchResponseData.OffsetFetchResponsePartitions()
.setPartitionIndex(1)
.setCommittedOffset(offset)
.setCommittedLeaderEpoch(leaderEpoch.get)
.setMetadata(metadata)
).asJava)
).asJava),
offsetFetchResponse.group(g.groupId)
)
}
}
// test that after adding all necessary ACLs, we get no partition level or group level errors
// from the offsetFetch response
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResources(2))
offsetFetchResponse = connectAndReceive[OffsetFetchResponse](offsetFetchRequest)
offsetFetchResponse.data.groups.asScala.map(_.groupId).foreach( groupId =>
verifyResponse(offsetFetchResponse.groupLevelError(groupId), offsetFetchResponse.partitionDataMap(groupId), partitionMap(groupId))
)
offsetFetchResponse.data.groups.forEach { group =>
assertEquals(Errors.NONE.code, group.errorCode)
group.topics.forEach { topic =>
topic.partitions.forEach { partition =>
assertEquals(Errors.NONE.code, partition.errorCode)
}
}
}
}
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames)