CodeStyle formatting to conform to basic Checkstyle rules.
The code-style rules follow google style with some changes: 1. Increase line length from 100 to 120 2. Disable JavaDoc related checkstyles as this needs more manual work. Both source and test code are checked for code-style
This commit is contained in:
committed by
vinoth chandar
parent
987f5d6b96
commit
788e4f2d2e
@@ -37,14 +37,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 org.apache.spark.util.SizeEstimator;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Comparator;
|
||||
@@ -53,6 +45,13 @@ 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;
|
||||
import org.apache.spark.util.SizeEstimator;
|
||||
|
||||
/**
|
||||
* IO Operation to append data onto an existing file.
|
||||
@@ -61,14 +60,13 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieAppendHandle.class);
|
||||
private static AtomicLong recordIndex = new AtomicLong(1);
|
||||
|
||||
private TableFileSystemView.RealtimeView fileSystemView;
|
||||
private final WriteStatus writeStatus;
|
||||
private final String fileId;
|
||||
private String partitionPath;
|
||||
private Iterator<HoodieRecord<T>> recordItr;
|
||||
List<IndexedRecord> recordList = new ArrayList<>();
|
||||
List<String> keysToDelete = new ArrayList<>();
|
||||
private TableFileSystemView.RealtimeView fileSystemView;
|
||||
private String partitionPath;
|
||||
private Iterator<HoodieRecord<T>> recordItr;
|
||||
private long recordsWritten = 0;
|
||||
private long recordsDeleted = 0;
|
||||
private long averageRecordSize = 0;
|
||||
@@ -76,11 +74,8 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
private Writer writer;
|
||||
private boolean doInit = true;
|
||||
|
||||
public HoodieAppendHandle(HoodieWriteConfig config,
|
||||
String commitTime,
|
||||
HoodieTable<T> hoodieTable,
|
||||
String fileId,
|
||||
Iterator<HoodieRecord<T>> recordItr) {
|
||||
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());
|
||||
@@ -93,42 +88,40 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
private void init(String partitionPath) {
|
||||
|
||||
// extract some information from the first record
|
||||
FileSlice fileSlice = fileSystemView.getLatestFileSlices(partitionPath)
|
||||
.filter(fileSlice1 -> fileSlice1.getDataFile().get().getFileId().equals(fileId))
|
||||
.findFirst().get();
|
||||
// HACK(vc) This also assumes a base file. It will break, if appending without one.
|
||||
String latestValidFilePath = fileSlice.getDataFile().get().getFileName();
|
||||
String baseCommitTime = FSUtils.getCommitTime(latestValidFilePath);
|
||||
writeStatus.getStat().setPrevCommit(baseCommitTime);
|
||||
writeStatus.setFileId(fileId);
|
||||
writeStatus.setPartitionPath(partitionPath);
|
||||
writeStatus.getStat().setFileId(fileId);
|
||||
this.partitionPath = partitionPath;
|
||||
FileSlice fileSlice = fileSystemView.getLatestFileSlices(partitionPath)
|
||||
.filter(fileSlice1 -> fileSlice1.getDataFile().get().getFileId().equals(fileId)).findFirst()
|
||||
.get();
|
||||
// HACK(vc) This also assumes a base file. It will break, if appending without one.
|
||||
String latestValidFilePath = fileSlice.getDataFile().get().getFileName();
|
||||
String baseCommitTime = FSUtils.getCommitTime(latestValidFilePath);
|
||||
writeStatus.getStat().setPrevCommit(baseCommitTime);
|
||||
writeStatus.setFileId(fileId);
|
||||
writeStatus.setPartitionPath(partitionPath);
|
||||
writeStatus.getStat().setFileId(fileId);
|
||||
this.partitionPath = partitionPath;
|
||||
|
||||
try {
|
||||
this.writer = HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath))
|
||||
.withFileId(fileId).overBaseCommit(baseCommitTime).withLogVersion(fileSlice.getLogFiles()
|
||||
.map(logFile -> logFile.getLogVersion())
|
||||
.max(Comparator.naturalOrder()).orElse(HoodieLogFile.LOGFILE_BASE_VERSION))
|
||||
.withSizeThreshold(config.getLogFileMaxSize())
|
||||
.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(partitionPath,
|
||||
FSUtils.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId));
|
||||
writeStatus.getStat().setPath(path.toString());
|
||||
try {
|
||||
this.writer = HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath))
|
||||
.withFileId(fileId).overBaseCommit(baseCommitTime).withLogVersion(
|
||||
fileSlice.getLogFiles().map(logFile -> logFile.getLogVersion())
|
||||
.max(Comparator.naturalOrder()).orElse(HoodieLogFile.LOGFILE_BASE_VERSION))
|
||||
.withSizeThreshold(config.getLogFileMaxSize()).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(partitionPath,
|
||||
FSUtils.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId));
|
||||
writeStatus.getStat().setPath(path.toString());
|
||||
}
|
||||
|
||||
private Optional<IndexedRecord> getIndexedRecord(HoodieRecord<T> hoodieRecord) {
|
||||
@@ -150,7 +143,8 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
}
|
||||
|
||||
writeStatus.markSuccess(hoodieRecord, recordMetadata);
|
||||
// deflate record payload after recording success. This will help users access payload as a part of marking
|
||||
// deflate record payload after recording success. This will help users access payload as a
|
||||
// part of marking
|
||||
// record successful.
|
||||
hoodieRecord.deflate();
|
||||
return avroRecord;
|
||||
@@ -165,7 +159,8 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
// to make sure we don't append records with older (shorter) schema than already appended
|
||||
public void doAppend() {
|
||||
|
||||
int maxBlockSize = config.getLogFileDataBlockMaxSize(); int numberOfRecords = 0;
|
||||
int maxBlockSize = config.getLogFileDataBlockMaxSize();
|
||||
int numberOfRecords = 0;
|
||||
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, commitTime);
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
|
||||
@@ -173,16 +168,17 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
HoodieRecord record = recordItr.next();
|
||||
// update the new location of the record, so we know where to find it next
|
||||
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId));
|
||||
if(doInit) {
|
||||
if (doInit) {
|
||||
init(record.getPartitionPath());
|
||||
averageRecordSize = SizeEstimator.estimate(record);
|
||||
doInit = false;
|
||||
}
|
||||
// Append if max number of records reached to achieve block size
|
||||
if(numberOfRecords >= (int) (maxBlockSize / averageRecordSize)) {
|
||||
// Recompute averageRecordSize before writing a new block and update existing value with avg of new and old
|
||||
if (numberOfRecords >= (int) (maxBlockSize / averageRecordSize)) {
|
||||
// Recompute averageRecordSize before writing a new block and update existing value with
|
||||
// avg of new and old
|
||||
logger.info("AvgRecordSize => " + averageRecordSize);
|
||||
averageRecordSize = (averageRecordSize + SizeEstimator.estimate(record))/2;
|
||||
averageRecordSize = (averageRecordSize + SizeEstimator.estimate(record)) / 2;
|
||||
doAppend(header);
|
||||
numberOfRecords = 0;
|
||||
}
|
||||
|
||||
@@ -68,9 +68,8 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
throws IOException {
|
||||
logger.info("Cleaning " + partitionPath + ", retaining latest " + config
|
||||
.getCleanerFileVersionsRetained() + " file versions. ");
|
||||
List<HoodieFileGroup> fileGroups =
|
||||
fileSystemView.getAllFileGroups(partitionPath)
|
||||
.collect(Collectors.toList());
|
||||
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()
|
||||
@@ -94,11 +93,9 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
FileSlice nextSlice = fileSliceIterator.next();
|
||||
HoodieDataFile dataFile = nextSlice.getDataFile().get();
|
||||
deletePaths.add(dataFile.getFileStatus().getPath().toString());
|
||||
if (hoodieTable.getMetaClient().getTableType()
|
||||
== HoodieTableType.MERGE_ON_READ) {
|
||||
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())
|
||||
deletePaths.addAll(nextSlice.getLogFiles().map(file -> file.getPath().toString())
|
||||
.collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
@@ -121,8 +118,8 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
private List<String> getFilesToCleanKeepingLatestCommits(String partitionPath)
|
||||
throws IOException {
|
||||
int commitsRetained = config.getCleanerCommitsRetained();
|
||||
logger.info(
|
||||
"Cleaning " + partitionPath + ", retaining latest " + commitsRetained + " commits. ");
|
||||
logger
|
||||
.info("Cleaning " + partitionPath + ", retaining latest " + commitsRetained + " commits. ");
|
||||
List<String> deletePaths = new ArrayList<>();
|
||||
|
||||
// Collect all the datafiles savepointed by all the savepoints
|
||||
@@ -132,15 +129,14 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
// 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());
|
||||
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);
|
||||
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.
|
||||
@@ -151,28 +147,26 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
// 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
|
||||
// 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))) {
|
||||
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)) {
|
||||
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 (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())
|
||||
deletePaths.addAll(aSlice.getLogFiles().map(file -> file.getPath().toString())
|
||||
.collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
@@ -190,9 +184,10 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
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
|
||||
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;
|
||||
}
|
||||
}
|
||||
@@ -213,8 +208,7 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
} else {
|
||||
throw new IllegalArgumentException("Unknown cleaning policy : " + policy.name());
|
||||
}
|
||||
logger.info(
|
||||
deletePaths.size() + " patterns used to delete in partition path:" + partitionPath);
|
||||
logger.info(deletePaths.size() + " patterns used to delete in partition path:" + partitionPath);
|
||||
|
||||
return deletePaths;
|
||||
}
|
||||
@@ -227,8 +221,8 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
int commitsRetained = config.getCleanerCommitsRetained();
|
||||
if (config.getCleanerPolicy() == HoodieCleaningPolicy.KEEP_LATEST_COMMITS
|
||||
&& commitTimeline.countInstants() > commitsRetained) {
|
||||
earliestCommitToRetain =
|
||||
commitTimeline.nthInstant(commitTimeline.countInstants() - commitsRetained);
|
||||
earliestCommitToRetain = commitTimeline
|
||||
.nthInstant(commitTimeline.countInstants() - commitsRetained);
|
||||
}
|
||||
return earliestCommitToRetain;
|
||||
}
|
||||
|
||||
@@ -40,13 +40,6 @@ 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 org.apache.avro.Schema;
|
||||
import org.apache.avro.file.DataFileStream;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
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;
|
||||
@@ -54,6 +47,11 @@ import java.util.Map;
|
||||
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.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Archiver to bound the growth of <action>.commit files
|
||||
@@ -76,11 +74,9 @@ public class HoodieCommitArchiveLog {
|
||||
private HoodieLogFormat.Writer openWriter() {
|
||||
try {
|
||||
if (this.writer == null) {
|
||||
return HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(archiveFilePath.getParent())
|
||||
return HoodieLogFormat.newWriterBuilder().onParentPath(archiveFilePath.getParent())
|
||||
.withFileId(archiveFilePath.getName())
|
||||
.withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION)
|
||||
.withFs(metaClient.getFs())
|
||||
.withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFs(metaClient.getFs())
|
||||
.overBaseCommit("").build();
|
||||
} else {
|
||||
return this.writer;
|
||||
@@ -136,21 +132,19 @@ public class HoodieCommitArchiveLog {
|
||||
.getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION))
|
||||
.filterCompletedInstants();
|
||||
Stream<HoodieInstant> instants = cleanAndRollbackTimeline.getInstants()
|
||||
.collect(Collectors.groupingBy(s -> s.getAction()))
|
||||
.entrySet()
|
||||
.stream()
|
||||
.map(i -> {
|
||||
.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());
|
||||
}).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
|
||||
//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.
|
||||
// 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
|
||||
@@ -169,16 +163,14 @@ public class HoodieCommitArchiveLog {
|
||||
log.info("Deleting instants " + archivedInstants);
|
||||
boolean success = true;
|
||||
for (HoodieInstant archivedInstant : archivedInstants) {
|
||||
Path commitFile =
|
||||
new Path(metaClient.getMetaPath(), archivedInstant.getFileName());
|
||||
Path commitFile = new Path(metaClient.getMetaPath(), archivedInstant.getFileName());
|
||||
try {
|
||||
if (metaClient.getFs().exists(commitFile)) {
|
||||
success &= metaClient.getFs().delete(commitFile, false);
|
||||
log.info("Archived and deleted instant file " + commitFile);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to delete archived instant " + archivedInstant,
|
||||
e);
|
||||
throw new HoodieIOException("Failed to delete archived instant " + archivedInstant, e);
|
||||
}
|
||||
}
|
||||
return success;
|
||||
@@ -186,8 +178,8 @@ public class HoodieCommitArchiveLog {
|
||||
|
||||
public void archive(List<HoodieInstant> instants) throws HoodieCommitException {
|
||||
try {
|
||||
HoodieTimeline commitTimeline =
|
||||
metaClient.getActiveTimeline().getAllCommitsTimeline().filterCompletedInstants();
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveTimeline().getAllCommitsTimeline()
|
||||
.filterCompletedInstants();
|
||||
Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema();
|
||||
log.info("Wrapper schema " + wrapperSchema.toString());
|
||||
List<IndexedRecord> records = new ArrayList<>();
|
||||
@@ -247,6 +239,8 @@ public class HoodieCommitArchiveLog {
|
||||
archivedMetaWrapper.setActionType(ActionType.commit.name());
|
||||
break;
|
||||
}
|
||||
default:
|
||||
throw new UnsupportedOperationException("Action not fully supported yet");
|
||||
}
|
||||
return archivedMetaWrapper;
|
||||
}
|
||||
@@ -256,9 +250,8 @@ public class HoodieCommitArchiveLog {
|
||||
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);
|
||||
com.uber.hoodie.avro.model.HoodieCommitMetadata avroMetaData = mapper
|
||||
.convertValue(hoodieCommitMetadata, com.uber.hoodie.avro.model.HoodieCommitMetadata.class);
|
||||
return avroMetaData;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -49,8 +49,8 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
private long recordsWritten = 0;
|
||||
private long recordsDeleted = 0;
|
||||
|
||||
public HoodieCreateHandle(HoodieWriteConfig config, String commitTime,
|
||||
HoodieTable<T> hoodieTable, String 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());
|
||||
@@ -64,14 +64,11 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
}
|
||||
|
||||
try {
|
||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs,
|
||||
commitTime,
|
||||
new Path(config.getBasePath()),
|
||||
new Path(config.getBasePath(), partitionPath));
|
||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, commitTime,
|
||||
new Path(config.getBasePath()), new Path(config.getBasePath(), partitionPath));
|
||||
partitionMetadata.trySave(TaskContext.getPartitionId());
|
||||
this.storageWriter =
|
||||
HoodieStorageWriterFactory
|
||||
.getStorageWriter(commitTime, getStorageWriterPath(), hoodieTable, config, schema);
|
||||
this.storageWriter = HoodieStorageWriterFactory
|
||||
.getStorageWriter(commitTime, getStorageWriterPath(), hoodieTable, config, schema);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieInsertException(
|
||||
"Failed to initialize HoodieStorageWriter for path " + getStorageWriterPath(), e);
|
||||
@@ -81,13 +78,12 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
|
||||
/**
|
||||
* Determines whether we can accept the incoming records, into the current file, depending on
|
||||
*
|
||||
* <p>
|
||||
* - 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());
|
||||
return storageWriter.canWrite() && record.getPartitionPath().equals(status.getPartitionPath());
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -111,7 +107,8 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
recordsDeleted++;
|
||||
}
|
||||
status.markSuccess(record, recordMetadata);
|
||||
// deflate record payload after recording success. This will help users access payload as a part of marking
|
||||
// deflate record payload after recording success. This will help users access payload as a
|
||||
// part of marking
|
||||
// record successful.
|
||||
record.deflate();
|
||||
} catch (Throwable t) {
|
||||
@@ -126,9 +123,8 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
* 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);
|
||||
logger.info("Closing the file " + status.getFileId() + " as we are done with all the records "
|
||||
+ recordsWritten);
|
||||
try {
|
||||
storageWriter.close();
|
||||
|
||||
@@ -144,8 +140,7 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
|
||||
return status;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieInsertException("Failed to close the Insert Handle for path " + path,
|
||||
e);
|
||||
throw new HoodieInsertException("Failed to close the Insert Handle for path " + path, e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -39,11 +39,10 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
|
||||
protected final HoodieWriteConfig config;
|
||||
protected final FileSystem fs;
|
||||
protected final HoodieTable<T> hoodieTable;
|
||||
protected HoodieTimeline hoodieTimeline;
|
||||
protected final Schema schema;
|
||||
protected HoodieTimeline hoodieTimeline;
|
||||
|
||||
public HoodieIOHandle(HoodieWriteConfig config, String commitTime,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
public HoodieIOHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable) {
|
||||
this.commitTime = commitTime;
|
||||
this.config = config;
|
||||
this.fs = hoodieTable.getMetaClient().getFs();
|
||||
@@ -52,6 +51,32 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
|
||||
this.schema = createHoodieWriteSchema(config);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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, HoodieTable hoodieTable) {
|
||||
FileSystem fs = hoodieTable.getMetaClient().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 static Schema createHoodieWriteSchema(HoodieWriteConfig config) {
|
||||
return 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 {
|
||||
@@ -72,37 +97,7 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
|
||||
taskAttemptId));
|
||||
}
|
||||
|
||||
/**
|
||||
* 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,
|
||||
HoodieTable hoodieTable) {
|
||||
FileSystem fs = hoodieTable.getMetaClient().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 static Schema createHoodieWriteSchema(HoodieWriteConfig config) {
|
||||
return HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -26,14 +26,18 @@ import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.ReflectionUtils;
|
||||
import com.uber.hoodie.common.util.collection.ExternalSpillableMap;
|
||||
import com.uber.hoodie.common.util.collection.converter.StringConverter;
|
||||
import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter;
|
||||
import com.uber.hoodie.common.util.collection.converter.StringConverter;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
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.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -41,11 +45,6 @@ import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.TaskContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
|
||||
@SuppressWarnings("Duplicates")
|
||||
public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
|
||||
|
||||
@@ -62,59 +61,46 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
private long recordsDeleted = 0;
|
||||
private long updatedRecordsWritten = 0;
|
||||
|
||||
public HoodieMergeHandle(HoodieWriteConfig config,
|
||||
String commitTime,
|
||||
HoodieTable<T> hoodieTable,
|
||||
Iterator<HoodieRecord<T>> recordItr,
|
||||
String fileId) {
|
||||
public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
|
||||
Iterator<HoodieRecord<T>> recordItr, String fileId) {
|
||||
super(config, commitTime, hoodieTable);
|
||||
this.fileSystemView = hoodieTable.getROFileSystemView();
|
||||
init(fileId, init(fileId, recordItr));
|
||||
}
|
||||
|
||||
public HoodieMergeHandle(HoodieWriteConfig config,
|
||||
String commitTime,
|
||||
HoodieTable<T> hoodieTable,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords,
|
||||
String fileId) {
|
||||
public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords, String fileId) {
|
||||
super(config, commitTime, hoodieTable);
|
||||
this.fileSystemView = hoodieTable.getROFileSystemView();
|
||||
this.keyToNewRecords = keyToNewRecords;
|
||||
init(fileId, keyToNewRecords.get(keyToNewRecords.keySet().stream().findFirst().get()).getPartitionPath());
|
||||
init(fileId, keyToNewRecords.get(keyToNewRecords.keySet().stream().findFirst().get())
|
||||
.getPartitionPath());
|
||||
}
|
||||
|
||||
/**
|
||||
* Extract old file path, initialize StorageWriter and WriteStatus
|
||||
* @param fileId
|
||||
* @param partitionPath
|
||||
*/
|
||||
private void init(String fileId, String partitionPath) {
|
||||
WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName());
|
||||
writeStatus.setStat(new HoodieWriteStat());
|
||||
this.writeStatus = writeStatus;
|
||||
try {
|
||||
String latestValidFilePath = fileSystemView
|
||||
.getLatestDataFiles(partitionPath)
|
||||
.filter(dataFile -> dataFile.getFileId().equals(fileId))
|
||||
.findFirst()
|
||||
.get().getFileName();
|
||||
String latestValidFilePath = fileSystemView.getLatestDataFiles(partitionPath)
|
||||
.filter(dataFile -> dataFile.getFileId().equals(fileId)).findFirst().get().getFileName();
|
||||
writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath));
|
||||
|
||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs,
|
||||
commitTime,
|
||||
new Path(config.getBasePath()),
|
||||
new Path(config.getBasePath(), partitionPath));
|
||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, commitTime,
|
||||
new Path(config.getBasePath()), new Path(config.getBasePath(), partitionPath));
|
||||
partitionMetadata.trySave(TaskContext.getPartitionId());
|
||||
|
||||
oldFilePath = new Path(
|
||||
config.getBasePath() + "/" + partitionPath + "/"
|
||||
+ latestValidFilePath);
|
||||
config.getBasePath() + "/" + partitionPath + "/" + latestValidFilePath);
|
||||
String relativePath = new Path(partitionPath + "/" + FSUtils
|
||||
.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)).toString();
|
||||
newFilePath = new Path(config.getBasePath(), relativePath);
|
||||
if (config.shouldUseTempFolderForCopyOnWriteForMerge()) {
|
||||
this.tempPath = makeTempPath(partitionPath, TaskContext.getPartitionId(),
|
||||
fileId, TaskContext.get().stageId(), TaskContext.get().taskAttemptId());
|
||||
this.tempPath = makeTempPath(partitionPath, TaskContext.getPartitionId(), fileId,
|
||||
TaskContext.get().stageId(), TaskContext.get().taskAttemptId());
|
||||
}
|
||||
|
||||
// handle cases of partial failures, for update task
|
||||
@@ -122,8 +108,9 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
fs.delete(newFilePath, false);
|
||||
}
|
||||
|
||||
logger.info(String.format("Merging new data into oldPath %s, as newPath %s",
|
||||
oldFilePath.toString(), getStorageWriterPath().toString()));
|
||||
logger.info(String
|
||||
.format("Merging new data into oldPath %s, as newPath %s", oldFilePath.toString(),
|
||||
getStorageWriterPath().toString()));
|
||||
// file name is same for all records, in this bunch
|
||||
writeStatus.setFileId(fileId);
|
||||
writeStatus.setPartitionPath(partitionPath);
|
||||
@@ -143,9 +130,6 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
|
||||
/**
|
||||
* Load the new incoming records in a map and return partitionPath
|
||||
* @param fileId
|
||||
* @param newRecordsItr
|
||||
* @return
|
||||
*/
|
||||
private String init(String fileId, Iterator<HoodieRecord<T>> newRecordsItr) {
|
||||
try {
|
||||
@@ -153,7 +137,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
logger.info("MaxMemoryPerPartitionMerge => " + config.getMaxMemoryPerPartitionMerge());
|
||||
this.keyToNewRecords = new ExternalSpillableMap<>(config.getMaxMemoryPerPartitionMerge(),
|
||||
Optional.empty(), new StringConverter(), new HoodieRecordConverter(schema, config.getPayloadClass()));
|
||||
} catch(IOException io) {
|
||||
} catch (IOException io) {
|
||||
throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io);
|
||||
}
|
||||
String partitionPath = null;
|
||||
@@ -164,14 +148,14 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
// update the new location of the record, so we know where to find it next
|
||||
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId));
|
||||
}
|
||||
logger.debug("Number of entries in MemoryBasedMap => " +
|
||||
((ExternalSpillableMap) keyToNewRecords).getInMemoryMapNumEntries()
|
||||
+ "Total size in bytes of MemoryBasedMap => " +
|
||||
((ExternalSpillableMap) keyToNewRecords).getCurrentInMemoryMapSize()
|
||||
+ "Number of entries in DiskBasedMap => " +
|
||||
((ExternalSpillableMap) keyToNewRecords).getDiskBasedMapNumEntries()
|
||||
+ "Size of file spilled to disk => " +
|
||||
((ExternalSpillableMap) keyToNewRecords).getSizeOfFileOnDiskInBytes());
|
||||
logger.debug("Number of entries in MemoryBasedMap => "
|
||||
+ ((ExternalSpillableMap) keyToNewRecords).getInMemoryMapNumEntries()
|
||||
+ "Total size in bytes of MemoryBasedMap => "
|
||||
+ ((ExternalSpillableMap) keyToNewRecords).getCurrentInMemoryMapSize()
|
||||
+ "Number of entries in DiskBasedMap => "
|
||||
+ ((ExternalSpillableMap) keyToNewRecords).getDiskBasedMapNumEntries()
|
||||
+ "Size of file spilled to disk => "
|
||||
+ ((ExternalSpillableMap) keyToNewRecords).getSizeOfFileOnDiskInBytes());
|
||||
|
||||
return partitionPath;
|
||||
}
|
||||
@@ -189,7 +173,8 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
}
|
||||
|
||||
writeStatus.markSuccess(hoodieRecord, recordMetadata);
|
||||
// deflate record payload after recording success. This will help users access payload as a part of marking
|
||||
// deflate record payload after recording success. This will help users access payload as a
|
||||
// part of marking
|
||||
// record successful.
|
||||
hoodieRecord.deflate();
|
||||
return true;
|
||||
@@ -201,8 +186,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
}
|
||||
|
||||
/**
|
||||
* Go through an old record. Here if we detect a newer version shows up, we write the new one to
|
||||
* the file.
|
||||
* 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();
|
||||
@@ -213,12 +197,12 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
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.
|
||||
*/
|
||||
/* 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);
|
||||
@@ -236,10 +220,9 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
try {
|
||||
storageWriter.writeAvro(key, oldRecord);
|
||||
} catch (ClassCastException e) {
|
||||
logger.error(
|
||||
"Schema mismatch when rewriting old record " + oldRecord + " from file "
|
||||
+ getOldFilePath() + " to file " + getStorageWriterPath() + " with schema " + schema
|
||||
.toString(true));
|
||||
logger.error("Schema mismatch when rewriting old record " + oldRecord + " from file "
|
||||
+ getOldFilePath() + " to file " + getStorageWriterPath() + " 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 "
|
||||
|
||||
@@ -53,8 +53,8 @@ public class CompactionOperation implements Serializable {
|
||||
this.partitionPath = partitionPath;
|
||||
this.dataFileCommitTime = dataFile.getCommitTime();
|
||||
this.dataFileSize = dataFile.getFileSize();
|
||||
this.deltaFilePaths = logFiles.stream().map(s -> s.getPath().toString()).collect(
|
||||
Collectors.toList());
|
||||
this.deltaFilePaths = logFiles.stream().map(s -> s.getPath().toString())
|
||||
.collect(Collectors.toList());
|
||||
this.metrics = writeConfig.getCompactionStrategy()
|
||||
.captureMetrics(dataFile, partitionPath, logFiles);
|
||||
}
|
||||
|
||||
@@ -17,17 +17,15 @@
|
||||
package com.uber.hoodie.io.compact;
|
||||
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
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.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.Date;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
/**
|
||||
* A HoodieCompactor runs compaction on a hoodie table
|
||||
@@ -38,7 +36,7 @@ public interface HoodieCompactor extends Serializable {
|
||||
* Compact the delta files with the data files
|
||||
*/
|
||||
JavaRDD<WriteStatus> compact(JavaSparkContext jsc, final HoodieWriteConfig config,
|
||||
HoodieTable hoodieTable, String compactionCommitTime) throws Exception;
|
||||
HoodieTable hoodieTable, String compactionCommitTime) throws Exception;
|
||||
|
||||
|
||||
// Helper methods
|
||||
|
||||
@@ -17,6 +17,7 @@
|
||||
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;
|
||||
@@ -70,9 +71,8 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
}
|
||||
|
||||
private JavaRDD<WriteStatus> executeCompaction(JavaSparkContext jsc,
|
||||
List<CompactionOperation> operations,
|
||||
HoodieTable hoodieTable,
|
||||
HoodieWriteConfig config, String compactionCommitTime) throws IOException {
|
||||
List<CompactionOperation> operations, HoodieTable hoodieTable, HoodieWriteConfig config,
|
||||
String compactionCommitTime) throws IOException {
|
||||
|
||||
log.info("After filtering, Compacting " + operations + " files");
|
||||
return jsc.parallelize(operations, operations.size())
|
||||
@@ -80,18 +80,19 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
.flatMap(writeStatusesItr -> writeStatusesItr.iterator());
|
||||
}
|
||||
|
||||
private List<WriteStatus> compact(HoodieTable hoodieTable,
|
||||
HoodieWriteConfig config, CompactionOperation operation, String commitTime)
|
||||
throws IOException {
|
||||
private List<WriteStatus> compact(HoodieTable hoodieTable, HoodieWriteConfig config,
|
||||
CompactionOperation operation, String commitTime) throws IOException {
|
||||
FileSystem fs = hoodieTable.getMetaClient().getFs();
|
||||
Schema readerSchema =
|
||||
HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||
Schema readerSchema = HoodieAvroUtils
|
||||
.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||
|
||||
log.info("Compacting base " + operation.getDataFilePath() + " with delta files " + operation
|
||||
.getDeltaFilePaths() + " for commit " + commitTime);
|
||||
// TODO - FIX THIS
|
||||
// Reads the entire avro file. Always only specific blocks should be read from the avro file (failure recover).
|
||||
// Load all the delta commits since the last compaction commit and get all the blocks to be loaded and load it using CompositeAvroLogReader
|
||||
// Reads the entire avro file. Always only specific blocks should be read from the avro file
|
||||
// (failure recover).
|
||||
// Load all the delta commits since the last compaction commit and get all the blocks to be
|
||||
// loaded and load it using CompositeAvroLogReader
|
||||
// Since a DeltaCommit is not defined yet, reading all the records. revisit this soon.
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
String maxInstantTime = metaClient.getActiveTimeline()
|
||||
@@ -114,52 +115,47 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
Iterator<List<WriteStatus>> result = table
|
||||
.handleUpdate(commitTime, operation.getFileId(), scanner.getRecords());
|
||||
Iterable<List<WriteStatus>> resultIterable = () -> result;
|
||||
return StreamSupport.stream(resultIterable.spliterator(), false)
|
||||
.flatMap(Collection::stream)
|
||||
return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream)
|
||||
.map(s -> {
|
||||
s.getStat().setTotalRecordsToBeUpdate(scanner.getTotalRecordsToUpdate());
|
||||
s.getStat().setTotalLogFiles(scanner.getTotalLogFiles());
|
||||
s.getStat().setTotalLogRecords(scanner.getTotalLogRecords());
|
||||
s.getStat().setPartitionPath(operation.getPartitionPath());
|
||||
return s;
|
||||
})
|
||||
.collect(toList());
|
||||
}).collect(toList());
|
||||
}
|
||||
|
||||
private List<CompactionOperation> getCompactionWorkload(JavaSparkContext jsc,
|
||||
HoodieTable hoodieTable,
|
||||
HoodieWriteConfig config, String compactionCommitTime)
|
||||
HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime)
|
||||
throws IOException {
|
||||
|
||||
Preconditions.checkArgument(
|
||||
hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ,
|
||||
"HoodieRealtimeTableCompactor can only compact table of type "
|
||||
+ HoodieTableType.MERGE_ON_READ + " and not " + hoodieTable.getMetaClient()
|
||||
.getTableType().name());
|
||||
Preconditions
|
||||
.checkArgument(hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ,
|
||||
"HoodieRealtimeTableCompactor can only compact table of type "
|
||||
+ HoodieTableType.MERGE_ON_READ + " and not " + hoodieTable.getMetaClient()
|
||||
.getTableType().name());
|
||||
|
||||
//TODO : check if maxMemory is not greater than JVM or spark.executor memory
|
||||
// TODO - rollback any compactions in flight
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
log.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime);
|
||||
List<String> partitionPaths =
|
||||
FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
|
||||
List<String> partitionPaths = FSUtils
|
||||
.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
|
||||
config.shouldAssumeDatePartitioning());
|
||||
|
||||
TableFileSystemView.RealtimeView fileSystemView = hoodieTable.getRTFileSystemView();
|
||||
log.info("Compaction looking for files to compact in " + partitionPaths + " partitions");
|
||||
List<CompactionOperation> operations =
|
||||
jsc.parallelize(partitionPaths, partitionPaths.size())
|
||||
.flatMap((FlatMapFunction<String, CompactionOperation>) partitionPath -> fileSystemView
|
||||
.getLatestFileSlices(partitionPath)
|
||||
.map(s -> new CompactionOperation(s.getDataFile().get(),
|
||||
partitionPath,
|
||||
List<CompactionOperation> operations = jsc.parallelize(partitionPaths, partitionPaths.size())
|
||||
.flatMap((FlatMapFunction<String, CompactionOperation>) partitionPath -> fileSystemView
|
||||
.getLatestFileSlices(partitionPath).map(
|
||||
s -> new CompactionOperation(s.getDataFile().get(), partitionPath,
|
||||
s.getLogFiles().sorted(HoodieLogFile.getLogVersionComparator().reversed())
|
||||
.collect(Collectors.toList()), config))
|
||||
.filter(c -> !c.getDeltaFilePaths().isEmpty())
|
||||
.collect(toList()).iterator()).collect();
|
||||
.filter(c -> !c.getDeltaFilePaths().isEmpty()).collect(toList()).iterator()).collect();
|
||||
log.info("Total of " + operations.size() + " compactions are retrieved");
|
||||
|
||||
// Filter the compactions with the passed in filter. This lets us choose most effective compactions only
|
||||
// Filter the compactions with the passed in filter. This lets us choose most effective
|
||||
// compactions only
|
||||
operations = config.getCompactionStrategy().orderAndFilter(config, operations);
|
||||
if (operations.isEmpty()) {
|
||||
log.warn("After filtering, Nothing to compact for " + metaClient.getBasePath());
|
||||
|
||||
@@ -44,9 +44,9 @@ public class BoundedIOCompactionStrategy implements CompactionStrategy {
|
||||
List<HoodieLogFile> logFiles) {
|
||||
Map<String, Object> metrics = Maps.newHashMap();
|
||||
// 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);
|
||||
Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize)
|
||||
.filter(Optional::isPresent).map(Optional::get).reduce((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
|
||||
@@ -64,7 +64,8 @@ public class BoundedIOCompactionStrategy implements CompactionStrategy {
|
||||
@Override
|
||||
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
|
||||
// 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();
|
||||
long targetIORemaining = writeConfig.getTargetIOPerCompactionInMB();
|
||||
|
||||
@@ -28,7 +28,7 @@ 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.
|
||||
*
|
||||
* <p>
|
||||
* Implementation of CompactionStrategy cannot hold any state. Difference instantiations can be
|
||||
* passed in every time
|
||||
*
|
||||
@@ -41,9 +41,9 @@ 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.
|
||||
*
|
||||
* @param dataFile - Base file to compact
|
||||
* @param dataFile - Base file to compact
|
||||
* @param partitionPath - Partition path
|
||||
* @param logFiles - List of log files to compact with the base file
|
||||
* @param logFiles - List of log files to compact with the base file
|
||||
* @return Map[String, Object] - metrics captured
|
||||
*/
|
||||
Map<String, Object> captureMetrics(HoodieDataFile dataFile, String partitionPath,
|
||||
@@ -54,7 +54,7 @@ public interface CompactionStrategy extends Serializable {
|
||||
* order and filter out compactions
|
||||
*
|
||||
* @param writeConfig - HoodieWriteConfig - config for this compaction is passed in
|
||||
* @param operations - list of compactions collected
|
||||
* @param operations - list of compactions collected
|
||||
* @return list of compactions to perform in this run
|
||||
*/
|
||||
List<CompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
|
||||
|
||||
@@ -20,7 +20,6 @@ package com.uber.hoodie.io.compact.strategy;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.io.compact.CompactionOperation;
|
||||
|
||||
import java.text.ParseException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Comparator;
|
||||
@@ -30,20 +29,21 @@ import java.util.Locale;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* This strategy orders compactions in reverse order of creation of Hive Partitions.
|
||||
* It helps to compact data in latest partitions first and then older capped at the Total_IO allowed.
|
||||
* This strategy orders compactions in reverse order of creation of Hive Partitions. It helps to
|
||||
* compact data in latest partitions first and then older capped at the Total_IO allowed.
|
||||
*/
|
||||
public class DayBasedCompactionStrategy extends BoundedIOCompactionStrategy {
|
||||
|
||||
// For now, use SimpleDateFormat as default partition format
|
||||
private static String datePartitionFormat = "yyyy/MM/dd";
|
||||
// Sorts compaction in LastInFirstCompacted order
|
||||
private static Comparator<CompactionOperation> comparator = (CompactionOperation leftC, CompactionOperation rightC) -> {
|
||||
private static Comparator<CompactionOperation> comparator = (CompactionOperation leftC,
|
||||
CompactionOperation rightC) -> {
|
||||
try {
|
||||
Date left = new SimpleDateFormat(datePartitionFormat, Locale.ENGLISH)
|
||||
.parse(leftC.getPartitionPath());
|
||||
.parse(leftC.getPartitionPath());
|
||||
Date right = new SimpleDateFormat(datePartitionFormat, Locale.ENGLISH)
|
||||
.parse(rightC.getPartitionPath());
|
||||
.parse(rightC.getPartitionPath());
|
||||
return left.after(right) ? -1 : right.after(left) ? 1 : 0;
|
||||
} catch (ParseException e) {
|
||||
throw new HoodieException("Invalid Partition Date Format", e);
|
||||
@@ -55,8 +55,10 @@ public class DayBasedCompactionStrategy extends BoundedIOCompactionStrategy {
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<CompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig, List<CompactionOperation> operations) {
|
||||
public List<CompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
|
||||
List<CompactionOperation> operations) {
|
||||
// Iterate through the operations and accept operations as long as we are within the IO limit
|
||||
return super.orderAndFilter(writeConfig, operations.stream().sorted(comparator).collect(Collectors.toList()));
|
||||
return super.orderAndFilter(writeConfig,
|
||||
operations.stream().sorted(comparator).collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
@@ -44,9 +44,9 @@ public class LogFileSizeBasedCompactionStrategy extends BoundedIOCompactionStrat
|
||||
|
||||
Map<String, Object> metrics = super.captureMetrics(dataFile, partitionPath, logFiles);
|
||||
// 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);
|
||||
Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize)
|
||||
.filter(Optional::isPresent).map(Optional::get).reduce((size1, size2) -> size1 + size2)
|
||||
.orElse(0L);
|
||||
// save the metrics needed during the order
|
||||
metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize);
|
||||
return metrics;
|
||||
|
||||
@@ -36,8 +36,8 @@ import org.apache.spark.TaskContext;
|
||||
* 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> {
|
||||
public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends IndexedRecord> extends
|
||||
ParquetWriter<IndexedRecord> implements HoodieStorageWriter<R> {
|
||||
|
||||
private static AtomicLong recordIndex = new AtomicLong(1);
|
||||
|
||||
@@ -49,6 +49,29 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
|
||||
private final 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(file, parquetConfig.getHadoopConf()));
|
||||
this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
|
||||
this.fs = (HoodieWrapperFileSystem) this.file
|
||||
.getFileSystem(registerFileSystem(file, 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() * parquetConfig.getCompressionRatio());
|
||||
this.writeSupport = parquetConfig.getWriteSupport();
|
||||
this.commitTime = commitTime;
|
||||
this.schema = schema;
|
||||
}
|
||||
|
||||
private static Configuration registerFileSystem(Path file, Configuration conf) {
|
||||
Configuration returnConf = new Configuration(conf);
|
||||
String scheme = FSUtils.getFs(file.toString(), conf).getScheme();
|
||||
@@ -57,37 +80,12 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
|
||||
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(file, parquetConfig.getHadoopConf()));
|
||||
this.file =
|
||||
HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
|
||||
this.fs = (HoodieWrapperFileSystem) this.file
|
||||
.getFileSystem(registerFileSystem(file, 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() * parquetConfig.getCompressionRatio());
|
||||
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.addHoodieKeyToRecord((GenericRecord) avroRecord, record.getRecordKey(),
|
||||
record.getPartitionPath(), file.getName());
|
||||
HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord, commitTime, seqId);
|
||||
super.write(avroRecord);
|
||||
writeSupport.add(record.getRecordKey());
|
||||
|
||||
@@ -30,22 +30,22 @@ import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||
|
||||
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 {
|
||||
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, hoodieTable);
|
||||
}
|
||||
|
||||
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> newParquetStorageWriter(
|
||||
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R>
|
||||
newParquetStorageWriter(
|
||||
String commitTime, Path path, HoodieWriteConfig config, Schema schema,
|
||||
HoodieTable hoodieTable) throws IOException {
|
||||
BloomFilter filter =
|
||||
new BloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP());
|
||||
HoodieAvroWriteSupport writeSupport =
|
||||
new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter);
|
||||
BloomFilter filter = new BloomFilter(config.getBloomFilterNumEntries(),
|
||||
config.getBloomFilterFPP());
|
||||
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(
|
||||
new AvroSchemaConverter().convert(schema), schema, filter);
|
||||
|
||||
HoodieParquetConfig parquetConfig =
|
||||
new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP,
|
||||
|
||||
@@ -59,8 +59,8 @@ import org.apache.hadoop.util.Progressable;
|
||||
*/
|
||||
public class HoodieWrapperFileSystem extends FileSystem {
|
||||
|
||||
private static final Set<String> SUPPORT_SCHEMES;
|
||||
public static final String HOODIE_SCHEME_PREFIX = "hoodie-";
|
||||
private static final Set<String> SUPPORT_SCHEMES;
|
||||
|
||||
static {
|
||||
SUPPORT_SCHEMES = new HashSet<>();
|
||||
@@ -69,18 +69,50 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
SUPPORT_SCHEMES.add("s3");
|
||||
SUPPORT_SCHEMES.add("s3a");
|
||||
|
||||
|
||||
// Hoodie currently relies on underlying object store being fully
|
||||
// consistent so only regional buckets should be used.
|
||||
SUPPORT_SCHEMES.add("gs");
|
||||
SUPPORT_SCHEMES.add("viewfs");
|
||||
}
|
||||
|
||||
private ConcurrentMap<String, SizeAwareFSDataOutputStream> openStreams =
|
||||
new ConcurrentHashMap<>();
|
||||
private ConcurrentMap<String, SizeAwareFSDataOutputStream> openStreams = new
|
||||
ConcurrentHashMap<>();
|
||||
private FileSystem fileSystem;
|
||||
private URI uri;
|
||||
|
||||
public static Path convertToHoodiePath(Path file, Configuration conf) {
|
||||
try {
|
||||
String scheme = FSUtils.getFs(file.toString(), conf).getScheme();
|
||||
return convertPathWithScheme(file, getHoodieScheme(scheme));
|
||||
} catch (HoodieIOException e) {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
private static Path convertPathWithScheme(Path oldPath, String newScheme) {
|
||||
URI oldURI = oldPath.toUri();
|
||||
URI newURI;
|
||||
try {
|
||||
newURI = new URI(newScheme, oldURI.getUserInfo(), oldURI.getHost(), oldURI.getPort(),
|
||||
oldURI.getPath(), oldURI.getQuery(), oldURI.getFragment());
|
||||
return new Path(newURI);
|
||||
} catch (URISyntaxException e) {
|
||||
// TODO - Better Exception handling
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public static String getHoodieScheme(String scheme) {
|
||||
String newScheme;
|
||||
if (SUPPORT_SCHEMES.contains(scheme)) {
|
||||
newScheme = HOODIE_SCHEME_PREFIX + scheme;
|
||||
} else {
|
||||
throw new IllegalArgumentException(
|
||||
"BlockAlignedAvroParquetWriter does not support scheme " + scheme);
|
||||
}
|
||||
return newScheme;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initialize(URI uri, Configuration conf) throws IOException {
|
||||
// Get the default filesystem to decorate
|
||||
@@ -90,7 +122,8 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
path = new Path(path.toString().replace(HOODIE_SCHEME_PREFIX, ""));
|
||||
}
|
||||
this.fileSystem = FSUtils.getFs(path.toString(), conf);
|
||||
// Do not need to explicitly initialize the default filesystem, its done already in the above FileSystem.get
|
||||
// Do not need to explicitly initialize the default filesystem, its done already in the above
|
||||
// FileSystem.get
|
||||
// fileSystem.initialize(FileSystem.getDefaultUri(conf), conf);
|
||||
// fileSystem.setConf(conf);
|
||||
this.uri = uri;
|
||||
@@ -108,8 +141,7 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite,
|
||||
int bufferSize, short replication, long blockSize, Progressable progress)
|
||||
throws IOException {
|
||||
int bufferSize, short replication, long blockSize, Progressable progress) throws IOException {
|
||||
final Path translatedPath = convertToDefaultPath(f);
|
||||
return wrapOutputStream(f, fileSystem
|
||||
.create(translatedPath, permission, overwrite, bufferSize, replication, blockSize,
|
||||
@@ -122,8 +154,8 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
return fsDataOutputStream;
|
||||
}
|
||||
|
||||
SizeAwareFSDataOutputStream os =
|
||||
new SizeAwareFSDataOutputStream(fsDataOutputStream, new Runnable() {
|
||||
SizeAwareFSDataOutputStream os = new SizeAwareFSDataOutputStream(fsDataOutputStream,
|
||||
new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
openStreams.remove(path.getName());
|
||||
@@ -160,14 +192,13 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize)
|
||||
throws IOException {
|
||||
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize) throws IOException {
|
||||
return fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize,
|
||||
Progressable progress) throws IOException {
|
||||
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, Progressable progress)
|
||||
throws IOException {
|
||||
return fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize, progress);
|
||||
}
|
||||
|
||||
@@ -175,14 +206,12 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication,
|
||||
long blockSize, Progressable progress) throws IOException {
|
||||
return fileSystem
|
||||
.create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize,
|
||||
progress);
|
||||
.create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize, progress);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, FsPermission permission, EnumSet<CreateFlag> flags,
|
||||
int bufferSize, short replication, long blockSize, Progressable progress)
|
||||
throws IOException {
|
||||
int bufferSize, short replication, long blockSize, Progressable progress) throws IOException {
|
||||
return fileSystem
|
||||
.create(convertToDefaultPath(f), permission, flags, bufferSize, replication, blockSize,
|
||||
progress);
|
||||
@@ -197,7 +226,6 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
progress, checksumOpt);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication,
|
||||
long blockSize) throws IOException {
|
||||
@@ -205,7 +233,6 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
.create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream append(Path f, int bufferSize, Progressable progress)
|
||||
throws IOException {
|
||||
@@ -228,13 +255,13 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setWorkingDirectory(Path new_dir) {
|
||||
fileSystem.setWorkingDirectory(convertToDefaultPath(new_dir));
|
||||
public Path getWorkingDirectory() {
|
||||
return convertToHoodiePath(fileSystem.getWorkingDirectory());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path getWorkingDirectory() {
|
||||
return convertToHoodiePath(fileSystem.getWorkingDirectory());
|
||||
public void setWorkingDirectory(Path newDir) {
|
||||
fileSystem.setWorkingDirectory(convertToDefaultPath(newDir));
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -290,8 +317,7 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
}
|
||||
|
||||
@Override
|
||||
public BlockLocation[] getFileBlockLocations(Path p, long start, long len)
|
||||
throws IOException {
|
||||
public BlockLocation[] getFileBlockLocations(Path p, long start, long len) throws IOException {
|
||||
return fileSystem.getFileBlockLocations(convertToDefaultPath(p), start, len);
|
||||
}
|
||||
|
||||
@@ -319,17 +345,16 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
public FSDataOutputStream createNonRecursive(Path f, boolean overwrite, int bufferSize,
|
||||
short replication, long blockSize, Progressable progress) throws IOException {
|
||||
return fileSystem
|
||||
.createNonRecursive(convertToDefaultPath(f), overwrite, bufferSize, replication,
|
||||
blockSize, progress);
|
||||
.createNonRecursive(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize,
|
||||
progress);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, boolean overwrite,
|
||||
int bufferSize, short replication, long blockSize, Progressable progress)
|
||||
throws IOException {
|
||||
int bufferSize, short replication, long blockSize, Progressable progress) throws IOException {
|
||||
return fileSystem
|
||||
.createNonRecursive(convertToDefaultPath(f), permission, overwrite, bufferSize,
|
||||
replication, blockSize, progress);
|
||||
.createNonRecursive(convertToDefaultPath(f), permission, overwrite, bufferSize, replication,
|
||||
blockSize, progress);
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -418,20 +443,17 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus[] listStatus(Path f, PathFilter filter)
|
||||
throws IOException {
|
||||
public FileStatus[] listStatus(Path f, PathFilter filter) throws IOException {
|
||||
return fileSystem.listStatus(convertToDefaultPath(f), filter);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus[] listStatus(Path[] files)
|
||||
throws IOException {
|
||||
public FileStatus[] listStatus(Path[] files) throws IOException {
|
||||
return fileSystem.listStatus(convertDefaults(files));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus[] listStatus(Path[] files, PathFilter filter)
|
||||
throws IOException {
|
||||
public FileStatus[] listStatus(Path[] files, PathFilter filter) throws IOException {
|
||||
return fileSystem.listStatus(convertDefaults(files), filter);
|
||||
}
|
||||
|
||||
@@ -441,20 +463,17 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus[] globStatus(Path pathPattern, PathFilter filter)
|
||||
throws IOException {
|
||||
public FileStatus[] globStatus(Path pathPattern, PathFilter filter) throws IOException {
|
||||
return fileSystem.globStatus(convertToDefaultPath(pathPattern), filter);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f)
|
||||
throws IOException {
|
||||
public RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f) throws IOException {
|
||||
return fileSystem.listLocatedStatus(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override
|
||||
public RemoteIterator<LocatedFileStatus> listFiles(Path f, boolean recursive)
|
||||
throws IOException {
|
||||
public RemoteIterator<LocatedFileStatus> listFiles(Path f, boolean recursive) throws IOException {
|
||||
return fileSystem.listFiles(convertToDefaultPath(f), recursive);
|
||||
}
|
||||
|
||||
@@ -498,8 +517,8 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
@Override
|
||||
public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path src, Path dst)
|
||||
throws IOException {
|
||||
fileSystem.copyFromLocalFile(delSrc, overwrite, convertToDefaultPath(src),
|
||||
convertToDefaultPath(dst));
|
||||
fileSystem
|
||||
.copyFromLocalFile(delSrc, overwrite, convertToDefaultPath(src), convertToDefaultPath(dst));
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -525,15 +544,13 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile)
|
||||
throws IOException {
|
||||
return convertToHoodiePath(fileSystem.startLocalOutput(convertToDefaultPath(fsOutputFile),
|
||||
convertToDefaultPath(tmpLocalFile)));
|
||||
public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile) throws IOException {
|
||||
return convertToHoodiePath(fileSystem
|
||||
.startLocalOutput(convertToDefaultPath(fsOutputFile), convertToDefaultPath(tmpLocalFile)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile)
|
||||
throws IOException {
|
||||
public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile) throws IOException {
|
||||
fileSystem.completeLocalOutput(convertToDefaultPath(fsOutputFile),
|
||||
convertToDefaultPath(tmpLocalFile));
|
||||
}
|
||||
@@ -574,23 +591,18 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void access(Path path, FsAction mode)
|
||||
throws IOException {
|
||||
public void access(Path path, FsAction mode) throws IOException {
|
||||
fileSystem.access(convertToDefaultPath(path), mode);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void createSymlink(Path target, Path link, boolean createParent)
|
||||
throws
|
||||
IOException {
|
||||
public void createSymlink(Path target, Path link, boolean createParent) throws IOException {
|
||||
fileSystem
|
||||
.createSymlink(convertToDefaultPath(target), convertToDefaultPath(link), createParent);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus getFileLinkStatus(Path f)
|
||||
throws
|
||||
IOException {
|
||||
public FileStatus getFileLinkStatus(Path f) throws IOException {
|
||||
return fileSystem.getFileLinkStatus(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@@ -651,8 +663,7 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
|
||||
@Override
|
||||
public Path createSnapshot(Path path, String snapshotName) throws IOException {
|
||||
return convertToHoodiePath(
|
||||
fileSystem.createSnapshot(convertToDefaultPath(path), snapshotName));
|
||||
return convertToHoodiePath(fileSystem.createSnapshot(convertToDefaultPath(path), snapshotName));
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -718,8 +729,7 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, byte[]> getXAttrs(Path path, List<String> names)
|
||||
throws IOException {
|
||||
public Map<String, byte[]> getXAttrs(Path path, List<String> names) throws IOException {
|
||||
return fileSystem.getXAttrs(convertToDefaultPath(path), names);
|
||||
}
|
||||
|
||||
@@ -734,13 +744,13 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setConf(Configuration conf) {
|
||||
// ignore this. we will set conf on init
|
||||
public Configuration getConf() {
|
||||
return fileSystem.getConf();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Configuration getConf() {
|
||||
return fileSystem.getConf();
|
||||
public void setConf(Configuration conf) {
|
||||
// ignore this. we will set conf on init
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -762,15 +772,6 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
return convertPathWithScheme(oldPath, getHoodieScheme(fileSystem.getScheme()));
|
||||
}
|
||||
|
||||
public static Path convertToHoodiePath(Path file, Configuration conf) {
|
||||
try {
|
||||
String scheme = FSUtils.getFs(file.toString(), conf).getScheme();
|
||||
return convertPathWithScheme(file, getHoodieScheme(scheme));
|
||||
} catch (HoodieIOException e) {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
private Path convertToDefaultPath(Path oldPath) {
|
||||
return convertPathWithScheme(oldPath, fileSystem.getScheme());
|
||||
}
|
||||
@@ -783,30 +784,6 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
return psrcsNew;
|
||||
}
|
||||
|
||||
private static Path convertPathWithScheme(Path oldPath, String newScheme) {
|
||||
URI oldURI = oldPath.toUri();
|
||||
URI newURI;
|
||||
try {
|
||||
newURI = new URI(newScheme, oldURI.getUserInfo(), oldURI.getHost(), oldURI.getPort(),
|
||||
oldURI.getPath(), oldURI.getQuery(), oldURI.getFragment());
|
||||
return new Path(newURI);
|
||||
} catch (URISyntaxException e) {
|
||||
// TODO - Better Exception handling
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public static String getHoodieScheme(String scheme) {
|
||||
String newScheme;
|
||||
if (SUPPORT_SCHEMES.contains(scheme)) {
|
||||
newScheme = HOODIE_SCHEME_PREFIX + scheme;
|
||||
} else {
|
||||
throw new IllegalArgumentException(
|
||||
"BlockAlignedAvroParquetWriter does not support scheme " + scheme);
|
||||
}
|
||||
return newScheme;
|
||||
}
|
||||
|
||||
public long getBytesWritten(Path file) {
|
||||
if (openStreams.containsKey(file.getName())) {
|
||||
return openStreams.get(file.getName()).getBytesWritten();
|
||||
|
||||
Reference in New Issue
Block a user