Skip to content

Commit e501b62

Browse files
arpit09Arpit Varshney
and
Arpit Varshney
authored
[GOBBLIN-2118]Reduce no of network calls while fetching kafka offsets during startup (apache#4009)
* Reduce no of network calls to kafka * Add UT * Added UTs * Refactoring code * Removing whitespaces * Removing whitespaces * Removing whitespaces * Addressed review comments * Changing contains to add for the failedOffsetsGetList * Added null check --------- Co-authored-by: Arpit Varshney <[email protected]>
1 parent a55c89c commit e501b62

File tree

3 files changed

+143
-32
lines changed

3 files changed

+143
-32
lines changed

gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/kafka/client/GobblinKafkaConsumerClient.java

+19
Original file line numberDiff line numberDiff line change
@@ -66,6 +66,25 @@ public interface GobblinKafkaConsumerClient extends Closeable {
6666
*/
6767
public long getEarliestOffset(KafkaPartition partition) throws KafkaOffsetRetrievalFailureException;
6868

69+
/**
70+
* Get the earliest available offset for a {@link Collection} of {@link KafkaPartition}s. NOTE: The default implementation
71+
* is not efficient i.e. it will make a getEarliest() call for every {@link KafkaPartition}. Individual implementations
72+
* of {@link GobblinKafkaConsumerClient} should override this method to use more advanced APIs of the underlying KafkaConsumer
73+
* to retrieve the latest offsets for a collection of partitions.
74+
*
75+
* @param partitions for which earliest offset is retrieved
76+
*
77+
* @throws KafkaOffsetRetrievalFailureException - If the underlying kafka-client does not support getting the earliest offset
78+
*/
79+
public default Map<KafkaPartition, Long> getEarliestOffsets(final Collection<KafkaPartition> partitions)
80+
throws KafkaOffsetRetrievalFailureException {
81+
final Map<KafkaPartition, Long> offsetMap = Maps.newHashMap();
82+
for (final KafkaPartition partition : partitions) {
83+
offsetMap.put(partition, getEarliestOffset(partition));
84+
}
85+
return offsetMap;
86+
}
87+
6988
/**
7089
* Get the latest available offset for a <code>partition</code>
7190
*

gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/KafkaSource.java

+79-32
Original file line numberDiff line numberDiff line change
@@ -236,9 +236,9 @@ public List<WorkUnit> getWorkunitsForFilteredPartitions(SourceState state,
236236
try {
237237
Config config = ConfigUtils.propertiesToConfig(state.getProperties());
238238
GobblinKafkaConsumerClientFactory kafkaConsumerClientFactory = kafkaConsumerClientResolver
239-
.resolveClass(
240-
state.getProp(GOBBLIN_KAFKA_CONSUMER_CLIENT_FACTORY_CLASS,
241-
DEFAULT_GOBBLIN_KAFKA_CONSUMER_CLIENT_FACTORY_CLASS)).newInstance();
239+
.resolveClass(state.getProp(
240+
GOBBLIN_KAFKA_CONSUMER_CLIENT_FACTORY_CLASS,
241+
DEFAULT_GOBBLIN_KAFKA_CONSUMER_CLIENT_FACTORY_CLASS)).newInstance();
242242

243243
this.kafkaConsumerClient.set(kafkaConsumerClientFactory.create(config));
244244

@@ -440,29 +440,35 @@ private int calculateNumMappersForPacker(SourceState state,
440440
/*
441441
* This function need to be thread safe since it is called in the Runnable
442442
*/
443-
private List<WorkUnit> getWorkUnitsForTopic(KafkaTopic topic, SourceState state,
443+
public List<WorkUnit> getWorkUnitsForTopic(KafkaTopic topic, SourceState state,
444444
Optional<State> topicSpecificState, Optional<Set<Integer>> filteredPartitions) {
445445
Timer.Context context = this.metricContext.timer("isTopicQualifiedTimer").time();
446446
boolean topicQualified = isTopicQualified(topic);
447447
context.close();
448448

449-
List<WorkUnit> workUnits = Lists.newArrayList();
450-
List<KafkaPartition> topicPartitions = topic.getPartitions();
451-
for (KafkaPartition partition : topicPartitions) {
452-
if(filteredPartitions.isPresent() && !filteredPartitions.get().contains(partition.getId())) {
453-
continue;
454-
}
455-
WorkUnit workUnit = getWorkUnitForTopicPartition(partition, state, topicSpecificState);
456-
if (workUnit != null) {
457-
// For disqualified topics, for each of its workunits set the high watermark to be the same
458-
// as the low watermark, so that it will be skipped.
459-
if (!topicQualified) {
460-
skipWorkUnit(workUnit);
461-
}
462-
workUnit.setProp(NUM_TOPIC_PARTITIONS, topicPartitions.size());
463-
workUnits.add(workUnit);
464-
}
449+
final List<WorkUnit> workUnits = Lists.newArrayList();
450+
final List<KafkaPartition> topicPartitions = topic.getPartitions();
451+
Map<KafkaPartition, WorkUnit> workUnitMap;
452+
453+
if (filteredPartitions.isPresent()) {
454+
LOG.info("Filtered partitions for topic {} are {}", topic.getName(), filteredPartitions.get());
455+
final List<KafkaPartition> filteredPartitionsToBeProcessed = topicPartitions.stream()
456+
.filter(partition -> filteredPartitions.get().contains(partition.getId()))
457+
.collect(Collectors.toList());
458+
workUnitMap = getWorkUnits(filteredPartitionsToBeProcessed, state, topicSpecificState);
459+
} else {
460+
workUnitMap = getWorkUnits(topicPartitions, state, topicSpecificState);
461+
}
462+
463+
if (!topicQualified) {
464+
workUnitMap.values().forEach(KafkaSource::skipWorkUnit);
465+
}
466+
467+
for (WorkUnit workUnit : workUnitMap.values()) {
468+
workUnit.setProp(NUM_TOPIC_PARTITIONS, topicPartitions.size());
469+
workUnits.add(workUnit);
465470
}
471+
466472
this.partitionsToBeProcessed.addAll(topic.getPartitions());
467473
return workUnits;
468474
}
@@ -482,20 +488,61 @@ private static void skipWorkUnit(WorkUnit workUnit) {
482488
workUnit.setProp(ConfigurationKeys.WORK_UNIT_HIGH_WATER_MARK_KEY, workUnit.getLowWaterMark());
483489
}
484490

485-
private WorkUnit getWorkUnitForTopicPartition(KafkaPartition partition, SourceState state,
491+
/**
492+
* Get the workunits of all the partitions passed, this method fetches all the offsets for the partitions
493+
* at once from kafka, and for each partiton creates a workunit.
494+
* @param partitions
495+
* @param state
496+
* @param topicSpecificState
497+
* @return
498+
*/
499+
private Map<KafkaPartition, WorkUnit> getWorkUnits(Collection<KafkaPartition> partitions, SourceState state,
486500
Optional<State> topicSpecificState) {
487-
Offsets offsets = new Offsets();
488-
489-
boolean failedToGetKafkaOffsets = false;
490-
491-
try (Timer.Context context = this.metricContext.timer(OFFSET_FETCH_TIMER).time()) {
492-
offsets.setOffsetFetchEpochTime(System.currentTimeMillis());
493-
offsets.setEarliestOffset(this.kafkaConsumerClient.get().getEarliestOffset(partition));
494-
offsets.setLatestOffset(this.kafkaConsumerClient.get().getLatestOffset(partition));
495-
} catch (Throwable t) {
496-
failedToGetKafkaOffsets = true;
497-
LOG.error("Caught error in creating work unit for {}", partition, t);
501+
final Map<KafkaPartition, Offsets> partitionOffsetMap = Maps.newHashMap();
502+
final Set<KafkaPartition> failedOffsetsGetList = Sets.newHashSet();
503+
try (final Timer.Context context = this.metricContext.timer(OFFSET_FETCH_TIMER).time()) {
504+
// Fetch the offsets for all the partitions at once
505+
final Map<KafkaPartition, Long> earliestOffsetMap = this.kafkaConsumerClient.get().getEarliestOffsets(partitions);
506+
final Map<KafkaPartition, Long> latestOffsetMap = this.kafkaConsumerClient.get().getLatestOffsets(partitions);
507+
for (KafkaPartition partition : partitions) {
508+
final Offsets offsets = new Offsets();
509+
offsets.setOffsetFetchEpochTime(System.currentTimeMillis());
510+
// Check if both earliest and latest offset are fetched for the partition, then set the offsets
511+
if (earliestOffsetMap.containsKey(partition) && latestOffsetMap.containsKey(partition)) {
512+
offsets.setEarliestOffset(earliestOffsetMap.get(partition));
513+
offsets.setLatestOffset(latestOffsetMap.get(partition));
514+
offsets.setOffsetFetchEpochTime(System.currentTimeMillis());
515+
partitionOffsetMap.put(partition, offsets);
516+
// If either is not available, put it in the failed offsets list
517+
} else {
518+
failedOffsetsGetList.add(partition);
519+
}
520+
}
521+
LOG.info("Time taken to fetch offset for partitions {} is {} ms", partitions,
522+
TimeUnit.NANOSECONDS.toMillis(context.stop()));
523+
} catch (KafkaOffsetRetrievalFailureException e) {
524+
// When exception occurred while fetching earliest or latest offset for all the partitions,
525+
// add all the partitions to fetchOffsetsFailedPartitions
526+
failedOffsetsGetList.addAll(partitions);
527+
LOG.error("Caught error in creating work unit for {}", partitions, e);
498528
}
529+
if (!failedOffsetsGetList.isEmpty()) {
530+
LOG.error("Failed to fetch offsets for partitions {}", failedOffsetsGetList);
531+
}
532+
final Map<KafkaPartition, WorkUnit> workUnitMap = Maps.newHashMap();
533+
for (Map.Entry<KafkaPartition, Offsets> partitionOffset : partitionOffsetMap.entrySet()) {
534+
WorkUnit workUnit =
535+
getWorkUnitForTopicPartition(partitionOffset.getKey(), state, topicSpecificState, partitionOffset.getValue(),
536+
failedOffsetsGetList.contains(partitionOffset.getKey()));
537+
if (workUnit != null) {
538+
workUnitMap.put(partitionOffset.getKey(), workUnit);
539+
}
540+
}
541+
return workUnitMap;
542+
}
543+
544+
private WorkUnit getWorkUnitForTopicPartition(KafkaPartition partition, SourceState state,
545+
Optional<State> topicSpecificState, Offsets offsets, boolean failedToGetKafkaOffsets) {
499546

500547
long previousOffset = 0;
501548
long previousOffsetFetchEpochTime = 0;

gobblin-modules/gobblin-kafka-common/src/test/java/org/apache/gobblin/source/extractor/extract/kafka/KafkaSourceTest.java

+45
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@
2222
import java.io.IOException;
2323
import java.util.ArrayList;
2424
import java.util.Arrays;
25+
import java.util.Collection;
2526
import java.util.HashMap;
2627
import java.util.Iterator;
2728
import java.util.LinkedList;
@@ -71,6 +72,40 @@ public void testGetWorkunits() {
7172

7273
}
7374

75+
@Test
76+
public void testGetWorkunitsForTopic() {
77+
TestKafkaClient testKafkaClient = new TestKafkaClient();
78+
testKafkaClient.testTopics = testTopics;
79+
SourceState state = new SourceState();
80+
state.setProp(ConfigurationKeys.WRITER_OUTPUT_DIR, "TestPath");
81+
state.setProp(KafkaWorkUnitPacker.KAFKA_WORKUNIT_PACKER_TYPE, KafkaWorkUnitPacker.PackerType.CUSTOM);
82+
state.setProp(KafkaWorkUnitPacker.KAFKA_WORKUNIT_PACKER_CUSTOMIZED_TYPE,
83+
"org.apache.gobblin.source.extractor.extract.kafka.workunit.packer.KafkaTopicGroupingWorkUnitPacker");
84+
state.setProp(GOBBLIN_KAFKA_CONSUMER_CLIENT_FACTORY_CLASS, "MockTestKafkaConsumerClientFactory");
85+
List<KafkaTopic> kafkaTopicList = toKafkaTopicList(testTopics);
86+
TestKafkaSource testKafkaSource = new TestKafkaSource(testKafkaClient);
87+
88+
List<String> allTopics = testTopics;
89+
Map<String, List<Integer>> filteredTopicPartitionMap = new HashMap<>();
90+
filteredTopicPartitionMap.put(allTopics.get(0), new LinkedList<>());
91+
filteredTopicPartitionMap.put(allTopics.get(1), new LinkedList<>());
92+
filteredTopicPartitionMap.put(allTopics.get(2), new LinkedList<>());
93+
filteredTopicPartitionMap.get(allTopics.get(0)).addAll(Arrays.asList(0, 11));
94+
filteredTopicPartitionMap.get(allTopics.get(1)).addAll(Arrays.asList(2, 8, 10));
95+
filteredTopicPartitionMap.get(allTopics.get(2)).addAll(Arrays.asList(1, 3, 5, 7));
96+
testKafkaSource.getWorkunitsForFilteredPartitions(state, Optional.of(filteredTopicPartitionMap), Optional.of(3));
97+
98+
for (KafkaTopic topic : kafkaTopicList) {
99+
List<WorkUnit> rawWorkunitList =
100+
testKafkaSource.getWorkUnitsForTopic(topic, state, Optional.absent(), Optional.absent());
101+
// verify if the latest offset has been taken via testKafkaClient.getLatestOffsets()
102+
for (WorkUnit workUnit : rawWorkunitList) {
103+
Assert.assertEquals(
104+
Integer.valueOf(workUnit.getProp(ConfigurationKeys.WORK_UNIT_HIGH_WATER_MARK_KEY)).intValue(), 20);
105+
}
106+
}
107+
}
108+
74109
@Test
75110
public void testGetWorkunitsForFilteredPartitions() {
76111
TestKafkaClient testKafkaClient = new TestKafkaClient();
@@ -205,6 +240,16 @@ public List<KafkaTopic> getFilteredTopics(List<Pattern> blacklist, List<Pattern>
205240
return toKafkaTopicList(DatasetFilterUtils.filter(testTopics, blacklist, whitelist));
206241
}
207242

243+
@Override
244+
public Map<KafkaPartition, Long> getEarliestOffsets(Collection<KafkaPartition> partitions) {
245+
return partitions.stream().collect(Collectors.toMap(p -> p, p -> 10L));
246+
}
247+
248+
@Override
249+
public Map<KafkaPartition, Long> getLatestOffsets(Collection<KafkaPartition> partitions) {
250+
return partitions.stream().collect(Collectors.toMap(p -> p, p -> 20L));
251+
}
252+
208253
@Override
209254
public long getEarliestOffset(KafkaPartition partition)
210255
throws KafkaOffsetRetrievalFailureException {

0 commit comments

Comments
 (0)