Reformatting code per Google Code Style all over
This commit is contained in:
committed by
vinoth chandar
parent
5a62480a92
commit
e45679f5e2
@@ -36,13 +36,6 @@ import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieAppendException;
|
||||
import com.uber.hoodie.exception.HoodieUpsertException;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.TaskContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
@@ -50,155 +43,161 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.TaskContext;
|
||||
|
||||
/**
|
||||
* IO Operation to append data onto an existing file.
|
||||
*
|
||||
* @param <T>
|
||||
*/
|
||||
public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
|
||||
private static Logger logger = LogManager.getLogger(HoodieMergeHandle.class);
|
||||
private static AtomicLong recordIndex = new AtomicLong(1);
|
||||
|
||||
private final WriteStatus writeStatus;
|
||||
private final String fileId;
|
||||
private String partitionPath;
|
||||
private List<HoodieRecord<T>> records;
|
||||
private long recordsWritten = 0;
|
||||
private long recordsDeleted = 0;
|
||||
private HoodieLogFile currentLogFile;
|
||||
private Writer writer;
|
||||
private static Logger logger = LogManager.getLogger(HoodieMergeHandle.class);
|
||||
private static AtomicLong recordIndex = new AtomicLong(1);
|
||||
|
||||
public HoodieAppendHandle(HoodieWriteConfig config,
|
||||
String commitTime,
|
||||
HoodieTable<T> hoodieTable,
|
||||
String fileId,
|
||||
Iterator<HoodieRecord<T>> recordItr) {
|
||||
super(config, commitTime, hoodieTable);
|
||||
WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName());
|
||||
writeStatus.setStat(new HoodieDeltaWriteStat());
|
||||
this.writeStatus = writeStatus;
|
||||
this.fileId = fileId;
|
||||
init(recordItr);
|
||||
}
|
||||
private final WriteStatus writeStatus;
|
||||
private final String fileId;
|
||||
private String partitionPath;
|
||||
private List<HoodieRecord<T>> records;
|
||||
private long recordsWritten = 0;
|
||||
private long recordsDeleted = 0;
|
||||
private HoodieLogFile currentLogFile;
|
||||
private Writer writer;
|
||||
|
||||
private void init(Iterator<HoodieRecord<T>> recordItr) {
|
||||
List<HoodieRecord<T>> records = Lists.newArrayList();
|
||||
recordItr.forEachRemaining(record -> {
|
||||
records.add(record);
|
||||
// extract some information from the first record
|
||||
if (partitionPath == null) {
|
||||
partitionPath = record.getPartitionPath();
|
||||
// HACK(vc) This also assumes a base file. It will break, if appending without one.
|
||||
String latestValidFilePath =
|
||||
fileSystemView.getLatestDataFiles(record.getPartitionPath())
|
||||
.filter(dataFile -> dataFile.getFileId().equals(fileId))
|
||||
.findFirst().get().getFileName();
|
||||
String baseCommitTime = FSUtils.getCommitTime(latestValidFilePath);
|
||||
writeStatus.getStat().setPrevCommit(baseCommitTime);
|
||||
writeStatus.setFileId(fileId);
|
||||
writeStatus.setPartitionPath(record.getPartitionPath());
|
||||
writeStatus.getStat().setFileId(fileId);
|
||||
public HoodieAppendHandle(HoodieWriteConfig config,
|
||||
String commitTime,
|
||||
HoodieTable<T> hoodieTable,
|
||||
String fileId,
|
||||
Iterator<HoodieRecord<T>> recordItr) {
|
||||
super(config, commitTime, hoodieTable);
|
||||
WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName());
|
||||
writeStatus.setStat(new HoodieDeltaWriteStat());
|
||||
this.writeStatus = writeStatus;
|
||||
this.fileId = fileId;
|
||||
init(recordItr);
|
||||
}
|
||||
|
||||
try {
|
||||
this.writer = HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath))
|
||||
.withFileId(fileId).overBaseCommit(baseCommitTime)
|
||||
.withFs(fs).withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
||||
this.currentLogFile = writer.getLogFile();
|
||||
((HoodieDeltaWriteStat) writeStatus.getStat())
|
||||
.setLogVersion(currentLogFile.getLogVersion());
|
||||
((HoodieDeltaWriteStat) writeStatus.getStat())
|
||||
.setLogOffset(writer.getCurrentSize());
|
||||
} catch (Exception e) {
|
||||
logger.error("Error in update task at commit " + commitTime, e);
|
||||
writeStatus.setGlobalError(e);
|
||||
throw new HoodieUpsertException(
|
||||
"Failed to initialize HoodieUpdateHandle for FileId: " + fileId
|
||||
+ " on commit " + commitTime + " on HDFS path " + hoodieTable
|
||||
.getMetaClient().getBasePath() + partitionPath, e);
|
||||
}
|
||||
Path path = new Path(record.getPartitionPath(),
|
||||
FSUtils.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId));
|
||||
writeStatus.getStat().setPath(path.toString());
|
||||
}
|
||||
// update the new location of the record, so we know where to find it next
|
||||
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId));
|
||||
});
|
||||
this.records = records;
|
||||
}
|
||||
private void init(Iterator<HoodieRecord<T>> recordItr) {
|
||||
List<HoodieRecord<T>> records = Lists.newArrayList();
|
||||
recordItr.forEachRemaining(record -> {
|
||||
records.add(record);
|
||||
// extract some information from the first record
|
||||
if (partitionPath == null) {
|
||||
partitionPath = record.getPartitionPath();
|
||||
// HACK(vc) This also assumes a base file. It will break, if appending without one.
|
||||
String latestValidFilePath =
|
||||
fileSystemView.getLatestDataFiles(record.getPartitionPath())
|
||||
.filter(dataFile -> dataFile.getFileId().equals(fileId))
|
||||
.findFirst().get().getFileName();
|
||||
String baseCommitTime = FSUtils.getCommitTime(latestValidFilePath);
|
||||
writeStatus.getStat().setPrevCommit(baseCommitTime);
|
||||
writeStatus.setFileId(fileId);
|
||||
writeStatus.setPartitionPath(record.getPartitionPath());
|
||||
writeStatus.getStat().setFileId(fileId);
|
||||
|
||||
private Optional<IndexedRecord> getIndexedRecord(HoodieRecord<T> hoodieRecord) {
|
||||
Optional recordMetadata = hoodieRecord.getData().getMetadata();
|
||||
try {
|
||||
Optional<IndexedRecord> avroRecord = hoodieRecord.getData().getInsertValue(schema);
|
||||
|
||||
if(avroRecord.isPresent()) {
|
||||
String seqId = HoodieRecord.generateSequenceId(commitTime, TaskContext.getPartitionId(),
|
||||
recordIndex.getAndIncrement());
|
||||
HoodieAvroUtils
|
||||
.addHoodieKeyToRecord((GenericRecord) avroRecord.get(), hoodieRecord.getRecordKey(),
|
||||
hoodieRecord.getPartitionPath(), fileId);
|
||||
HoodieAvroUtils
|
||||
.addCommitMetadataToRecord((GenericRecord) avroRecord.get(), commitTime, seqId);
|
||||
recordsWritten++;
|
||||
} else {
|
||||
recordsDeleted++;
|
||||
}
|
||||
|
||||
hoodieRecord.deflate();
|
||||
writeStatus.markSuccess(hoodieRecord, recordMetadata);
|
||||
return avroRecord;
|
||||
this.writer = HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath))
|
||||
.withFileId(fileId).overBaseCommit(baseCommitTime)
|
||||
.withFs(fs).withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
||||
this.currentLogFile = writer.getLogFile();
|
||||
((HoodieDeltaWriteStat) writeStatus.getStat())
|
||||
.setLogVersion(currentLogFile.getLogVersion());
|
||||
((HoodieDeltaWriteStat) writeStatus.getStat())
|
||||
.setLogOffset(writer.getCurrentSize());
|
||||
} catch (Exception e) {
|
||||
logger.error("Error writing record " + hoodieRecord, e);
|
||||
writeStatus.markFailure(hoodieRecord, e, recordMetadata);
|
||||
logger.error("Error in update task at commit " + commitTime, e);
|
||||
writeStatus.setGlobalError(e);
|
||||
throw new HoodieUpsertException(
|
||||
"Failed to initialize HoodieUpdateHandle for FileId: " + fileId
|
||||
+ " on commit " + commitTime + " on HDFS path " + hoodieTable
|
||||
.getMetaClient().getBasePath() + partitionPath, e);
|
||||
}
|
||||
return Optional.empty();
|
||||
}
|
||||
Path path = new Path(record.getPartitionPath(),
|
||||
FSUtils.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId));
|
||||
writeStatus.getStat().setPath(path.toString());
|
||||
}
|
||||
// update the new location of the record, so we know where to find it next
|
||||
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId));
|
||||
});
|
||||
this.records = records;
|
||||
}
|
||||
|
||||
public void doAppend() {
|
||||
private Optional<IndexedRecord> getIndexedRecord(HoodieRecord<T> hoodieRecord) {
|
||||
Optional recordMetadata = hoodieRecord.getData().getMetadata();
|
||||
try {
|
||||
Optional<IndexedRecord> avroRecord = hoodieRecord.getData().getInsertValue(schema);
|
||||
|
||||
List<IndexedRecord> recordList = new ArrayList<>();
|
||||
List<String> keysToDelete = new ArrayList<>();
|
||||
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
|
||||
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, commitTime);
|
||||
records.stream().forEach(record -> {
|
||||
Optional<IndexedRecord> indexedRecord = getIndexedRecord(record);
|
||||
if(indexedRecord.isPresent()) {
|
||||
recordList.add(indexedRecord.get());
|
||||
} else {
|
||||
keysToDelete.add(record.getRecordKey());
|
||||
}
|
||||
});
|
||||
try {
|
||||
if(recordList.size() > 0) {
|
||||
writer = writer.appendBlock(new HoodieAvroDataBlock(recordList, schema, metadata));
|
||||
}
|
||||
if(keysToDelete.size() > 0) {
|
||||
writer = writer.appendBlock(new HoodieDeleteBlock(keysToDelete.stream().toArray(String[]::new), metadata));
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new HoodieAppendException(
|
||||
"Failed while appeding records to " + currentLogFile.getPath(), e);
|
||||
}
|
||||
}
|
||||
if (avroRecord.isPresent()) {
|
||||
String seqId = HoodieRecord.generateSequenceId(commitTime, TaskContext.getPartitionId(),
|
||||
recordIndex.getAndIncrement());
|
||||
HoodieAvroUtils
|
||||
.addHoodieKeyToRecord((GenericRecord) avroRecord.get(), hoodieRecord.getRecordKey(),
|
||||
hoodieRecord.getPartitionPath(), fileId);
|
||||
HoodieAvroUtils
|
||||
.addCommitMetadataToRecord((GenericRecord) avroRecord.get(), commitTime, seqId);
|
||||
recordsWritten++;
|
||||
} else {
|
||||
recordsDeleted++;
|
||||
}
|
||||
|
||||
public void close() {
|
||||
try {
|
||||
if (writer != null) {
|
||||
writer.close();
|
||||
}
|
||||
writeStatus.getStat().setNumWrites(recordsWritten);
|
||||
writeStatus.getStat().setNumDeletes(recordsDeleted);
|
||||
writeStatus.getStat().setTotalWriteErrors(writeStatus.getFailedRecords().size());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
|
||||
}
|
||||
hoodieRecord.deflate();
|
||||
writeStatus.markSuccess(hoodieRecord, recordMetadata);
|
||||
return avroRecord;
|
||||
} catch (Exception e) {
|
||||
logger.error("Error writing record " + hoodieRecord, e);
|
||||
writeStatus.markFailure(hoodieRecord, e, recordMetadata);
|
||||
}
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
public WriteStatus getWriteStatus() {
|
||||
return writeStatus;
|
||||
public void doAppend() {
|
||||
|
||||
List<IndexedRecord> recordList = new ArrayList<>();
|
||||
List<String> keysToDelete = new ArrayList<>();
|
||||
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
|
||||
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, commitTime);
|
||||
records.stream().forEach(record -> {
|
||||
Optional<IndexedRecord> indexedRecord = getIndexedRecord(record);
|
||||
if (indexedRecord.isPresent()) {
|
||||
recordList.add(indexedRecord.get());
|
||||
} else {
|
||||
keysToDelete.add(record.getRecordKey());
|
||||
}
|
||||
});
|
||||
try {
|
||||
if (recordList.size() > 0) {
|
||||
writer = writer.appendBlock(new HoodieAvroDataBlock(recordList, schema, metadata));
|
||||
}
|
||||
if (keysToDelete.size() > 0) {
|
||||
writer = writer.appendBlock(
|
||||
new HoodieDeleteBlock(keysToDelete.stream().toArray(String[]::new), metadata));
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new HoodieAppendException(
|
||||
"Failed while appeding records to " + currentLogFile.getPath(), e);
|
||||
}
|
||||
}
|
||||
|
||||
public void close() {
|
||||
try {
|
||||
if (writer != null) {
|
||||
writer.close();
|
||||
}
|
||||
writeStatus.getStat().setNumWrites(recordsWritten);
|
||||
writeStatus.getStat().setNumDeletes(recordsDeleted);
|
||||
writeStatus.getStat().setTotalWriteErrors(writeStatus.getFailedRecords().size());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
|
||||
}
|
||||
}
|
||||
|
||||
public WriteStatus getWriteStatus() {
|
||||
return writeStatus;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
@@ -27,226 +27,212 @@ import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Cleaner is responsible for garbage collecting older files in a given partition path, such that
|
||||
* <p>
|
||||
* 1) It provides sufficient time for existing queries running on older versions, to finish
|
||||
* <p>
|
||||
* 2) It bounds the growth of the files in the file system
|
||||
* <p>
|
||||
* TODO: Should all cleaning be done based on {@link com.uber.hoodie.common.model.HoodieCommitMetadata}
|
||||
* <p> 1) It provides sufficient time for existing queries running on older versions, to finish <p>
|
||||
* 2) It bounds the growth of the files in the file system <p> TODO: Should all cleaning be done
|
||||
* based on {@link com.uber.hoodie.common.model.HoodieCommitMetadata}
|
||||
*/
|
||||
public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
private static Logger logger = LogManager.getLogger(HoodieCleanHelper.class);
|
||||
|
||||
private final TableFileSystemView fileSystemView;
|
||||
private final HoodieTimeline commitTimeline;
|
||||
private HoodieTable<T> hoodieTable;
|
||||
private HoodieWriteConfig config;
|
||||
private FileSystem fs;
|
||||
private static Logger logger = LogManager.getLogger(HoodieCleanHelper.class);
|
||||
|
||||
public HoodieCleanHelper(HoodieTable<T> hoodieTable, HoodieWriteConfig config) {
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.fileSystemView = hoodieTable.getCompletedFileSystemView();
|
||||
this.commitTimeline = hoodieTable.getCompletedCommitTimeline();
|
||||
this.config = config;
|
||||
this.fs = hoodieTable.getFs();
|
||||
private final TableFileSystemView fileSystemView;
|
||||
private final HoodieTimeline commitTimeline;
|
||||
private HoodieTable<T> hoodieTable;
|
||||
private HoodieWriteConfig config;
|
||||
private FileSystem fs;
|
||||
|
||||
public HoodieCleanHelper(HoodieTable<T> hoodieTable, HoodieWriteConfig config) {
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.fileSystemView = hoodieTable.getCompletedFileSystemView();
|
||||
this.commitTimeline = hoodieTable.getCompletedCommitTimeline();
|
||||
this.config = config;
|
||||
this.fs = hoodieTable.getFs();
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Selects the older versions of files for cleaning, such that it bounds the number of versions of
|
||||
* each file. This policy is useful, if you are simply interested in querying the table, and you
|
||||
* don't want too many versions for a single file (i.e run it with versionsRetained = 1)
|
||||
*/
|
||||
private List<String> getFilesToCleanKeepingLatestVersions(String partitionPath)
|
||||
throws IOException {
|
||||
logger.info("Cleaning " + partitionPath + ", retaining latest " + config
|
||||
.getCleanerFileVersionsRetained() + " file versions. ");
|
||||
List<HoodieFileGroup> fileGroups =
|
||||
fileSystemView.getAllFileGroups(partitionPath)
|
||||
.collect(Collectors.toList());
|
||||
List<String> deletePaths = new ArrayList<>();
|
||||
// Collect all the datafiles savepointed by all the savepoints
|
||||
List<String> savepointedFiles = hoodieTable.getSavepoints().stream()
|
||||
.flatMap(s -> hoodieTable.getSavepointedDataFiles(s)).collect(Collectors.toList());
|
||||
|
||||
for (HoodieFileGroup fileGroup : fileGroups) {
|
||||
int keepVersions = config.getCleanerFileVersionsRetained();
|
||||
Iterator<FileSlice> fileSliceIterator = fileGroup.getAllFileSlices().iterator();
|
||||
while (fileSliceIterator.hasNext() && keepVersions > 0) {
|
||||
// Skip this most recent version
|
||||
FileSlice nextSlice = fileSliceIterator.next();
|
||||
HoodieDataFile dataFile = nextSlice.getDataFile().get();
|
||||
if (savepointedFiles.contains(dataFile.getFileName())) {
|
||||
// do not clean up a savepoint data file
|
||||
continue;
|
||||
}
|
||||
keepVersions--;
|
||||
}
|
||||
// Delete the remaining files
|
||||
while (fileSliceIterator.hasNext()) {
|
||||
FileSlice nextSlice = fileSliceIterator.next();
|
||||
HoodieDataFile dataFile = nextSlice.getDataFile().get();
|
||||
deletePaths.add(dataFile.getFileStatus().getPath().toString());
|
||||
if (hoodieTable.getMetaClient().getTableType()
|
||||
== HoodieTableType.MERGE_ON_READ) {
|
||||
// If merge on read, then clean the log files for the commits as well
|
||||
deletePaths.addAll(nextSlice.getLogFiles()
|
||||
.map(file -> file.getPath().toString())
|
||||
.collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
}
|
||||
return deletePaths;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Selects the older versions of files for cleaning, such that it bounds the number of versions of each file.
|
||||
* This policy is useful, if you are simply interested in querying the table, and you don't want too many
|
||||
* versions for a single file (i.e run it with versionsRetained = 1)
|
||||
*
|
||||
* @param partitionPath
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
private List<String> getFilesToCleanKeepingLatestVersions(String partitionPath)
|
||||
throws IOException {
|
||||
logger.info("Cleaning " + partitionPath + ", retaining latest " + config
|
||||
.getCleanerFileVersionsRetained() + " file versions. ");
|
||||
List<HoodieFileGroup> fileGroups =
|
||||
fileSystemView.getAllFileGroups(partitionPath)
|
||||
.collect(Collectors.toList());
|
||||
List<String> deletePaths = new ArrayList<>();
|
||||
// Collect all the datafiles savepointed by all the savepoints
|
||||
List<String> savepointedFiles = hoodieTable.getSavepoints().stream()
|
||||
.flatMap(s -> hoodieTable.getSavepointedDataFiles(s)).collect(Collectors.toList());
|
||||
/**
|
||||
* Selects the versions for file for cleaning, such that it <p> - Leaves the latest version of the
|
||||
* file untouched - For older versions, - It leaves all the commits untouched which has occured in
|
||||
* last <code>config.getCleanerCommitsRetained()</code> commits - It leaves ONE commit before this
|
||||
* window. We assume that the max(query execution time) == commit_batch_time *
|
||||
* config.getCleanerCommitsRetained(). This is 12 hours by default. This is essential to leave the
|
||||
* file used by the query thats running for the max time. <p> This provides the effect of having
|
||||
* lookback into all changes that happened in the last X commits. (eg: if you retain 24 commits,
|
||||
* and commit batch time is 30 mins, then you have 12 hrs of lookback) <p> This policy is the
|
||||
* default.
|
||||
*/
|
||||
private List<String> getFilesToCleanKeepingLatestCommits(String partitionPath)
|
||||
throws IOException {
|
||||
int commitsRetained = config.getCleanerCommitsRetained();
|
||||
logger.info(
|
||||
"Cleaning " + partitionPath + ", retaining latest " + commitsRetained + " commits. ");
|
||||
List<String> deletePaths = new ArrayList<>();
|
||||
|
||||
for (HoodieFileGroup fileGroup : fileGroups) {
|
||||
int keepVersions = config.getCleanerFileVersionsRetained();
|
||||
Iterator<FileSlice> fileSliceIterator = fileGroup.getAllFileSlices().iterator();
|
||||
while (fileSliceIterator.hasNext() && keepVersions > 0) {
|
||||
// Skip this most recent version
|
||||
FileSlice nextSlice = fileSliceIterator.next();
|
||||
HoodieDataFile dataFile = nextSlice.getDataFile().get();
|
||||
if(savepointedFiles.contains(dataFile.getFileName())) {
|
||||
// do not clean up a savepoint data file
|
||||
continue;
|
||||
}
|
||||
keepVersions--;
|
||||
}
|
||||
// Delete the remaining files
|
||||
while (fileSliceIterator.hasNext()) {
|
||||
FileSlice nextSlice = fileSliceIterator.next();
|
||||
HoodieDataFile dataFile = nextSlice.getDataFile().get();
|
||||
deletePaths.add(dataFile.getFileStatus().getPath().toString());
|
||||
if (hoodieTable.getMetaClient().getTableType()
|
||||
== HoodieTableType.MERGE_ON_READ) {
|
||||
// If merge on read, then clean the log files for the commits as well
|
||||
deletePaths.addAll(nextSlice.getLogFiles()
|
||||
.map(file -> file.getPath().toString())
|
||||
.collect(Collectors.toList()));
|
||||
}
|
||||
// Collect all the datafiles savepointed by all the savepoints
|
||||
List<String> savepointedFiles = hoodieTable.getSavepoints().stream()
|
||||
.flatMap(s -> hoodieTable.getSavepointedDataFiles(s)).collect(Collectors.toList());
|
||||
|
||||
// determine if we have enough commits, to start cleaning.
|
||||
if (commitTimeline.countInstants() > commitsRetained) {
|
||||
HoodieInstant earliestCommitToRetain = getEarliestCommitToRetain().get();
|
||||
List<HoodieFileGroup> fileGroups =
|
||||
fileSystemView.getAllFileGroups(partitionPath)
|
||||
.collect(Collectors.toList());
|
||||
for (HoodieFileGroup fileGroup : fileGroups) {
|
||||
List<FileSlice> fileSliceList = fileGroup.getAllFileSlices().collect(Collectors.toList());
|
||||
HoodieDataFile dataFile = fileSliceList.get(0).getDataFile().get();
|
||||
String lastVersion = dataFile.getCommitTime();
|
||||
String lastVersionBeforeEarliestCommitToRetain =
|
||||
getLatestVersionBeforeCommit(fileSliceList, earliestCommitToRetain);
|
||||
|
||||
// Ensure there are more than 1 version of the file (we only clean old files from updates)
|
||||
// i.e always spare the last commit.
|
||||
for (FileSlice aSlice : fileSliceList) {
|
||||
HoodieDataFile aFile = aSlice.getDataFile().get();
|
||||
String fileCommitTime = aFile.getCommitTime();
|
||||
if (savepointedFiles.contains(aFile.getFileName())) {
|
||||
// do not clean up a savepoint data file
|
||||
continue;
|
||||
}
|
||||
// Dont delete the latest commit and also the last commit before the earliest commit we are retaining
|
||||
// The window of commit retain == max query run time. So a query could be running which still
|
||||
// uses this file.
|
||||
if (fileCommitTime.equals(lastVersion) || (
|
||||
lastVersionBeforeEarliestCommitToRetain != null && fileCommitTime
|
||||
.equals(lastVersionBeforeEarliestCommitToRetain))) {
|
||||
// move on to the next file
|
||||
continue;
|
||||
}
|
||||
|
||||
// Always keep the last commit
|
||||
if (HoodieTimeline.compareTimestamps(
|
||||
earliestCommitToRetain.getTimestamp(),
|
||||
fileCommitTime,
|
||||
HoodieTimeline.GREATER)) {
|
||||
// this is a commit, that should be cleaned.
|
||||
deletePaths.add(aFile.getFileStatus().getPath().toString());
|
||||
if (hoodieTable.getMetaClient().getTableType()
|
||||
== HoodieTableType.MERGE_ON_READ) {
|
||||
// If merge on read, then clean the log files for the commits as well
|
||||
deletePaths.addAll(aSlice.getLogFiles()
|
||||
.map(file -> file.getPath().toString())
|
||||
.collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
}
|
||||
return deletePaths;
|
||||
}
|
||||
}
|
||||
|
||||
return deletePaths;
|
||||
}
|
||||
|
||||
/**
|
||||
* Selects the versions for file for cleaning, such that it
|
||||
* <p>
|
||||
* - Leaves the latest version of the file untouched
|
||||
* - For older versions,
|
||||
* - It leaves all the commits untouched which has occured in last <code>config.getCleanerCommitsRetained()</code> commits
|
||||
* - It leaves ONE commit before this window. We assume that the max(query execution time) == commit_batch_time * config.getCleanerCommitsRetained(). This is 12 hours by default.
|
||||
* This is essential to leave the file used by the query thats running for the max time.
|
||||
* <p>
|
||||
* This provides the effect of having lookback into all changes that happened in the last X
|
||||
* commits. (eg: if you retain 24 commits, and commit batch time is 30 mins, then you have 12 hrs of lookback)
|
||||
* <p>
|
||||
* This policy is the default.
|
||||
*
|
||||
* @param partitionPath
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
private List<String> getFilesToCleanKeepingLatestCommits(String partitionPath)
|
||||
throws IOException {
|
||||
int commitsRetained = config.getCleanerCommitsRetained();
|
||||
logger.info(
|
||||
"Cleaning " + partitionPath + ", retaining latest " + commitsRetained + " commits. ");
|
||||
List<String> deletePaths = new ArrayList<>();
|
||||
|
||||
// Collect all the datafiles savepointed by all the savepoints
|
||||
List<String> savepointedFiles = hoodieTable.getSavepoints().stream()
|
||||
.flatMap(s -> hoodieTable.getSavepointedDataFiles(s)).collect(Collectors.toList());
|
||||
|
||||
// determine if we have enough commits, to start cleaning.
|
||||
if (commitTimeline.countInstants() > commitsRetained) {
|
||||
HoodieInstant earliestCommitToRetain = getEarliestCommitToRetain().get();
|
||||
List<HoodieFileGroup> fileGroups =
|
||||
fileSystemView.getAllFileGroups(partitionPath)
|
||||
.collect(Collectors.toList());
|
||||
for (HoodieFileGroup fileGroup : fileGroups) {
|
||||
List<FileSlice> fileSliceList = fileGroup.getAllFileSlices().collect(Collectors.toList());
|
||||
HoodieDataFile dataFile = fileSliceList.get(0).getDataFile().get();
|
||||
String lastVersion = dataFile.getCommitTime();
|
||||
String lastVersionBeforeEarliestCommitToRetain =
|
||||
getLatestVersionBeforeCommit(fileSliceList, earliestCommitToRetain);
|
||||
|
||||
// Ensure there are more than 1 version of the file (we only clean old files from updates)
|
||||
// i.e always spare the last commit.
|
||||
for (FileSlice aSlice : fileSliceList) {
|
||||
HoodieDataFile aFile = aSlice.getDataFile().get();
|
||||
String fileCommitTime = aFile.getCommitTime();
|
||||
if(savepointedFiles.contains(aFile.getFileName())) {
|
||||
// do not clean up a savepoint data file
|
||||
continue;
|
||||
}
|
||||
// Dont delete the latest commit and also the last commit before the earliest commit we are retaining
|
||||
// The window of commit retain == max query run time. So a query could be running which still
|
||||
// uses this file.
|
||||
if (fileCommitTime.equals(lastVersion) || (
|
||||
lastVersionBeforeEarliestCommitToRetain != null && fileCommitTime
|
||||
.equals(lastVersionBeforeEarliestCommitToRetain))) {
|
||||
// move on to the next file
|
||||
continue;
|
||||
}
|
||||
|
||||
// Always keep the last commit
|
||||
if (HoodieTimeline.compareTimestamps(
|
||||
earliestCommitToRetain.getTimestamp(),
|
||||
fileCommitTime,
|
||||
HoodieTimeline.GREATER)) {
|
||||
// this is a commit, that should be cleaned.
|
||||
deletePaths.add(aFile.getFileStatus().getPath().toString());
|
||||
if (hoodieTable.getMetaClient().getTableType()
|
||||
== HoodieTableType.MERGE_ON_READ) {
|
||||
// If merge on read, then clean the log files for the commits as well
|
||||
deletePaths.addAll(aSlice.getLogFiles()
|
||||
.map(file -> file.getPath().toString())
|
||||
.collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return deletePaths;
|
||||
/**
|
||||
* Gets the latest version < commitTime. This version file could still be used by queries.
|
||||
*/
|
||||
private String getLatestVersionBeforeCommit(List<FileSlice> fileSliceList,
|
||||
HoodieInstant commitTime) {
|
||||
for (FileSlice file : fileSliceList) {
|
||||
String fileCommitTime = file.getDataFile().get().getCommitTime();
|
||||
if (HoodieTimeline.compareTimestamps(commitTime.getTimestamp(), fileCommitTime,
|
||||
HoodieTimeline.GREATER)) {
|
||||
// fileList is sorted on the reverse, so the first commit we find <= commitTime is the one we want
|
||||
return fileCommitTime;
|
||||
}
|
||||
}
|
||||
// There is no version of this file which is <= commitTime
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the latest version < commitTime. This version file could still be used by queries.
|
||||
*/
|
||||
private String getLatestVersionBeforeCommit(List<FileSlice> fileSliceList,
|
||||
HoodieInstant commitTime) {
|
||||
for (FileSlice file : fileSliceList) {
|
||||
String fileCommitTime = file.getDataFile().get().getCommitTime();
|
||||
if (HoodieTimeline.compareTimestamps(commitTime.getTimestamp(), fileCommitTime,
|
||||
HoodieTimeline.GREATER)) {
|
||||
// fileList is sorted on the reverse, so the first commit we find <= commitTime is the one we want
|
||||
return fileCommitTime;
|
||||
}
|
||||
}
|
||||
// There is no version of this file which is <= commitTime
|
||||
return null;
|
||||
/**
|
||||
* Returns files to be cleaned for the given partitionPath based on cleaning policy.
|
||||
*/
|
||||
public List<String> getDeletePaths(String partitionPath) throws IOException {
|
||||
HoodieCleaningPolicy policy = config.getCleanerPolicy();
|
||||
List<String> deletePaths;
|
||||
if (policy == HoodieCleaningPolicy.KEEP_LATEST_COMMITS) {
|
||||
deletePaths = getFilesToCleanKeepingLatestCommits(partitionPath);
|
||||
} else if (policy == HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) {
|
||||
deletePaths = getFilesToCleanKeepingLatestVersions(partitionPath);
|
||||
} else {
|
||||
throw new IllegalArgumentException("Unknown cleaning policy : " + policy.name());
|
||||
}
|
||||
logger.info(
|
||||
deletePaths.size() + " patterns used to delete in partition path:" + partitionPath);
|
||||
|
||||
/**
|
||||
* Returns files to be cleaned for the given partitionPath based on cleaning policy.
|
||||
*/
|
||||
public List<String> getDeletePaths(String partitionPath) throws IOException {
|
||||
HoodieCleaningPolicy policy = config.getCleanerPolicy();
|
||||
List<String> deletePaths;
|
||||
if (policy == HoodieCleaningPolicy.KEEP_LATEST_COMMITS) {
|
||||
deletePaths = getFilesToCleanKeepingLatestCommits(partitionPath);
|
||||
} else if (policy == HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) {
|
||||
deletePaths = getFilesToCleanKeepingLatestVersions(partitionPath);
|
||||
} else {
|
||||
throw new IllegalArgumentException("Unknown cleaning policy : " + policy.name());
|
||||
}
|
||||
logger.info(
|
||||
deletePaths.size() + " patterns used to delete in partition path:" + partitionPath);
|
||||
return deletePaths;
|
||||
}
|
||||
|
||||
return deletePaths;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns earliest commit to retain based on cleaning policy.
|
||||
*/
|
||||
public Optional<HoodieInstant> getEarliestCommitToRetain() {
|
||||
Optional<HoodieInstant> earliestCommitToRetain = Optional.empty();
|
||||
int commitsRetained = config.getCleanerCommitsRetained();
|
||||
if (config.getCleanerPolicy() == HoodieCleaningPolicy.KEEP_LATEST_COMMITS
|
||||
&& commitTimeline.countInstants() > commitsRetained) {
|
||||
earliestCommitToRetain =
|
||||
commitTimeline.nthInstant(commitTimeline.countInstants() - commitsRetained);
|
||||
}
|
||||
return earliestCommitToRetain;
|
||||
/**
|
||||
* Returns earliest commit to retain based on cleaning policy.
|
||||
*/
|
||||
public Optional<HoodieInstant> getEarliestCommitToRetain() {
|
||||
Optional<HoodieInstant> earliestCommitToRetain = Optional.empty();
|
||||
int commitsRetained = config.getCleanerCommitsRetained();
|
||||
if (config.getCleanerPolicy() == HoodieCleaningPolicy.KEEP_LATEST_COMMITS
|
||||
&& commitTimeline.countInstants() > commitsRetained) {
|
||||
earliestCommitToRetain =
|
||||
commitTimeline.nthInstant(commitTimeline.countInstants() - commitsRetained);
|
||||
}
|
||||
return earliestCommitToRetain;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -39,6 +39,12 @@ import com.uber.hoodie.exception.HoodieCommitException;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
@@ -46,225 +52,233 @@ import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* Archiver to bound the growth of <action>.commit files
|
||||
*/
|
||||
public class HoodieCommitArchiveLog {
|
||||
private static Logger log = LogManager.getLogger(HoodieCommitArchiveLog.class);
|
||||
|
||||
private final Path archiveFilePath;
|
||||
private final FileSystem fs;
|
||||
private final HoodieWriteConfig config;
|
||||
private HoodieLogFormat.Writer writer;
|
||||
private static Logger log = LogManager.getLogger(HoodieCommitArchiveLog.class);
|
||||
|
||||
public HoodieCommitArchiveLog(HoodieWriteConfig config, FileSystem fs) {
|
||||
this.fs = fs;
|
||||
this.config = config;
|
||||
this.archiveFilePath = HoodieArchivedTimeline
|
||||
.getArchiveLogPath(config.getBasePath() + "/" + HoodieTableMetaClient.METAFOLDER_NAME);
|
||||
private final Path archiveFilePath;
|
||||
private final FileSystem fs;
|
||||
private final HoodieWriteConfig config;
|
||||
private HoodieLogFormat.Writer writer;
|
||||
|
||||
public HoodieCommitArchiveLog(HoodieWriteConfig config, FileSystem fs) {
|
||||
this.fs = fs;
|
||||
this.config = config;
|
||||
this.archiveFilePath = HoodieArchivedTimeline
|
||||
.getArchiveLogPath(config.getBasePath() + "/" + HoodieTableMetaClient.METAFOLDER_NAME);
|
||||
}
|
||||
|
||||
private HoodieLogFormat.Writer openWriter() {
|
||||
try {
|
||||
if (this.writer == null) {
|
||||
return HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(archiveFilePath.getParent())
|
||||
.withFileId(archiveFilePath.getName())
|
||||
.withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION)
|
||||
.withFs(fs)
|
||||
.overBaseCommit("").build();
|
||||
} else {
|
||||
return this.writer;
|
||||
}
|
||||
} catch (InterruptedException | IOException e) {
|
||||
throw new HoodieException("Unable to initialize HoodieLogFormat writer", e);
|
||||
}
|
||||
}
|
||||
|
||||
private void close() {
|
||||
try {
|
||||
if (this.writer != null) {
|
||||
this.writer.close();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Unable to close HoodieLogFormat writer", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if commits need to be archived. If yes, archive commits.
|
||||
*/
|
||||
public boolean archiveIfRequired() {
|
||||
try {
|
||||
List<HoodieInstant> instantsToArchive = getInstantsToArchive().collect(Collectors.toList());
|
||||
boolean success = true;
|
||||
if (instantsToArchive.iterator().hasNext()) {
|
||||
this.writer = openWriter();
|
||||
log.info("Archiving instants " + instantsToArchive);
|
||||
archive(instantsToArchive);
|
||||
success = deleteArchivedInstants(instantsToArchive);
|
||||
} else {
|
||||
log.info("No Instants to archive");
|
||||
}
|
||||
return success;
|
||||
} finally {
|
||||
close();
|
||||
}
|
||||
}
|
||||
|
||||
private Stream<HoodieInstant> getInstantsToArchive() {
|
||||
|
||||
// TODO : rename to max/minInstantsToKeep
|
||||
int maxCommitsToKeep = config.getMaxCommitsToKeep();
|
||||
int minCommitsToKeep = config.getMinCommitsToKeep();
|
||||
|
||||
HoodieTable table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
|
||||
// GroupBy each action and limit each action timeline to maxCommitsToKeep
|
||||
HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline()
|
||||
.getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION,
|
||||
HoodieTimeline.ROLLBACK_ACTION));
|
||||
Stream<HoodieInstant> instants = cleanAndRollbackTimeline.getInstants()
|
||||
.collect(Collectors.groupingBy(s -> s.getAction()))
|
||||
.entrySet()
|
||||
.stream()
|
||||
.map(i -> {
|
||||
if (i.getValue().size() > maxCommitsToKeep) {
|
||||
return i.getValue().subList(0, i.getValue().size() - minCommitsToKeep);
|
||||
} else {
|
||||
return new ArrayList<HoodieInstant>();
|
||||
}
|
||||
})
|
||||
.flatMap(i -> i.stream());
|
||||
|
||||
//TODO (na) : Add a way to return actions associated with a timeline and then merge/unify with logic above to avoid Stream.concats
|
||||
HoodieTimeline commitTimeline = table.getCompletedCommitTimeline();
|
||||
// We cannot have any holes in the commit timeline. We cannot archive any commits which are made after the first savepoint present.
|
||||
Optional<HoodieInstant> firstSavepoint = table.getCompletedSavepointTimeline().firstInstant();
|
||||
if (!commitTimeline.empty() && commitTimeline.countInstants() > maxCommitsToKeep) {
|
||||
// Actually do the commits
|
||||
instants = Stream.concat(instants, commitTimeline.getInstants().filter(s -> {
|
||||
// if no savepoint present, then dont filter
|
||||
return !(firstSavepoint.isPresent() && HoodieTimeline
|
||||
.compareTimestamps(firstSavepoint.get().getTimestamp(), s.getTimestamp(),
|
||||
HoodieTimeline.LESSER_OR_EQUAL));
|
||||
}).limit(commitTimeline.countInstants() - minCommitsToKeep));
|
||||
}
|
||||
|
||||
private HoodieLogFormat.Writer openWriter() {
|
||||
try {
|
||||
if(this.writer == null) {
|
||||
return HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(archiveFilePath.getParent())
|
||||
.withFileId(archiveFilePath.getName())
|
||||
.withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION)
|
||||
.withFs(fs)
|
||||
.overBaseCommit("").build();
|
||||
} else {
|
||||
return this.writer;
|
||||
}
|
||||
} catch(InterruptedException | IOException e) {
|
||||
throw new HoodieException("Unable to initialize HoodieLogFormat writer", e);
|
||||
return instants;
|
||||
}
|
||||
|
||||
private boolean deleteArchivedInstants(List<HoodieInstant> archivedInstants) {
|
||||
log.info("Deleting instants " + archivedInstants);
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
|
||||
boolean success = true;
|
||||
for (HoodieInstant archivedInstant : archivedInstants) {
|
||||
Path commitFile =
|
||||
new Path(metaClient.getMetaPath(), archivedInstant.getFileName());
|
||||
try {
|
||||
if (fs.exists(commitFile)) {
|
||||
success &= fs.delete(commitFile, false);
|
||||
log.info("Archived and deleted instant file " + commitFile);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to delete archived instant " + archivedInstant,
|
||||
e);
|
||||
}
|
||||
}
|
||||
return success;
|
||||
}
|
||||
|
||||
private void close() {
|
||||
try {
|
||||
if(this.writer != null) {
|
||||
this.writer.close();
|
||||
}
|
||||
} catch(IOException e) {
|
||||
throw new HoodieException("Unable to close HoodieLogFormat writer", e);
|
||||
}
|
||||
public void archive(List<HoodieInstant> instants) throws HoodieCommitException {
|
||||
|
||||
try {
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
HoodieTimeline commitTimeline =
|
||||
metaClient.getActiveTimeline().getAllCommitsTimeline().filterCompletedInstants();
|
||||
|
||||
Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema();
|
||||
log.info("Wrapper schema " + wrapperSchema.toString());
|
||||
List<IndexedRecord> records = new ArrayList<>();
|
||||
for (HoodieInstant hoodieInstant : instants) {
|
||||
records.add(convertToAvroRecord(commitTimeline, hoodieInstant));
|
||||
}
|
||||
HoodieAvroDataBlock block = new HoodieAvroDataBlock(records, wrapperSchema);
|
||||
this.writer = writer.appendBlock(block);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieCommitException("Failed to archive commits", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if commits need to be archived. If yes, archive commits.
|
||||
*/
|
||||
public boolean archiveIfRequired() {
|
||||
try {
|
||||
List<HoodieInstant> instantsToArchive = getInstantsToArchive().collect(Collectors.toList());
|
||||
boolean success = true;
|
||||
if (instantsToArchive.iterator().hasNext()) {
|
||||
this.writer = openWriter();
|
||||
log.info("Archiving instants " + instantsToArchive);
|
||||
archive(instantsToArchive);
|
||||
success = deleteArchivedInstants(instantsToArchive);
|
||||
} else {
|
||||
log.info("No Instants to archive");
|
||||
}
|
||||
return success;
|
||||
} finally {
|
||||
close();
|
||||
}
|
||||
public Path getArchiveFilePath() {
|
||||
return archiveFilePath;
|
||||
}
|
||||
|
||||
private IndexedRecord convertToAvroRecord(HoodieTimeline commitTimeline,
|
||||
HoodieInstant hoodieInstant) throws IOException {
|
||||
HoodieArchivedMetaEntry archivedMetaWrapper = new HoodieArchivedMetaEntry();
|
||||
archivedMetaWrapper.setCommitTime(hoodieInstant.getTimestamp());
|
||||
switch (hoodieInstant.getAction()) {
|
||||
case HoodieTimeline.CLEAN_ACTION: {
|
||||
archivedMetaWrapper.setHoodieCleanMetadata(AvroUtils
|
||||
.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(),
|
||||
HoodieCleanMetadata.class));
|
||||
archivedMetaWrapper.setActionType(ActionType.clean.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.COMMIT_ACTION: {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get());
|
||||
archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata));
|
||||
archivedMetaWrapper.setActionType(ActionType.commit.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.COMPACTION_ACTION: {
|
||||
com.uber.hoodie.common.model.HoodieCompactionMetadata compactionMetadata = com.uber.hoodie.common.model.HoodieCompactionMetadata
|
||||
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get());
|
||||
archivedMetaWrapper
|
||||
.setHoodieCompactionMetadata(compactionMetadataConverter(compactionMetadata));
|
||||
archivedMetaWrapper.setActionType(ActionType.compaction.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.ROLLBACK_ACTION: {
|
||||
archivedMetaWrapper.setHoodieRollbackMetadata(AvroUtils
|
||||
.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(),
|
||||
HoodieRollbackMetadata.class));
|
||||
archivedMetaWrapper.setActionType(ActionType.rollback.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.SAVEPOINT_ACTION: {
|
||||
archivedMetaWrapper.setHoodieSavePointMetadata(AvroUtils
|
||||
.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(),
|
||||
HoodieSavepointMetadata.class));
|
||||
archivedMetaWrapper.setActionType(ActionType.savepoint.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.DELTA_COMMIT_ACTION: {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get());
|
||||
archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata));
|
||||
archivedMetaWrapper.setActionType(ActionType.commit.name());
|
||||
break;
|
||||
}
|
||||
}
|
||||
return archivedMetaWrapper;
|
||||
}
|
||||
|
||||
private Stream<HoodieInstant> getInstantsToArchive() {
|
||||
private com.uber.hoodie.avro.model.HoodieCommitMetadata commitMetadataConverter(
|
||||
HoodieCommitMetadata hoodieCommitMetadata) {
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
//Need this to ignore other public get() methods
|
||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
com.uber.hoodie.avro.model.HoodieCommitMetadata avroMetaData =
|
||||
mapper.convertValue(hoodieCommitMetadata,
|
||||
com.uber.hoodie.avro.model.HoodieCommitMetadata.class);
|
||||
return avroMetaData;
|
||||
}
|
||||
|
||||
// TODO : rename to max/minInstantsToKeep
|
||||
int maxCommitsToKeep = config.getMaxCommitsToKeep();
|
||||
int minCommitsToKeep = config.getMinCommitsToKeep();
|
||||
|
||||
HoodieTable table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
|
||||
// GroupBy each action and limit each action timeline to maxCommitsToKeep
|
||||
HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline().getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION,
|
||||
HoodieTimeline.ROLLBACK_ACTION));
|
||||
Stream<HoodieInstant> instants = cleanAndRollbackTimeline.getInstants()
|
||||
.collect(Collectors.groupingBy(s -> s.getAction()))
|
||||
.entrySet()
|
||||
.stream()
|
||||
.map(i -> {
|
||||
if (i.getValue().size() > maxCommitsToKeep) {
|
||||
return i.getValue().subList(0, i.getValue().size() - minCommitsToKeep);
|
||||
} else {
|
||||
return new ArrayList<HoodieInstant>();
|
||||
}
|
||||
})
|
||||
.flatMap(i -> i.stream());
|
||||
|
||||
//TODO (na) : Add a way to return actions associated with a timeline and then merge/unify with logic above to avoid Stream.concats
|
||||
HoodieTimeline commitTimeline = table.getCompletedCommitTimeline();
|
||||
// We cannot have any holes in the commit timeline. We cannot archive any commits which are made after the first savepoint present.
|
||||
Optional<HoodieInstant> firstSavepoint = table.getCompletedSavepointTimeline().firstInstant();
|
||||
if (!commitTimeline.empty() && commitTimeline.countInstants() > maxCommitsToKeep) {
|
||||
// Actually do the commits
|
||||
instants = Stream.concat(instants, commitTimeline.getInstants().filter(s -> {
|
||||
// if no savepoint present, then dont filter
|
||||
return !(firstSavepoint.isPresent() && HoodieTimeline
|
||||
.compareTimestamps(firstSavepoint.get().getTimestamp(), s.getTimestamp(),
|
||||
HoodieTimeline.LESSER_OR_EQUAL));
|
||||
}).limit(commitTimeline.countInstants() - minCommitsToKeep));
|
||||
}
|
||||
|
||||
return instants;
|
||||
}
|
||||
|
||||
private boolean deleteArchivedInstants(List<HoodieInstant> archivedInstants) {
|
||||
log.info("Deleting instants " + archivedInstants);
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
|
||||
boolean success = true;
|
||||
for (HoodieInstant archivedInstant : archivedInstants) {
|
||||
Path commitFile =
|
||||
new Path(metaClient.getMetaPath(), archivedInstant.getFileName());
|
||||
try {
|
||||
if (fs.exists(commitFile)) {
|
||||
success &= fs.delete(commitFile, false);
|
||||
log.info("Archived and deleted instant file " + commitFile);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to delete archived instant " + archivedInstant,
|
||||
e);
|
||||
}
|
||||
}
|
||||
return success;
|
||||
}
|
||||
|
||||
public void archive(List<HoodieInstant> instants) throws HoodieCommitException {
|
||||
|
||||
try {
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
HoodieTimeline commitTimeline =
|
||||
metaClient.getActiveTimeline().getAllCommitsTimeline().filterCompletedInstants();
|
||||
|
||||
Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema();
|
||||
log.info("Wrapper schema " + wrapperSchema.toString());
|
||||
List<IndexedRecord> records = new ArrayList<>();
|
||||
for (HoodieInstant hoodieInstant : instants) {
|
||||
records.add(convertToAvroRecord(commitTimeline, hoodieInstant));
|
||||
}
|
||||
HoodieAvroDataBlock block = new HoodieAvroDataBlock(records, wrapperSchema);
|
||||
this.writer = writer.appendBlock(block);
|
||||
} catch(Exception e) {
|
||||
throw new HoodieCommitException("Failed to archive commits", e);
|
||||
}
|
||||
}
|
||||
|
||||
public Path getArchiveFilePath() {
|
||||
return archiveFilePath;
|
||||
}
|
||||
|
||||
private IndexedRecord convertToAvroRecord(HoodieTimeline commitTimeline, HoodieInstant hoodieInstant) throws IOException {
|
||||
HoodieArchivedMetaEntry archivedMetaWrapper = new HoodieArchivedMetaEntry();
|
||||
archivedMetaWrapper.setCommitTime(hoodieInstant.getTimestamp());
|
||||
switch(hoodieInstant.getAction()) {
|
||||
case HoodieTimeline.CLEAN_ACTION:{
|
||||
archivedMetaWrapper.setHoodieCleanMetadata(AvroUtils.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieCleanMetadata.class));
|
||||
archivedMetaWrapper.setActionType(ActionType.clean.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.COMMIT_ACTION:{
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get());
|
||||
archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata));
|
||||
archivedMetaWrapper.setActionType(ActionType.commit.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.COMPACTION_ACTION:{
|
||||
com.uber.hoodie.common.model.HoodieCompactionMetadata compactionMetadata = com.uber.hoodie.common.model.HoodieCompactionMetadata
|
||||
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get());
|
||||
archivedMetaWrapper.setHoodieCompactionMetadata(compactionMetadataConverter(compactionMetadata));
|
||||
archivedMetaWrapper.setActionType(ActionType.compaction.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.ROLLBACK_ACTION:{
|
||||
archivedMetaWrapper.setHoodieRollbackMetadata(AvroUtils.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieRollbackMetadata.class));
|
||||
archivedMetaWrapper.setActionType(ActionType.rollback.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.SAVEPOINT_ACTION:{
|
||||
archivedMetaWrapper.setHoodieSavePointMetadata(AvroUtils.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieSavepointMetadata.class));
|
||||
archivedMetaWrapper.setActionType(ActionType.savepoint.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.DELTA_COMMIT_ACTION:{
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get());
|
||||
archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata));
|
||||
archivedMetaWrapper.setActionType(ActionType.commit.name());
|
||||
break;
|
||||
}
|
||||
}
|
||||
return archivedMetaWrapper;
|
||||
}
|
||||
|
||||
private com.uber.hoodie.avro.model.HoodieCommitMetadata commitMetadataConverter(HoodieCommitMetadata hoodieCommitMetadata) {
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
//Need this to ignore other public get() methods
|
||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
com.uber.hoodie.avro.model.HoodieCommitMetadata avroMetaData =
|
||||
mapper.convertValue(hoodieCommitMetadata, com.uber.hoodie.avro.model.HoodieCommitMetadata.class);
|
||||
return avroMetaData;
|
||||
}
|
||||
|
||||
private com.uber.hoodie.avro.model.HoodieCompactionMetadata compactionMetadataConverter(HoodieCompactionMetadata hoodieCompactionMetadata) {
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
com.uber.hoodie.avro.model.HoodieCompactionMetadata avroMetaData = mapper.convertValue(hoodieCompactionMetadata,
|
||||
com.uber.hoodie.avro.model.HoodieCompactionMetadata.class);
|
||||
return avroMetaData;
|
||||
}
|
||||
private com.uber.hoodie.avro.model.HoodieCompactionMetadata compactionMetadataConverter(
|
||||
HoodieCompactionMetadata hoodieCompactionMetadata) {
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
com.uber.hoodie.avro.model.HoodieCompactionMetadata avroMetaData = mapper
|
||||
.convertValue(hoodieCompactionMetadata,
|
||||
com.uber.hoodie.avro.model.HoodieCompactionMetadata.class);
|
||||
return avroMetaData;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -29,116 +29,111 @@ import com.uber.hoodie.exception.HoodieInsertException;
|
||||
import com.uber.hoodie.io.storage.HoodieStorageWriter;
|
||||
import com.uber.hoodie.io.storage.HoodieStorageWriterFactory;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import java.io.IOException;
|
||||
import java.util.Optional;
|
||||
import java.util.UUID;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.TaskContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Optional;
|
||||
import java.util.UUID;
|
||||
|
||||
public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
|
||||
private static Logger logger = LogManager.getLogger(HoodieCreateHandle.class);
|
||||
|
||||
private final WriteStatus status;
|
||||
private final HoodieStorageWriter<IndexedRecord> storageWriter;
|
||||
private final Path path;
|
||||
private long recordsWritten = 0;
|
||||
private long recordsDeleted = 0;
|
||||
private static Logger logger = LogManager.getLogger(HoodieCreateHandle.class);
|
||||
|
||||
public HoodieCreateHandle(HoodieWriteConfig config, String commitTime,
|
||||
HoodieTable<T> hoodieTable, String partitionPath) {
|
||||
super(config, commitTime, hoodieTable);
|
||||
this.status = ReflectionUtils.loadClass(config.getWriteStatusClassName());
|
||||
status.setFileId(UUID.randomUUID().toString());
|
||||
status.setPartitionPath(partitionPath);
|
||||
private final WriteStatus status;
|
||||
private final HoodieStorageWriter<IndexedRecord> storageWriter;
|
||||
private final Path path;
|
||||
private long recordsWritten = 0;
|
||||
private long recordsDeleted = 0;
|
||||
|
||||
this.path = makeNewPath(partitionPath, TaskContext.getPartitionId(), status.getFileId());
|
||||
try {
|
||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs,
|
||||
commitTime,
|
||||
new Path(config.getBasePath()),
|
||||
new Path(config.getBasePath(), partitionPath));
|
||||
partitionMetadata.trySave(TaskContext.getPartitionId());
|
||||
this.storageWriter =
|
||||
HoodieStorageWriterFactory.getStorageWriter(commitTime, path, hoodieTable, config, schema);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieInsertException(
|
||||
"Failed to initialize HoodieStorageWriter for path " + path, e);
|
||||
}
|
||||
logger.info("New InsertHandle for partition :" + partitionPath);
|
||||
public HoodieCreateHandle(HoodieWriteConfig config, String commitTime,
|
||||
HoodieTable<T> hoodieTable, String partitionPath) {
|
||||
super(config, commitTime, hoodieTable);
|
||||
this.status = ReflectionUtils.loadClass(config.getWriteStatusClassName());
|
||||
status.setFileId(UUID.randomUUID().toString());
|
||||
status.setPartitionPath(partitionPath);
|
||||
|
||||
this.path = makeNewPath(partitionPath, TaskContext.getPartitionId(), status.getFileId());
|
||||
try {
|
||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs,
|
||||
commitTime,
|
||||
new Path(config.getBasePath()),
|
||||
new Path(config.getBasePath(), partitionPath));
|
||||
partitionMetadata.trySave(TaskContext.getPartitionId());
|
||||
this.storageWriter =
|
||||
HoodieStorageWriterFactory
|
||||
.getStorageWriter(commitTime, path, hoodieTable, config, schema);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieInsertException(
|
||||
"Failed to initialize HoodieStorageWriter for path " + path, e);
|
||||
}
|
||||
logger.info("New InsertHandle for partition :" + partitionPath);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether we can accept the incoming records, into the current file, depending on
|
||||
*
|
||||
* - Whether it belongs to the same partitionPath as existing records
|
||||
* - Whether the current file written bytes lt max file size
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public boolean canWrite(HoodieRecord record) {
|
||||
return storageWriter.canWrite() && record.getPartitionPath()
|
||||
.equals(status.getPartitionPath());
|
||||
/**
|
||||
* Determines whether we can accept the incoming records, into the current file, depending on
|
||||
*
|
||||
* - Whether it belongs to the same partitionPath as existing records - Whether the current file
|
||||
* written bytes lt max file size
|
||||
*/
|
||||
public boolean canWrite(HoodieRecord record) {
|
||||
return storageWriter.canWrite() && record.getPartitionPath()
|
||||
.equals(status.getPartitionPath());
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform the actual writing of the given record into the backing file.
|
||||
*/
|
||||
public void write(HoodieRecord record) {
|
||||
Optional recordMetadata = record.getData().getMetadata();
|
||||
try {
|
||||
Optional<IndexedRecord> avroRecord = record.getData().getInsertValue(schema);
|
||||
|
||||
if (avroRecord.isPresent()) {
|
||||
storageWriter.writeAvroWithMetadata(avroRecord.get(), record);
|
||||
// update the new location of record, so we know where to find it next
|
||||
record.setNewLocation(new HoodieRecordLocation(commitTime, status.getFileId()));
|
||||
recordsWritten++;
|
||||
} else {
|
||||
recordsDeleted++;
|
||||
}
|
||||
record.deflate();
|
||||
status.markSuccess(record, recordMetadata);
|
||||
} catch (Throwable t) {
|
||||
// Not throwing exception from here, since we don't want to fail the entire job
|
||||
// for a single record
|
||||
status.markFailure(record, t, recordMetadata);
|
||||
logger.error("Error writing record " + record, t);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform the actual writing of the given record into the backing file.
|
||||
*
|
||||
* @param record
|
||||
*/
|
||||
public void write(HoodieRecord record) {
|
||||
Optional recordMetadata = record.getData().getMetadata();
|
||||
try {
|
||||
Optional<IndexedRecord> avroRecord = record.getData().getInsertValue(schema);
|
||||
/**
|
||||
* Performs actions to durably, persist the current changes and returns a WriteStatus object
|
||||
*/
|
||||
public WriteStatus close() {
|
||||
logger.info(
|
||||
"Closing the file " + status.getFileId() + " as we are done with all the records "
|
||||
+ recordsWritten);
|
||||
try {
|
||||
storageWriter.close();
|
||||
|
||||
if(avroRecord.isPresent()) {
|
||||
storageWriter.writeAvroWithMetadata(avroRecord.get(), record);
|
||||
// update the new location of record, so we know where to find it next
|
||||
record.setNewLocation(new HoodieRecordLocation(commitTime, status.getFileId()));
|
||||
recordsWritten++;
|
||||
} else {
|
||||
recordsDeleted++;
|
||||
}
|
||||
record.deflate();
|
||||
status.markSuccess(record, recordMetadata);
|
||||
} catch (Throwable t) {
|
||||
// Not throwing exception from here, since we don't want to fail the entire job
|
||||
// for a single record
|
||||
status.markFailure(record, t, recordMetadata);
|
||||
logger.error("Error writing record " + record, t);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs actions to durably, persist the current changes and returns a WriteStatus object
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public WriteStatus close() {
|
||||
logger.info(
|
||||
"Closing the file " + status.getFileId() + " as we are done with all the records "
|
||||
+ recordsWritten);
|
||||
try {
|
||||
storageWriter.close();
|
||||
|
||||
HoodieWriteStat stat = new HoodieWriteStat();
|
||||
stat.setNumWrites(recordsWritten);
|
||||
stat.setNumDeletes(recordsDeleted);
|
||||
stat.setPrevCommit(HoodieWriteStat.NULL_COMMIT);
|
||||
stat.setFileId(status.getFileId());
|
||||
String relativePath = path.toString().replace(new Path(config.getBasePath()) + "/", "");
|
||||
stat.setPath(relativePath);
|
||||
stat.setTotalWriteBytes(FSUtils.getFileSize(fs, path));
|
||||
stat.setTotalWriteErrors(status.getFailedRecords().size());
|
||||
status.setStat(stat);
|
||||
|
||||
return status;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieInsertException("Failed to close the Insert Handle for path " + path,
|
||||
e);
|
||||
}
|
||||
HoodieWriteStat stat = new HoodieWriteStat();
|
||||
stat.setNumWrites(recordsWritten);
|
||||
stat.setNumDeletes(recordsDeleted);
|
||||
stat.setPrevCommit(HoodieWriteStat.NULL_COMMIT);
|
||||
stat.setFileId(status.getFileId());
|
||||
String relativePath = path.toString().replace(new Path(config.getBasePath()) + "/", "");
|
||||
stat.setPath(relativePath);
|
||||
stat.setTotalWriteBytes(FSUtils.getFileSize(fs, path));
|
||||
stat.setTotalWriteErrors(status.getFailedRecords().size());
|
||||
status.setStat(stat);
|
||||
|
||||
return status;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieInsertException("Failed to close the Insert Handle for path " + path,
|
||||
e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -24,6 +24,7 @@ import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import java.io.IOException;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
@@ -31,68 +32,67 @@ import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
|
||||
private static Logger logger = LogManager.getLogger(HoodieIOHandle.class);
|
||||
protected final String commitTime;
|
||||
protected final HoodieWriteConfig config;
|
||||
protected final FileSystem fs;
|
||||
protected final HoodieTable<T> hoodieTable;
|
||||
protected HoodieTimeline hoodieTimeline;
|
||||
protected TableFileSystemView.ReadOptimizedView fileSystemView;
|
||||
protected final Schema schema;
|
||||
|
||||
public HoodieIOHandle(HoodieWriteConfig config, String commitTime,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
this.commitTime = commitTime;
|
||||
this.config = config;
|
||||
this.fs = FSUtils.getFs();
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.hoodieTimeline = hoodieTable.getCompletedCommitTimeline();
|
||||
this.fileSystemView = hoodieTable.getROFileSystemView();
|
||||
this.schema =
|
||||
HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||
private static Logger logger = LogManager.getLogger(HoodieIOHandle.class);
|
||||
protected final String commitTime;
|
||||
protected final HoodieWriteConfig config;
|
||||
protected final FileSystem fs;
|
||||
protected final HoodieTable<T> hoodieTable;
|
||||
protected HoodieTimeline hoodieTimeline;
|
||||
protected TableFileSystemView.ReadOptimizedView fileSystemView;
|
||||
protected final Schema schema;
|
||||
|
||||
public HoodieIOHandle(HoodieWriteConfig config, String commitTime,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
this.commitTime = commitTime;
|
||||
this.config = config;
|
||||
this.fs = FSUtils.getFs();
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.hoodieTimeline = hoodieTable.getCompletedCommitTimeline();
|
||||
this.fileSystemView = hoodieTable.getROFileSystemView();
|
||||
this.schema =
|
||||
HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||
}
|
||||
|
||||
public Path makeNewPath(String partitionPath, int taskPartitionId, String fileName) {
|
||||
Path path = new Path(config.getBasePath(), partitionPath);
|
||||
try {
|
||||
fs.mkdirs(path); // create a new partition as needed.
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to make dir " + path, e);
|
||||
}
|
||||
|
||||
public Path makeNewPath(String partitionPath, int taskPartitionId, String fileName) {
|
||||
Path path = new Path(config.getBasePath(), partitionPath);
|
||||
try {
|
||||
fs.mkdirs(path); // create a new partition as needed.
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to make dir " + path, e);
|
||||
return new Path(path.toString(),
|
||||
FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName));
|
||||
}
|
||||
|
||||
/**
|
||||
* Deletes any new tmp files written during the current commit, into the partition
|
||||
*/
|
||||
public static void cleanupTmpFilesFromCurrentCommit(HoodieWriteConfig config,
|
||||
String commitTime,
|
||||
String partitionPath,
|
||||
int taskPartitionId) {
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
try {
|
||||
FileStatus[] prevFailedFiles = fs.globStatus(new Path(String
|
||||
.format("%s/%s/%s", config.getBasePath(), partitionPath,
|
||||
FSUtils.maskWithoutFileId(commitTime, taskPartitionId))));
|
||||
if (prevFailedFiles != null) {
|
||||
logger.info("Deleting " + prevFailedFiles.length
|
||||
+ " files generated by previous failed attempts.");
|
||||
for (FileStatus status : prevFailedFiles) {
|
||||
fs.delete(status.getPath(), false);
|
||||
}
|
||||
|
||||
return new Path(path.toString(),
|
||||
FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName));
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to cleanup Temp files from commit " + commitTime,
|
||||
e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Deletes any new tmp files written during the current commit, into the partition
|
||||
*/
|
||||
public static void cleanupTmpFilesFromCurrentCommit(HoodieWriteConfig config,
|
||||
String commitTime,
|
||||
String partitionPath,
|
||||
int taskPartitionId) {
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
try {
|
||||
FileStatus[] prevFailedFiles = fs.globStatus(new Path(String
|
||||
.format("%s/%s/%s", config.getBasePath(), partitionPath,
|
||||
FSUtils.maskWithoutFileId(commitTime, taskPartitionId))));
|
||||
if (prevFailedFiles != null) {
|
||||
logger.info("Deleting " + prevFailedFiles.length
|
||||
+ " files generated by previous failed attempts.");
|
||||
for (FileStatus status : prevFailedFiles) {
|
||||
fs.delete(status.getPath(), false);
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to cleanup Temp files from commit " + commitTime,
|
||||
e);
|
||||
}
|
||||
}
|
||||
|
||||
public Schema getSchema() {
|
||||
return schema;
|
||||
}
|
||||
public Schema getSchema() {
|
||||
return schema;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -16,19 +16,23 @@
|
||||
|
||||
package com.uber.hoodie.io;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
|
||||
import com.uber.hoodie.common.util.ReflectionUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieWriteStat;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.ReflectionUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieUpsertException;
|
||||
import com.uber.hoodie.io.storage.HoodieStorageWriter;
|
||||
import com.uber.hoodie.io.storage.HoodieStorageWriterFactory;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.Optional;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -36,197 +40,197 @@ import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.TaskContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.Optional;
|
||||
|
||||
@SuppressWarnings("Duplicates")
|
||||
public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
|
||||
private static Logger logger = LogManager.getLogger(HoodieMergeHandle.class);
|
||||
|
||||
private WriteStatus writeStatus;
|
||||
private HashMap<String, HoodieRecord<T>> keyToNewRecords;
|
||||
private HoodieStorageWriter<IndexedRecord> storageWriter;
|
||||
private Path newFilePath;
|
||||
private Path oldFilePath;
|
||||
private long recordsWritten = 0;
|
||||
private long recordsDeleted = 0;
|
||||
private long updatedRecordsWritten = 0;
|
||||
private static Logger logger = LogManager.getLogger(HoodieMergeHandle.class);
|
||||
|
||||
public HoodieMergeHandle(HoodieWriteConfig config,
|
||||
String commitTime,
|
||||
HoodieTable<T> hoodieTable,
|
||||
Iterator<HoodieRecord<T>> recordItr,
|
||||
String fileId) {
|
||||
super(config, commitTime, hoodieTable);
|
||||
init(fileId, recordItr);
|
||||
}
|
||||
private WriteStatus writeStatus;
|
||||
private HashMap<String, HoodieRecord<T>> keyToNewRecords;
|
||||
private HoodieStorageWriter<IndexedRecord> storageWriter;
|
||||
private Path newFilePath;
|
||||
private Path oldFilePath;
|
||||
private long recordsWritten = 0;
|
||||
private long recordsDeleted = 0;
|
||||
private long updatedRecordsWritten = 0;
|
||||
|
||||
/**
|
||||
* Load the new incoming records in a map, and extract the old file path.
|
||||
*/
|
||||
private void init(String fileId, Iterator<HoodieRecord<T>> newRecordsItr) {
|
||||
WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName());
|
||||
writeStatus.setStat(new HoodieWriteStat());
|
||||
this.writeStatus = writeStatus;
|
||||
this.keyToNewRecords = new HashMap<>();
|
||||
public HoodieMergeHandle(HoodieWriteConfig config,
|
||||
String commitTime,
|
||||
HoodieTable<T> hoodieTable,
|
||||
Iterator<HoodieRecord<T>> recordItr,
|
||||
String fileId) {
|
||||
super(config, commitTime, hoodieTable);
|
||||
init(fileId, recordItr);
|
||||
}
|
||||
|
||||
try {
|
||||
// Load the new records in a map
|
||||
while (newRecordsItr.hasNext()) {
|
||||
HoodieRecord<T> record = newRecordsItr.next();
|
||||
// If the first record, we need to extract some info out
|
||||
if (oldFilePath == null) {
|
||||
String latestValidFilePath = fileSystemView
|
||||
.getLatestDataFiles(record.getPartitionPath())
|
||||
.filter(dataFile -> dataFile.getFileId().equals(fileId))
|
||||
.findFirst()
|
||||
.get().getFileName();
|
||||
writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath));
|
||||
/**
|
||||
* Load the new incoming records in a map, and extract the old file path.
|
||||
*/
|
||||
private void init(String fileId, Iterator<HoodieRecord<T>> newRecordsItr) {
|
||||
WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName());
|
||||
writeStatus.setStat(new HoodieWriteStat());
|
||||
this.writeStatus = writeStatus;
|
||||
this.keyToNewRecords = new HashMap<>();
|
||||
|
||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs,
|
||||
commitTime,
|
||||
new Path(config.getBasePath()),
|
||||
new Path(config.getBasePath(), record.getPartitionPath()));
|
||||
partitionMetadata.trySave(TaskContext.getPartitionId());
|
||||
try {
|
||||
// Load the new records in a map
|
||||
while (newRecordsItr.hasNext()) {
|
||||
HoodieRecord<T> record = newRecordsItr.next();
|
||||
// If the first record, we need to extract some info out
|
||||
if (oldFilePath == null) {
|
||||
String latestValidFilePath = fileSystemView
|
||||
.getLatestDataFiles(record.getPartitionPath())
|
||||
.filter(dataFile -> dataFile.getFileId().equals(fileId))
|
||||
.findFirst()
|
||||
.get().getFileName();
|
||||
writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath));
|
||||
|
||||
oldFilePath = new Path(
|
||||
config.getBasePath() + "/" + record.getPartitionPath() + "/"
|
||||
+ latestValidFilePath);
|
||||
String relativePath = new Path( record.getPartitionPath() + "/" + FSUtils
|
||||
.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)).toString();
|
||||
newFilePath = new Path(config.getBasePath(), relativePath);
|
||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs,
|
||||
commitTime,
|
||||
new Path(config.getBasePath()),
|
||||
new Path(config.getBasePath(), record.getPartitionPath()));
|
||||
partitionMetadata.trySave(TaskContext.getPartitionId());
|
||||
|
||||
// handle cases of partial failures, for update task
|
||||
if (fs.exists(newFilePath)) {
|
||||
fs.delete(newFilePath, false);
|
||||
}
|
||||
oldFilePath = new Path(
|
||||
config.getBasePath() + "/" + record.getPartitionPath() + "/"
|
||||
+ latestValidFilePath);
|
||||
String relativePath = new Path(record.getPartitionPath() + "/" + FSUtils
|
||||
.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)).toString();
|
||||
newFilePath = new Path(config.getBasePath(), relativePath);
|
||||
|
||||
logger.info(String.format("Merging new data into oldPath %s, as newPath %s",
|
||||
oldFilePath.toString(), newFilePath.toString()));
|
||||
// file name is same for all records, in this bunch
|
||||
writeStatus.setFileId(fileId);
|
||||
writeStatus.setPartitionPath(record.getPartitionPath());
|
||||
writeStatus.getStat().setFileId(fileId);
|
||||
writeStatus.getStat().setPath(relativePath);
|
||||
}
|
||||
keyToNewRecords.put(record.getRecordKey(), record);
|
||||
// update the new location of the record, so we know where to find it next
|
||||
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId));
|
||||
}
|
||||
// Create the writer for writing the new version file
|
||||
storageWriter = HoodieStorageWriterFactory
|
||||
.getStorageWriter(commitTime, newFilePath, hoodieTable, config, schema);
|
||||
// handle cases of partial failures, for update task
|
||||
if (fs.exists(newFilePath)) {
|
||||
fs.delete(newFilePath, false);
|
||||
}
|
||||
|
||||
} catch (Exception e) {
|
||||
logger.error("Error in update task at commit " + commitTime, e);
|
||||
writeStatus.setGlobalError(e);
|
||||
throw new HoodieUpsertException(
|
||||
"Failed to initialize HoodieUpdateHandle for FileId: " + fileId + " on commit "
|
||||
+ commitTime + " on path " + hoodieTable.getMetaClient().getBasePath(), e);
|
||||
logger.info(String.format("Merging new data into oldPath %s, as newPath %s",
|
||||
oldFilePath.toString(), newFilePath.toString()));
|
||||
// file name is same for all records, in this bunch
|
||||
writeStatus.setFileId(fileId);
|
||||
writeStatus.setPartitionPath(record.getPartitionPath());
|
||||
writeStatus.getStat().setFileId(fileId);
|
||||
writeStatus.getStat().setPath(relativePath);
|
||||
}
|
||||
keyToNewRecords.put(record.getRecordKey(), record);
|
||||
// update the new location of the record, so we know where to find it next
|
||||
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId));
|
||||
}
|
||||
// Create the writer for writing the new version file
|
||||
storageWriter = HoodieStorageWriterFactory
|
||||
.getStorageWriter(commitTime, newFilePath, hoodieTable, config, schema);
|
||||
|
||||
} catch (Exception e) {
|
||||
logger.error("Error in update task at commit " + commitTime, e);
|
||||
writeStatus.setGlobalError(e);
|
||||
throw new HoodieUpsertException(
|
||||
"Failed to initialize HoodieUpdateHandle for FileId: " + fileId + " on commit "
|
||||
+ commitTime + " on path " + hoodieTable.getMetaClient().getBasePath(), e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private boolean writeUpdateRecord(HoodieRecord<T> hoodieRecord, Optional<IndexedRecord> indexedRecord) {
|
||||
Optional recordMetadata = hoodieRecord.getData().getMetadata();
|
||||
try {
|
||||
if(indexedRecord.isPresent()) {
|
||||
storageWriter.writeAvroWithMetadata(indexedRecord.get(), hoodieRecord);
|
||||
recordsWritten++;
|
||||
updatedRecordsWritten++;
|
||||
} else {
|
||||
recordsDeleted++;
|
||||
}
|
||||
private boolean writeUpdateRecord(HoodieRecord<T> hoodieRecord,
|
||||
Optional<IndexedRecord> indexedRecord) {
|
||||
Optional recordMetadata = hoodieRecord.getData().getMetadata();
|
||||
try {
|
||||
if (indexedRecord.isPresent()) {
|
||||
storageWriter.writeAvroWithMetadata(indexedRecord.get(), hoodieRecord);
|
||||
recordsWritten++;
|
||||
updatedRecordsWritten++;
|
||||
} else {
|
||||
recordsDeleted++;
|
||||
}
|
||||
|
||||
hoodieRecord.deflate();
|
||||
writeStatus.markSuccess(hoodieRecord, recordMetadata);
|
||||
return true;
|
||||
} catch (Exception e) {
|
||||
logger.error("Error writing record "+ hoodieRecord, e);
|
||||
writeStatus.markFailure(hoodieRecord, e, recordMetadata);
|
||||
}
|
||||
return false;
|
||||
hoodieRecord.deflate();
|
||||
writeStatus.markSuccess(hoodieRecord, recordMetadata);
|
||||
return true;
|
||||
} catch (Exception e) {
|
||||
logger.error("Error writing record " + hoodieRecord, e);
|
||||
writeStatus.markFailure(hoodieRecord, e, recordMetadata);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Go through an old record. Here if we detect a newer version shows up, we write the new one to the file.
|
||||
*/
|
||||
public void write(GenericRecord oldRecord) {
|
||||
String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key);
|
||||
boolean copyOldRecord = true;
|
||||
if (keyToNewRecords.containsKey(key)) {
|
||||
try {
|
||||
Optional<IndexedRecord> combinedAvroRecord = hoodieRecord.getData().combineAndGetUpdateValue(oldRecord, schema);
|
||||
if (writeUpdateRecord(hoodieRecord, combinedAvroRecord)) {
|
||||
/**
|
||||
* Go through an old record. Here if we detect a newer version shows up, we write the new one to
|
||||
* the file.
|
||||
*/
|
||||
public void write(GenericRecord oldRecord) {
|
||||
String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key);
|
||||
boolean copyOldRecord = true;
|
||||
if (keyToNewRecords.containsKey(key)) {
|
||||
try {
|
||||
Optional<IndexedRecord> combinedAvroRecord = hoodieRecord.getData()
|
||||
.combineAndGetUpdateValue(oldRecord, schema);
|
||||
if (writeUpdateRecord(hoodieRecord, combinedAvroRecord)) {
|
||||
/* ONLY WHEN
|
||||
* 1) we have an update for this key AND
|
||||
* 2) We are able to successfully write the the combined new value
|
||||
*
|
||||
* We no longer need to copy the old record over.
|
||||
*/
|
||||
copyOldRecord = false;
|
||||
}
|
||||
keyToNewRecords.remove(key);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieUpsertException("Failed to combine/merge new record with old value in storage, for new record {"
|
||||
+ keyToNewRecords.get(key) + "}, old value {" + oldRecord + "}", e);
|
||||
}
|
||||
}
|
||||
|
||||
if (copyOldRecord) {
|
||||
// this should work as it is, since this is an existing record
|
||||
String errMsg = "Failed to merge old record into new file for key " + key + " from old file "
|
||||
+ getOldFilePath() + " to new file " + newFilePath;
|
||||
try {
|
||||
storageWriter.writeAvro(key, oldRecord);
|
||||
} catch (ClassCastException e) {
|
||||
logger.error(
|
||||
"Schema mismatch when rewriting old record " + oldRecord + " from file "
|
||||
+ getOldFilePath() + " to file " + newFilePath + " with schema " + schema
|
||||
.toString(true));
|
||||
throw new HoodieUpsertException(errMsg, e);
|
||||
} catch (IOException e) {
|
||||
logger.error("Failed to merge old record into new file for key " + key + " from old file "
|
||||
+ getOldFilePath() + " to new file " + newFilePath, e);
|
||||
throw new HoodieUpsertException(errMsg, e);
|
||||
}
|
||||
recordsWritten ++;
|
||||
copyOldRecord = false;
|
||||
}
|
||||
keyToNewRecords.remove(key);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieUpsertException(
|
||||
"Failed to combine/merge new record with old value in storage, for new record {"
|
||||
+ keyToNewRecords.get(key) + "}, old value {" + oldRecord + "}", e);
|
||||
}
|
||||
}
|
||||
|
||||
public void close() {
|
||||
try {
|
||||
// write out any pending records (this can happen when inserts are turned into updates)
|
||||
Iterator<String> pendingRecordsItr = keyToNewRecords.keySet().iterator();
|
||||
while (pendingRecordsItr.hasNext()) {
|
||||
String key = pendingRecordsItr.next();
|
||||
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key);
|
||||
writeUpdateRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(schema));
|
||||
}
|
||||
keyToNewRecords.clear();
|
||||
|
||||
if (storageWriter != null) {
|
||||
storageWriter.close();
|
||||
}
|
||||
|
||||
writeStatus.getStat().setTotalWriteBytes(FSUtils.getFileSize(fs, newFilePath));
|
||||
writeStatus.getStat().setNumWrites(recordsWritten);
|
||||
writeStatus.getStat().setNumDeletes(recordsDeleted);
|
||||
writeStatus.getStat().setNumUpdateWrites(updatedRecordsWritten);
|
||||
writeStatus.getStat().setTotalWriteErrors(writeStatus.getFailedRecords().size());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
|
||||
}
|
||||
if (copyOldRecord) {
|
||||
// this should work as it is, since this is an existing record
|
||||
String errMsg = "Failed to merge old record into new file for key " + key + " from old file "
|
||||
+ getOldFilePath() + " to new file " + newFilePath;
|
||||
try {
|
||||
storageWriter.writeAvro(key, oldRecord);
|
||||
} catch (ClassCastException e) {
|
||||
logger.error(
|
||||
"Schema mismatch when rewriting old record " + oldRecord + " from file "
|
||||
+ getOldFilePath() + " to file " + newFilePath + " with schema " + schema
|
||||
.toString(true));
|
||||
throw new HoodieUpsertException(errMsg, e);
|
||||
} catch (IOException e) {
|
||||
logger.error("Failed to merge old record into new file for key " + key + " from old file "
|
||||
+ getOldFilePath() + " to new file " + newFilePath, e);
|
||||
throw new HoodieUpsertException(errMsg, e);
|
||||
}
|
||||
recordsWritten++;
|
||||
}
|
||||
}
|
||||
|
||||
public Path getOldFilePath() {
|
||||
return oldFilePath;
|
||||
}
|
||||
public void close() {
|
||||
try {
|
||||
// write out any pending records (this can happen when inserts are turned into updates)
|
||||
Iterator<String> pendingRecordsItr = keyToNewRecords.keySet().iterator();
|
||||
while (pendingRecordsItr.hasNext()) {
|
||||
String key = pendingRecordsItr.next();
|
||||
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key);
|
||||
writeUpdateRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(schema));
|
||||
}
|
||||
keyToNewRecords.clear();
|
||||
|
||||
public WriteStatus getWriteStatus() {
|
||||
return writeStatus;
|
||||
if (storageWriter != null) {
|
||||
storageWriter.close();
|
||||
}
|
||||
|
||||
writeStatus.getStat().setTotalWriteBytes(FSUtils.getFileSize(fs, newFilePath));
|
||||
writeStatus.getStat().setNumWrites(recordsWritten);
|
||||
writeStatus.getStat().setNumDeletes(recordsDeleted);
|
||||
writeStatus.getStat().setNumUpdateWrites(updatedRecordsWritten);
|
||||
writeStatus.getStat().setTotalWriteErrors(writeStatus.getFailedRecords().size());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
|
||||
}
|
||||
}
|
||||
|
||||
public Path getOldFilePath() {
|
||||
return oldFilePath;
|
||||
}
|
||||
|
||||
public WriteStatus getWriteStatus() {
|
||||
return writeStatus;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,7 +18,6 @@ package com.uber.hoodie.io.compact;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.io.compact.strategy.CompactionStrategy;
|
||||
import java.io.Serializable;
|
||||
@@ -27,8 +26,8 @@ import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Encapsulates all the needed information about a compaction
|
||||
* and make a decision whether this compaction is effective or not
|
||||
* Encapsulates all the needed information about a compaction and make a decision whether this
|
||||
* compaction is effective or not
|
||||
*
|
||||
* @see CompactionStrategy
|
||||
*/
|
||||
|
||||
@@ -22,29 +22,28 @@ import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.Date;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
/**
|
||||
* A HoodieCompactor runs compaction on a hoodie table
|
||||
*/
|
||||
public interface HoodieCompactor extends Serializable {
|
||||
/**
|
||||
* Compact the delta files with the data files
|
||||
* @throws Exception
|
||||
*/
|
||||
HoodieCompactionMetadata compact(JavaSparkContext jsc, final HoodieWriteConfig config,
|
||||
HoodieTable hoodieTable) throws Exception;
|
||||
|
||||
/**
|
||||
* Compact the delta files with the data files
|
||||
*/
|
||||
HoodieCompactionMetadata compact(JavaSparkContext jsc, final HoodieWriteConfig config,
|
||||
HoodieTable hoodieTable) throws Exception;
|
||||
|
||||
|
||||
// Helper methods
|
||||
default String startCompactionCommit(HoodieTable hoodieTable) {
|
||||
String commitTime = HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date());
|
||||
HoodieActiveTimeline activeTimeline = hoodieTable.getActiveTimeline();
|
||||
activeTimeline
|
||||
.createInflight(new HoodieInstant(true, HoodieTimeline.COMPACTION_ACTION, commitTime));
|
||||
return commitTime;
|
||||
}
|
||||
// Helper methods
|
||||
default String startCompactionCommit(HoodieTable hoodieTable) {
|
||||
String commitTime = HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date());
|
||||
HoodieActiveTimeline activeTimeline = hoodieTable.getActiveTimeline();
|
||||
activeTimeline
|
||||
.createInflight(new HoodieInstant(true, HoodieTimeline.COMPACTION_ACTION, commitTime));
|
||||
return commitTime;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -16,14 +16,14 @@
|
||||
|
||||
package com.uber.hoodie.io.compact;
|
||||
|
||||
import static java.util.stream.Collectors.toList;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.CompactionWriteStat;
|
||||
import com.uber.hoodie.common.model.HoodieAvroPayload;
|
||||
import com.uber.hoodie.common.model.HoodieCompactionMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
@@ -36,7 +36,12 @@ import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieCompactionException;
|
||||
import com.uber.hoodie.table.HoodieCopyOnWriteTable;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.StreamSupport;
|
||||
import org.apache.avro.Schema;
|
||||
@@ -46,18 +51,10 @@ import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.FlatMapFunction;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
|
||||
import static java.util.stream.Collectors.*;
|
||||
|
||||
/**
|
||||
* HoodieRealtimeTableCompactor compacts a hoodie table with merge on read storage.
|
||||
* Computes all possible compactions, passes it through a CompactionFilter and executes
|
||||
* all the compactions and writes a new version of base files and make a normal commit
|
||||
* HoodieRealtimeTableCompactor compacts a hoodie table with merge on read storage. Computes all
|
||||
* possible compactions, passes it through a CompactionFilter and executes all the compactions and
|
||||
* writes a new version of base files and make a normal commit
|
||||
*
|
||||
* @see HoodieCompactor
|
||||
*/
|
||||
@@ -80,7 +77,8 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
String compactionCommit = startCompactionCommit(hoodieTable);
|
||||
log.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommit);
|
||||
List<String> partitionPaths =
|
||||
FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), config.shouldAssumeDatePartitioning());
|
||||
FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
|
||||
config.shouldAssumeDatePartitioning());
|
||||
|
||||
log.info("Compaction looking for files to compact in " + partitionPaths + " partitions");
|
||||
List<CompactionOperation> operations =
|
||||
@@ -89,7 +87,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
.getRTFileSystemView()
|
||||
.getLatestFileSlices(partitionPath)
|
||||
.map(s -> new CompactionOperation(s.getDataFile().get(),
|
||||
partitionPath, s.getLogFiles().collect(Collectors.toList()), config))
|
||||
partitionPath, s.getLogFiles().collect(Collectors.toList()), config))
|
||||
.collect(toList()).iterator()).collect();
|
||||
log.info("Total of " + operations.size() + " compactions are retrieved");
|
||||
|
||||
@@ -150,14 +148,15 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
// Since a DeltaCommit is not defined yet, reading all the records. revisit this soon.
|
||||
|
||||
String maxInstantTime = metaClient.getActiveTimeline()
|
||||
.getTimelineOfActions(
|
||||
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION,
|
||||
HoodieTimeline.COMPACTION_ACTION,
|
||||
HoodieTimeline.DELTA_COMMIT_ACTION))
|
||||
.filterCompletedInstants().lastInstant().get().getTimestamp();
|
||||
.getTimelineOfActions(
|
||||
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION,
|
||||
HoodieTimeline.COMPACTION_ACTION,
|
||||
HoodieTimeline.DELTA_COMMIT_ACTION))
|
||||
.filterCompletedInstants().lastInstant().get().getTimestamp();
|
||||
|
||||
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, metaClient.getBasePath(),
|
||||
operation.getDeltaFilePaths(), readerSchema, maxInstantTime);
|
||||
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs,
|
||||
metaClient.getBasePath(),
|
||||
operation.getDeltaFilePaths(), readerSchema, maxInstantTime);
|
||||
if (!scanner.iterator().hasNext()) {
|
||||
return Lists.newArrayList();
|
||||
}
|
||||
|
||||
@@ -28,8 +28,8 @@ import java.util.Map;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* CompactionStrategy which looks at total IO to be done for the compaction (read + write)
|
||||
* and limits the list of compactions to be under a configured limit on the IO
|
||||
* CompactionStrategy which looks at total IO to be done for the compaction (read + write) and
|
||||
* limits the list of compactions to be under a configured limit on the IO
|
||||
*
|
||||
* @see CompactionStrategy
|
||||
*/
|
||||
@@ -46,7 +46,7 @@ public class BoundedIOCompactionStrategy implements CompactionStrategy {
|
||||
// Total size of all the log files
|
||||
Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(
|
||||
Optional::isPresent).map(Optional::get).reduce(
|
||||
(size1, size2) -> size1 + size2).orElse(0L);
|
||||
(size1, size2) -> size1 + size2).orElse(0L);
|
||||
// Total read will be the base file + all the log files
|
||||
Long totalIORead = FSUtils.getSizeInMB(dataFile.getFileSize() + totalLogFileSize);
|
||||
// Total write will be similar to the size of the base file
|
||||
@@ -62,7 +62,8 @@ public class BoundedIOCompactionStrategy implements CompactionStrategy {
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<CompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig, List<CompactionOperation> operations) {
|
||||
public List<CompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
|
||||
List<CompactionOperation> operations) {
|
||||
// Iterate through the operations in order and accept operations as long as we are within the IO limit
|
||||
// Preserves the original ordering of compactions
|
||||
List<CompactionOperation> finalOperations = Lists.newArrayList();
|
||||
|
||||
@@ -25,12 +25,12 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Strategy for compaction. Pluggable implementation of define how compaction should be done.
|
||||
* The implementations of this interface can capture the relevant metrics to order and filter
|
||||
* the final list of compaction operation to run in a single compaction.
|
||||
* Strategy for compaction. Pluggable implementation of define how compaction should be done. The
|
||||
* implementations of this interface can capture the relevant metrics to order and filter the final
|
||||
* list of compaction operation to run in a single compaction.
|
||||
*
|
||||
* Implementation of CompactionStrategy cannot hold any state.
|
||||
* Difference instantiations can be passed in every time
|
||||
* Implementation of CompactionStrategy cannot hold any state. Difference instantiations can be
|
||||
* passed in every time
|
||||
*
|
||||
* @see com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor
|
||||
* @see CompactionOperation
|
||||
@@ -38,8 +38,8 @@ import java.util.Map;
|
||||
public interface CompactionStrategy extends Serializable {
|
||||
|
||||
/**
|
||||
* Callback hook when a CompactionOperation is created. Individual strategies can
|
||||
* capture the metrics they need to decide on the priority.
|
||||
* Callback hook when a CompactionOperation is created. Individual strategies can capture the
|
||||
* metrics they need to decide on the priority.
|
||||
*
|
||||
* @param dataFile - Base file to compact
|
||||
* @param partitionPath - Partition path
|
||||
@@ -50,8 +50,8 @@ public interface CompactionStrategy extends Serializable {
|
||||
List<HoodieLogFile> logFiles);
|
||||
|
||||
/**
|
||||
* Order and Filter the list of compactions. Use the metrics captured with the
|
||||
* captureMetrics to order and filter out compactions
|
||||
* Order and Filter the list of compactions. Use the metrics captured with the captureMetrics to
|
||||
* order and filter out compactions
|
||||
*
|
||||
* @param writeConfig - HoodieWriteConfig - config for this compaction is passed in
|
||||
* @param operations - list of compactions collected
|
||||
|
||||
@@ -27,8 +27,8 @@ import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* LogFileSizeBasedCompactionStrategy orders the compactions based on the total log files size
|
||||
* and limits the compactions within a configured IO bound
|
||||
* LogFileSizeBasedCompactionStrategy orders the compactions based on the total log files size and
|
||||
* limits the compactions within a configured IO bound
|
||||
*
|
||||
* @see BoundedIOCompactionStrategy
|
||||
* @see CompactionStrategy
|
||||
|
||||
@@ -25,9 +25,9 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* UnBoundedCompactionStrategy will not change ordering or filter any compaction.
|
||||
* It is a pass-through and will compact all the base files which has a log file.
|
||||
* This usually means no-intelligence on compaction.
|
||||
* UnBoundedCompactionStrategy will not change ordering or filter any compaction. It is a
|
||||
* pass-through and will compact all the base files which has a log file. This usually means
|
||||
* no-intelligence on compaction.
|
||||
*
|
||||
* @see CompactionStrategy
|
||||
*/
|
||||
|
||||
@@ -17,50 +17,50 @@
|
||||
package com.uber.hoodie.io.storage;
|
||||
|
||||
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||
|
||||
public class HoodieParquetConfig {
|
||||
private HoodieAvroWriteSupport writeSupport;
|
||||
private CompressionCodecName compressionCodecName;
|
||||
private int blockSize;
|
||||
private int pageSize;
|
||||
private int maxFileSize;
|
||||
private Configuration hadoopConf;
|
||||
|
||||
public HoodieParquetConfig(HoodieAvroWriteSupport writeSupport,
|
||||
CompressionCodecName compressionCodecName, int blockSize, int pageSize, int maxFileSize,
|
||||
Configuration hadoopConf) {
|
||||
this.writeSupport = writeSupport;
|
||||
this.compressionCodecName = compressionCodecName;
|
||||
this.blockSize = blockSize;
|
||||
this.pageSize = pageSize;
|
||||
this.maxFileSize = maxFileSize;
|
||||
this.hadoopConf = hadoopConf;
|
||||
}
|
||||
private HoodieAvroWriteSupport writeSupport;
|
||||
private CompressionCodecName compressionCodecName;
|
||||
private int blockSize;
|
||||
private int pageSize;
|
||||
private int maxFileSize;
|
||||
private Configuration hadoopConf;
|
||||
|
||||
public HoodieAvroWriteSupport getWriteSupport() {
|
||||
return writeSupport;
|
||||
}
|
||||
public HoodieParquetConfig(HoodieAvroWriteSupport writeSupport,
|
||||
CompressionCodecName compressionCodecName, int blockSize, int pageSize, int maxFileSize,
|
||||
Configuration hadoopConf) {
|
||||
this.writeSupport = writeSupport;
|
||||
this.compressionCodecName = compressionCodecName;
|
||||
this.blockSize = blockSize;
|
||||
this.pageSize = pageSize;
|
||||
this.maxFileSize = maxFileSize;
|
||||
this.hadoopConf = hadoopConf;
|
||||
}
|
||||
|
||||
public CompressionCodecName getCompressionCodecName() {
|
||||
return compressionCodecName;
|
||||
}
|
||||
public HoodieAvroWriteSupport getWriteSupport() {
|
||||
return writeSupport;
|
||||
}
|
||||
|
||||
public int getBlockSize() {
|
||||
return blockSize;
|
||||
}
|
||||
public CompressionCodecName getCompressionCodecName() {
|
||||
return compressionCodecName;
|
||||
}
|
||||
|
||||
public int getPageSize() {
|
||||
return pageSize;
|
||||
}
|
||||
public int getBlockSize() {
|
||||
return blockSize;
|
||||
}
|
||||
|
||||
public int getMaxFileSize() {
|
||||
return maxFileSize;
|
||||
}
|
||||
public int getPageSize() {
|
||||
return pageSize;
|
||||
}
|
||||
|
||||
public Configuration getHadoopConf() {
|
||||
return hadoopConf;
|
||||
}
|
||||
public int getMaxFileSize() {
|
||||
return maxFileSize;
|
||||
}
|
||||
|
||||
public Configuration getHadoopConf() {
|
||||
return hadoopConf;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -20,6 +20,8 @@ import com.uber.hoodie.avro.HoodieAvroWriteSupport;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
@@ -30,79 +32,76 @@ import org.apache.parquet.hadoop.ParquetFileWriter;
|
||||
import org.apache.parquet.hadoop.ParquetWriter;
|
||||
import org.apache.spark.TaskContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
/**
|
||||
* HoodieParquetWriter extends the ParquetWriter to help limit the size of underlying file.
|
||||
* Provides a way to check if the current file can take more records with the <code>canWrite()</code>
|
||||
*
|
||||
* @param <T>
|
||||
* HoodieParquetWriter extends the ParquetWriter to help limit the size of underlying file. Provides
|
||||
* a way to check if the current file can take more records with the <code>canWrite()</code>
|
||||
*/
|
||||
public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends IndexedRecord>
|
||||
extends ParquetWriter<IndexedRecord> implements HoodieStorageWriter<R> {
|
||||
private static double STREAM_COMPRESSION_RATIO = 0.1;
|
||||
private static AtomicLong recordIndex = new AtomicLong(1);
|
||||
|
||||
private static double STREAM_COMPRESSION_RATIO = 0.1;
|
||||
private static AtomicLong recordIndex = new AtomicLong(1);
|
||||
|
||||
|
||||
private final Path file;
|
||||
private final HoodieWrapperFileSystem fs;
|
||||
private final long maxFileSize;
|
||||
private final HoodieAvroWriteSupport writeSupport;
|
||||
private final String commitTime;
|
||||
private final Schema schema;
|
||||
private final Path file;
|
||||
private final HoodieWrapperFileSystem fs;
|
||||
private final long maxFileSize;
|
||||
private final HoodieAvroWriteSupport writeSupport;
|
||||
private final String commitTime;
|
||||
private final Schema schema;
|
||||
|
||||
|
||||
private static Configuration registerFileSystem(Configuration conf) {
|
||||
Configuration returnConf = new Configuration(conf);
|
||||
String scheme = FileSystem.getDefaultUri(conf).getScheme();
|
||||
returnConf.set("fs." + HoodieWrapperFileSystem.getHoodieScheme(scheme) + ".impl",
|
||||
HoodieWrapperFileSystem.class.getName());
|
||||
return returnConf;
|
||||
}
|
||||
private static Configuration registerFileSystem(Configuration conf) {
|
||||
Configuration returnConf = new Configuration(conf);
|
||||
String scheme = FileSystem.getDefaultUri(conf).getScheme();
|
||||
returnConf.set("fs." + HoodieWrapperFileSystem.getHoodieScheme(scheme) + ".impl",
|
||||
HoodieWrapperFileSystem.class.getName());
|
||||
return returnConf;
|
||||
}
|
||||
|
||||
public HoodieParquetWriter(String commitTime, Path file,
|
||||
HoodieParquetConfig parquetConfig, Schema schema) throws IOException {
|
||||
super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
|
||||
ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(),
|
||||
parquetConfig.getCompressionCodecName(), parquetConfig.getBlockSize(),
|
||||
parquetConfig.getPageSize(), parquetConfig.getPageSize(),
|
||||
ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED,
|
||||
ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, ParquetWriter.DEFAULT_WRITER_VERSION,
|
||||
registerFileSystem(parquetConfig.getHadoopConf()));
|
||||
this.file =
|
||||
HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
|
||||
this.fs = (HoodieWrapperFileSystem) this.file
|
||||
.getFileSystem(registerFileSystem(parquetConfig.getHadoopConf()));
|
||||
// We cannot accurately measure the snappy compressed output file size. We are choosing a conservative 10%
|
||||
// TODO - compute this compression ratio dynamically by looking at the bytes written to the stream and the actual file size reported by HDFS
|
||||
this.maxFileSize = parquetConfig.getMaxFileSize() + Math
|
||||
.round(parquetConfig.getMaxFileSize() * STREAM_COMPRESSION_RATIO);
|
||||
this.writeSupport = parquetConfig.getWriteSupport();
|
||||
this.commitTime = commitTime;
|
||||
this.schema = schema;
|
||||
}
|
||||
public HoodieParquetWriter(String commitTime, Path file,
|
||||
HoodieParquetConfig parquetConfig, Schema schema) throws IOException {
|
||||
super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
|
||||
ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(),
|
||||
parquetConfig.getCompressionCodecName(), parquetConfig.getBlockSize(),
|
||||
parquetConfig.getPageSize(), parquetConfig.getPageSize(),
|
||||
ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED,
|
||||
ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, ParquetWriter.DEFAULT_WRITER_VERSION,
|
||||
registerFileSystem(parquetConfig.getHadoopConf()));
|
||||
this.file =
|
||||
HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
|
||||
this.fs = (HoodieWrapperFileSystem) this.file
|
||||
.getFileSystem(registerFileSystem(parquetConfig.getHadoopConf()));
|
||||
// We cannot accurately measure the snappy compressed output file size. We are choosing a conservative 10%
|
||||
// TODO - compute this compression ratio dynamically by looking at the bytes written to the stream and the actual file size reported by HDFS
|
||||
this.maxFileSize = parquetConfig.getMaxFileSize() + Math
|
||||
.round(parquetConfig.getMaxFileSize() * STREAM_COMPRESSION_RATIO);
|
||||
this.writeSupport = parquetConfig.getWriteSupport();
|
||||
this.commitTime = commitTime;
|
||||
this.schema = schema;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void writeAvroWithMetadata(R avroRecord, HoodieRecord record) throws IOException {
|
||||
String seqId = HoodieRecord.generateSequenceId(commitTime, TaskContext.getPartitionId(),
|
||||
recordIndex.getAndIncrement());
|
||||
HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord,
|
||||
record.getRecordKey(),
|
||||
record.getPartitionPath(),
|
||||
file.getName());
|
||||
HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord, commitTime, seqId);
|
||||
super.write(avroRecord);
|
||||
writeSupport.add(record.getRecordKey());
|
||||
}
|
||||
@Override
|
||||
public void writeAvroWithMetadata(R avroRecord, HoodieRecord record) throws IOException {
|
||||
String seqId = HoodieRecord.generateSequenceId(commitTime, TaskContext.getPartitionId(),
|
||||
recordIndex.getAndIncrement());
|
||||
HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord,
|
||||
record.getRecordKey(),
|
||||
record.getPartitionPath(),
|
||||
file.getName());
|
||||
HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord, commitTime, seqId);
|
||||
super.write(avroRecord);
|
||||
writeSupport.add(record.getRecordKey());
|
||||
}
|
||||
|
||||
public boolean canWrite() {
|
||||
return fs.getBytesWritten(file) < maxFileSize;
|
||||
}
|
||||
public boolean canWrite() {
|
||||
return fs.getBytesWritten(file) < maxFileSize;
|
||||
}
|
||||
|
||||
@Override public void writeAvro(String key, IndexedRecord object) throws IOException {
|
||||
super.write(object);
|
||||
writeSupport.add(key);
|
||||
}
|
||||
@Override
|
||||
public void writeAvro(String key, IndexedRecord object) throws IOException {
|
||||
super.write(object);
|
||||
writeSupport.add(key);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -17,13 +17,16 @@
|
||||
package com.uber.hoodie.io.storage;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import java.io.IOException;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public interface HoodieStorageWriter<R extends IndexedRecord> {
|
||||
void writeAvroWithMetadata(R newRecord, HoodieRecord record) throws IOException;
|
||||
boolean canWrite();
|
||||
void close() throws IOException;
|
||||
void writeAvro(String key, R oldRecord) throws IOException;
|
||||
|
||||
void writeAvroWithMetadata(R newRecord, HoodieRecord record) throws IOException;
|
||||
|
||||
boolean canWrite();
|
||||
|
||||
void close() throws IOException;
|
||||
|
||||
void writeAvro(String key, R oldRecord) throws IOException;
|
||||
}
|
||||
|
||||
@@ -16,42 +16,42 @@
|
||||
|
||||
package com.uber.hoodie.io.storage;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
|
||||
import com.uber.hoodie.common.BloomFilter;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import java.io.IOException;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.parquet.avro.AvroSchemaConverter;
|
||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class HoodieStorageWriterFactory {
|
||||
public static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> getStorageWriter(
|
||||
String commitTime, Path path, HoodieTable<T> hoodieTable, HoodieWriteConfig config, Schema schema)
|
||||
throws IOException {
|
||||
//TODO - based on the metadata choose the implementation of HoodieStorageWriter
|
||||
// Currently only parquet is supported
|
||||
return newParquetStorageWriter(commitTime, path, config, schema);
|
||||
}
|
||||
|
||||
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> newParquetStorageWriter(
|
||||
String commitTime, Path path, HoodieWriteConfig config, Schema schema) throws IOException {
|
||||
BloomFilter filter =
|
||||
new BloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP());
|
||||
HoodieAvroWriteSupport writeSupport =
|
||||
new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter);
|
||||
public static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> getStorageWriter(
|
||||
String commitTime, Path path, HoodieTable<T> hoodieTable, HoodieWriteConfig config,
|
||||
Schema schema)
|
||||
throws IOException {
|
||||
//TODO - based on the metadata choose the implementation of HoodieStorageWriter
|
||||
// Currently only parquet is supported
|
||||
return newParquetStorageWriter(commitTime, path, config, schema);
|
||||
}
|
||||
|
||||
HoodieParquetConfig parquetConfig =
|
||||
new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP,
|
||||
config.getParquetBlockSize(), config.getParquetPageSize(),
|
||||
config.getParquetMaxFileSize(), FSUtils.getFs().getConf());
|
||||
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> newParquetStorageWriter(
|
||||
String commitTime, Path path, HoodieWriteConfig config, Schema schema) throws IOException {
|
||||
BloomFilter filter =
|
||||
new BloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP());
|
||||
HoodieAvroWriteSupport writeSupport =
|
||||
new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter);
|
||||
|
||||
return new HoodieParquetWriter<>(commitTime, path, parquetConfig, schema);
|
||||
}
|
||||
HoodieParquetConfig parquetConfig =
|
||||
new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP,
|
||||
config.getParquetBlockSize(), config.getParquetPageSize(),
|
||||
config.getParquetMaxFileSize(), FSUtils.getFs().getConf());
|
||||
|
||||
return new HoodieParquetWriter<>(commitTime, path, parquetConfig, schema);
|
||||
}
|
||||
}
|
||||
|
||||
File diff suppressed because it is too large
Load Diff
@@ -16,44 +16,47 @@
|
||||
|
||||
package com.uber.hoodie.io.storage;
|
||||
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
|
||||
/**
|
||||
* Wrapper over <code>FSDataOutputStream</code> to keep track of the size of the written bytes.
|
||||
* This gives a cheap way to check on the underlying file size.
|
||||
* Wrapper over <code>FSDataOutputStream</code> to keep track of the size of the written bytes. This
|
||||
* gives a cheap way to check on the underlying file size.
|
||||
*/
|
||||
public class SizeAwareFSDataOutputStream extends FSDataOutputStream {
|
||||
// A callback to call when the output stream is closed.
|
||||
private final Runnable closeCallback;
|
||||
// Keep track of the bytes written
|
||||
private final AtomicLong bytesWritten = new AtomicLong(0L);
|
||||
|
||||
public SizeAwareFSDataOutputStream(FSDataOutputStream out, Runnable closeCallback)
|
||||
throws IOException {
|
||||
super(out);
|
||||
this.closeCallback = closeCallback;
|
||||
}
|
||||
// A callback to call when the output stream is closed.
|
||||
private final Runnable closeCallback;
|
||||
// Keep track of the bytes written
|
||||
private final AtomicLong bytesWritten = new AtomicLong(0L);
|
||||
|
||||
@Override public synchronized void write(byte[] b, int off, int len) throws IOException {
|
||||
bytesWritten.addAndGet(len);
|
||||
super.write(b, off, len);
|
||||
}
|
||||
public SizeAwareFSDataOutputStream(FSDataOutputStream out, Runnable closeCallback)
|
||||
throws IOException {
|
||||
super(out);
|
||||
this.closeCallback = closeCallback;
|
||||
}
|
||||
|
||||
@Override public void write(byte[] b) throws IOException {
|
||||
bytesWritten.addAndGet(b.length);
|
||||
super.write(b);
|
||||
}
|
||||
@Override
|
||||
public synchronized void write(byte[] b, int off, int len) throws IOException {
|
||||
bytesWritten.addAndGet(len);
|
||||
super.write(b, off, len);
|
||||
}
|
||||
|
||||
@Override public void close() throws IOException {
|
||||
super.close();
|
||||
closeCallback.run();
|
||||
}
|
||||
@Override
|
||||
public void write(byte[] b) throws IOException {
|
||||
bytesWritten.addAndGet(b.length);
|
||||
super.write(b);
|
||||
}
|
||||
|
||||
public long getBytesWritten() {
|
||||
return bytesWritten.get();
|
||||
}
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
super.close();
|
||||
closeCallback.run();
|
||||
}
|
||||
|
||||
public long getBytesWritten() {
|
||||
return bytesWritten.get();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user