|
30 | 30 |
|
31 | 31 | import com.google.api.core.ApiFutures;
|
32 | 32 | import com.google.api.core.SettableApiFuture;
|
| 33 | +import com.google.api.gax.rpc.StatusCode.Code; |
33 | 34 | import com.google.cloud.pubsublite.AdminClient;
|
34 | 35 | import com.google.cloud.pubsublite.CloudZone;
|
35 | 36 | import com.google.cloud.pubsublite.Offset;
|
|
38 | 39 | import com.google.cloud.pubsublite.SubscriptionPath;
|
39 | 40 | import com.google.cloud.pubsublite.TopicName;
|
40 | 41 | import com.google.cloud.pubsublite.TopicPath;
|
| 42 | +import com.google.cloud.pubsublite.internal.CheckedApiException; |
41 | 43 | import com.google.cloud.pubsublite.internal.CursorClient;
|
42 | 44 | import com.google.cloud.pubsublite.internal.TopicStatsClient;
|
43 | 45 | import com.google.cloud.pubsublite.internal.testing.UnitTestExamples;
|
|
53 | 55 | import com.google.common.collect.ImmutableSet;
|
54 | 56 | import com.google.common.collect.Multimaps;
|
55 | 57 | import com.google.common.reflect.ImmutableTypeToInstanceMap;
|
| 58 | +import com.google.protobuf.util.Timestamps; |
56 | 59 | import java.time.Duration;
|
| 60 | +import java.util.HashMap; |
57 | 61 | import java.util.List;
|
58 | 62 | import java.util.Map;
|
| 63 | +import java.util.Optional; |
59 | 64 | import java.util.concurrent.atomic.AtomicReference;
|
60 | 65 | import java.util.regex.Pattern;
|
61 | 66 | import org.apache.kafka.clients.consumer.Consumer;
|
62 | 67 | import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
|
63 | 68 | import org.apache.kafka.clients.consumer.ConsumerRecord;
|
64 | 69 | import org.apache.kafka.clients.consumer.ConsumerRecords;
|
65 | 70 | import org.apache.kafka.clients.consumer.OffsetAndMetadata;
|
| 71 | +import org.apache.kafka.clients.consumer.OffsetAndTimestamp; |
66 | 72 | import org.apache.kafka.clients.consumer.OffsetCommitCallback;
|
67 | 73 | import org.apache.kafka.common.KafkaException;
|
68 | 74 | import org.apache.kafka.common.PartitionInfo;
|
69 | 75 | import org.apache.kafka.common.TopicPartition;
|
| 76 | +import org.apache.kafka.common.errors.BrokerNotAvailableException; |
70 | 77 | import org.apache.kafka.common.errors.TimeoutException;
|
71 |
| -import org.apache.kafka.common.errors.UnsupportedVersionException; |
72 | 78 | import org.junit.Before;
|
73 | 79 | import org.junit.Test;
|
74 | 80 | import org.junit.runner.RunWith;
|
@@ -139,11 +145,6 @@ public void unsupportedOperations() {
|
139 | 145 | UnsupportedOperationException.class,
|
140 | 146 | () ->
|
141 | 147 | 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)); |
147 | 148 | }
|
148 | 149 |
|
149 | 150 | @Test
|
@@ -481,6 +482,44 @@ public void endOffsets() {
|
481 | 482 | assertThat(output).isEqualTo(ImmutableMap.of(partition2, 22L, partition4, 44L));
|
482 | 483 | }
|
483 | 484 |
|
| 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 | + |
484 | 523 | @Test
|
485 | 524 | public void close() {
|
486 | 525 | consumer.close();
|
|
0 commit comments