Skip to content

Commit a785e53

Browse files
authored
feat: Implement Consumer.offsetsForTimes (#123)
Retrieves cursors for event timestamps from the TopicStatsClient to implement PubsubLiteConsumer.offsetsForTimes.
1 parent 3ff2cdf commit a785e53

File tree

2 files changed

+78
-9
lines changed

2 files changed

+78
-9
lines changed

src/main/java/com/google/cloud/pubsublite/kafka/PubsubLiteConsumer.java

Lines changed: 33 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -34,11 +34,14 @@
3434
import com.google.cloud.pubsublite.proto.Cursor;
3535
import com.google.cloud.pubsublite.proto.SeekRequest;
3636
import com.google.cloud.pubsublite.proto.SeekRequest.NamedTarget;
37+
import com.google.common.base.Preconditions;
3738
import com.google.common.collect.ImmutableMap;
3839
import com.google.common.collect.ImmutableSet;
3940
import com.google.common.flogger.GoogleLogger;
41+
import com.google.protobuf.util.Timestamps;
4042
import java.time.Duration;
4143
import java.util.Collection;
44+
import java.util.HashMap;
4245
import java.util.HashSet;
4346
import java.util.List;
4447
import java.util.Map;
@@ -59,7 +62,6 @@
5962
import org.apache.kafka.common.MetricName;
6063
import org.apache.kafka.common.PartitionInfo;
6164
import org.apache.kafka.common.TopicPartition;
62-
import org.apache.kafka.common.errors.UnsupportedVersionException;
6365

6466
/**
6567
* A class that uses a SingleSubscriptionConsumer to remove the duplicate methods from the kafka
@@ -466,8 +468,36 @@ public Map<TopicPartition, OffsetAndTimestamp> offsetsForTimes(Map<TopicPartitio
466468
@Override
467469
public Map<TopicPartition, OffsetAndTimestamp> offsetsForTimes(
468470
Map<TopicPartition, Long> map, Duration duration) {
469-
throw new UnsupportedVersionException(
470-
"Pub/Sub Lite does not support Consumer backlog introspection.");
471+
try {
472+
Map<TopicPartition, ApiFuture<Optional<Cursor>>> cursors =
473+
map.entrySet().stream()
474+
.collect(
475+
Collectors.toMap(
476+
entry -> entry.getKey(),
477+
entry ->
478+
topicStatsClient.computeCursorForEventTime(
479+
topicPath,
480+
checkTopicGetPartition(entry.getKey()),
481+
Timestamps.fromMillis(entry.getValue()))));
482+
ApiFutures.allAsList(cursors.values()).get(duration.toMillis(), TimeUnit.MILLISECONDS);
483+
484+
Map<TopicPartition, OffsetAndTimestamp> output = new HashMap<>();
485+
for (Map.Entry<TopicPartition, ApiFuture<Optional<Cursor>>> entry : cursors.entrySet()) {
486+
// As per KafkaConsumer.offsetsForTimes, null is returned if there is no result for the
487+
// partition.
488+
OffsetAndTimestamp offsetAndTime = null;
489+
Optional<Cursor> cursor = entry.getValue().get();
490+
if (cursor.isPresent()) {
491+
offsetAndTime =
492+
new OffsetAndTimestamp(
493+
cursor.get().getOffset(), Preconditions.checkNotNull(map.get(entry.getKey())));
494+
}
495+
output.put(entry.getKey(), offsetAndTime);
496+
}
497+
return output;
498+
} catch (Throwable t) {
499+
throw toKafka(t);
500+
}
471501
}
472502

473503
@Override

src/test/java/com/google/cloud/pubsublite/kafka/PubsubLiteConsumerTest.java

Lines changed: 45 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,7 @@
3030

3131
import com.google.api.core.ApiFutures;
3232
import com.google.api.core.SettableApiFuture;
33+
import com.google.api.gax.rpc.StatusCode.Code;
3334
import com.google.cloud.pubsublite.AdminClient;
3435
import com.google.cloud.pubsublite.CloudZone;
3536
import com.google.cloud.pubsublite.Offset;
@@ -38,6 +39,7 @@
3839
import com.google.cloud.pubsublite.SubscriptionPath;
3940
import com.google.cloud.pubsublite.TopicName;
4041
import com.google.cloud.pubsublite.TopicPath;
42+
import com.google.cloud.pubsublite.internal.CheckedApiException;
4143
import com.google.cloud.pubsublite.internal.CursorClient;
4244
import com.google.cloud.pubsublite.internal.TopicStatsClient;
4345
import com.google.cloud.pubsublite.internal.testing.UnitTestExamples;
@@ -53,22 +55,26 @@
5355
import com.google.common.collect.ImmutableSet;
5456
import com.google.common.collect.Multimaps;
5557
import com.google.common.reflect.ImmutableTypeToInstanceMap;
58+
import com.google.protobuf.util.Timestamps;
5659
import java.time.Duration;
60+
import java.util.HashMap;
5761
import java.util.List;
5862
import java.util.Map;
63+
import java.util.Optional;
5964
import java.util.concurrent.atomic.AtomicReference;
6065
import java.util.regex.Pattern;
6166
import org.apache.kafka.clients.consumer.Consumer;
6267
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
6368
import org.apache.kafka.clients.consumer.ConsumerRecord;
6469
import org.apache.kafka.clients.consumer.ConsumerRecords;
6570
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
71+
import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
6672
import org.apache.kafka.clients.consumer.OffsetCommitCallback;
6773
import org.apache.kafka.common.KafkaException;
6874
import org.apache.kafka.common.PartitionInfo;
6975
import org.apache.kafka.common.TopicPartition;
76+
import org.apache.kafka.common.errors.BrokerNotAvailableException;
7077
import org.apache.kafka.common.errors.TimeoutException;
71-
import org.apache.kafka.common.errors.UnsupportedVersionException;
7278
import org.junit.Before;
7379
import org.junit.Test;
7480
import org.junit.runner.RunWith;
@@ -139,11 +145,6 @@ public void unsupportedOperations() {
139145
UnsupportedOperationException.class,
140146
() ->
141147
consumer.subscribe(ImmutableList.of("a", "b"), mock(ConsumerRebalanceListener.class)));
142-
assertThrows(
143-
UnsupportedVersionException.class, () -> consumer.offsetsForTimes(ImmutableMap.of()));
144-
assertThrows(
145-
UnsupportedVersionException.class,
146-
() -> consumer.offsetsForTimes(ImmutableMap.of(), Duration.ZERO));
147148
}
148149

149150
@Test
@@ -481,6 +482,44 @@ public void endOffsets() {
481482
assertThat(output).isEqualTo(ImmutableMap.of(partition2, 22L, partition4, 44L));
482483
}
483484

485+
@Test
486+
public void offsetsForTimes() {
487+
TopicPartition partition2 = new TopicPartition(example(TopicPath.class).toString(), 2);
488+
TopicPartition partition4 = new TopicPartition(example(TopicPath.class).toString(), 4);
489+
when(topicStatsClient.computeCursorForEventTime(
490+
example(TopicPath.class), Partition.of(2), Timestamps.fromMillis(2000)))
491+
.thenReturn(
492+
ApiFutures.immediateFuture(Optional.of(Cursor.newBuilder().setOffset(22).build())));
493+
when(topicStatsClient.computeCursorForEventTime(
494+
example(TopicPath.class), Partition.of(4), Timestamps.fromMillis(4000)))
495+
.thenReturn(ApiFutures.immediateFuture(Optional.empty()));
496+
Map<TopicPartition, OffsetAndTimestamp> output =
497+
consumer.offsetsForTimes(ImmutableMap.of(partition2, 2000L, partition4, 4000L));
498+
499+
Map<TopicPartition, OffsetAndTimestamp> expected = new HashMap<>();
500+
expected.put(partition2, new OffsetAndTimestamp(22, 2000));
501+
expected.put(partition4, null);
502+
assertThat(output).isEqualTo(expected);
503+
}
504+
505+
@Test
506+
public void offsetsForTimesFailure() {
507+
TopicPartition partition2 = new TopicPartition(example(TopicPath.class).toString(), 2);
508+
TopicPartition partition4 = new TopicPartition(example(TopicPath.class).toString(), 4);
509+
when(topicStatsClient.computeCursorForEventTime(
510+
example(TopicPath.class), Partition.of(2), Timestamps.fromMillis(2000)))
511+
.thenReturn(
512+
ApiFutures.immediateFuture(Optional.of(Cursor.newBuilder().setOffset(22).build())));
513+
when(topicStatsClient.computeCursorForEventTime(
514+
example(TopicPath.class), Partition.of(4), Timestamps.fromMillis(4000)))
515+
.thenReturn(
516+
ApiFutures.immediateFailedFuture(new CheckedApiException(Code.UNAVAILABLE).underlying));
517+
518+
assertThrows(
519+
BrokerNotAvailableException.class,
520+
() -> consumer.offsetsForTimes(ImmutableMap.of(partition2, 2000L, partition4, 4000L)));
521+
}
522+
484523
@Test
485524
public void close() {
486525
consumer.close();

0 commit comments

Comments
 (0)