bvaradar commented on code in PR #8376:
URL: https://github.com/apache/hudi/pull/8376#discussion_r1176055401
##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java:
##########
@@ -107,48 +112,119 @@ public static String offsetsToStr(OffsetRange[] ranges) {
* @param numEvents maximum number of events to read.
*/
public static OffsetRange[] computeOffsetRanges(Map<TopicPartition, Long>
fromOffsetMap,
- Map<TopicPartition, Long>
toOffsetMap, long numEvents) {
-
- Comparator<OffsetRange> byPartition =
Comparator.comparing(OffsetRange::partition);
-
- // Create initial offset ranges for each 'to' partition, with from = to
offsets.
+ Map<TopicPartition, Long>
toOffsetMap,
+ long numEvents,
+ long minPartitions) {
+ // Create initial offset ranges for each 'to' partition, with default
from = 0 offsets.
OffsetRange[] ranges = toOffsetMap.keySet().stream().map(tp -> {
long fromOffset = fromOffsetMap.getOrDefault(tp, 0L);
- return OffsetRange.create(tp, fromOffset, fromOffset);
+ return OffsetRange.create(tp, fromOffset, toOffsetMap.get(tp));
})
- .sorted(byPartition)
+ .sorted(SORT_BY_PARTITION)
.collect(Collectors.toList())
.toArray(new OffsetRange[toOffsetMap.size()]);
+ LOG.debug("numEvents {}, minPartitions {}, ranges {}", numEvents,
minPartitions, ranges);
+ boolean needSplitToMinPartitions = minPartitions > toOffsetMap.size();
+ long totalEvents = totalNewMessages(ranges);
long allocedEvents = 0;
Set<Integer> exhaustedPartitions = new HashSet<>();
+ List<OffsetRange> finalRanges = new ArrayList<>();
+ // choose the numEvents with min(totalEvents, numEvents)
+ numEvents = Math.min(totalEvents, numEvents);
Review Comment:
Instead of changing the function argument (numEvents), can you create a new
local variable
##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java:
##########
@@ -107,48 +112,119 @@ public static String offsetsToStr(OffsetRange[] ranges) {
* @param numEvents maximum number of events to read.
*/
public static OffsetRange[] computeOffsetRanges(Map<TopicPartition, Long>
fromOffsetMap,
- Map<TopicPartition, Long>
toOffsetMap, long numEvents) {
-
- Comparator<OffsetRange> byPartition =
Comparator.comparing(OffsetRange::partition);
-
- // Create initial offset ranges for each 'to' partition, with from = to
offsets.
+ Map<TopicPartition, Long>
toOffsetMap,
+ long numEvents,
+ long minPartitions) {
+ // Create initial offset ranges for each 'to' partition, with default
from = 0 offsets.
OffsetRange[] ranges = toOffsetMap.keySet().stream().map(tp -> {
long fromOffset = fromOffsetMap.getOrDefault(tp, 0L);
- return OffsetRange.create(tp, fromOffset, fromOffset);
+ return OffsetRange.create(tp, fromOffset, toOffsetMap.get(tp));
})
- .sorted(byPartition)
+ .sorted(SORT_BY_PARTITION)
.collect(Collectors.toList())
.toArray(new OffsetRange[toOffsetMap.size()]);
+ LOG.debug("numEvents {}, minPartitions {}, ranges {}", numEvents,
minPartitions, ranges);
+ boolean needSplitToMinPartitions = minPartitions > toOffsetMap.size();
+ long totalEvents = totalNewMessages(ranges);
long allocedEvents = 0;
Set<Integer> exhaustedPartitions = new HashSet<>();
+ List<OffsetRange> finalRanges = new ArrayList<>();
+ // choose the numEvents with min(totalEvents, numEvents)
+ numEvents = Math.min(totalEvents, numEvents);
+
// keep going until we have events to allocate and partitions still not
exhausted.
while (allocedEvents < numEvents && exhaustedPartitions.size() <
toOffsetMap.size()) {
- long remainingEvents = numEvents - allocedEvents;
- long eventsPerPartition =
- (long) Math.ceil((1.0 * remainingEvents) / (toOffsetMap.size()
- exhaustedPartitions.size()));
-
// Allocate the remaining events to non-exhausted partitions, in round
robin fashion
+ Set<Integer> allocatedPartitionsThisLoop = new
HashSet<>(exhaustedPartitions);
for (int i = 0; i < ranges.length; i++) {
+ long remainingEvents = numEvents - allocedEvents;
+ long remainingPartitions = toOffsetMap.size() -
allocatedPartitionsThisLoop.size();
+ // if need tp split into minPartitions, recalculate the
remainingPartitions
+ if (needSplitToMinPartitions) {
+ remainingPartitions = minPartitions - finalRanges.size();
+ }
+ long eventsPerPartition = (long) Math.ceil((1.0 * remainingEvents) /
remainingPartitions);
+
OffsetRange range = ranges[i];
- if (!exhaustedPartitions.contains(range.partition())) {
- long toOffsetMax = toOffsetMap.get(range.topicPartition());
- long toOffset = Math.min(toOffsetMax, range.untilOffset() +
eventsPerPartition);
- if (toOffset == toOffsetMax) {
- exhaustedPartitions.add(range.partition());
- }
- allocedEvents += toOffset - range.untilOffset();
- // We need recompute toOffset if allocedEvents larger than
numEvents.
- if (allocedEvents > numEvents) {
- long offsetsToAdd = Math.min(eventsPerPartition, (numEvents -
allocedEvents));
- toOffset = Math.min(toOffsetMax, toOffset + offsetsToAdd);
- }
- ranges[i] = OffsetRange.create(range.topicPartition(),
range.fromOffset(), toOffset);
+ if (exhaustedPartitions.contains(range.partition())) {
+ continue;
}
+
+ long toOffset = Math.min(range.untilOffset(), range.fromOffset() +
eventsPerPartition);
+ if (toOffset == range.untilOffset()) {
+ exhaustedPartitions.add(range.partition());
+ }
+ allocedEvents += toOffset - range.fromOffset();
+ // We need recompute toOffset if allocedEvents larger than numEvents.
+ if (allocedEvents > numEvents) {
+ long offsetsToAdd = Math.min(eventsPerPartition, (numEvents -
allocedEvents));
+ toOffset = Math.min(range.untilOffset(), toOffset + offsetsToAdd);
+ }
+ OffsetRange thisRange = OffsetRange.create(range.topicPartition(),
range.fromOffset(), toOffset);
+ // filter out the empty ranges
+ if (thisRange.count() > 0) {
+ finalRanges.add(thisRange);
+ }
+ ranges[i] = OffsetRange.create(range.topicPartition(),
range.fromOffset() + thisRange.count(), range.untilOffset());
+ allocatedPartitionsThisLoop.add(range.partition());
}
}
- return ranges;
+ if (!needSplitToMinPartitions) {
+ LOG.debug("final ranges merged by topic partition {}",
Arrays.toString(mergeRangesByTopicPartition(finalRanges.toArray(new
OffsetRange[0]))));
+ return mergeRangesByTopicPartition(finalRanges.toArray(new
OffsetRange[0]));
+ }
+ finalRanges.sort(SORT_BY_PARTITION);
+ LOG.debug("final ranges {}", Arrays.toString(finalRanges.toArray(new
OffsetRange[0])));
+ return finalRanges.toArray(new OffsetRange[0]);
+ }
+
+ // the number of the result ranges can be less or more than minPartitions
due to rounding
+ public static OffsetRange[] splitRangesToMinPartitions(OffsetRange[]
oldRanges, long minPartitions) {
Review Comment:
Looks like this method is not being used. Can we remove it ?
--
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]