1
0

Small File Size correction handling for MOR table type

This commit is contained in:
Nishith Agarwal
2018-01-10 21:10:22 -08:00
committed by vinoth chandar
parent 2116815261
commit 30049383f5
3 changed files with 192 additions and 29 deletions

View File

@@ -20,9 +20,11 @@ import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.HoodieRollbackStat;
import com.uber.hoodie.common.model.FileSlice;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
@@ -36,18 +38,20 @@ import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieCompactionException;
import com.uber.hoodie.exception.HoodieRollbackException;
import com.uber.hoodie.exception.HoodieUpsertException;
import com.uber.hoodie.io.HoodieAppendHandle;
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.Partitioner;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.Function;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
@@ -57,7 +61,6 @@ import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
/**
* Implementation of a more real-time read-optimized Hoodie Table where
* <p>
@@ -78,21 +81,89 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
private static Logger logger = LogManager.getLogger(HoodieMergeOnReadTable.class);
public HoodieMergeOnReadTable(HoodieWriteConfig config,
HoodieTableMetaClient metaClient) {
// UpsertPartitioner for MergeOnRead table type
private MergeOnReadUpsertPartitioner mergeOnReadUpsertPartitioner;
public HoodieMergeOnReadTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
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) {
throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner.");
}
mergeOnReadUpsertPartitioner = new MergeOnReadUpsertPartitioner(profile);
return mergeOnReadUpsertPartitioner;
}
@Override
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileId,
Iterator<HoodieRecord<T>> recordItr) throws IOException {
logger.info("Merging updates for commit " + commitTime + " for file " + fileId);
HoodieAppendHandle<T> appendHandle =
new HoodieAppendHandle<>(config, commitTime, this, fileId, recordItr);
appendHandle.doAppend();
appendHandle.close();
return Collections.singletonList(Collections.singletonList(appendHandle.getWriteStatus()))
.iterator();
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);
appendHandle.doAppend();
appendHandle.close();
return Collections.singletonList(Collections.singletonList(appendHandle.getWriteStatus()))
.iterator();
}
}
@Override