[HUDI-2501] Add HoodieData abstraction and refactor compaction actions in hudi-client module (#3741)
This commit is contained in:
@@ -22,7 +22,8 @@ import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
import org.apache.hudi.sink.utils.NonThrownExecutor;
|
||||
import org.apache.hudi.table.action.compact.FlinkCompactHelpers;
|
||||
import org.apache.hudi.table.HoodieFlinkCopyOnWriteTable;
|
||||
import org.apache.hudi.table.action.compact.HoodieFlinkMergeOnReadTableCompactor;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.annotation.VisibleForTesting;
|
||||
@@ -98,7 +99,17 @@ public class CompactFunction extends ProcessFunction<CompactionPlanEvent, Compac
|
||||
}
|
||||
|
||||
private void doCompaction(String instantTime, CompactionOperation compactionOperation, Collector<CompactionCommitEvent> collector) throws IOException {
|
||||
List<WriteStatus> writeStatuses = FlinkCompactHelpers.compact(writeClient, instantTime, compactionOperation);
|
||||
HoodieFlinkMergeOnReadTableCompactor compactor = new HoodieFlinkMergeOnReadTableCompactor();
|
||||
List<WriteStatus> writeStatuses = compactor.compact(
|
||||
new HoodieFlinkCopyOnWriteTable<>(
|
||||
writeClient.getConfig(),
|
||||
writeClient.getEngineContext(),
|
||||
writeClient.getHoodieTable().getMetaClient()),
|
||||
writeClient.getHoodieTable().getMetaClient(),
|
||||
writeClient.getConfig(),
|
||||
compactionOperation,
|
||||
instantTime,
|
||||
writeClient.getHoodieTable().getTaskContextSupplier());
|
||||
collector.collect(new CompactionCommitEvent(instantTime, compactionOperation.getFileId(), writeStatuses, taskID));
|
||||
}
|
||||
|
||||
|
||||
@@ -84,7 +84,7 @@ public class CompactionPlanOperator extends AbstractStreamOperator<CompactionPla
|
||||
public void notifyCheckpointComplete(long checkpointId) {
|
||||
try {
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
CompactionUtil.rollbackCompaction(table, writeClient, conf);
|
||||
CompactionUtil.rollbackCompaction(table, conf);
|
||||
scheduleCompaction(table, checkpointId);
|
||||
} catch (Throwable throwable) {
|
||||
// make it fail-safe
|
||||
|
||||
@@ -99,7 +99,7 @@ public class HoodieFlinkCompactor {
|
||||
HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime);
|
||||
if (timeline.containsInstant(inflightInstant)) {
|
||||
LOG.info("Rollback inflight compaction instant: [" + compactionInstantTime + "]");
|
||||
writeClient.rollbackInflightCompaction(inflightInstant, table);
|
||||
table.rollbackInflightCompaction(inflightInstant);
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
}
|
||||
|
||||
|
||||
@@ -18,7 +18,6 @@
|
||||
|
||||
package org.apache.hudi.util;
|
||||
|
||||
import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.TableSchemaResolver;
|
||||
@@ -111,7 +110,7 @@ public class CompactionUtil {
|
||||
}
|
||||
}
|
||||
|
||||
public static void rollbackCompaction(HoodieFlinkTable<?> table, HoodieFlinkWriteClient writeClient, Configuration conf) {
|
||||
public static void rollbackCompaction(HoodieFlinkTable<?> table, Configuration conf) {
|
||||
String curInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
int deltaSeconds = conf.getInteger(FlinkOptions.COMPACTION_DELTA_SECONDS);
|
||||
HoodieTimeline inflightCompactionTimeline = table.getActiveTimeline()
|
||||
@@ -121,7 +120,7 @@ public class CompactionUtil {
|
||||
&& StreamerUtil.instantTimeDiffSeconds(curInstantTime, instant.getTimestamp()) >= deltaSeconds);
|
||||
inflightCompactionTimeline.getInstants().forEach(inflightInstant -> {
|
||||
LOG.info("Rollback the pending compaction instant: " + inflightInstant);
|
||||
writeClient.rollbackInflightCompaction(inflightInstant, table);
|
||||
table.rollbackInflightCompaction(inflightInstant);
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
});
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user