You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2021/06/23 04:41:49 UTC

[GitHub] [hudi] vinothchandar commented on a change in pull request #3092: [HUDI-1910] Commit Offset to Kafka after successful Hudi commit

vinothchandar commented on a change in pull request #3092:
URL: https://github.com/apache/hudi/pull/3092#discussion_r656753261



##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java
##########
@@ -157,29 +161,23 @@ public static long totalNewMessages(OffsetRange[] ranges) {
 
     private static final String KAFKA_TOPIC_NAME = "hoodie.deltastreamer.source.kafka.topic";
     private static final String MAX_EVENTS_FROM_KAFKA_SOURCE_PROP = "hoodie.deltastreamer.kafka.source.maxEvents";
+    public static final String ENABLE_KAFKA_COMMIT_OFFSET = "hoodie.deltastreamer.source.kafka.enable.commit.offset";

Review comment:
       IIRC all these are just kafka configs right?

##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
##########
@@ -473,7 +476,9 @@ public void refreshTimeline() throws IOException {
       boolean success = writeClient.commit(instantTime, writeStatusRDD, Option.of(checkpointCommitMetadata));
       if (success) {
         LOG.info("Commit " + instantTime + " successful!");
-
+        if (this.props.getBoolean(ENABLE_KAFKA_COMMIT_OFFSET, DEFAULT_ENABLE_KAFKA_COMMIT_OFFSET)) {

Review comment:
       I am trying to see how we can move such source specific logic back to the source. Ideally `DeltaSync` should be agnostic of Kafka etc. Can we add an `onCommit()` or  some callback on the Source? 

##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java
##########
@@ -157,29 +161,23 @@ public static long totalNewMessages(OffsetRange[] ranges) {
 
     private static final String KAFKA_TOPIC_NAME = "hoodie.deltastreamer.source.kafka.topic";
     private static final String MAX_EVENTS_FROM_KAFKA_SOURCE_PROP = "hoodie.deltastreamer.kafka.source.maxEvents";
+    public static final String ENABLE_KAFKA_COMMIT_OFFSET = "hoodie.deltastreamer.source.kafka.enable.commit.offset";
     // "auto.offset.reset" is kafka native config param. Do not change the config param name.
     public static final String KAFKA_AUTO_OFFSET_RESET = "auto.offset.reset";
     private static final KafkaResetOffsetStrategies DEFAULT_KAFKA_AUTO_OFFSET_RESET = KafkaResetOffsetStrategies.LATEST;
     public static final long DEFAULT_MAX_EVENTS_FROM_KAFKA_SOURCE = 5000000;
     public static long maxEventsFromKafkaSource = DEFAULT_MAX_EVENTS_FROM_KAFKA_SOURCE;
+    public static final Boolean DEFAULT_ENABLE_KAFKA_COMMIT_OFFSET = false;

Review comment:
       can we move this closer to the prop name?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org