lanyuanxiaoyao commented on code in PR #5677:
URL: https://github.com/apache/hudi/pull/5677#discussion_r914710865
##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java:
##########
@@ -218,74 +228,112 @@ private void compact() throws Exception {
}
// fetch the instant based on the configured execution sequence
- HoodieTimeline timeline =
table.getActiveTimeline().filterPendingCompactionTimeline();
- Option<HoodieInstant> requested =
CompactionUtil.isLIFO(cfg.compactionSeq) ? timeline.lastInstant() :
timeline.firstInstant();
- if (!requested.isPresent()) {
+ HoodieTimeline timeline = table.getActiveTimeline();
+ List<HoodieInstant> requested = ((CompactionPlanSelectStrategy)
ReflectionUtils.loadClass(cfg.compactionPlanSelectStrategy))
+ .select(timeline.filterPendingCompactionTimeline(), cfg);
+ if (requested.isEmpty()) {
// do nothing.
LOG.info("No compaction plan scheduled, turns on the compaction plan
schedule with --schedule option");
return;
}
- String compactionInstantTime = requested.get().getTimestamp();
-
- HoodieInstant inflightInstant =
HoodieTimeline.getCompactionInflightInstant(compactionInstantTime);
- if (timeline.containsInstant(inflightInstant)) {
- LOG.info("Rollback inflight compaction instant: [" +
compactionInstantTime + "]");
- table.rollbackInflightCompaction(inflightInstant);
- table.getMetaClient().reloadActiveTimeline();
- }
+ List<String> compactionInstantTimes =
requested.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList());
+ compactionInstantTimes.forEach(timestamp -> {
+ HoodieInstant inflightInstant =
HoodieTimeline.getCompactionInflightInstant(timestamp);
+ if (timeline.containsInstant(inflightInstant)) {
+ LOG.info("Rollback inflight compaction instant: [" + timestamp +
"]");
+ table.rollbackInflightCompaction(inflightInstant);
+ table.getMetaClient().reloadActiveTimeline();
+ }
+ });
- // generate compaction plan
+ // generate timestamp and compaction plan pair
// should support configurable commit metadata
- HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan(
- table.getMetaClient(), compactionInstantTime);
-
- if (compactionPlan == null || (compactionPlan.getOperations() == null)
- || (compactionPlan.getOperations().isEmpty())) {
+ List<Pair<String, HoodieCompactionPlan>> compactionPlans =
compactionInstantTimes.stream()
+ .map(timestamp -> {
+ try {
+ return Pair.of(timestamp,
CompactionUtils.getCompactionPlan(table.getMetaClient(), timestamp));
+ } catch (IOException e) {
+ throw new HoodieException(e);
+ }
+ })
+ // reject empty compaction plan
+ .filter(pair -> !(pair.getRight() == null
+ || pair.getRight().getOperations() == null
+ || pair.getRight().getOperations().isEmpty()))
+ .collect(Collectors.toList());
+
+ if (compactionPlans.isEmpty()) {
// No compaction plan, do nothing and return.
- LOG.info("No compaction plan for instant " + compactionInstantTime);
+ LOG.info("No compaction plan for instant " + String.join(",",
compactionInstantTimes));
return;
}
- HoodieInstant instant =
HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime);
+ List<HoodieInstant> instants =
compactionInstantTimes.stream().map(HoodieTimeline::getCompactionRequestedInstant).collect(Collectors.toList());
HoodieTimeline pendingCompactionTimeline =
table.getActiveTimeline().filterPendingCompactionTimeline();
- if (!pendingCompactionTimeline.containsInstant(instant)) {
- // this means that the compaction plan was written to auxiliary
path(.tmp)
- // but not the meta path(.hoodie), this usually happens when the job
crush
- // exceptionally.
-
- // clean the compaction plan in auxiliary path and cancels the
compaction.
-
- LOG.warn("The compaction plan was fetched through the auxiliary
path(.tmp) but not the meta path(.hoodie).\n"
- + "Clean the compaction plan in auxiliary path and cancels the
compaction");
- CompactionUtil.cleanInstant(table.getMetaClient(), instant);
- return;
+ for (HoodieInstant instant : instants) {
+ if (!pendingCompactionTimeline.containsInstant(instant)) {
+ // this means that the compaction plan was written to auxiliary
path(.tmp)
+ // but not the meta path(.hoodie), this usually happens when the job
crush
+ // exceptionally.
+ // clean the compaction plan in auxiliary path and cancels the
compaction.
+ LOG.warn("The compaction plan was fetched through the auxiliary
path(.tmp) but not the meta path(.hoodie).\n"
+ + "Clean the compaction plan in auxiliary path and cancels the
compaction");
+ CompactionUtil.cleanInstant(table.getMetaClient(), instant);
+ return;
Review Comment:
I think that you are right. I don't know much about the operation here, so I
kept the original logic. If you can confirm that removing this 'return' has no
effect, I can 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]