@Override
public void handle(OffsetCommitResponse commitResponse, RequestFuture future) {
sensors.commitLatency.record(response.requestLatencyMs());
Set unauthorizedTopics = new HashSet<>();
for (Map.Entry entry : commitResponse.responseData().entrySet()) {
TopicPartition tp = entry.getKey();
OffsetAndMetadata offsetAndMetadata = this.offsets.get(tp);
long offset = offsetAndMetadata.offset();
Errors error = entry.getValue();
if (error == Errors.NONE) {
log.debug("Group {} committed offset {} for partition {}", groupId, offset, tp);
if (subscriptions.isAssigned(tp))
// update the local cache only if the partition is still assigned
subscriptions.committed(tp, offsetAndMetadata);
} else if (error == Errors.GROUP_AUTHORIZATION_FAILED) {
log.error("Not authorized to commit offsets for group {}", groupId);
future.raise(new GroupAuthorizationException(groupId));
return;
} else if (error == Errors.TOPIC_AUTHORIZATION_FAILED) {
unauthorizedTopics.add(tp.topic());
} else if (error == Errors.OFFSET_METADATA_TOO_LARGE
|| error == Errors.INVALID_COMMIT_OFFSET_SIZE) {
// raise the error to the user
log.debug("Offset commit for group {} failed on partition {}: {}", groupId, tp, error.message());
future.raise(error);
return;
} else if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS) {
// just retry
log.debug("Offset commit for group {} failed: {}", groupId, error.message());
future.raise(error);
return;
} else if (error == Errors.COORDINATOR_NOT_AVAILABLE
|| error == Errors.NOT_COORDINATOR
|| error == Errors.REQUEST_TIMED_OUT) {
log.debug("Offset commit for group {} failed: {}", groupId, error.message());
coordinatorDead();
future.raise(error);
return;
} else if (error == Errors.UNKNOWN_MEMBER_ID
|| error == Errors.ILLEGAL_GENERATION
|| error == Errors.REBALANCE_IN_PROGRESS) {
// need to re-join group
log.debug("Offset commit for group {} failed: {}", groupId, error.message());
resetGeneration();
future.raise(new CommitFailedException());
return;
} else if (error == Errors.UNKNOWN_TOPIC_OR_PARTITION) {
log.debug("Offset commit for group {} failed on partition {}: {}", groupId, tp, error.message());
future.raise(new KafkaException("Partition " + tp + " may not exist or user may not have Describe access to topic"));
return;
} else {
log.error("Group {} failed to commit partition {} at offset {}: {}", groupId, tp, offset, error.message());
future.raise(new KafkaException("Unexpected error in commit: " + error.message()));
return;
}
}
if (!unauthorizedTopics.isEmpty()) {
log.error("Not authorized to commit to topics {} for group {}", unauthorizedTopics, groupId);
future.raise(new TopicAuthorizationException(unauthorizedTopics));
} else {
future.complete(null);
}
}