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

@@ -62,19 +62,12 @@ import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.Function;
/**
* Implementation of a more real-time read-optimized Hoodie Table where
* <p>
* INSERTS - Same as HoodieCopyOnWriteTable - Produce new files, block aligned to desired size (or)
* Merge with the smallest existing file, to expand it
* </p>
* <p>
* UPDATES - Appends the changes to a rolling log file maintained per file Id. Compaction merges the
* log file into the base file.
* </p>
* <p>
* WARNING - MOR table type does not support nested rollbacks, every rollback must be followed by an
* attempted commit action
* </p>
* Implementation of a more real-time read-optimized Hoodie Table where <p> INSERTS - Same as
* HoodieCopyOnWriteTable - Produce new files, block aligned to desired size (or) Merge with the
* smallest existing file, to expand it </p> <p> UPDATES - Appends the changes to a rolling log file
* maintained per file Id. Compaction merges the log file into the base file. </p> <p> WARNING - MOR
* table type does not support nested rollbacks, every rollback must be followed by an attempted
* commit action </p>
*/
public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
HoodieCopyOnWriteTable<T> {
@@ -88,57 +81,6 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
super(config, metaClient);
}
/**
* UpsertPartitioner for MergeOnRead table type, this allows auto correction of small parquet
* files to larger ones without the need for an index in the logFile.
*/
class MergeOnReadUpsertPartitioner extends HoodieCopyOnWriteTable.UpsertPartitioner {
MergeOnReadUpsertPartitioner(WorkloadProfile profile) {
super(profile);
}
@Override
protected List<SmallFile> getSmallFiles(String partitionPath) {
// smallFiles only for partitionPath
List<SmallFile> smallFileLocations = new ArrayList<>();
// Init here since this class (and member variables) might not have been initialized
HoodieTimeline commitTimeline = getCompletedCommitTimeline();
if (!commitTimeline.empty()) {
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
// find smallest file in partition and append to it
Optional<FileSlice> smallFileSlice = getRTFileSystemView()
.getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp())
.filter(fileSlice -> fileSlice.getLogFiles().count() < 1 &&
fileSlice.getDataFile().get().getFileSize() < config.getParquetSmallFileLimit())
.sorted((FileSlice left, FileSlice right) ->
left.getDataFile().get().getFileSize() < right.getDataFile().get().getFileSize() ? -1 : 1)
.findFirst();
if(smallFileSlice.isPresent()) {
String filename = smallFileSlice.get().getDataFile().get().getFileName();
SmallFile sf = new SmallFile();
sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename),
FSUtils.getFileId(filename));
sf.sizeBytes = smallFileSlice.get().getDataFile().get().getFileSize();
smallFileLocations.add(sf);
// Update the global small files list
smallFiles.add(sf);
}
}
return smallFileLocations;
}
public List<String> getSmallFileIds() {
return (List<String>) smallFiles.stream().map(smallFile -> ((SmallFile) smallFile).location.getFileId())
.collect(Collectors.toList());
}
}
@Override
public Partitioner getUpsertPartitioner(WorkloadProfile profile) {
if (profile == null) {
@@ -150,15 +92,16 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
@Override
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileId,
Iterator<HoodieRecord<T>> recordItr) throws IOException {
Iterator<HoodieRecord<T>> recordItr) throws IOException {
logger.info("Merging updates for commit " + commitTime + " for file " + fileId);
if(mergeOnReadUpsertPartitioner.getSmallFileIds().contains(fileId)) {
logger.info("Small file corrections for updates for commit " + commitTime + " for file " + fileId);
if (mergeOnReadUpsertPartitioner.getSmallFileIds().contains(fileId)) {
logger.info(
"Small file corrections for updates for commit " + commitTime + " for file " + fileId);
return super.handleUpdate(commitTime, fileId, recordItr);
} else {
HoodieAppendHandle<T> appendHandle =
new HoodieAppendHandle<>(config, commitTime, this, fileId, recordItr);
HoodieAppendHandle<T> appendHandle = new HoodieAppendHandle<>(config, commitTime, this,
fileId, recordItr);
appendHandle.doAppend();
appendHandle.close();
return Collections.singletonList(Collections.singletonList(appendHandle.getWriteStatus()))
@@ -202,13 +145,11 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
if (commits.size() > 1) {
throw new UnsupportedOperationException("Nested Rollbacks are not supported");
}
Map<String, HoodieInstant> commitsAndCompactions =
this.getActiveTimeline()
.getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION,
HoodieActiveTimeline.DELTA_COMMIT_ACTION))
.getInstants()
.filter(i -> commits.contains(i.getTimestamp()))
.collect(Collectors.toMap(i -> i.getTimestamp(), i -> i));
Map<String, HoodieInstant> commitsAndCompactions = this.getActiveTimeline()
.getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION,
HoodieActiveTimeline.DELTA_COMMIT_ACTION)).getInstants()
.filter(i -> commits.contains(i.getTimestamp()))
.collect(Collectors.toMap(i -> i.getTimestamp(), i -> i));
// Atomically un-publish all non-inflight commits
commitsAndCompactions.entrySet().stream().map(entry -> entry.getValue())
@@ -218,9 +159,9 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
Long startTime = System.currentTimeMillis();
List<HoodieRollbackStat> allRollbackStats = jsc.parallelize
(FSUtils.getAllPartitionPaths(this.metaClient.getFs(),
this.getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning()))
List<HoodieRollbackStat> allRollbackStats = jsc.parallelize(FSUtils
.getAllPartitionPaths(this.metaClient.getFs(), this.getMetaClient().getBasePath(),
config.shouldAssumeDatePartitioning()))
.map((Function<String, List<HoodieRollbackStat>>) partitionPath -> {
return commits.stream().map(commit -> {
HoodieInstant instant = commitsAndCompactions.get(commit);
@@ -228,56 +169,63 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
switch (instant.getAction()) {
case HoodieTimeline.COMMIT_ACTION:
try {
Map<FileStatus, Boolean> results = super.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
hoodieRollbackStats = HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
.withDeletedFileResults(results).build();
Map<FileStatus, Boolean> results = super
.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
hoodieRollbackStats = HoodieRollbackStat.newBuilder()
.withPartitionPath(partitionPath).withDeletedFileResults(results).build();
break;
} catch (IOException io) {
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
}
case HoodieTimeline.DELTA_COMMIT_ACTION:
try {
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(this.getCommitTimeline().getInstantDetails(new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get());
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
this.getCommitTimeline().getInstantDetails(
new HoodieInstant(true, instant.getAction(), instant.getTimestamp()))
.get());
// read commit file and (either append delete blocks or delete file)
Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
Map<FileStatus, Long> filesToNumBlocksRollback = new HashMap<>();
// we do not know fileIds for inserts (first inserts are parquet files), delete all parquet files for the corresponding failed commit, if present (same as COW)
// we do not know fileIds for inserts (first inserts are parquet files), delete
// all parquet files for the corresponding failed commit, if present (same as COW)
filesToDeletedStatus = super
.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
// append rollback blocks for updates
if(commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) {
if (commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) {
commitMetadata.getPartitionToWriteStats().get(partitionPath).stream()
.filter(wStat -> {
return wStat != null && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT
return wStat != null
&& wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT
&& wStat.getPrevCommit() != null;
})
.forEach(wStat -> {
}).forEach(wStat -> {
HoodieLogFormat.Writer writer = null;
try {
writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(new Path(this.getMetaClient().getBasePath(), partitionPath))
writer = HoodieLogFormat.newWriterBuilder().onParentPath(
new Path(this.getMetaClient().getBasePath(), partitionPath))
.withFileId(wStat.getFileId()).overBaseCommit(wStat.getPrevCommit())
.withFs(this.metaClient.getFs())
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
Long numRollbackBlocks = 0L;
// generate metadata
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
Map<HoodieLogBlock.HeaderMetadataType, String> header =
Maps.newHashMap();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME,
metaClient.getActiveTimeline().lastInstant().get().getTimestamp());
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, commit);
header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE,
String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME,
commit);
header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, String
.valueOf(
HoodieCommandBlock.HoodieCommandBlockTypeEnum
.ROLLBACK_PREVIOUS_BLOCK
.ordinal()));
// if update belongs to an existing log file
writer = writer.appendBlock(new HoodieCommandBlock(
header));
writer = writer.appendBlock(new HoodieCommandBlock(header));
numRollbackBlocks++;
filesToNumBlocksRollback
.put(this.getMetaClient().getFs().getFileStatus(writer.getLogFile().getPath()),
numRollbackBlocks);
filesToNumBlocksRollback.put(this.getMetaClient().getFs()
.getFileStatus(writer.getLogFile().getPath()), numRollbackBlocks);
} catch (IOException | InterruptedException io) {
throw new HoodieRollbackException(
"Failed to rollback for commit " + commit, io);
@@ -289,7 +237,8 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
}
}
});
hoodieRollbackStats = HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
hoodieRollbackStats = HoodieRollbackStat.newBuilder()
.withPartitionPath(partitionPath)
.withDeletedFileResults(filesToDeletedStatus)
.withRollbackBlockAppendResults(filesToNumBlocksRollback).build();
}
@@ -297,17 +246,19 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
} catch (IOException io) {
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
}
default:
break;
}
return hoodieRollbackStats;
}).collect(Collectors.toList());
}).flatMap(x -> x.iterator()).filter(x -> x != null).collect();
commitsAndCompactions.entrySet().stream()
.map(entry -> new HoodieInstant(true, entry.getValue().getAction(),
entry.getValue().getTimestamp()))
.forEach(this.getActiveTimeline()::deleteInflight);
commitsAndCompactions.entrySet().stream().map(
entry -> new HoodieInstant(true, entry.getValue().getAction(),
entry.getValue().getTimestamp())).forEach(this.getActiveTimeline()::deleteInflight);
logger.debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime));
logger
.debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime));
return allRollbackStats;
}
@@ -317,4 +268,56 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
// do nothing for MOR tables
return Optional.empty();
}
/**
* UpsertPartitioner for MergeOnRead table type, this allows auto correction of small parquet
* files to larger ones without the need for an index in the logFile.
*/
class MergeOnReadUpsertPartitioner extends HoodieCopyOnWriteTable.UpsertPartitioner {
MergeOnReadUpsertPartitioner(WorkloadProfile profile) {
super(profile);
}
@Override
protected List<SmallFile> getSmallFiles(String partitionPath) {
// smallFiles only for partitionPath
List<SmallFile> smallFileLocations = new ArrayList<>();
// Init here since this class (and member variables) might not have been initialized
HoodieTimeline commitTimeline = getCompletedCommitTimeline();
if (!commitTimeline.empty()) {
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
// find smallest file in partition and append to it
Optional<FileSlice> smallFileSlice = getRTFileSystemView()
.getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).filter(
fileSlice -> fileSlice.getLogFiles().count() < 1
&& fileSlice.getDataFile().get().getFileSize() < config
.getParquetSmallFileLimit()).sorted((FileSlice left, FileSlice right) ->
left.getDataFile().get().getFileSize() < right.getDataFile().get().getFileSize()
? -1 : 1).findFirst();
if (smallFileSlice.isPresent()) {
String filename = smallFileSlice.get().getDataFile().get().getFileName();
SmallFile sf = new SmallFile();
sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename),
FSUtils.getFileId(filename));
sf.sizeBytes = smallFileSlice.get().getDataFile().get().getFileSize();
smallFileLocations.add(sf);
// Update the global small files list
smallFiles.add(sf);
}
}
return smallFileLocations;
}
public List<String> getSmallFileIds() {
return (List<String>) smallFiles.stream()
.map(smallFile -> ((SmallFile) smallFile).location.getFileId())
.collect(Collectors.toList());
}
}
}