1
0

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:
Balaji Varadarajan
2018-03-20 16:29:20 -07:00
committed by vinoth chandar
parent 987f5d6b96
commit 788e4f2d2e
200 changed files with 6209 additions and 5975 deletions

View File

@@ -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;
}