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);
|
||||
}
|
||||
|
||||
@@ -27,6 +27,7 @@ import com.uber.hoodie.common.HoodieClientTestUtils;
|
||||
import com.uber.hoodie.common.HoodieTestDataGenerator;
|
||||
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.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
@@ -398,13 +399,13 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
**/
|
||||
|
||||
private void validateDeltaCommit(String latestDeltaCommit,
|
||||
final Map<String, Pair<String, HoodieCompactionOperation>> fileIdToCompactionOperation,
|
||||
final Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> fgIdToCompactionOperation,
|
||||
HoodieWriteConfig cfg) throws IOException {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
List<FileSlice> fileSliceList = getCurrentLatestFileSlices(table, cfg);
|
||||
fileSliceList.forEach(fileSlice -> {
|
||||
Pair<String, HoodieCompactionOperation> opPair = fileIdToCompactionOperation.get(fileSlice.getFileId());
|
||||
Pair<String, HoodieCompactionOperation> opPair = fgIdToCompactionOperation.get(fileSlice.getFileGroupId());
|
||||
if (opPair != null) {
|
||||
System.out.println("FileSlice :" + fileSlice);
|
||||
assertTrue("Expect baseInstant to match compaction Instant",
|
||||
@@ -430,7 +431,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
pendingCompactions.stream().map(pc -> pc.getKey().getTimestamp()).sorted().collect(Collectors.toList());
|
||||
assertEquals(expPendingCompactionInstants, gotPendingCompactionInstants);
|
||||
|
||||
Map<String, Pair<String, HoodieCompactionOperation>> fileIdToCompactionOperation =
|
||||
Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> fgIdToCompactionOperation =
|
||||
CompactionUtils.getAllPendingCompactionOperations(metaClient);
|
||||
|
||||
if (insertFirst) {
|
||||
@@ -451,7 +452,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
List<HoodieDataFile> dataFilesToRead = getCurrentLatestDataFiles(hoodieTable, cfg);
|
||||
assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit",
|
||||
dataFilesToRead.stream().findAny().isPresent());
|
||||
validateDeltaCommit(firstInstant, fileIdToCompactionOperation, cfg);
|
||||
validateDeltaCommit(firstInstant, fgIdToCompactionOperation, cfg);
|
||||
}
|
||||
|
||||
int numRecords = records.size();
|
||||
@@ -459,7 +460,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
records = dataGen.generateUpdates(instantTime, numRecords);
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
createNextDeltaCommit(instantTime, records, client, metaClient, cfg, false);
|
||||
validateDeltaCommit(instantTime, fileIdToCompactionOperation, cfg);
|
||||
validateDeltaCommit(instantTime, fgIdToCompactionOperation, cfg);
|
||||
}
|
||||
return records;
|
||||
}
|
||||
|
||||
@@ -33,6 +33,7 @@ import com.uber.hoodie.common.model.HoodieCleaningPolicy;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
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.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
@@ -204,16 +205,14 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
|
||||
insertFirstBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn);
|
||||
|
||||
Map<String, String> selectedFileIdForCompaction = new HashMap<>();
|
||||
Map<String, FileSlice> compactionFileIdToLatestFileSlice = new HashMap<>();
|
||||
Map<HoodieFileGroupId, FileSlice> compactionFileIdToLatestFileSlice = new HashMap<>();
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metadata, getConfig(), jsc);
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
TableFileSystemView fsView = table.getFileSystemView();
|
||||
Optional<Boolean> added = fsView.getAllFileGroups(partitionPath).findFirst()
|
||||
.map(fg -> {
|
||||
selectedFileIdForCompaction.put(fg.getId(), partitionPath);
|
||||
fg.getLatestFileSlice().map(fs -> compactionFileIdToLatestFileSlice.put(fg.getId(), fs));
|
||||
fg.getLatestFileSlice().map(fs -> compactionFileIdToLatestFileSlice.put(fg.getFileGroupId(), fs));
|
||||
return true;
|
||||
});
|
||||
if (added.isPresent()) {
|
||||
@@ -224,7 +223,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
|
||||
// Create workload with selected file-slices
|
||||
List<Pair<String, FileSlice>> partitionFileSlicePairs = compactionFileIdToLatestFileSlice.entrySet().stream()
|
||||
.map(e -> Pair.of(selectedFileIdForCompaction.get(e.getKey()), e.getValue())).collect(Collectors.toList());
|
||||
.map(e -> Pair.of(e.getKey().getPartitionPath(), e.getValue())).collect(Collectors.toList());
|
||||
HoodieCompactionPlan compactionPlan =
|
||||
CompactionUtils.buildFromFileSlices(partitionFileSlicePairs, Optional.empty(), Optional.empty());
|
||||
List<String> instantTimes = HoodieTestUtils.monotonicIncreasingCommitTimestamps(9, 1);
|
||||
@@ -270,18 +269,18 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
List<HoodieFileGroup> fileGroups = fsView.getAllFileGroups(partitionPath).collect(Collectors.toList());
|
||||
|
||||
for (HoodieFileGroup fileGroup : fileGroups) {
|
||||
if (selectedFileIdForCompaction.containsKey(fileGroup.getId())) {
|
||||
if (compactionFileIdToLatestFileSlice.containsKey(fileGroup.getFileGroupId())) {
|
||||
// Ensure latest file-slice selected for compaction is retained
|
||||
Optional<HoodieDataFile> dataFileForCompactionPresent =
|
||||
fileGroup.getAllDataFiles().filter(df -> {
|
||||
return compactionFileIdToLatestFileSlice.get(fileGroup.getId())
|
||||
return compactionFileIdToLatestFileSlice.get(fileGroup.getFileGroupId())
|
||||
.getBaseInstantTime().equals(df.getCommitTime());
|
||||
}).findAny();
|
||||
Assert.assertTrue("Data File selected for compaction is retained",
|
||||
dataFileForCompactionPresent.isPresent());
|
||||
} else {
|
||||
// file has no more than max versions
|
||||
String fileId = fileGroup.getId();
|
||||
String fileId = fileGroup.getFileGroupId().getFileId();
|
||||
List<HoodieDataFile> dataFiles = fileGroup.getAllDataFiles().collect(Collectors.toList());
|
||||
|
||||
assertTrue("fileId " + fileId + " has more than " + maxVersions + " versions",
|
||||
|
||||
@@ -334,7 +334,7 @@ public class TestCompactionAdminClient extends TestHoodieClientBase {
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
|
||||
// Call the main unschedule API
|
||||
client.unscheduleCompactionFileId(op.getFileId(), false, false);
|
||||
client.unscheduleCompactionFileId(op.getFileGroupId(), false, false);
|
||||
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
final HoodieTableFileSystemView newFsView =
|
||||
|
||||
Reference in New Issue
Block a user