dtspslhh commented on code in PR #3798:
URL: https://github.com/apache/gobblin/pull/3798#discussion_r1372313166
##########
gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/KafkaSource.java:
##########
@@ -257,32 +277,42 @@ public String apply(KafkaTopic topic) {
Stopwatch createWorkUnitStopwatch = Stopwatch.createStarted();
for (KafkaTopic topic : topics) {
+ LOG.info("Discovered topic " + topic);
+ if (topic.getTopicSpecificState().isPresent()) {
+ topicSpecificStateMap.computeIfAbsent(topic.getName(), k -> new
State())
+ .addAllIfNotExist(topic.getTopicSpecificState().get());
+ }
+ Optional<Set<Integer>> partitionIDSet = Optional.absent();
+ if(filteredTopicPartition.isPresent()) {
+ List<Integer> list =
java.util.Optional.ofNullable(filteredTopicPartitionMap.get(topic.getName()))
+ .orElse(new ArrayList<>());
+ partitionIDSet = Optional.of(new HashSet<>(list));
+ LOG.info("Compute the workunit for topic {} with num of filtered
partitions: {}",
+ topic.getName(), list.size());
+ }
+
threadPool.submit(
new WorkUnitCreator(topic, state,
Optional.fromNullable(topicSpecificStateMap.get(topic.getName())),
- workUnits));
+ kafkaTopicWorkunitMap, partitionIDSet));
}
ExecutorsUtils.shutdownExecutorService(threadPool, Optional.of(LOG), 1L,
TimeUnit.HOURS);
- LOG.info(String.format("Created workunits for %d topics in %d seconds",
workUnits.size(),
+ LOG.info(String.format("Created workunits for %d topics in %d seconds",
kafkaTopicWorkunitMap.size(),
createWorkUnitStopwatch.elapsed(TimeUnit.SECONDS)));
// Create empty WorkUnits for skipped partitions (i.e., partitions that
have previous offsets,
// but aren't processed).
- createEmptyWorkUnitsForSkippedPartitions(workUnits,
topicSpecificStateMap, state);
- //determine the number of mappers
- int maxMapperNum =
- state.getPropAsInt(ConfigurationKeys.MR_JOB_MAX_MAPPERS_KEY,
ConfigurationKeys.DEFAULT_MR_JOB_MAX_MAPPERS);
+ createEmptyWorkUnitsForSkippedPartitions(kafkaTopicWorkunitMap,
topicSpecificStateMap, state);
+
KafkaWorkUnitPacker kafkaWorkUnitPacker =
KafkaWorkUnitPacker.getInstance(this, state, Optional.of(this.metricContext));
- int numOfMultiWorkunits = maxMapperNum;
- if(state.contains(ConfigurationKeys.MR_TARGET_MAPPER_SIZE)) {
- double totalEstDataSize =
kafkaWorkUnitPacker.setWorkUnitEstSizes(workUnits);
- LOG.info(String.format("The total estimated data size is %.2f",
totalEstDataSize));
- double targetMapperSize =
state.getPropAsDouble(ConfigurationKeys.MR_TARGET_MAPPER_SIZE);
- numOfMultiWorkunits = (int) (totalEstDataSize / targetMapperSize) + 1;
- numOfMultiWorkunits = Math.min(numOfMultiWorkunits, maxMapperNum);
+ int numOfMultiWorkunits = minContainer.or(0);
+ if(state.contains(ConfigurationKeys.MR_JOB_MAX_MAPPERS_KEY)) {
+ numOfMultiWorkunits = Math.max(numOfMultiWorkunits,
+ calculateNumMappersForPacker(state, kafkaWorkUnitPacker,
kafkaTopicWorkunitMap));
}
- addTopicSpecificPropsToWorkUnits(workUnits, topicSpecificStateMap);
- List<WorkUnit> workUnitList = kafkaWorkUnitPacker.pack(workUnits,
numOfMultiWorkunits);
+
+ addTopicSpecificPropsToWorkUnits(kafkaTopicWorkunitMap,
topicSpecificStateMap);
+ List<WorkUnit> workUnitList =
kafkaWorkUnitPacker.pack(kafkaTopicWorkunitMap, numOfMultiWorkunits);
Review Comment:
the design is to able to handle multiple topics.
My idea is in
[KafkaTopicGroupingWorkUnitPacker](https://github.com/apache/gobblin/blob/master/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/workunit/packer/KafkaTopicGroupingWorkUnitPacker.java#L231),
before pack all WU, we construct a priority queue based on WU weight, and keep
splitting MultiWorkUnits, until the minContainer count reaches or can't split
anymore
--
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]