veenaypatil commented on a change in pull request #3092:
URL: https://github.com/apache/hudi/pull/3092#discussion_r655334095
##########
File path:
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
##########
@@ -506,6 +509,8 @@ public void refreshTimeline() throws IOException {
return Pair.of(scheduledCompactionInstant, writeStatusRDD);
}
+
+
Review comment:
removed
##########
File path:
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java
##########
@@ -297,7 +295,37 @@ public String getTopicName() {
return topicName;
}
- public HashMap<String, Object> getKafkaParams() {
+ public Map<String, Object> getKafkaParams() {
+ return kafkaParams;
+ }
+
+ private static Map<String, Object> excludeHoodieConfigs(TypedProperties
props) {
+ Map<String, Object> kafkaParams = new HashMap<>();
+ props.keySet().stream().filter(prop -> {
+ // In order to prevent printing unnecessary warn logs, here filter out
the hoodie
+ // configuration items before passing to kafkaParams
+ return !prop.toString().startsWith("hoodie.");
+ }).forEach(prop -> {
+ kafkaParams.put(prop.toString(), props.get(prop.toString()));
+ });
return kafkaParams;
}
+
+ /**
+ * Commit offsets to Kafka only after hoodie commit is successful.
+ * @param checkpointStr checkpoint string containing offsets.
+ * @param props properties for Kafka consumer.
+ */
+ public static void commitOffsetToKafka(String checkpointStr, TypedProperties
props) {
+ DataSourceUtils.checkRequiredProperties(props,
Collections.singletonList(ConsumerConfig.GROUP_ID_CONFIG));
+ Map<TopicPartition, Long> offsetMap =
KafkaOffsetGen.CheckpointUtils.strToOffsets(checkpointStr);
+ Map<String, Object> kafkaParams =
KafkaOffsetGen.excludeHoodieConfigs(props);
+ Map<TopicPartition, OffsetAndMetadata> offsetAndMetadataMap = new
HashMap<>(offsetMap.size());
+ try (KafkaConsumer consumer = new KafkaConsumer(kafkaParams)) {
+ offsetMap.forEach((topicPartition, offset) ->
offsetAndMetadataMap.put(topicPartition, new OffsetAndMetadata(offset)));
+ consumer.commitSync(offsetAndMetadataMap);
+ } catch (CommitFailedException | TimeoutException e) {
+ LOG.warn("Committing offsets to Kafka failed, this does not impact
processing of records", e);
Review comment:
committing offset to Kafka is optional for the user, failure of single
commit should not fail the complete job, hence logged this exception.
##########
File path:
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java
##########
@@ -297,7 +295,37 @@ public String getTopicName() {
return topicName;
}
- public HashMap<String, Object> getKafkaParams() {
+ public Map<String, Object> getKafkaParams() {
+ return kafkaParams;
+ }
+
+ private static Map<String, Object> excludeHoodieConfigs(TypedProperties
props) {
Review comment:
Are you proposing to create `KafkaOffsetGen` object in `DeltaSync` as
well to call `commitOffsetToKafka` method ? I wanted to avoid doing it as we
are already doing so in respective Source classes. Also, I saw
`excludeHoodieConfigs ` as helper method, hence created static
##########
File path:
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
##########
@@ -473,7 +474,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(KafkaOffsetGen.Config.ENABLE_KAFKA_COMMIT_OFFSET,KafkaOffsetGen.Config.DEFAULT_ENABLE_KAFKA_COMMIT_OFFSET))
{
Review comment:
done
##########
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.enable.kafka.commit.offset";
Review comment:
make sense, updated code, did not rename
`hoodie.deltastreamer.kafka.source.maxEvents` in this PR, will discuss more on
compatibility
##########
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.enable.kafka.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;
}
- private final HashMap<String, Object> kafkaParams;
+ private final Map<String, Object> kafkaParams;
private final TypedProperties props;
protected final String topicName;
private KafkaResetOffsetStrategies autoResetValue;
public KafkaOffsetGen(TypedProperties props) {
this.props = props;
-
- kafkaParams = new HashMap<>();
- props.keySet().stream().filter(prop -> {
- // In order to prevent printing unnecessary warn logs, here filter out
the hoodie
- // configuration items before passing to kafkaParams
- return !prop.toString().startsWith("hoodie.");
- }).forEach(prop -> {
- kafkaParams.put(prop.toString(), props.get(prop.toString()));
- });
+ kafkaParams = KafkaOffsetGen.excludeHoodieConfigs(props);
Review comment:
done
##########
File path:
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
##########
@@ -473,7 +474,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(KafkaOffsetGen.Config.ENABLE_KAFKA_COMMIT_OFFSET,KafkaOffsetGen.Config.DEFAULT_ENABLE_KAFKA_COMMIT_OFFSET))
{
+ KafkaOffsetGen.commitOffsetToKafka(checkpointStr, this.props);
Review comment:
done
--
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:
[email protected]