YuweiXiao commented on code in PR #5677:
URL: https://github.com/apache/hudi/pull/5677#discussion_r881602226
##########
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:
Why 'return' here? E.g., if we clean one of the instant, it should be ok for
others to proceed.
##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/InstantCompactionPlanSelectStrategy.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.sink.compact.strategy;
+
+import java.util.Collections;
+import java.util.List;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.sink.compact.FlinkCompactionConfig;
+
+/**
+ * Specify the compaction plan instant to compact
+ */
+public class InstantCompactionPlanSelectStrategy implements
CompactionPlanSelectStrategy {
Review Comment:
maybe we could extend this strategy to multi-instant selection. E.g.,
config="001,002,008" will compact three instant.
--
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]