wangxianghu commented on a change in pull request #3175:
URL: https://github.com/apache/hudi/pull/3175#discussion_r660220267



##########
File path: 
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java
##########
@@ -327,4 +330,19 @@ public void commitOffsetToKafka(String checkpointStr) {
       LOG.warn("Committing offsets to Kafka failed, this does not impact 
processing of records", e);
     }
   }
+
+  private Map<TopicPartition, Long> getGroupOffsets(KafkaConsumer consumer, 
Set<TopicPartition> topicPartitions) {
+    Map<TopicPartition, Long> fromOffsets = new HashMap<>();
+    for (TopicPartition topicPartition : topicPartitions) {
+      OffsetAndMetadata committedOffsetAndMetadata = 
consumer.committed(topicPartition);
+      if (committedOffsetAndMetadata != null) {
+        fromOffsets.put(topicPartition, committedOffsetAndMetadata.offset());
+      } else {
+        LOG.warn("There are no commits associated with this consumer group, 
starting to consume form latest offset");
+        fromOffsets = consumer.endOffsets(topicPartitions);
+        break;

Review comment:
       This logic seems  the same as `LATEST` ?

##########
File path: 
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java
##########
@@ -327,4 +330,19 @@ public void commitOffsetToKafka(String checkpointStr) {
       LOG.warn("Committing offsets to Kafka failed, this does not impact 
processing of records", e);
     }
   }
+
+  private Map<TopicPartition, Long> getGroupOffsets(KafkaConsumer consumer, 
Set<TopicPartition> topicPartitions) {
+    Map<TopicPartition, Long> fromOffsets = new HashMap<>();
+    for (TopicPartition topicPartition : topicPartitions) {
+      OffsetAndMetadata committedOffsetAndMetadata = 
consumer.committed(topicPartition);
+      if (committedOffsetAndMetadata != null) {
+        fromOffsets.put(topicPartition, committedOffsetAndMetadata.offset());
+      } else {
+        LOG.warn("There are no commits associated with this consumer group, 
starting to consume form latest offset");
+        fromOffsets = consumer.endOffsets(topicPartitions);
+        break;

Review comment:
       @veenaypatil I mean in this feature, is there any difference between 
`latest` and `none`
   
   copied from 
`org.apache.kafka.clients.consumer.ConsumerConfig#AUTO_OFFSET_RESET_CONFIG`
   ```
       /**
        * <code>auto.offset.reset</code>
        */
       public static final String AUTO_OFFSET_RESET_CONFIG = 
"auto.offset.reset";
       public static final String AUTO_OFFSET_RESET_DOC = "What to do when 
there is no initial offset in Kafka or if the current offset does not exist any 
more on the server (e.g. because that data has been deleted): <ul><li>earliest: 
automatically reset the offset to the earliest offset<li>latest: automatically 
reset the offset to the latest offset</li><li>none: throw exception to the 
consumer if no previous offset is found for the consumer's 
group</li><li>anything else: throw exception to the consumer.</li></ul>";
   ```

##########
File path: 
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java
##########
@@ -327,4 +330,19 @@ public void commitOffsetToKafka(String checkpointStr) {
       LOG.warn("Committing offsets to Kafka failed, this does not impact 
processing of records", e);
     }
   }
+
+  private Map<TopicPartition, Long> getGroupOffsets(KafkaConsumer consumer, 
Set<TopicPartition> topicPartitions) {
+    Map<TopicPartition, Long> fromOffsets = new HashMap<>();
+    for (TopicPartition topicPartition : topicPartitions) {
+      OffsetAndMetadata committedOffsetAndMetadata = 
consumer.committed(topicPartition);
+      if (committedOffsetAndMetadata != null) {
+        fromOffsets.put(topicPartition, committedOffsetAndMetadata.offset());
+      } else {
+        LOG.warn("There are no commits associated with this consumer group, 
starting to consume form latest offset");
+        fromOffsets = consumer.endOffsets(topicPartitions);
+        break;

Review comment:
       > @wangxianghu
   > 
   > Yes, if you update the value to earliest/latest in this test case the 
consumer will start reading either from 0th offset (earliest) or 500th offset 
(latest), instead it should start from 250th offset (as this is the last 
committed offset)
   > 
   > 
https://github.com/apache/hudi/blob/80a1f1cceca52f266057b948f48cf20f5d273184/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/helpers/TestKafkaOffsetGen.java#L139
   > 
   > I actually don't like the NONE option here and wanted to use GROUP but the 
consumer will throw an exception in that case
   
   IIUC, if you have committed the offset to kafka, when you start the consumer 
with the same group.id as before and set `auto.offset.set` to `latest`, it will 
continue to consume offset committed last time(which means 250th as you 
mentioned), right ?

##########
File path: 
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java
##########
@@ -327,4 +330,19 @@ public void commitOffsetToKafka(String checkpointStr) {
       LOG.warn("Committing offsets to Kafka failed, this does not impact 
processing of records", e);
     }
   }
+
+  private Map<TopicPartition, Long> getGroupOffsets(KafkaConsumer consumer, 
Set<TopicPartition> topicPartitions) {
+    Map<TopicPartition, Long> fromOffsets = new HashMap<>();
+    for (TopicPartition topicPartition : topicPartitions) {
+      OffsetAndMetadata committedOffsetAndMetadata = 
consumer.committed(topicPartition);
+      if (committedOffsetAndMetadata != null) {
+        fromOffsets.put(topicPartition, committedOffsetAndMetadata.offset());
+      } else {
+        LOG.warn("There are no commits associated with this consumer group, 
starting to consume form latest offset");
+        fromOffsets = consumer.endOffsets(topicPartitions);
+        break;

Review comment:
       > @wangxianghu yes, but that's not happening, I think it is because of 
the way we are explicitly setting the `fromOffsets` here - 
https://github.com/apache/hudi/blob/master/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java#L221
 which moves the consumer to end, that is it starts reading from 500th offset 
for partition0 and partition1 in test case
   > 
   > Copying doc for `KafkaConsumer#endOffsets`
   > 
   > ```
   > Get the end offsets for the given partitions. In the default 
read_uncommitted isolation level, the end offset is the high watermark (that 
is, the offset of the last successfully replicated message plus one)
   > ```
   
   you are right.




-- 
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]


Reply via email to