Skip to content

Commit

Permalink
KAFKA-14821 Implement the listOffsets API with AdminApiDriver (apache…
Browse files Browse the repository at this point in the history
…#13432)

We are handling complex workflows ListOffsets by chaining together MetadataCall instances and ListOffsetsCall instances, there are many complex and error-prone logic. In this PR we rewrote it with the `AdminApiDriver` infra, notable changes better than old logic:
1. Retry lookup stage on receiving `NOT_LEADER_OR_FOLLOWER` and `LEADER_NOT_AVAILABLE`, whereas in the past we failed the partition directly without retry.
2. Removing class field `supportsMaxTimestamp` and calculating it on the fly to avoid the mutable state, this won't change any behavior of  the client.
3. Retry fulfillment stage on `RetriableException`, whereas in the past we just retry fulfillment stage on `InvalidMetadataException`, this means we will retry on `TimeoutException` and other `RetriableException`.

We also `handleUnsupportedVersionException` to `AdminApiHandler` and `AdminApiLookupStrategy`, they are used to keep consistency with old logic, and we can continue improvise them. 

Reviewers: Ziming Deng <[email protected]>, David Jacot <[email protected]>
  • Loading branch information
dimitarndimitrov authored Apr 20, 2023
1 parent f5de4da commit e14dd80
Show file tree
Hide file tree
Showing 12 changed files with 771 additions and 483 deletions.

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

import org.apache.kafka.common.Node;
import org.apache.kafka.common.errors.DisconnectException;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.requests.AbstractRequest;
import org.apache.kafka.common.requests.AbstractResponse;
import org.apache.kafka.common.requests.FindCoordinatorRequest.NoBatchedFindCoordinatorsException;
Expand Down Expand Up @@ -260,12 +261,31 @@ public void onFailure(
.filter(future.lookupKeys()::contains)
.collect(Collectors.toSet());
retryLookup(keysToUnmap);
} else if (t instanceof UnsupportedVersionException) {
if (spec.scope instanceof FulfillmentScope) {
int brokerId = ((FulfillmentScope) spec.scope).destinationBrokerId;
Map<K, Throwable> unrecoverableFailures =
handler.handleUnsupportedVersionException(
brokerId,
(UnsupportedVersionException) t,
spec.keys);
completeExceptionally(unrecoverableFailures);
} else {
Map<K, Throwable> unrecoverableLookupFailures =
handler.lookupStrategy().handleUnsupportedVersionException(
(UnsupportedVersionException) t,
spec.keys);
completeLookupExceptionally(unrecoverableLookupFailures);
Set<K> keysToUnmap = spec.keys.stream()
.filter(k -> !unrecoverableLookupFailures.containsKey(k))
.collect(Collectors.toSet());
retryLookup(keysToUnmap);
}
} else {
Map<K, Throwable> errors = spec.keys.stream().collect(Collectors.toMap(
Function.identity(),
key -> t
));

if (spec.scope instanceof FulfillmentScope) {
completeExceptionally(errors);
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
package org.apache.kafka.clients.admin.internals;

import org.apache.kafka.common.Node;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.requests.AbstractRequest;
import org.apache.kafka.common.requests.AbstractResponse;

Expand Down Expand Up @@ -70,6 +71,23 @@ public interface AdminApiHandler<K, V> {
*/
ApiResult<K, V> handleResponse(Node broker, Set<K> keys, AbstractResponse response);

/**
* Callback that is invoked when a fulfillment request hits an UnsupportedVersionException.
* Keys for which the exception cannot be handled and the request shouldn't be retried must be mapped
* to an error and returned. The request will then be retried for the remainder of the keys.
*
* @return The failure mappings for the keys for which the exception cannot be handled and the
* request shouldn't be retried. If the exception cannot be handled all initial keys will be in
* the returned map.
*/
default Map<K, Throwable> handleUnsupportedVersionException(
int brokerId,
UnsupportedVersionException exception,
Set<K> keys
) {
return keys.stream().collect(Collectors.toMap(k -> k, k -> exception));
}

/**
* Get the lookup strategy that is responsible for finding the brokerId
* which will handle each respective key.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,8 @@
*/
package org.apache.kafka.clients.admin.internals;

import java.util.stream.Collectors;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.requests.AbstractRequest;
import org.apache.kafka.common.requests.AbstractResponse;

Expand Down Expand Up @@ -81,6 +83,23 @@ public interface AdminApiLookupStrategy<T> {
*/
LookupResult<T> handleResponse(Set<T> keys, AbstractResponse response);

/**
* Callback that is invoked when a lookup request hits an UnsupportedVersionException.
* Keys for which the exception cannot be handled and the request shouldn't be retried must be mapped
* to an error and returned. The remainder of the keys will then be unmapped and the lookup request will
* be retried for them.
*
* @return The failure mappings for the keys for which the exception cannot be handled and the
* request shouldn't be retried. If the exception cannot be handled all initial keys will be in
* the returned map.
*/
default Map<T, Throwable> handleUnsupportedVersionException(
UnsupportedVersionException exception,
Set<T> keys
) {
return keys.stream().collect(Collectors.toMap(k -> k, k -> exception));
}

class LookupResult<K> {
// This is the set of keys that have been completed by the lookup phase itself.
// The driver will not attempt lookup or fulfillment for completed keys.
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,207 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.clients.admin.internals;

import java.util.ArrayList;
import java.util.Collection;
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.stream.Collectors;
import org.apache.kafka.clients.admin.ListOffsetsOptions;
import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo;
import org.apache.kafka.clients.admin.internals.AdminApiFuture.SimpleAdminApiFuture;
import org.apache.kafka.clients.admin.internals.AdminApiHandler.Batched;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.ApiException;
import org.apache.kafka.common.errors.RetriableException;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition;
import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic;
import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.requests.AbstractResponse;
import org.apache.kafka.common.requests.ListOffsetsRequest;
import org.apache.kafka.common.requests.ListOffsetsResponse;
import org.apache.kafka.common.utils.CollectionUtils;
import org.apache.kafka.common.utils.LogContext;
import org.slf4j.Logger;

public final class ListOffsetsHandler extends Batched<TopicPartition, ListOffsetsResultInfo> {

private final Map<TopicPartition, Long> offsetTimestampsByPartition;
private final ListOffsetsOptions options;
private final Logger log;
private final AdminApiLookupStrategy<TopicPartition> lookupStrategy;

public ListOffsetsHandler(
Map<TopicPartition, Long> offsetTimestampsByPartition,
ListOffsetsOptions options,
LogContext logContext
) {
this.offsetTimestampsByPartition = offsetTimestampsByPartition;
this.options = options;
this.log = logContext.logger(ListOffsetsHandler.class);
this.lookupStrategy = new PartitionLeaderStrategy(logContext);
}

@Override
public String apiName() {
return "listOffsets";
}

@Override
public AdminApiLookupStrategy<TopicPartition> lookupStrategy() {
return this.lookupStrategy;
}

@Override
ListOffsetsRequest.Builder buildBatchedRequest(int brokerId, Set<TopicPartition> keys) {
Map<String, ListOffsetsTopic> topicsByName = CollectionUtils.groupPartitionsByTopic(
keys,
topicName -> new ListOffsetsTopic().setName(topicName),
(listOffsetsTopic, partitionId) -> {
TopicPartition topicPartition = new TopicPartition(listOffsetsTopic.name(), partitionId);
long offsetTimestamp = offsetTimestampsByPartition.get(topicPartition);
listOffsetsTopic.partitions().add(
new ListOffsetsPartition()
.setPartitionIndex(partitionId)
.setTimestamp(offsetTimestamp));
});
boolean supportsMaxTimestamp = keys
.stream()
.anyMatch(key -> offsetTimestampsByPartition.get(key) == ListOffsetsRequest.MAX_TIMESTAMP);

return ListOffsetsRequest.Builder
.forConsumer(true, options.isolationLevel(), supportsMaxTimestamp)
.setTargetTimes(new ArrayList<>(topicsByName.values()));
}

@Override
public ApiResult<TopicPartition, ListOffsetsResultInfo> handleResponse(
Node broker,
Set<TopicPartition> keys,
AbstractResponse abstractResponse
) {
ListOffsetsResponse response = (ListOffsetsResponse) abstractResponse;
Map<TopicPartition, ListOffsetsResultInfo> completed = new HashMap<>();
Map<TopicPartition, Throwable> failed = new HashMap<>();
List<TopicPartition> unmapped = new ArrayList<>();
Set<TopicPartition> retriable = new HashSet<>();

for (ListOffsetsTopicResponse topic : response.topics()) {
for (ListOffsetsPartitionResponse partition : topic.partitions()) {
TopicPartition topicPartition = new TopicPartition(topic.name(), partition.partitionIndex());
Errors error = Errors.forCode(partition.errorCode());
if (!offsetTimestampsByPartition.containsKey(topicPartition)) {
log.warn("ListOffsets response includes unknown topic partition {}", topicPartition);
} else if (error == Errors.NONE) {
Optional<Integer> leaderEpoch = (partition.leaderEpoch() == ListOffsetsResponse.UNKNOWN_EPOCH)
? Optional.empty()
: Optional.of(partition.leaderEpoch());
completed.put(
topicPartition,
new ListOffsetsResultInfo(partition.offset(), partition.timestamp(), leaderEpoch));
} else {
handlePartitionError(topicPartition, error, failed, unmapped, retriable);
}
}
}

// Sanity-check if the current leader for these partitions returned results for all of them
for (TopicPartition topicPartition : keys) {
if (unmapped.isEmpty()
&& !completed.containsKey(topicPartition)
&& !failed.containsKey(topicPartition)
&& !retriable.contains(topicPartition)
) {
ApiException sanityCheckException = new ApiException(
"The response from broker " + broker.id() +
" did not contain a result for topic partition " + topicPartition);
log.error(
"ListOffsets request for topic partition {} failed sanity check",
topicPartition,
sanityCheckException);
failed.put(topicPartition, sanityCheckException);
}
}

return new ApiResult<>(completed, failed, unmapped);
}

private void handlePartitionError(
TopicPartition topicPartition,
Errors error,
Map<TopicPartition, Throwable> failed,
List<TopicPartition> unmapped,
Set<TopicPartition> retriable
) {
if (error == Errors.NOT_LEADER_OR_FOLLOWER || error == Errors.LEADER_NOT_AVAILABLE) {
log.debug(
"ListOffsets lookup request for topic partition {} will be retried due to invalid leader metadata {}",
topicPartition,
error);
unmapped.add(topicPartition);
} else if (error.exception() instanceof RetriableException) {
log.debug(
"ListOffsets fulfillment request for topic partition {} will be retried due to {}",
topicPartition,
error);
retriable.add(topicPartition);
} else {
log.error(
"ListOffsets request for topic partition {} failed due to an unexpected error {}",
topicPartition,
error);
failed.put(topicPartition, error.exception());
}
}

@Override
public Map<TopicPartition, Throwable> handleUnsupportedVersionException(
int brokerId, UnsupportedVersionException exception, Set<TopicPartition> keys
) {
log.warn("Broker " + brokerId + " does not support MAX_TIMESTAMP offset specs");
Map<TopicPartition, Throwable> maxTimestampPartitions = new HashMap<>();
for (TopicPartition topicPartition : keys) {
Long offsetTimestamp = offsetTimestampsByPartition.get(topicPartition);
if (offsetTimestamp == ListOffsetsRequest.MAX_TIMESTAMP) {
maxTimestampPartitions.put(topicPartition, exception);
}
}
// If there are no partitions with MAX_TIMESTAMP specs the UnsupportedVersionException cannot be handled
// and all partitions should be failed here.
// Otherwise, just the partitions with MAX_TIMESTAMP specs should be failed here and the fulfillment stage
// will later be retried for the potentially empty set of partitions with non-MAX_TIMESTAMP specs.
if (maxTimestampPartitions.isEmpty()) {
return keys.stream().collect(Collectors.toMap(k -> k, k -> exception));
} else {
return maxTimestampPartitions;
}
}

public static SimpleAdminApiFuture<TopicPartition, ListOffsetsResultInfo> newFuture(
Collection<TopicPartition> topicPartitions
) {
return AdminApiFuture.forKeys(new HashSet<>(topicPartitions));
}
}
Loading

0 comments on commit e14dd80

Please sign in to comment.