You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/07/06 12:55:27 UTC

[GitHub] [hudi] garyli1019 commented on a diff in pull request #5677: [HUDI-4152] Flink offline compaction support compacting multi compaction plan at once

garyli1019 commented on code in PR #5677:
URL: https://github.com/apache/hudi/pull/5677#discussion_r914786140


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java:
##########
@@ -218,74 +228,104 @@ 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

Review Comment:
   empty plan need some special handling, like commit an empty meta. otherwise it will stay there forever.



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java:
##########
@@ -218,74 +228,104 @@ 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;
+        }
       }
 
       // get compactionParallelism.
       int compactionParallelism = conf.getInteger(FlinkOptions.COMPACTION_TASKS) == -1
-          ? compactionPlan.getOperations().size() : conf.getInteger(FlinkOptions.COMPACTION_TASKS);
+          ? Math.toIntExact(compactionPlans.stream().mapToLong(pair -> pair.getRight().getOperations().size()).sum())

Review Comment:
   is this the total number of file groups?



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java:
##########
@@ -218,74 +228,104 @@ 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;
+        }
       }
 
       // get compactionParallelism.
       int compactionParallelism = conf.getInteger(FlinkOptions.COMPACTION_TASKS) == -1
-          ? compactionPlan.getOperations().size() : conf.getInteger(FlinkOptions.COMPACTION_TASKS);
+          ? Math.toIntExact(compactionPlans.stream().mapToLong(pair -> pair.getRight().getOperations().size()).sum())
+          : conf.getInteger(FlinkOptions.COMPACTION_TASKS);
 
-      LOG.info("Start to compaction for instant " + compactionInstantTime);
+      LOG.info("Start to compaction for instant " + compactionInstantTimes);
 
       // Mark instant as compaction inflight
-      table.getActiveTimeline().transitionCompactionRequestedToInflight(instant);
+      for (HoodieInstant instant : instants) {
+        table.getActiveTimeline().transitionCompactionRequestedToInflight(instant);
+      }
       table.getMetaClient().reloadActiveTimeline();
 
-      env.addSource(new CompactionPlanSourceFunction(compactionPlan, compactionInstantTime))
+      // use side-output to make operations that is in the same plan to be placed in the same stream
+      // keyby() cannot sure that different operations are in the different stream
+      DataStream<CompactionPlanEvent> source = env.addSource(new MultiCompactionPlanSourceFunction(compactionPlans))
           .name("compaction_source")
-          .uid("uid_compaction_source")
-          .rebalance()
+          .uid("uid_compaction_source");
+
+      SingleOutputStreamOperator<Void> operator = source.rebalance()
           .transform("compact_task",
               TypeInformation.of(CompactionCommitEvent.class),
               new ProcessOperator<>(new CompactFunction(conf)))
           .setParallelism(compactionParallelism)
-          .addSink(new CompactionCommitSink(conf))
-          .name("clean_commits")
-          .uid("uid_clean_commits")
+          .process(new ProcessFunction<CompactionCommitEvent, Void>() {
+            @Override
+            public void processElement(CompactionCommitEvent event, ProcessFunction<CompactionCommitEvent, Void>.Context context, Collector<Void> out) {
+              context.output(new OutputTag<>(event.getInstant(), TypeInformation.of(CompactionCommitEvent.class)), event);
+            }
+          })
+          .name("group_by_compaction_plan")
+          .uid("uid_group_by_compaction_plan")
           .setParallelism(1);
 
-      env.execute("flink_hudi_compaction_" + compactionInstantTime);
+      compactionPlans.forEach(pair ->
+          operator.getSideOutput(new OutputTag<>(pair.getLeft(), TypeInformation.of(CompactionCommitEvent.class)))
+              .addSink(new CompactionCommitSink(conf))

Review Comment:
   how could the sink function know when all the compaction instants are completed without having the list of compaction plan?



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org