diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml
index 4a062627e7fdb..2f3186ce67147 100644
--- a/checkstyle/suppressions.xml
+++ b/checkstyle/suppressions.xml
@@ -12,7 +12,7 @@
+ files="(MessageDataGenerator|FieldSpec|AssignorConfiguration).java"/>
the type of element
@@ -674,6 +676,20 @@ public static Set mkSet(T... elems) {
return result;
}
+ /**
+ * Creates a sorted set
+ * @param elems the elements
+ * @param the type of element, must be comparable
+ * @return SortedSet
+ */
+ @SafeVarargs
+ public static > SortedSet mkSortedSet(T... elems) {
+ SortedSet result = new TreeSet<>();
+ for (T elem : elems)
+ result.add(elem);
+ return result;
+ }
+
/**
* Creates a map entry (for use with {@link Utils#mkMap(java.util.Map.Entry[])})
*
diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
index 2807e62e1b7cc..8487268b4b551 100644
--- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
+++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
@@ -19,6 +19,7 @@
import java.util.LinkedList;
import java.util.TreeMap;
import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeoutException;
import java.util.function.Function;
import java.util.stream.Collectors;
import org.apache.kafka.clients.admin.Admin;
@@ -27,6 +28,7 @@
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.Metric;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.TopicPartition;
@@ -1217,17 +1219,9 @@ public Map> allLocalStorePartitionLags() {
}
log.debug("Current changelog positions: {}", allChangelogPositions);
- final Map allEndOffsets;
- try {
- allEndOffsets = adminClient.listOffsets(
- allPartitions.stream()
- .collect(Collectors.toMap(Function.identity(), tp -> OffsetSpec.latest()))
- ).all().get();
- } catch (final RuntimeException | InterruptedException | ExecutionException e) {
- throw new StreamsException("Unable to obtain end offsets from kafka", e);
- }
-
+ final Map allEndOffsets = fetchEndOffsetsWithoutTimeout(allPartitions, adminClient);
log.debug("Current end offsets :{}", allEndOffsets);
+
for (final Map.Entry entry : allEndOffsets.entrySet()) {
// Avoiding an extra admin API lookup by computing lags for not-yet-started restorations
// from zero instead of the real "earliest offset" for the changelog.
@@ -1244,4 +1238,28 @@ public Map> allLocalStorePartitionLags() {
return Collections.unmodifiableMap(localStorePartitionLags);
}
+
+ static Map fetchEndOffsetsWithoutTimeout(final Collection partitions,
+ final Admin adminClient) {
+ return fetchEndOffsets(partitions, adminClient, null);
+ }
+
+ public static Map fetchEndOffsets(final Collection partitions,
+ final Admin adminClient,
+ final Duration timeout) {
+ final Map endOffsets;
+ try {
+ final KafkaFuture