FileSystem View must treat same fileIds present in different partitions as different file-groups and handle pending compaction correctly
This commit is contained in:
committed by
vinoth chandar
parent
363df2c12e
commit
3ae6cb4ed5
@@ -25,6 +25,7 @@ import com.uber.hoodie.avro.model.HoodieCompactionPlan;
|
||||
import com.uber.hoodie.common.model.CompactionOperation;
|
||||
import com.uber.hoodie.common.model.FileSlice;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroupId;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
@@ -145,26 +146,28 @@ public class CompactionAdminClient implements Serializable {
|
||||
*
|
||||
* This operation MUST be executed with compactions and writer turned OFF.
|
||||
*
|
||||
* @param fileId FileId to be unscheduled
|
||||
* @param fgId FileGroupId to be unscheduled
|
||||
* @param skipValidation Skip validation
|
||||
* @param dryRun Dry Run Mode
|
||||
*/
|
||||
public List<RenameOpResult> unscheduleCompactionFileId(String fileId,
|
||||
public List<RenameOpResult> unscheduleCompactionFileId(HoodieFileGroupId fgId,
|
||||
boolean skipValidation, boolean dryRun) throws Exception {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> renameActions =
|
||||
getRenamingActionsForUnschedulingCompactionForFileId(metaClient, fileId, Optional.absent(), skipValidation);
|
||||
getRenamingActionsForUnschedulingCompactionForFileId(metaClient, fgId,
|
||||
Optional.absent(), skipValidation);
|
||||
|
||||
List<RenameOpResult> res = runRenamingOps(metaClient, renameActions, 1, dryRun);
|
||||
|
||||
if (!dryRun && !res.isEmpty() && res.get(0).isExecuted() && res.get(0).isSuccess()) {
|
||||
// Ready to remove this file-Id from compaction request
|
||||
Pair<String, HoodieCompactionOperation> compactionOperationWithInstant =
|
||||
CompactionUtils.getAllPendingCompactionOperations(metaClient).get(fileId);
|
||||
CompactionUtils.getAllPendingCompactionOperations(metaClient).get(fgId);
|
||||
HoodieCompactionPlan plan = CompactionUtils
|
||||
.getCompactionPlan(metaClient, compactionOperationWithInstant.getKey());
|
||||
List<HoodieCompactionOperation> newOps = plan.getOperations().stream()
|
||||
.filter(op -> !op.getFileId().equals(fileId)).collect(Collectors.toList());
|
||||
.filter(op -> (!op.getFileId().equals(fgId.getFileId()))
|
||||
&& (!op.getPartitionPath().equals(fgId.getPartitionPath()))).collect(Collectors.toList());
|
||||
HoodieCompactionPlan newPlan =
|
||||
HoodieCompactionPlan.newBuilder().setOperations(newOps).setExtraMetadata(plan.getExtraMetadata()).build();
|
||||
HoodieInstant inflight = new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION,
|
||||
@@ -465,23 +468,23 @@ public class CompactionAdminClient implements Serializable {
|
||||
* writer (ingestion/compaction) is running.
|
||||
*
|
||||
* @param metaClient Hoodie Table MetaClient
|
||||
* @param fileId FileId to remove compaction
|
||||
* @param fgId FileGroupId to remove compaction
|
||||
* @param fsViewOpt Cached File System View
|
||||
* @param skipValidation Skip Validation
|
||||
* @return list of pairs of log-files (old, new) and for each pair, rename must be done to successfully unschedule
|
||||
* compaction.
|
||||
*/
|
||||
public List<Pair<HoodieLogFile, HoodieLogFile>> getRenamingActionsForUnschedulingCompactionForFileId(
|
||||
HoodieTableMetaClient metaClient, String fileId, Optional<HoodieTableFileSystemView> fsViewOpt,
|
||||
boolean skipValidation) throws IOException {
|
||||
Map<String, Pair<String, HoodieCompactionOperation>> allPendingCompactions =
|
||||
HoodieTableMetaClient metaClient, HoodieFileGroupId fgId,
|
||||
Optional<HoodieTableFileSystemView> fsViewOpt, boolean skipValidation) throws IOException {
|
||||
Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> allPendingCompactions =
|
||||
CompactionUtils.getAllPendingCompactionOperations(metaClient);
|
||||
if (allPendingCompactions.containsKey(fileId)) {
|
||||
Pair<String, HoodieCompactionOperation> opWithInstant = allPendingCompactions.get(fileId);
|
||||
if (allPendingCompactions.containsKey(fgId)) {
|
||||
Pair<String, HoodieCompactionOperation> opWithInstant = allPendingCompactions.get(fgId);
|
||||
return getRenamingActionsForUnschedulingCompactionOperation(metaClient, opWithInstant.getKey(),
|
||||
CompactionOperation.convertFromAvroRecordInstance(opWithInstant.getValue()), fsViewOpt, skipValidation);
|
||||
}
|
||||
throw new HoodieException("FileId " + fileId + " not in pending compaction");
|
||||
throw new HoodieException("FileGroupId " + fgId + " not in pending compaction");
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -118,7 +118,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
baseInstantTime = fileSlice.get().getBaseInstantTime();
|
||||
} else {
|
||||
// This means there is no base data file, start appending to a new log file
|
||||
fileSlice = Optional.of(new FileSlice(baseInstantTime, this.fileId));
|
||||
fileSlice = Optional.of(new FileSlice(partitionPath, baseInstantTime, this.fileId));
|
||||
logger.info("New InsertHandle for partition :" + partitionPath);
|
||||
}
|
||||
writeStatus.getStat().setPrevCommit(baseInstantTime);
|
||||
|
||||
@@ -21,6 +21,7 @@ import com.uber.hoodie.common.model.FileSlice;
|
||||
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroup;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroupId;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
@@ -52,7 +53,7 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
|
||||
private final TableFileSystemView fileSystemView;
|
||||
private final HoodieTimeline commitTimeline;
|
||||
private final Map<String, CompactionOperation> fileIdToPendingCompactionOperations;
|
||||
private final Map<HoodieFileGroupId, CompactionOperation> fgIdToPendingCompactionOperations;
|
||||
private HoodieTable<T> hoodieTable;
|
||||
private HoodieWriteConfig config;
|
||||
|
||||
@@ -61,8 +62,8 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
this.fileSystemView = hoodieTable.getCompletedFileSystemView();
|
||||
this.commitTimeline = hoodieTable.getCompletedCommitTimeline();
|
||||
this.config = config;
|
||||
this.fileIdToPendingCompactionOperations =
|
||||
((HoodieTableFileSystemView)hoodieTable.getRTFileSystemView()).getFileIdToPendingCompaction().entrySet()
|
||||
this.fgIdToPendingCompactionOperations =
|
||||
((HoodieTableFileSystemView)hoodieTable.getRTFileSystemView()).getFgIdToPendingCompaction().entrySet()
|
||||
.stream().map(entry -> Pair.of(entry.getKey(), entry.getValue().getValue()))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
}
|
||||
@@ -249,7 +250,7 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
* @return true if file slice needs to be preserved, false otherwise.
|
||||
*/
|
||||
private boolean isFileSliceNeededForPendingCompaction(FileSlice fileSlice) {
|
||||
CompactionOperation op = fileIdToPendingCompactionOperations.get(fileSlice.getFileId());
|
||||
CompactionOperation op = fgIdToPendingCompactionOperations.get(fileSlice.getFileGroupId());
|
||||
if (null != op) {
|
||||
// If file slice's instant time is newer or same as that of operation, do not clean
|
||||
return HoodieTimeline.compareTimestamps(fileSlice.getBaseInstantTime(), op.getBaseInstantTime(),
|
||||
|
||||
@@ -18,6 +18,7 @@ package com.uber.hoodie.io.compact;
|
||||
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.avro.model.HoodieCompactionPlan;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroupId;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import java.io.IOException;
|
||||
@@ -38,12 +39,13 @@ public interface HoodieCompactor extends Serializable {
|
||||
* @param hoodieTable Hoodie Table
|
||||
* @param config Hoodie Write Configuration
|
||||
* @param compactionCommitTime scheduled compaction commit time
|
||||
* @param fgIdsInPendingCompactions partition-fileId pairs for which compaction is pending
|
||||
* @return Compaction Plan
|
||||
* @throws IOException when encountering errors
|
||||
*/
|
||||
HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc,
|
||||
HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime,
|
||||
Set<String> fileIdsWithPendingCompactions)
|
||||
Set<HoodieFileGroupId> fgIdsInPendingCompactions)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
|
||||
@@ -26,6 +26,7 @@ import com.uber.hoodie.avro.model.HoodieCompactionOperation;
|
||||
import com.uber.hoodie.avro.model.HoodieCompactionPlan;
|
||||
import com.uber.hoodie.common.model.CompactionOperation;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroupId;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.model.HoodieWriteStat.RuntimeStats;
|
||||
@@ -160,7 +161,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
@Override
|
||||
public HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc,
|
||||
HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime,
|
||||
Set<String> fileIdsWithPendingCompactions) throws IOException {
|
||||
Set<HoodieFileGroupId> fgIdsInPendingCompactions) throws IOException {
|
||||
|
||||
totalLogFiles = new LongAccumulator();
|
||||
totalFileSlices = new LongAccumulator();
|
||||
@@ -190,7 +191,8 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
jsc.parallelize(partitionPaths, partitionPaths.size())
|
||||
.flatMap((FlatMapFunction<String, CompactionOperation>) partitionPath -> fileSystemView
|
||||
.getLatestFileSlices(partitionPath)
|
||||
.filter(slice -> !fileIdsWithPendingCompactions.contains(slice.getFileId()))
|
||||
.filter(slice ->
|
||||
!fgIdsInPendingCompactions.contains(slice.getFileGroupId()))
|
||||
.map(
|
||||
s -> {
|
||||
List<HoodieLogFile> logFiles = s.getLogFiles().sorted(HoodieLogFile
|
||||
@@ -215,11 +217,11 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
// compactions only
|
||||
HoodieCompactionPlan compactionPlan = config.getCompactionStrategy().generateCompactionPlan(config, operations,
|
||||
CompactionUtils.getAllPendingCompactionPlans(metaClient).stream().map(Pair::getValue).collect(toList()));
|
||||
Preconditions.checkArgument(compactionPlan.getOperations().stream()
|
||||
.filter(op -> fileIdsWithPendingCompactions.contains(op.getFileId())).count() == 0,
|
||||
Preconditions.checkArgument(compactionPlan.getOperations().stream().noneMatch(
|
||||
op -> fgIdsInPendingCompactions.contains(new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()))),
|
||||
"Bad Compaction Plan. FileId MUST NOT have multiple pending compactions. "
|
||||
+ "Please fix your strategy implementation."
|
||||
+ "FileIdsWithPendingCompactions :" + fileIdsWithPendingCompactions
|
||||
+ "FileIdsWithPendingCompactions :" + fgIdsInPendingCompactions
|
||||
+ ", Selected workload :" + compactionPlan);
|
||||
if (compactionPlan.getOperations().isEmpty()) {
|
||||
log.warn("After filtering, Nothing to compact for " + metaClient.getBasePath());
|
||||
|
||||
@@ -147,7 +147,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
try {
|
||||
return compactor.generateCompactionPlan(jsc, this, config, instantTime,
|
||||
new HashSet<>(((HoodieTableFileSystemView)getRTFileSystemView())
|
||||
.getFileIdToPendingCompaction().keySet()));
|
||||
.getFgIdToPendingCompaction().keySet()));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user