[HUDI-130] Paths written in compaction plan needs to be relative to base-path
This commit is contained in:
committed by
Balaji Varadarajan
parent
e4c91ed13f
commit
d8be818ac9
@@ -239,8 +239,9 @@ public class CompactionAdminClient extends AbstractHoodieClient {
|
||||
FileSlice merged =
|
||||
fileSystemView.getLatestMergedFileSlicesBeforeOrOn(op.getPartitionPath(), lastInstant.getTimestamp())
|
||||
.filter(fs -> fs.getFileId().equals(op.getFileId())).findFirst().get();
|
||||
final int maxVersion = op.getDeltaFilePaths().stream().map(lf -> FSUtils.getFileVersionFromLog(new Path(lf)))
|
||||
.reduce((x, y) -> x > y ? x : y).orElse(0);
|
||||
final int maxVersion =
|
||||
op.getDeltaFileNames().stream().map(lf -> FSUtils.getFileVersionFromLog(new Path(lf)))
|
||||
.reduce((x, y) -> x > y ? x : y).orElse(0);
|
||||
List<HoodieLogFile> logFilesToBeMoved =
|
||||
merged.getLogFiles().filter(lf -> lf.getLogVersion() > maxVersion).collect(Collectors.toList());
|
||||
return logFilesToBeMoved.stream().map(lf -> {
|
||||
@@ -291,28 +292,34 @@ public class CompactionAdminClient extends AbstractHoodieClient {
|
||||
if (fileSliceOptional.isPresent()) {
|
||||
FileSlice fs = fileSliceOptional.get();
|
||||
Option<HoodieDataFile> df = fs.getDataFile();
|
||||
if (operation.getDataFilePath().isPresent()) {
|
||||
String expPath =
|
||||
metaClient.getFs().getFileStatus(new Path(operation.getDataFilePath().get())).getPath().toString();
|
||||
Preconditions.checkArgument(df.isPresent(),
|
||||
"Data File must be present. File Slice was : " + fs + ", operation :" + operation);
|
||||
if (operation.getDataFileName().isPresent()) {
|
||||
String expPath = metaClient.getFs().getFileStatus(new Path(
|
||||
FSUtils.getPartitionPath(metaClient.getBasePath(), operation.getPartitionPath()),
|
||||
new Path(operation.getDataFileName().get()))).getPath()
|
||||
.toString();
|
||||
Preconditions.checkArgument(df.isPresent(), "Data File must be present. File Slice was : "
|
||||
+ fs + ", operation :" + operation);
|
||||
Preconditions.checkArgument(df.get().getPath().equals(expPath),
|
||||
"Base Path in operation is specified as " + expPath + " but got path " + df.get().getPath());
|
||||
}
|
||||
Set<HoodieLogFile> logFilesInFileSlice = fs.getLogFiles().collect(Collectors.toSet());
|
||||
Set<HoodieLogFile> logFilesInCompactionOp = operation.getDeltaFilePaths().stream().map(dp -> {
|
||||
try {
|
||||
FileStatus[] fileStatuses = metaClient.getFs().listStatus(new Path(dp));
|
||||
Preconditions.checkArgument(fileStatuses.length == 1, "Expect only 1 file-status");
|
||||
return new HoodieLogFile(fileStatuses[0]);
|
||||
} catch (FileNotFoundException fe) {
|
||||
throw new CompactionValidationException(fe.getMessage());
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||
}
|
||||
}).collect(Collectors.toSet());
|
||||
Set<HoodieLogFile> missing = logFilesInCompactionOp.stream().filter(lf -> !logFilesInFileSlice.contains(lf))
|
||||
.collect(Collectors.toSet());
|
||||
Set<HoodieLogFile> logFilesInCompactionOp = operation.getDeltaFileNames().stream()
|
||||
.map(dp -> {
|
||||
try {
|
||||
FileStatus[] fileStatuses = metaClient.getFs().listStatus(
|
||||
new Path(FSUtils.getPartitionPath(metaClient.getBasePath(), operation.getPartitionPath()),
|
||||
new Path(dp)));
|
||||
Preconditions.checkArgument(fileStatuses.length == 1, "Expect only 1 file-status");
|
||||
return new HoodieLogFile(fileStatuses[0]);
|
||||
} catch (FileNotFoundException fe) {
|
||||
throw new CompactionValidationException(fe.getMessage());
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||
}
|
||||
}).collect(Collectors.toSet());
|
||||
Set<HoodieLogFile> missing =
|
||||
logFilesInCompactionOp.stream().filter(lf -> !logFilesInFileSlice.contains(lf))
|
||||
.collect(Collectors.toSet());
|
||||
Preconditions.checkArgument(missing.isEmpty(),
|
||||
"All log files specified in compaction operation is not present. Missing :" + missing + ", Exp :"
|
||||
+ logFilesInCompactionOp + ", Got :" + logFilesInFileSlice);
|
||||
|
||||
@@ -57,6 +57,7 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.util.AvroUtils;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
@@ -1176,13 +1177,14 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
*/
|
||||
private JavaRDD<WriteStatus> runCompaction(HoodieInstant compactionInstant, HoodieActiveTimeline activeTimeline,
|
||||
boolean autoCommit) throws IOException {
|
||||
HoodieCompactionPlan compactionPlan =
|
||||
AvroUtils.deserializeCompactionPlan(activeTimeline.getInstantAuxiliaryDetails(compactionInstant).get());
|
||||
HoodieTableMetaClient metaClient = createMetaClient(true);
|
||||
HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan(metaClient,
|
||||
compactionInstant.getTimestamp());
|
||||
|
||||
// Mark instant as compaction inflight
|
||||
activeTimeline.transitionCompactionRequestedToInflight(compactionInstant);
|
||||
compactionTimer = metrics.getCompactionCtx();
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTableMetaClient metaClient = createMetaClient(true);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
JavaRDD<WriteStatus> statuses = table.compact(jsc, compactionInstant.getTimestamp(), compactionPlan);
|
||||
// Force compaction action
|
||||
|
||||
@@ -32,6 +32,7 @@ import java.util.stream.Collectors;
|
||||
import java.util.stream.StreamSupport;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
@@ -98,10 +99,10 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
private List<WriteStatus> compact(HoodieCopyOnWriteTable hoodieCopyOnWriteTable, HoodieTableMetaClient metaClient,
|
||||
HoodieWriteConfig config, CompactionOperation operation, String commitTime) throws IOException {
|
||||
FileSystem fs = metaClient.getFs();
|
||||
Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||
|
||||
log.info("Compacting base " + operation.getDataFilePath() + " with delta files " + operation.getDeltaFilePaths()
|
||||
+ " for commit " + commitTime);
|
||||
Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||
log.info("Compacting base " + operation.getDataFileName() + " with delta files " + operation
|
||||
.getDeltaFileNames() + " for commit " + commitTime);
|
||||
// TODO - FIX THIS
|
||||
// Reads the entire avro file. Always only specific blocks should be read from the avro file
|
||||
// (failure recover).
|
||||
@@ -113,15 +114,21 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION))
|
||||
.filterCompletedInstants().lastInstant().get().getTimestamp();
|
||||
log.info("MaxMemoryPerCompaction => " + config.getMaxMemoryPerCompaction());
|
||||
HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, metaClient.getBasePath(),
|
||||
operation.getDeltaFilePaths(), readerSchema, maxInstantTime, config.getMaxMemoryPerCompaction(),
|
||||
config.getCompactionLazyBlockReadEnabled(), config.getCompactionReverseLogReadEnabled(),
|
||||
config.getMaxDFSStreamBufferSize(), config.getSpillableMapBasePath());
|
||||
|
||||
List<String> logFiles = operation.getDeltaFileNames().stream()
|
||||
.map(p -> new Path(FSUtils.getPartitionPath(metaClient.getBasePath(), operation.getPartitionPath()),
|
||||
p).toString()).collect(toList());
|
||||
HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs,
|
||||
metaClient.getBasePath(), logFiles, readerSchema, maxInstantTime,
|
||||
config.getMaxMemoryPerCompaction(), config.getCompactionLazyBlockReadEnabled(),
|
||||
config.getCompactionReverseLogReadEnabled(), config.getMaxDFSStreamBufferSize(),
|
||||
config.getSpillableMapBasePath());
|
||||
if (!scanner.iterator().hasNext()) {
|
||||
return Lists.<WriteStatus>newArrayList();
|
||||
}
|
||||
|
||||
Option<HoodieDataFile> oldDataFileOpt = operation.getBaseFile();
|
||||
Option<HoodieDataFile> oldDataFileOpt = operation.getBaseFile(metaClient.getBasePath(),
|
||||
operation.getPartitionPath());
|
||||
|
||||
// Compacting is very similar to applying updates to existing file
|
||||
Iterator<List<WriteStatus>> result;
|
||||
@@ -182,22 +189,28 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
|
||||
RealtimeView fileSystemView = hoodieTable.getRTFileSystemView();
|
||||
log.info("Compaction looking for files to compact in " + partitionPaths + " partitions");
|
||||
List<HoodieCompactionOperation> operations = jsc.parallelize(partitionPaths, partitionPaths.size())
|
||||
.flatMap((FlatMapFunction<String, CompactionOperation>) partitionPath -> fileSystemView
|
||||
.getLatestFileSlices(partitionPath)
|
||||
.filter(slice -> !fgIdsInPendingCompactions.contains(slice.getFileGroupId())).map(s -> {
|
||||
List<HoodieLogFile> logFiles =
|
||||
s.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList());
|
||||
totalLogFiles.add((long) logFiles.size());
|
||||
totalFileSlices.add(1L);
|
||||
// Avro generated classes are not inheriting Serializable. Using CompactionOperation POJO
|
||||
// for spark Map operations and collecting them finally in Avro generated classes for storing
|
||||
// into meta files.
|
||||
Option<HoodieDataFile> dataFile = s.getDataFile();
|
||||
return new CompactionOperation(dataFile, partitionPath, logFiles,
|
||||
config.getCompactionStrategy().captureMetrics(config, dataFile, partitionPath, logFiles));
|
||||
}).filter(c -> !c.getDeltaFilePaths().isEmpty()).collect(toList()).iterator())
|
||||
.collect().stream().map(CompactionUtils::buildHoodieCompactionOperation).collect(toList());
|
||||
List<HoodieCompactionOperation> operations =
|
||||
jsc.parallelize(partitionPaths, partitionPaths.size())
|
||||
.flatMap((FlatMapFunction<String, CompactionOperation>) partitionPath -> fileSystemView
|
||||
.getLatestFileSlices(partitionPath)
|
||||
.filter(slice ->
|
||||
!fgIdsInPendingCompactions.contains(slice.getFileGroupId()))
|
||||
.map(
|
||||
s -> {
|
||||
List<HoodieLogFile> logFiles = s.getLogFiles().sorted(HoodieLogFile
|
||||
.getLogFileComparator()).collect(Collectors.toList());
|
||||
totalLogFiles.add((long) logFiles.size());
|
||||
totalFileSlices.add(1L);
|
||||
// Avro generated classes are not inheriting Serializable. Using CompactionOperation POJO
|
||||
// for spark Map operations and collecting them finally in Avro generated classes for storing
|
||||
// into meta files.
|
||||
Option<HoodieDataFile> dataFile = s.getDataFile();
|
||||
return new CompactionOperation(dataFile, partitionPath, logFiles,
|
||||
config.getCompactionStrategy().captureMetrics(config, dataFile, partitionPath, logFiles));
|
||||
})
|
||||
.filter(c -> !c.getDeltaFileNames().isEmpty())
|
||||
.collect(toList()).iterator()).collect().stream().map(CompactionUtils::buildHoodieCompactionOperation)
|
||||
.collect(toList());
|
||||
log.info("Total of " + operations.size() + " compactions are retrieved");
|
||||
log.info("Total number of latest files slices " + totalFileSlices.value());
|
||||
log.info("Total number of log files " + totalLogFiles.value());
|
||||
|
||||
@@ -26,6 +26,7 @@ import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
@@ -94,7 +95,9 @@ public abstract class CompactionStrategy implements Serializable {
|
||||
List<HoodieCompactionOperation> operations, List<HoodieCompactionPlan> pendingCompactionPlans) {
|
||||
// Strategy implementation can overload this method to set specific compactor-id
|
||||
return HoodieCompactionPlan.newBuilder()
|
||||
.setOperations(orderAndFilter(writeConfig, operations, pendingCompactionPlans)).build();
|
||||
.setOperations(orderAndFilter(writeConfig, operations, pendingCompactionPlans))
|
||||
.setVersion(CompactionUtils.LATEST_COMPACTION_METADATA_VERSION)
|
||||
.build();
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
Reference in New Issue
Block a user