dengziming commented on a change in pull request #18145:
URL: https://github.com/apache/flink/pull/18145#discussion_r791334012
##########
File path:
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java
##########
@@ -281,27 +283,50 @@ protected AbstractPartitionDiscoverer
createPartitionDiscoverer(
protected Map<KafkaTopicPartition, Long> fetchOffsetsWithTimestamp(
Collection<KafkaTopicPartition> partitions, long timestamp) {
- Map<TopicPartition, Long> partitionOffsetsRequest = new
HashMap<>(partitions.size());
+ Map<TopicPartition, OffsetSpec> partitionOffsetsRequest = new
HashMap<>(partitions.size());
for (KafkaTopicPartition partition : partitions) {
partitionOffsetsRequest.put(
- new TopicPartition(partition.getTopic(),
partition.getPartition()), timestamp);
+ new TopicPartition(partition.getTopic(),
partition.getPartition()),
+ OffsetSpec.forTimestamp(timestamp));
}
final Map<KafkaTopicPartition, Long> result = new
HashMap<>(partitions.size());
// use a short-lived consumer to fetch the offsets;
// this is ok because this is a one-time operation that happens only
on startup
- try (KafkaConsumer<?, ?> consumer = new KafkaConsumer(properties)) {
- for (Map.Entry<TopicPartition, OffsetAndTimestamp>
partitionToOffset :
-
consumer.offsetsForTimes(partitionOffsetsRequest).entrySet()) {
-
+ try (Admin adminClient = Admin.create(properties)) {
+ Map<TopicPartition, Long> topicPartitionOffsets =
+ adminClient
+ .listOffsets(partitionOffsetsRequest)
+ .all()
+ .thenApply(
+ info -> {
+ Map<TopicPartition, Long> offsets =
new HashMap<>();
+ info.forEach(
+ (tp, listOffsetsResultInfo) ->
{
+ if (listOffsetsResultInfo
!= null) {
+ offsets.put(
+ tp,
listOffsetsResultInfo.offset());
+ }
+ });
+ return offsets;
+ })
+ .get();
+ for (Map.Entry<TopicPartition, Long> partitionToOffset :
+ topicPartitionOffsets.entrySet()) {
result.put(
new KafkaTopicPartition(
partitionToOffset.getKey().topic(),
partitionToOffset.getKey().partition()),
(partitionToOffset.getValue() == null)
? null
- : partitionToOffset.getValue().offset());
+ : partitionToOffset.getValue());
Review comment:
Good catch!
##########
File path:
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java
##########
@@ -281,29 +283,41 @@ protected AbstractPartitionDiscoverer
createPartitionDiscoverer(
protected Map<KafkaTopicPartition, Long> fetchOffsetsWithTimestamp(
Collection<KafkaTopicPartition> partitions, long timestamp) {
- Map<TopicPartition, Long> partitionOffsetsRequest = new
HashMap<>(partitions.size());
+ Map<TopicPartition, OffsetSpec> partitionOffsetsRequest = new
HashMap<>(partitions.size());
for (KafkaTopicPartition partition : partitions) {
partitionOffsetsRequest.put(
- new TopicPartition(partition.getTopic(),
partition.getPartition()), timestamp);
+ new TopicPartition(partition.getTopic(),
partition.getPartition()),
+ OffsetSpec.forTimestamp(timestamp));
}
- final Map<KafkaTopicPartition, Long> result = new
HashMap<>(partitions.size());
// use a short-lived consumer to fetch the offsets;
// this is ok because this is a one-time operation that happens only
on startup
- try (KafkaConsumer<?, ?> consumer = new KafkaConsumer(properties)) {
- for (Map.Entry<TopicPartition, OffsetAndTimestamp>
partitionToOffset :
-
consumer.offsetsForTimes(partitionOffsetsRequest).entrySet()) {
-
- result.put(
- new KafkaTopicPartition(
- partitionToOffset.getKey().topic(),
- partitionToOffset.getKey().partition()),
- (partitionToOffset.getValue() == null)
- ? null
- : partitionToOffset.getValue().offset());
- }
+ try (Admin adminClient = Admin.create(properties)) {
Review comment:
Yes, I reverted this change since it is already marked as deprecated.
##########
File path:
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumerator.java
##########
@@ -558,25 +533,100 @@ public PartitionOffsetsRetrieverImpl(
}
}
+ /**
+ * List offsets for the specified partitions and OffsetSpec. This
operation enables to find
+ * the beginning offset, end offset as well as the offset matching a
timestamp in
+ * partitions.
+ *
+ * @see KafkaAdminClient#listOffsets(Map)
+ * @param topicPartitionOffsets The mapping from partition to the
OffsetSpec to look up.
+ * @return The list offsets result.
+ */
+ private Map<TopicPartition, ListOffsetsResult.ListOffsetsResultInfo>
listOffsets(
+ Map<TopicPartition, OffsetSpec> topicPartitionOffsets) {
+ try {
+ return adminClient
+ .listOffsets(topicPartitionOffsets)
+ .all()
+ .thenApply(
+ result -> {
+ Map<TopicPartition,
ListOffsetsResult.ListOffsetsResultInfo>
+ offsets = new HashMap<>();
+ result.forEach(
+ (tp, listOffsetsResultInfo) -> {
+ if (listOffsetsResultInfo !=
null) {
+ offsets.put(tp,
listOffsetsResultInfo);
+ }
+ });
+ return offsets;
+ })
+ .get();
+ } catch (InterruptedException e) {
+ throw new FlinkRuntimeException(
Review comment:
Thank you for this reminder, I also find there is another
`InterruptedException` ad `committedOffsets` so I revised it too.
##########
File path:
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumerator.java
##########
@@ -558,25 +533,100 @@ public PartitionOffsetsRetrieverImpl(
}
}
+ /**
+ * List offsets for the specified partitions and OffsetSpec. This
operation enables to find
+ * the beginning offset, end offset as well as the offset matching a
timestamp in
+ * partitions.
+ *
+ * @see KafkaAdminClient#listOffsets(Map)
+ * @param topicPartitionOffsets The mapping from partition to the
OffsetSpec to look up.
+ * @return The list offsets result.
+ */
+ private Map<TopicPartition, ListOffsetsResult.ListOffsetsResultInfo>
listOffsets(
+ Map<TopicPartition, OffsetSpec> topicPartitionOffsets) {
+ try {
+ return adminClient
+ .listOffsets(topicPartitionOffsets)
+ .all()
+ .thenApply(
+ result -> {
+ Map<TopicPartition,
ListOffsetsResult.ListOffsetsResultInfo>
+ offsets = new HashMap<>();
+ result.forEach(
+ (tp, listOffsetsResultInfo) -> {
+ if (listOffsetsResultInfo !=
null) {
+ offsets.put(tp,
listOffsetsResultInfo);
+ }
+ });
+ return offsets;
+ })
+ .get();
+ } catch (InterruptedException e) {
+ throw new FlinkRuntimeException(
Review comment:
Thank you for this reminder, I also find there is another
`InterruptedException` in `committedOffsets` so I revised it too.
##########
File path:
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
##########
@@ -152,7 +152,7 @@ public static void startClusters(boolean secureMode) throws
Exception {
public static void startClusters(KafkaTestEnvironment.Config
environmentConfig)
throws Exception {
- kafkaServer = constructKafkaTestEnvionment();
+ kafkaServer = constructKafkaTestEnvironment();
Review comment:
I already rebased my commits and made this change in a separate commit
as other PRs, do you mean I should submit another PR for this hotfix?
##########
File path:
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
##########
@@ -152,7 +152,7 @@ public static void startClusters(boolean secureMode) throws
Exception {
public static void startClusters(KafkaTestEnvironment.Config
environmentConfig)
throws Exception {
- kafkaServer = constructKafkaTestEnvionment();
+ kafkaServer = constructKafkaTestEnvironment();
Review comment:
I already rebased my commits and made this change in a separate commit
as other PRs, do you mean I should submit another PR for this hotfix? this is
my first PR so maybe I misoperated them.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]