-
Notifications
You must be signed in to change notification settings - Fork 2.4k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Browse files
Browse the repository at this point in the history
…#2227) Co-authored-by: Sivabalan Narayanan <sivabala@uber.com>
- Loading branch information
1 parent
06dc7c7
commit 617cc24
Showing
5 changed files
with
213 additions
and
30 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -21,6 +21,7 @@ | |
import org.apache.hudi.DataSourceUtils; | ||
import org.apache.hudi.common.config.TypedProperties; | ||
import org.apache.hudi.common.util.Option; | ||
import org.apache.hudi.exception.HoodieDeltaStreamerException; | ||
import org.apache.hudi.exception.HoodieException; | ||
import org.apache.hudi.exception.HoodieNotSupportedException; | ||
|
||
|
@@ -40,6 +41,8 @@ | |
import java.util.List; | ||
import java.util.Map; | ||
import java.util.Set; | ||
import java.util.regex.Matcher; | ||
import java.util.regex.Pattern; | ||
import java.util.stream.Collectors; | ||
|
||
/** | ||
|
@@ -49,6 +52,12 @@ public class KafkaOffsetGen { | |
|
||
private static final Logger LOG = LogManager.getLogger(KafkaOffsetGen.class); | ||
|
||
/** | ||
* kafka checkpoint Pattern. | ||
* Format: topic_name,partition_num:offset,partition_num:offset,.... | ||
*/ | ||
private final Pattern pattern = Pattern.compile(".*,.*:.*"); | ||
|
||
public static class CheckpointUtils { | ||
|
||
/** | ||
|
@@ -148,23 +157,38 @@ public static class Config { | |
|
||
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"; | ||
private static final KafkaResetOffsetStrategies DEFAULT_AUTO_RESET_OFFSET = KafkaResetOffsetStrategies.LATEST; | ||
private static final String KAFKA_AUTO_RESET_OFFSETS = "hoodie.deltastreamer.source.kafka.auto.reset.offsets"; | ||
This comment has been minimized.
Sorry, something went wrong. |
||
private static final KafkaResetOffsetStrategies DEFAULT_KAFKA_AUTO_RESET_OFFSETS = KafkaResetOffsetStrategies.LATEST; | ||
public static final long DEFAULT_MAX_EVENTS_FROM_KAFKA_SOURCE = 5000000; | ||
public static long maxEventsFromKafkaSource = DEFAULT_MAX_EVENTS_FROM_KAFKA_SOURCE; | ||
} | ||
|
||
private final HashMap<String, Object> kafkaParams; | ||
private final TypedProperties props; | ||
protected final String topicName; | ||
private KafkaResetOffsetStrategies autoResetValue; | ||
|
||
public KafkaOffsetGen(TypedProperties props) { | ||
this.props = props; | ||
|
||
kafkaParams = new HashMap<>(); | ||
for (Object prop : props.keySet()) { | ||
kafkaParams.put(prop.toString(), props.get(prop.toString())); | ||
} | ||
DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.KAFKA_TOPIC_NAME)); | ||
topicName = props.getString(Config.KAFKA_TOPIC_NAME); | ||
String kafkaAutoResetOffsetsStr = props.getString(Config.KAFKA_AUTO_RESET_OFFSETS, Config.DEFAULT_KAFKA_AUTO_RESET_OFFSETS.name()); | ||
boolean found = false; | ||
for (KafkaResetOffsetStrategies entry: KafkaResetOffsetStrategies.values()) { | ||
if (entry.name().toLowerCase().equals(kafkaAutoResetOffsetsStr)) { | ||
found = true; | ||
autoResetValue = entry; | ||
break; | ||
} | ||
} | ||
if (!found) { | ||
throw new HoodieDeltaStreamerException(Config.KAFKA_AUTO_RESET_OFFSETS + " config set to unknown value " + kafkaAutoResetOffsetsStr); | ||
} | ||
} | ||
|
||
public OffsetRange[] getNextOffsetRanges(Option<String> lastCheckpointStr, long sourceLimit, HoodieDeltaStreamerMetrics metrics) { | ||
|
@@ -186,8 +210,6 @@ public OffsetRange[] getNextOffsetRanges(Option<String> lastCheckpointStr, long | |
fromOffsets = checkupValidOffsets(consumer, lastCheckpointStr, topicPartitions); | ||
This comment has been minimized.
Sorry, something went wrong.
nsivabalan
Author
Contributor
|
||
metrics.updateDeltaStreamerKafkaDelayCountMetrics(delayOffsetCalculation(lastCheckpointStr, topicPartitions, consumer)); | ||
} else { | ||
KafkaResetOffsetStrategies autoResetValue = KafkaResetOffsetStrategies | ||
.valueOf(props.getString("auto.offset.reset", Config.DEFAULT_AUTO_RESET_OFFSET.toString()).toUpperCase()); | ||
switch (autoResetValue) { | ||
case EARLIEST: | ||
fromOffsets = consumer.beginningOffsets(topicPartitions); | ||
|
@@ -227,12 +249,23 @@ public OffsetRange[] getNextOffsetRanges(Option<String> lastCheckpointStr, long | |
// else return earliest offsets | ||
private Map<TopicPartition, Long> checkupValidOffsets(KafkaConsumer consumer, | ||
Option<String> lastCheckpointStr, Set<TopicPartition> topicPartitions) { | ||
Map<TopicPartition, Long> checkpointOffsets = CheckpointUtils.strToOffsets(lastCheckpointStr.get()); | ||
Map<TopicPartition, Long> earliestOffsets = consumer.beginningOffsets(topicPartitions); | ||
if (checkTopicCheckpoint(lastCheckpointStr)) { | ||
Map<TopicPartition, Long> checkpointOffsets = CheckpointUtils.strToOffsets(lastCheckpointStr.get()); | ||
boolean checkpointOffsetReseter = checkpointOffsets.entrySet().stream() | ||
.anyMatch(offset -> offset.getValue() < earliestOffsets.get(offset.getKey())); | ||
return checkpointOffsetReseter ? earliestOffsets : checkpointOffsets; | ||
} | ||
|
||
switch (autoResetValue) { | ||
case EARLIEST: | ||
return earliestOffsets; | ||
case LATEST: | ||
return consumer.endOffsets(topicPartitions); | ||
default: | ||
throw new HoodieNotSupportedException("Auto reset value must be one of 'earliest' or 'latest' "); | ||
} | ||
|
||
boolean checkpointOffsetReseter = checkpointOffsets.entrySet().stream() | ||
.anyMatch(offset -> offset.getValue() < earliestOffsets.get(offset.getKey())); | ||
return checkpointOffsetReseter ? earliestOffsets : checkpointOffsets; | ||
} | ||
|
||
private Long delayOffsetCalculation(Option<String> lastCheckpointStr, Set<TopicPartition> topicPartitions, KafkaConsumer consumer) { | ||
|
@@ -257,6 +290,11 @@ public boolean checkTopicExists(KafkaConsumer consumer) { | |
return result.containsKey(topicName); | ||
} | ||
|
||
private boolean checkTopicCheckpoint(Option<String> lastCheckpointStr) { | ||
Matcher matcher = pattern.matcher(lastCheckpointStr.get()); | ||
return matcher.matches(); | ||
} | ||
|
||
public String getTopicName() { | ||
return topicName; | ||
} | ||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
@liujinhui1994 : Guess we might have to revisit this. I didn't realize this param ("auto.reset.offsets") actually a Kafka config. And so we have to maintain the same config name. Don't think we can change the param name here. My bad.