1
0

[HUDI-296] Explore use of spotless to auto fix formatting errors (#945)

- Add spotless format fixing to project
- One time reformatting for conformity
- Build fails for formatting changes and mvn spotless:apply autofixes them
This commit is contained in:
leesf
2019-10-10 20:19:40 +08:00
committed by vinoth chandar
parent 834c591955
commit b19bed442d
381 changed files with 7350 additions and 9064 deletions

View File

@@ -51,10 +51,9 @@ import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* HoodieInputFormat which understands the Hoodie File Structure and filters files based on the
* Hoodie Mode. If paths that does not correspond to a hoodie dataset then they are passed in as is
* (as what FileInputFormat.listStatus() would do). The JobConf could have paths from multipe
* Hoodie/Non-Hoodie datasets
* HoodieInputFormat which understands the Hoodie File Structure and filters files based on the Hoodie Mode. If paths
* that does not correspond to a hoodie dataset then they are passed in as is (as what FileInputFormat.listStatus()
* would do). The JobConf could have paths from multipe Hoodie/Non-Hoodie datasets
*/
@UseFileSplitsFromInputFormat
public class HoodieParquetInputFormat extends MapredParquetInputFormat implements Configurable {
@@ -86,22 +85,19 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
String mode = HoodieHiveUtil.readMode(Job.getInstance(job), tableName);
// Get all commits, delta commits, compactions, as all of them produce a base parquet file
// today
HoodieTimeline timeline = metadata.getActiveTimeline().getCommitsTimeline()
.filterCompletedInstants();
ReadOptimizedView roView = new HoodieTableFileSystemView(metadata,
timeline, statuses);
HoodieTimeline timeline = metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
ReadOptimizedView roView = new HoodieTableFileSystemView(metadata, timeline, statuses);
if (HoodieHiveUtil.INCREMENTAL_SCAN_MODE.equals(mode)) {
// this is of the form commitTs_partition_sequenceNumber
String lastIncrementalTs = HoodieHiveUtil
.readStartCommitTime(Job.getInstance(job), tableName);
String lastIncrementalTs = HoodieHiveUtil.readStartCommitTime(Job.getInstance(job), tableName);
// Total number of commits to return in this batch. Set this to -1 to get all the commits.
Integer maxCommits = HoodieHiveUtil.readMaxCommits(Job.getInstance(job), tableName);
LOG.info("Last Incremental timestamp was set as " + lastIncrementalTs);
List<String> commitsToReturn = timeline.findInstantsAfter(lastIncrementalTs, maxCommits)
.getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList());
List<HoodieDataFile> filteredFiles = roView.getLatestDataFilesInRange(commitsToReturn)
.collect(Collectors.toList());
List<String> commitsToReturn = timeline.findInstantsAfter(lastIncrementalTs, maxCommits).getInstants()
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
List<HoodieDataFile> filteredFiles =
roView.getLatestDataFilesInRange(commitsToReturn).collect(Collectors.toList());
for (HoodieDataFile filteredFile : filteredFiles) {
LOG.info("Processing incremental hoodie file - " + filteredFile.getPath());
filteredFile = checkFileStatus(filteredFile);
@@ -110,8 +106,7 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
LOG.info("Total paths to process after hoodie incremental filter " + filteredFiles.size());
} else {
// filter files on the latest commit found
List<HoodieDataFile> filteredFiles = roView.getLatestDataFiles()
.collect(Collectors.toList());
List<HoodieDataFile> filteredFiles = roView.getLatestDataFiles().collect(Collectors.toList());
LOG.info("Total paths to process after hoodie filter " + filteredFiles.size());
for (HoodieDataFile filteredFile : filteredFiles) {
if (LOG.isDebugEnabled()) {
@@ -127,10 +122,9 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
}
/**
* Checks the file status for a race condition which can set the file size to 0. 1.
* HiveInputFormat does super.listStatus() and gets back a FileStatus[] 2. Then it creates the
* HoodieTableMetaClient for the paths listed. 3. Generation of splits looks at FileStatus size to
* create splits, which skips this file
* Checks the file status for a race condition which can set the file size to 0. 1. HiveInputFormat does
* super.listStatus() and gets back a FileStatus[] 2. Then it creates the HoodieTableMetaClient for the paths listed.
* 3. Generation of splits looks at FileStatus size to create splits, which skips this file
*/
private HoodieDataFile checkFileStatus(HoodieDataFile dataFile) throws IOException {
Path dataPath = dataFile.getFileStatus().getPath();
@@ -146,24 +140,22 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
}
}
private Map<HoodieTableMetaClient, List<FileStatus>> groupFileStatus(FileStatus[] fileStatuses)
throws IOException {
private Map<HoodieTableMetaClient, List<FileStatus>> groupFileStatus(FileStatus[] fileStatuses) throws IOException {
// This assumes the paths for different tables are grouped together
Map<HoodieTableMetaClient, List<FileStatus>> grouped = new HashMap<>();
HoodieTableMetaClient metadata = null;
String nonHoodieBasePath = null;
for (FileStatus status : fileStatuses) {
if (!status.getPath().getName().endsWith(".parquet")) {
//FIXME(vc): skip non parquet files for now. This wont be needed once log file name start
// FIXME(vc): skip non parquet files for now. This wont be needed once log file name start
// with "."
continue;
}
if ((metadata == null && nonHoodieBasePath == null) || (metadata == null && !status.getPath()
.toString().contains(nonHoodieBasePath)) || (metadata != null && !status.getPath()
.toString().contains(metadata.getBasePath()))) {
if ((metadata == null && nonHoodieBasePath == null)
|| (metadata == null && !status.getPath().toString().contains(nonHoodieBasePath))
|| (metadata != null && !status.getPath().toString().contains(metadata.getBasePath()))) {
try {
metadata = getTableMetaClient(status.getPath().getFileSystem(conf),
status.getPath().getParent());
metadata = getTableMetaClient(status.getPath().getFileSystem(conf), status.getPath().getParent());
nonHoodieBasePath = null;
} catch (DatasetNotFoundException | InvalidDatasetException e) {
LOG.info("Handling a non-hoodie path " + status.getPath());
@@ -188,29 +180,28 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
}
@Override
public RecordReader<NullWritable, ArrayWritable> getRecordReader(final InputSplit split,
final JobConf job, final Reporter reporter) throws IOException {
public RecordReader<NullWritable, ArrayWritable> getRecordReader(final InputSplit split, final JobConf job,
final Reporter reporter) throws IOException {
// TODO enable automatic predicate pushdown after fixing issues
// FileSplit fileSplit = (FileSplit) split;
// HoodieTableMetadata metadata = getTableMetadata(fileSplit.getPath().getParent());
// String tableName = metadata.getTableName();
// String mode = HoodieHiveUtil.readMode(job, tableName);
// FileSplit fileSplit = (FileSplit) split;
// HoodieTableMetadata metadata = getTableMetadata(fileSplit.getPath().getParent());
// String tableName = metadata.getTableName();
// String mode = HoodieHiveUtil.readMode(job, tableName);
// if (HoodieHiveUtil.INCREMENTAL_SCAN_MODE.equals(mode)) {
// FilterPredicate predicate = constructHoodiePredicate(job, tableName, split);
// LOG.info("Setting parquet predicate push down as " + predicate);
// ParquetInputFormat.setFilterPredicate(job, predicate);
//clearOutExistingPredicate(job);
// }
// if (HoodieHiveUtil.INCREMENTAL_SCAN_MODE.equals(mode)) {
// FilterPredicate predicate = constructHoodiePredicate(job, tableName, split);
// LOG.info("Setting parquet predicate push down as " + predicate);
// ParquetInputFormat.setFilterPredicate(job, predicate);
// clearOutExistingPredicate(job);
// }
return super.getRecordReader(split, job, reporter);
}
/**
* Read the table metadata from a data path. This assumes certain hierarchy of files which should
* be changed once a better way is figured out to pass in the hoodie meta directory
* Read the table metadata from a data path. This assumes certain hierarchy of files which should be changed once a
* better way is figured out to pass in the hoodie meta directory
*/
protected static HoodieTableMetaClient getTableMetaClient(FileSystem fs, Path dataPath)
throws IOException {
protected static HoodieTableMetaClient getTableMetaClient(FileSystem fs, Path dataPath) throws IOException {
int levels = HoodieHiveUtil.DEFAULT_LEVELS_TO_BASEPATH;
if (HoodiePartitionMetadata.hasPartitionMetadata(fs, dataPath)) {
HoodiePartitionMetadata metadata = new HoodiePartitionMetadata(fs, dataPath);

View File

@@ -37,22 +37,22 @@ import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* Given a path is a part of - Hoodie dataset = accepts ONLY the latest version of each path -
* Non-Hoodie dataset = then always accept
* Given a path is a part of - Hoodie dataset = accepts ONLY the latest version of each path - Non-Hoodie dataset = then
* always accept
* <p>
* We can set this filter, on a query engine's Hadoop Config and if it respects path filters, then
* you should be able to query both hoodie and non-hoodie datasets as you would normally do.
* We can set this filter, on a query engine's Hadoop Config and if it respects path filters, then you should be able to
* query both hoodie and non-hoodie datasets as you would normally do.
* <p>
* hadoopConf.setClass("mapreduce.input.pathFilter.class", org.apache.hudi.hadoop
* .HoodieROTablePathFilter.class, org.apache.hadoop.fs.PathFilter.class)
* hadoopConf.setClass("mapreduce.input.pathFilter.class", org.apache.hudi.hadoop .HoodieROTablePathFilter.class,
* org.apache.hadoop.fs.PathFilter.class)
*/
public class HoodieROTablePathFilter implements PathFilter, Serializable {
private static final transient Logger LOG = LogManager.getLogger(HoodieROTablePathFilter.class);
/**
* Its quite common, to have all files from a given partition path be passed into accept(), cache
* the check for hoodie metadata for known partition paths and the latest versions of files
* Its quite common, to have all files from a given partition path be passed into accept(), cache the check for hoodie
* metadata for known partition paths and the latest versions of files
*/
private HashMap<String, HashSet<Path>> hoodiePathCache;
@@ -135,19 +135,16 @@ public class HoodieROTablePathFilter implements PathFilter, Serializable {
if (baseDir != null) {
try {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(),
baseDir.toString());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), baseDir.toString());
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient,
metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(),
fs.listStatus(folder));
List<HoodieDataFile> latestFiles = fsView.getLatestDataFiles()
.collect(Collectors.toList());
metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(), fs.listStatus(folder));
List<HoodieDataFile> latestFiles = fsView.getLatestDataFiles().collect(Collectors.toList());
// populate the cache
if (!hoodiePathCache.containsKey(folder.toString())) {
hoodiePathCache.put(folder.toString(), new HashSet<>());
}
LOG.info("Based on hoodie metadata from base path: " + baseDir.toString() + ", caching "
+ latestFiles.size() + " files under " + folder);
LOG.info("Based on hoodie metadata from base path: " + baseDir.toString() + ", caching " + latestFiles.size()
+ " files under " + folder);
for (HoodieDataFile lfile : latestFiles) {
hoodiePathCache.get(folder.toString()).add(new Path(lfile.getPath()));
}

View File

@@ -25,12 +25,11 @@ import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapred.RecordReader;
/**
* Record Reader for parquet. Records read from this reader is safe to be
* buffered for concurrent processing.
* Record Reader for parquet. Records read from this reader is safe to be buffered for concurrent processing.
*
* In concurrent producer/consumer pattern, where the record is read and buffered by one thread and processed in
* another thread, we need to ensure new instance of ArrayWritable is buffered. ParquetReader createKey/Value is unsafe
* as it gets reused for subsequent fetch. This wrapper makes ParquetReader safe for this use-case.
* In concurrent producer/consumer pattern, where the record is read and buffered by one thread and processed in another
* thread, we need to ensure new instance of ArrayWritable is buffered. ParquetReader createKey/Value is unsafe as it
* gets reused for subsequent fetch. This wrapper makes ParquetReader safe for this use-case.
*/
public class SafeParquetRecordReaderWrapper implements RecordReader<NullWritable, ArrayWritable> {
@@ -62,11 +61,9 @@ public class SafeParquetRecordReaderWrapper implements RecordReader<NullWritable
}
/**
* We could be in concurrent fetch and read env.
* We need to ensure new ArrayWritable as ParquetReader implementation reuses same
* ArrayWritable for all reads which will cause corruption when buffering.
* So, we create a new ArrayWritable here with Value class from parquetReader's value
* and an empty array.
* We could be in concurrent fetch and read env. We need to ensure new ArrayWritable as ParquetReader implementation
* reuses same ArrayWritable for all reads which will cause corruption when buffering. So, we create a new
* ArrayWritable here with Value class from parquetReader's value and an empty array.
*/
@Override
public ArrayWritable createValue() {

View File

@@ -26,8 +26,8 @@ import java.lang.annotation.RetentionPolicy;
import java.lang.annotation.Target;
/**
* When annotated on a InputFormat, informs the query engines, that they should use the FileSplits
* provided by the input format to execute the queries
* When annotated on a InputFormat, informs the query engines, that they should use the FileSplits provided by the input
* format to execute the queries
*/
@Inherited
@Documented

View File

@@ -76,14 +76,13 @@ import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* This is just a copy of the org.apache.hadoop.hive.ql.io.CombineHiveInputFormat from Hive 2.x
* Search for **MOD** to see minor modifications to support custom inputformat in CombineHiveInputFormat.
* See https://issues.apache.org/jira/browse/HIVE-9771
* This is just a copy of the org.apache.hadoop.hive.ql.io.CombineHiveInputFormat from Hive 2.x Search for **MOD** to
* see minor modifications to support custom inputformat in CombineHiveInputFormat. See
* https://issues.apache.org/jira/browse/HIVE-9771
* <p>
* <p>
* CombineHiveInputFormat is a parameterized InputFormat which looks at the path
* name and determine the correct InputFormat for that path name from
* mapredPlan.pathToPartitionInfo(). It can be used to read files with different
* CombineHiveInputFormat is a parameterized InputFormat which looks at the path name and determine the correct
* InputFormat for that path name from mapredPlan.pathToPartitionInfo(). It can be used to read files with different
* input format in the same map-reduce job.
*
* NOTE : This class is implemented to work with Hive 2.x +
@@ -116,19 +115,15 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
public Set<Integer> call() throws Exception {
Set<Integer> nonCombinablePathIndices = new HashSet<Integer>();
for (int i = 0; i < length; i++) {
PartitionDesc part =
HiveFileFormatUtils.getPartitionDescFromPathRecursively(
pathToPartitionInfo, paths[i + start],
IOPrepareCache.get().allocatePartitionDescMap());
PartitionDesc part = HiveFileFormatUtils.getPartitionDescFromPathRecursively(pathToPartitionInfo,
paths[i + start], IOPrepareCache.get().allocatePartitionDescMap());
// Use HiveInputFormat if any of the paths is not splittable
Class<? extends InputFormat> inputFormatClass = part.getInputFileFormatClass();
InputFormat<WritableComparable, Writable> inputFormat =
getInputFormatFromCache(inputFormatClass, conf);
InputFormat<WritableComparable, Writable> inputFormat = getInputFormatFromCache(inputFormatClass, conf);
if (inputFormat instanceof AvoidSplitCombination
&& ((AvoidSplitCombination) inputFormat).shouldSkipCombine(paths[i + start], conf)) {
if (LOG.isDebugEnabled()) {
LOG.debug("The path [" + paths[i + start]
+ "] is being parked for HiveInputFormat.getSplits");
LOG.debug("The path [" + paths[i + start] + "] is being parked for HiveInputFormat.getSplits");
}
nonCombinablePathIndices.add(i + start);
}
@@ -138,10 +133,9 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
}
/**
* CombineHiveInputSplit encapsulates an InputSplit with its corresponding
* inputFormatClassName. A CombineHiveInputSplit comprises of multiple chunks
* from different files. Since, they belong to a single directory, there is a
* single inputformat for all the chunks.
* CombineHiveInputSplit encapsulates an InputSplit with its corresponding inputFormatClassName. A
* CombineHiveInputSplit comprises of multiple chunks from different files. Since, they belong to a single directory,
* there is a single inputformat for all the chunks.
*/
public static class CombineHiveInputSplit extends InputSplitShim {
@@ -150,16 +144,14 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
private Map<Path, PartitionDesc> pathToPartitionInfo;
public CombineHiveInputSplit() throws IOException {
this(ShimLoader.getHadoopShims().getCombineFileInputFormat()
.getInputSplitShim());
this(ShimLoader.getHadoopShims().getCombineFileInputFormat().getInputSplitShim());
}
public CombineHiveInputSplit(CombineFileSplit inputSplitShim) throws IOException {
this(inputSplitShim.getJob(), inputSplitShim);
}
public CombineHiveInputSplit(JobConf job, CombineFileSplit inputSplitShim)
throws IOException {
public CombineHiveInputSplit(JobConf job, CombineFileSplit inputSplitShim) throws IOException {
this(job, inputSplitShim, null);
}
@@ -176,9 +168,8 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
// CombinedSplit.
Path[] ipaths = inputSplitShim.getPaths();
if (ipaths.length > 0) {
PartitionDesc part = HiveFileFormatUtils
.getPartitionDescFromPathRecursively(this.pathToPartitionInfo,
ipaths[0], IOPrepareCache.get().getPartitionDescMap());
PartitionDesc part = HiveFileFormatUtils.getPartitionDescFromPathRecursively(this.pathToPartitionInfo,
ipaths[0], IOPrepareCache.get().getPartitionDescMap());
inputFormatClassName = part.getInputFileFormatClass().getName();
}
}
@@ -307,9 +298,8 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
// extract all the inputFormatClass names for each chunk in the
// CombinedSplit.
PartitionDesc part =
HiveFileFormatUtils.getPartitionDescFromPathRecursively(pathToPartitionInfo,
inputSplitShim.getPath(0), IOPrepareCache.get().getPartitionDescMap());
PartitionDesc part = HiveFileFormatUtils.getPartitionDescFromPathRecursively(pathToPartitionInfo,
inputSplitShim.getPath(0), IOPrepareCache.get().getPartitionDescMap());
// create a new InputFormat instance if this is the first time to see
// this class
@@ -328,8 +318,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
private final String inputFormatClassName;
private final String deserializerClassName;
public CombinePathInputFormat(List<Operator<? extends OperatorDesc>> opList,
String inputFormatClassName,
public CombinePathInputFormat(List<Operator<? extends OperatorDesc>> opList, String inputFormatClassName,
String deserializerClassName) {
this.opList = opList;
this.inputFormatClassName = inputFormatClassName;
@@ -340,10 +329,9 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
public boolean equals(Object o) {
if (o instanceof CombinePathInputFormat) {
CombinePathInputFormat mObj = (CombinePathInputFormat) o;
return (opList.equals(mObj.opList))
&& (inputFormatClassName.equals(mObj.inputFormatClassName))
&& (deserializerClassName == null ? (mObj.deserializerClassName == null) :
deserializerClassName.equals(mObj.deserializerClassName));
return (opList.equals(mObj.opList)) && (inputFormatClassName.equals(mObj.inputFormatClassName))
&& (deserializerClassName == null ? (mObj.deserializerClassName == null)
: deserializerClassName.equals(mObj.deserializerClassName));
}
return false;
}
@@ -357,16 +345,14 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
/**
* Create Hive splits based on CombineFileSplit.
*/
private InputSplit[] getCombineSplits(JobConf job, int numSplits,
Map<Path, PartitionDesc> pathToPartitionInfo)
private InputSplit[] getCombineSplits(JobConf job, int numSplits, Map<Path, PartitionDesc> pathToPartitionInfo)
throws IOException {
init(job);
Map<Path, ArrayList<String>> pathToAliases = mrwork.getPathToAliases();
Map<String, Operator<? extends OperatorDesc>> aliasToWork =
mrwork.getAliasToWork();
Map<String, Operator<? extends OperatorDesc>> aliasToWork = mrwork.getAliasToWork();
/** MOD - Initialize a custom combine input format shim that will call listStatus on the custom inputFormat **/
HoodieCombineHiveInputFormat.HoodieCombineFileInputFormatShim
combine = new HoodieCombineHiveInputFormat.HoodieCombineFileInputFormatShim();
HoodieCombineHiveInputFormat.HoodieCombineFileInputFormatShim combine =
new HoodieCombineHiveInputFormat.HoodieCombineFileInputFormatShim();
InputSplit[] splits = null;
if (combine == null) {
@@ -385,13 +371,12 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
List<Path> inpDirs = new ArrayList<Path>();
List<Path> inpFiles = new ArrayList<Path>();
Map<CombinePathInputFormat, CombineFilter> poolMap =
new HashMap<CombinePathInputFormat, CombineFilter>();
Map<CombinePathInputFormat, CombineFilter> poolMap = new HashMap<CombinePathInputFormat, CombineFilter>();
Set<Path> poolSet = new HashSet<Path>();
for (Path path : paths) {
PartitionDesc part = HiveFileFormatUtils.getPartitionDescFromPathRecursively(
pathToPartitionInfo, path, IOPrepareCache.get().allocatePartitionDescMap());
PartitionDesc part = HiveFileFormatUtils.getPartitionDescFromPathRecursively(pathToPartitionInfo, path,
IOPrepareCache.get().allocatePartitionDescMap());
TableDesc tableDesc = part.getTableDesc();
if ((tableDesc != null) && tableDesc.isNonNative()) {
return super.getSplits(job, numSplits);
@@ -418,7 +403,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
}
FileSystem inpFs = path.getFileSystem(job);
//don't combine if inputformat is a SymlinkTextInputFormat
// don't combine if inputformat is a SymlinkTextInputFormat
if (inputFormat instanceof SymlinkTextInputFormat) {
splits = super.getSplits(job, numSplits);
return splits;
@@ -431,24 +416,21 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
List<Operator<? extends OperatorDesc>> opList = null;
if (!mrwork.isMapperCannotSpanPartns()) {
//if mapper can span partitions, make sure a splits does not contain multiple
// if mapper can span partitions, make sure a splits does not contain multiple
// opList + inputFormatClassName + deserializerClassName combination
// This is done using the Map of CombinePathInputFormat to PathFilter
opList = HiveFileFormatUtils.doGetWorksFromPath(
pathToAliases, aliasToWork, filterPath);
opList = HiveFileFormatUtils.doGetWorksFromPath(pathToAliases, aliasToWork, filterPath);
CombinePathInputFormat combinePathInputFormat =
new CombinePathInputFormat(opList, inputFormatClassName, deserializerClassName);
f = poolMap.get(combinePathInputFormat);
if (f == null) {
f = new CombineFilter(filterPath);
LOG.info("CombineHiveInputSplit creating pool for "
+ path + "; using filter path " + filterPath);
LOG.info("CombineHiveInputSplit creating pool for " + path + "; using filter path " + filterPath);
combine.createPool(job, f);
poolMap.put(combinePathInputFormat, f);
} else {
LOG.info("CombineHiveInputSplit: pool is already created for "
+ path + "; using filter path " + filterPath);
LOG.info("CombineHiveInputSplit: pool is already created for " + path + "; using filter path " + filterPath);
f.addPath(filterPath);
}
} else {
@@ -470,8 +452,8 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
// Processing directories
List<CombineFileSplit> iss = new ArrayList<CombineFileSplit>();
if (!mrwork.isMapperCannotSpanPartns()) {
//mapper can span partitions
//combine into as few as one split, subject to the PathFilters set
// mapper can span partitions
// combine into as few as one split, subject to the PathFilters set
// using combine.createPool.
iss = Arrays.asList(combine.getSplits(job, 1));
} else {
@@ -507,8 +489,8 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
@VisibleForTesting
public Set<Integer> getNonCombinablePathIndices(JobConf job, Path[] paths, int numThreads)
throws ExecutionException, InterruptedException {
LOG.info("Total number of paths: " + paths.length
+ ", launching " + numThreads + " threads to check non-combinable ones.");
LOG.info("Total number of paths: " + paths.length + ", launching " + numThreads
+ " threads to check non-combinable ones.");
int numPathPerThread = (int) Math.ceil((double) paths.length / numThreads);
ExecutorService executor = Executors.newFixedThreadPool(numThreads);
@@ -517,8 +499,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
for (int i = 0; i < numThreads; i++) {
int start = i * numPathPerThread;
int length = i != numThreads - 1 ? numPathPerThread : paths.length - start;
futureList.add(executor.submit(
new CheckNonCombinablePathCallable(paths, start, length, job)));
futureList.add(executor.submit(new CheckNonCombinablePathCallable(paths, start, length, job)));
}
Set<Integer> nonCombinablePathIndices = new HashSet<Integer>();
for (Future<Set<Integer>> future : futureList) {
@@ -572,15 +553,13 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
// Store the previous value for the path specification
String oldPaths = job.get(org.apache.hadoop.mapreduce.lib.input.FileInputFormat.INPUT_DIR);
if (LOG.isDebugEnabled()) {
LOG.debug("The received input paths are: [" + oldPaths
+ "] against the property "
LOG.debug("The received input paths are: [" + oldPaths + "] against the property "
+ org.apache.hadoop.mapreduce.lib.input.FileInputFormat.INPUT_DIR);
}
// Process the normal splits
if (nonCombinablePaths.size() > 0) {
FileInputFormat.setInputPaths(job,
nonCombinablePaths.toArray(new Path[nonCombinablePaths.size()]));
FileInputFormat.setInputPaths(job, nonCombinablePaths.toArray(new Path[nonCombinablePaths.size()]));
InputSplit[] splits = super.getSplits(job, numSplits);
for (InputSplit split : splits) {
result.add(split);
@@ -589,10 +568,9 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
// Process the combine splits
if (combinablePaths.size() > 0) {
FileInputFormat.setInputPaths(job,
combinablePaths.toArray(new Path[combinablePaths.size()]));
Map<Path, PartitionDesc> pathToPartitionInfo = this.pathToPartitionInfo != null
? this.pathToPartitionInfo : Utilities.getMapWork(job).getPathToPartitionInfo();
FileInputFormat.setInputPaths(job, combinablePaths.toArray(new Path[combinablePaths.size()]));
Map<Path, PartitionDesc> pathToPartitionInfo = this.pathToPartitionInfo != null ? this.pathToPartitionInfo
: Utilities.getMapWork(job).getPathToPartitionInfo();
InputSplit[] splits = getCombineSplits(job, numSplits, pathToPartitionInfo);
for (InputSplit split : splits) {
result.add(split);
@@ -614,8 +592,8 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
return result.toArray(new InputSplit[result.size()]);
}
private void processPaths(JobConf job, CombineFileInputFormatShim combine,
List<CombineFileSplit> iss, Path... path) throws IOException {
private void processPaths(JobConf job, CombineFileInputFormatShim combine, List<CombineFileSplit> iss, Path... path)
throws IOException {
JobConf currJob = new JobConf(job);
FileInputFormat.setInputPaths(currJob, path);
iss.addAll(Arrays.asList(combine.getSplits(currJob, 1)));
@@ -645,19 +623,17 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
/**
* This function is used to sample inputs for clauses like "TABLESAMPLE(1 PERCENT)"
* <p>
* First, splits are grouped by alias they are for. If one split serves more than one
* alias or not for any sampled alias, we just directly add it to returned list.
* Then we find a list of exclusive splits for every alias to be sampled.
* For each alias, we start from position of seedNumber%totalNumber, and keep add
* splits until the total size hits percentage.
* First, splits are grouped by alias they are for. If one split serves more than one alias or not for any sampled
* alias, we just directly add it to returned list. Then we find a list of exclusive splits for every alias to be
* sampled. For each alias, we start from position of seedNumber%totalNumber, and keep add splits until the total size
* hits percentage.
*
* @return the sampled splits
*/
private List<CombineFileSplit> sampleSplits(List<CombineFileSplit> splits) {
HashMap<String, SplitSample> nameToSamples = mrwork.getNameToSplitSample();
List<CombineFileSplit> retLists = new ArrayList<CombineFileSplit>();
Map<String, ArrayList<CombineFileSplit>> aliasToSplitList =
new HashMap<String, ArrayList<CombineFileSplit>>();
Map<String, ArrayList<CombineFileSplit>> aliasToSplitList = new HashMap<String, ArrayList<CombineFileSplit>>();
Map<Path, ArrayList<String>> pathToAliases = mrwork.getPathToAliases();
Map<Path, ArrayList<String>> pathToAliasesNoScheme = removeScheme(pathToAliases);
@@ -667,14 +643,13 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
String alias = null;
for (Path path : split.getPaths()) {
boolean schemeless = path.toUri().getScheme() == null;
List<String> l = HiveFileFormatUtils.doGetAliasesFromPath(
schemeless ? pathToAliasesNoScheme : pathToAliases, path);
List<String> l =
HiveFileFormatUtils.doGetAliasesFromPath(schemeless ? pathToAliasesNoScheme : pathToAliases, path);
// a path for a split unqualified the split from being sampled if:
// 1. it serves more than one alias
// 2. the alias it serves is not sampled
// 3. it serves different alias than another path for the same split
if (l.size() != 1 || !nameToSamples.containsKey(l.get(0))
|| (alias != null && l.get(0) != alias)) {
if (l.size() != 1 || !nameToSamples.containsKey(l.get(0)) || (alias != null && l.get(0) != alias)) {
alias = null;
break;
}
@@ -739,12 +714,10 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
}
/**
* Create a generic Hive RecordReader than can iterate over all chunks in a
* CombinedFileSplit.
* Create a generic Hive RecordReader than can iterate over all chunks in a CombinedFileSplit.
*/
@Override
public RecordReader getRecordReader(InputSplit split, JobConf job,
Reporter reporter) throws IOException {
public RecordReader getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
if (!(split instanceof CombineHiveInputSplit)) {
return super.getRecordReader(split, job, reporter);
}
@@ -762,10 +735,8 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
pushProjectionsAndFilters(job, inputFormatClass, hsplit.getPath(0));
return ShimLoader.getHadoopShims().getCombineFileInputFormat()
.getRecordReader(job,
(CombineFileSplit) split, reporter,
CombineHiveRecordReader.class);
return ShimLoader.getHadoopShims().getCombineFileInputFormat().getRecordReader(job, (CombineFileSplit) split,
reporter, CombineHiveRecordReader.class);
}
static class CombineFilter implements PathFilter {
@@ -813,8 +784,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
}
/**
* This is a marker interface that is used to identify the formats where
* combine split generation is not applicable
* This is a marker interface that is used to identify the formats where combine split generation is not applicable
*/
public interface AvoidSplitCombination {
@@ -823,8 +793,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
/**
* **MOD** this is the implementation of CombineFileInputFormat which is a copy of
* org.apache.hadoop.hive.shims.HadoopShimsSecure.CombineFileInputFormatShim
* with changes in listStatus
* org.apache.hadoop.hive.shims.HadoopShimsSecure.CombineFileInputFormatShim with changes in listStatus
*/
public static class HoodieCombineFileInputFormatShim<K, V> extends CombineFileInputFormat<K, V>
implements org.apache.hadoop.hive.shims.HadoopShims.CombineFileInputFormatShim<K, V> {
@@ -832,8 +801,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
private boolean hoodieFilter = false;
private boolean isRealTime = false;
public HoodieCombineFileInputFormatShim() {
}
public HoodieCombineFileInputFormatShim() {}
public Path[] getInputPathsShim(JobConf conf) {
try {
@@ -848,8 +816,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
}
@Override
public RecordReader<K, V> getRecordReader(InputSplit split, JobConf job, Reporter reporter)
throws IOException {
public RecordReader<K, V> getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
throw new IOException("CombineFileInputFormat.getRecordReader not needed.");
}
@@ -866,8 +833,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
input = new HoodieParquetInputFormat();
}
input.setConf(job.getConfiguration());
result = new ArrayList<FileStatus>(
Arrays.asList(input.listStatus(new JobConf(job.getConfiguration()))));
result = new ArrayList<FileStatus>(Arrays.asList(input.listStatus(new JobConf(job.getConfiguration()))));
} else {
result = super.listStatus(job);
}
@@ -903,14 +869,12 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
for (int pos = 0; pos < splits.length; ++pos) {
CombineFileSplit split = (CombineFileSplit) splits[pos];
if (split.getPaths().length > 0) {
inputSplitShims.add(
new HadoopShimsSecure.InputSplitShim(job, split.getPaths(), split.getStartOffsets(),
split.getLengths(), split.getLocations()));
inputSplitShims.add(new HadoopShimsSecure.InputSplitShim(job, split.getPaths(), split.getStartOffsets(),
split.getLengths(), split.getLocations()));
}
}
return (CombineFileSplit[]) inputSplitShims
.toArray(new HadoopShimsSecure.InputSplitShim[inputSplitShims.size()]);
return (CombineFileSplit[]) inputSplitShims.toArray(new HadoopShimsSecure.InputSplitShim[inputSplitShims.size()]);
}
public HadoopShimsSecure.InputSplitShim getInputSplitShim() throws IOException {

View File

@@ -58,8 +58,7 @@ import org.apache.parquet.hadoop.ParquetFileReader;
import org.apache.parquet.schema.MessageType;
/**
* Record Reader implementation to merge fresh avro data with base parquet data, to support real
* time queries.
* Record Reader implementation to merge fresh avro data with base parquet data, to support real time queries.
*/
public abstract class AbstractRealtimeRecordReader {
@@ -69,8 +68,7 @@ public abstract class AbstractRealtimeRecordReader {
// used to choose a trade off between IO vs Memory when performing compaction process
// Depending on outputfile size and memory provided, choose true to avoid OOM for large file
// size + small memory
public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP =
"compaction.lazy.block.read.enabled";
public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP = "compaction.lazy.block.read.enabled";
public static final String DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED = "true";
// Property to set the max memory for dfs inputstream buffer size
@@ -104,8 +102,7 @@ public abstract class AbstractRealtimeRecordReader {
baseFileSchema = readSchema(jobConf, split.getPath());
init();
} catch (IOException e) {
throw new HoodieIOException(
"Could not create HoodieRealtimeRecordReader on path " + this.split.getPath(), e);
throw new HoodieIOException("Could not create HoodieRealtimeRecordReader on path " + this.split.getPath(), e);
}
}
@@ -116,8 +113,8 @@ public abstract class AbstractRealtimeRecordReader {
}
/**
* Reads the schema from the parquet file. This is different from ParquetUtils as it uses the
* twitter parquet to support hive 1.1.0
* Reads the schema from the parquet file. This is different from ParquetUtils as it uses the twitter parquet to
* support hive 1.1.0
*/
private static MessageType readSchema(Configuration conf, Path parquetFilePath) {
try {
@@ -157,19 +154,19 @@ public abstract class AbstractRealtimeRecordReader {
}
/**
* Given a comma separated list of field names and positions at which they appear on Hive, return
* a ordered list of field names, that can be passed onto storage.
* Given a comma separated list of field names and positions at which they appear on Hive, return a ordered list of
* field names, that can be passed onto storage.
*/
private static List<String> orderFields(String fieldNameCsv, String fieldOrderCsv, List<String> partitioningFields) {
String[] fieldOrders = fieldOrderCsv.split(",");
List<String> fieldNames = Arrays.stream(fieldNameCsv.split(","))
.filter(fn -> !partitioningFields.contains(fn)).collect(Collectors.toList());
List<String> fieldNames = Arrays.stream(fieldNameCsv.split(",")).filter(fn -> !partitioningFields.contains(fn))
.collect(Collectors.toList());
// Hive does not provide ids for partitioning fields, so check for lengths excluding that.
if (fieldNames.size() != fieldOrders.length) {
throw new HoodieException(String
.format("Error ordering fields for storage read. #fieldNames: %d, #fieldPositions: %d",
throw new HoodieException(
String.format("Error ordering fields for storage read. #fieldNames: %d, #fieldPositions: %d",
fieldNames.size(), fieldOrders.length));
}
TreeMap<Integer, String> orderedFieldMap = new TreeMap<>();
@@ -180,18 +177,17 @@ public abstract class AbstractRealtimeRecordReader {
}
/**
* Generate a reader schema off the provided writeSchema, to just project out the provided
* columns
* Generate a reader schema off the provided writeSchema, to just project out the provided columns
*/
public static Schema generateProjectionSchema(Schema writeSchema, List<String> fieldNames) {
/**
* Avro & Presto field names seems to be case sensitive (support fields differing only in case)
* whereas Hive/Impala/SparkSQL(default) are case-insensitive. Spark allows this to be configurable
* using spark.sql.caseSensitive=true
* Avro & Presto field names seems to be case sensitive (support fields differing only in case) whereas
* Hive/Impala/SparkSQL(default) are case-insensitive. Spark allows this to be configurable using
* spark.sql.caseSensitive=true
*
* For a RT table setup with no delta-files (for a latest file-slice) -> we translate parquet schema to Avro
* Here the field-name case is dependent on parquet schema. Hive (1.x/2.x/CDH) translate column projections
* to lower-cases
* For a RT table setup with no delta-files (for a latest file-slice) -> we translate parquet schema to Avro Here
* the field-name case is dependent on parquet schema. Hive (1.x/2.x/CDH) translate column projections to
* lower-cases
*
*/
List<Schema.Field> projectedFields = new ArrayList<>();
@@ -201,16 +197,14 @@ public abstract class AbstractRealtimeRecordReader {
Schema.Field field = schemaFieldsMap.get(fn.toLowerCase());
if (field == null) {
throw new HoodieException("Field " + fn + " not found in log schema. Query cannot proceed! "
+ "Derived Schema Fields: "
+ new ArrayList<>(schemaFieldsMap.keySet()));
+ "Derived Schema Fields: " + new ArrayList<>(schemaFieldsMap.keySet()));
} else {
projectedFields
.add(new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultValue()));
projectedFields.add(new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultValue()));
}
}
Schema projectedSchema = Schema
.createRecord(writeSchema.getName(), writeSchema.getDoc(), writeSchema.getNamespace(), writeSchema.isError());
Schema projectedSchema = Schema.createRecord(writeSchema.getName(), writeSchema.getDoc(),
writeSchema.getNamespace(), writeSchema.isError());
projectedSchema.setFields(projectedFields);
return projectedSchema;
}
@@ -295,16 +289,16 @@ public abstract class AbstractRealtimeRecordReader {
}
/**
* Hive implementation of ParquetRecordReader results in partition columns not present in the original parquet file
* to also be part of the projected schema. Hive expects the record reader implementation to return the row in its
* Hive implementation of ParquetRecordReader results in partition columns not present in the original parquet file to
* also be part of the projected schema. Hive expects the record reader implementation to return the row in its
* entirety (with un-projected column having null values). As we use writerSchema for this, make sure writer schema
* also includes partition columns
*
* @param schema Schema to be changed
*/
private static Schema addPartitionFields(Schema schema, List<String> partitioningFields) {
final Set<String> firstLevelFieldNames = schema.getFields().stream().map(Field::name)
.map(String::toLowerCase).collect(Collectors.toSet());
final Set<String> firstLevelFieldNames =
schema.getFields().stream().map(Field::name).map(String::toLowerCase).collect(Collectors.toSet());
List<String> fieldsToAdd = partitioningFields.stream().map(String::toLowerCase)
.filter(x -> !firstLevelFieldNames.contains(x)).collect(Collectors.toList());
@@ -313,12 +307,12 @@ public abstract class AbstractRealtimeRecordReader {
/**
* Goes through the log files in reverse order and finds the schema from the last available data block. If not, falls
* back to the schema from the latest parquet file. Finally, sets the partition column and projection fields into
* the job conf.
* back to the schema from the latest parquet file. Finally, sets the partition column and projection fields into the
* job conf.
*/
private void init() throws IOException {
Schema schemaFromLogFile = LogReaderUtils
.readLatestSchemaFromLogFiles(split.getBasePath(), split.getDeltaFilePaths(), jobConf);
Schema schemaFromLogFile =
LogReaderUtils.readLatestSchemaFromLogFiles(split.getBasePath(), split.getDeltaFilePaths(), jobConf);
if (schemaFromLogFile == null) {
writerSchema = new AvroSchemaConverter().convert(baseFileSchema);
LOG.debug("Writer Schema From Parquet => " + writerSchema.getFields());
@@ -332,10 +326,8 @@ public abstract class AbstractRealtimeRecordReader {
partitionFields.length() > 0 ? Arrays.stream(partitionFields.split(",")).collect(Collectors.toList())
: new ArrayList<>();
writerSchema = addPartitionFields(writerSchema, partitioningFields);
List<String> projectionFields = orderFields(
jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR),
jobConf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR),
partitioningFields);
List<String> projectionFields = orderFields(jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR),
jobConf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR), partitioningFields);
// TODO(vc): In the future, the reader schema should be updated based on log files & be able
// to null out fields not present before
readerSchema = generateProjectionSchema(writerSchema, projectionFields);
@@ -353,8 +345,8 @@ public abstract class AbstractRealtimeRecordReader {
public long getMaxCompactionMemoryInBytes() {
// jobConf.getMemoryForMapTask() returns in MB
return (long) Math.ceil(Double
.valueOf(jobConf.get(COMPACTION_MEMORY_FRACTION_PROP, DEFAULT_COMPACTION_MEMORY_FRACTION))
* jobConf.getMemoryForMapTask() * 1024 * 1024L);
return (long) Math
.ceil(Double.valueOf(jobConf.get(COMPACTION_MEMORY_FRACTION_PROP, DEFAULT_COMPACTION_MEMORY_FRACTION))
* jobConf.getMemoryForMapTask() * 1024 * 1024L);
}
}

View File

@@ -77,19 +77,18 @@ public class HoodieParquetRealtimeInputFormat extends HoodieParquetInputFormat i
@Override
public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
Stream<FileSplit> fileSplits = Arrays.stream(super.getSplits(job, numSplits))
.map(is -> (FileSplit) is);
Stream<FileSplit> fileSplits = Arrays.stream(super.getSplits(job, numSplits)).map(is -> (FileSplit) is);
// obtain all unique parent folders for splits
Map<Path, List<FileSplit>> partitionsToParquetSplits = fileSplits
.collect(Collectors.groupingBy(split -> split.getPath().getParent()));
Map<Path, List<FileSplit>> partitionsToParquetSplits =
fileSplits.collect(Collectors.groupingBy(split -> split.getPath().getParent()));
// TODO(vc): Should we handle also non-hoodie splits here?
Map<String, HoodieTableMetaClient> metaClientMap = new HashMap<>();
Map<Path, HoodieTableMetaClient> partitionsToMetaClient = partitionsToParquetSplits.keySet()
.stream().collect(Collectors.toMap(Function.identity(), p -> {
Map<Path, HoodieTableMetaClient> partitionsToMetaClient =
partitionsToParquetSplits.keySet().stream().collect(Collectors.toMap(Function.identity(), p -> {
// find if we have a metaclient already for this partition.
Option<String> matchingBasePath = Option.fromJavaOptional(metaClientMap.keySet().stream()
.filter(basePath -> p.toString().startsWith(basePath)).findFirst());
Option<String> matchingBasePath = Option.fromJavaOptional(
metaClientMap.keySet().stream().filter(basePath -> p.toString().startsWith(basePath)).findFirst());
if (matchingBasePath.isPresent()) {
return metaClientMap.get(matchingBasePath.get());
}
@@ -109,23 +108,20 @@ public class HoodieParquetRealtimeInputFormat extends HoodieParquetInputFormat i
partitionsToParquetSplits.keySet().stream().forEach(partitionPath -> {
// for each partition path obtain the data & log file groupings, then map back to inputsplits
HoodieTableMetaClient metaClient = partitionsToMetaClient.get(partitionPath);
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient,
metaClient.getActiveTimeline());
String relPartitionPath = FSUtils
.getRelativePartitionPath(new Path(metaClient.getBasePath()), partitionPath);
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline());
String relPartitionPath = FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), partitionPath);
try {
// Both commit and delta-commits are included - pick the latest completed one
Option<HoodieInstant> latestCompletedInstant =
metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant();
Stream<FileSlice> latestFileSlices = latestCompletedInstant.map(instant ->
fsView.getLatestMergedFileSlicesBeforeOrOn(relPartitionPath, instant.getTimestamp()))
Stream<FileSlice> latestFileSlices = latestCompletedInstant
.map(instant -> fsView.getLatestMergedFileSlicesBeforeOrOn(relPartitionPath, instant.getTimestamp()))
.orElse(Stream.empty());
// subgroup splits again by file id & match with log files.
Map<String, List<FileSplit>> groupedInputSplits = partitionsToParquetSplits
.get(partitionPath).stream()
Map<String, List<FileSplit>> groupedInputSplits = partitionsToParquetSplits.get(partitionPath).stream()
.collect(Collectors.groupingBy(split -> FSUtils.getFileId(split.getPath().getName())));
latestFileSlices.forEach(fileSlice -> {
List<FileSplit> dataFileSplits = groupedInputSplits.get(fileSlice.getFileId());
@@ -135,21 +131,18 @@ public class HoodieParquetRealtimeInputFormat extends HoodieParquetInputFormat i
.map(logFile -> logFile.getPath().toString()).collect(Collectors.toList());
// Get the maxCommit from the last delta or compaction or commit - when
// bootstrapped from COW table
String maxCommitTime = metaClient.getActiveTimeline().getTimelineOfActions(
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION,
HoodieTimeline.DELTA_COMMIT_ACTION)).filterCompletedInstants().lastInstant()
.get().getTimestamp();
rtSplits.add(
new HoodieRealtimeFileSplit(split, metaClient.getBasePath(), logFilePaths,
maxCommitTime));
String maxCommitTime = metaClient
.getActiveTimeline().getTimelineOfActions(Sets.newHashSet(HoodieTimeline.COMMIT_ACTION,
HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION))
.filterCompletedInstants().lastInstant().get().getTimestamp();
rtSplits.add(new HoodieRealtimeFileSplit(split, metaClient.getBasePath(), logFilePaths, maxCommitTime));
} catch (IOException e) {
throw new HoodieIOException("Error creating hoodie real time split ", e);
}
});
});
} catch (Exception e) {
throw new HoodieException("Error obtaining data file/log file grouping: " + partitionPath,
e);
throw new HoodieException("Error obtaining data file/log file grouping: " + partitionPath, e);
}
});
LOG.info("Returning a total splits of " + rtSplits.size());
@@ -167,8 +160,7 @@ public class HoodieParquetRealtimeInputFormat extends HoodieParquetInputFormat i
/**
* Add a field to the existing fields projected
*/
private static Configuration addProjectionField(Configuration conf, String fieldName,
int fieldIndex) {
private static Configuration addProjectionField(Configuration conf, String fieldName, int fieldIndex) {
String readColNames = conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, "");
String readColIds = conf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, "");
@@ -186,8 +178,7 @@ public class HoodieParquetRealtimeInputFormat extends HoodieParquetInputFormat i
conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, readColNamesPrefix + fieldName);
conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, readColIdsPrefix + fieldIndex);
if (LOG.isDebugEnabled()) {
LOG.debug(String.format(
"Adding extra column " + fieldName + ", to enable log merging cols (%s) ids (%s) ",
LOG.debug(String.format("Adding extra column " + fieldName + ", to enable log merging cols (%s) ids (%s) ",
conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR),
conf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR)));
}
@@ -197,22 +188,21 @@ public class HoodieParquetRealtimeInputFormat extends HoodieParquetInputFormat i
private static synchronized Configuration addRequiredProjectionFields(Configuration configuration) {
// Need this to do merge records in HoodieRealtimeRecordReader
configuration = addProjectionField(configuration, HoodieRecord.RECORD_KEY_METADATA_FIELD,
HOODIE_RECORD_KEY_COL_POS);
configuration = addProjectionField(configuration, HoodieRecord.COMMIT_TIME_METADATA_FIELD,
HOODIE_COMMIT_TIME_COL_POS);
configuration = addProjectionField(configuration, HoodieRecord.PARTITION_PATH_METADATA_FIELD,
HOODIE_PARTITION_PATH_COL_POS);
configuration =
addProjectionField(configuration, HoodieRecord.RECORD_KEY_METADATA_FIELD, HOODIE_RECORD_KEY_COL_POS);
configuration =
addProjectionField(configuration, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HOODIE_COMMIT_TIME_COL_POS);
configuration =
addProjectionField(configuration, HoodieRecord.PARTITION_PATH_METADATA_FIELD, HOODIE_PARTITION_PATH_COL_POS);
return configuration;
}
@Override
public RecordReader<NullWritable, ArrayWritable> getRecordReader(final InputSplit split,
final JobConf job, final Reporter reporter) throws IOException {
public RecordReader<NullWritable, ArrayWritable> getRecordReader(final InputSplit split, final JobConf job,
final Reporter reporter) throws IOException {
LOG.info("Before adding Hoodie columns, Projections :" + job
.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR) + ", Ids :"
+ job.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR));
LOG.info("Before adding Hoodie columns, Projections :" + job.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR)
+ ", Ids :" + job.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR));
// Hive (across all versions) fails for queries like select count(`_hoodie_commit_time`) from table;
// In this case, the projection fields gets removed. Looking at HiveInputFormat implementation, in some cases
@@ -222,13 +212,11 @@ public class HoodieParquetRealtimeInputFormat extends HoodieParquetInputFormat i
// TO fix this, hoodie columns are appended late at the time record-reader gets built instead of construction time.
this.conf = addRequiredProjectionFields(job);
LOG.info("Creating record reader with readCols :" + job
.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR) + ", Ids :"
+ job.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR));
LOG.info("Creating record reader with readCols :" + job.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR)
+ ", Ids :" + job.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR));
// sanity check
Preconditions.checkArgument(split instanceof HoodieRealtimeFileSplit,
"HoodieRealtimeRecordReader can only work on HoodieRealtimeFileSplit and not with "
+ split);
"HoodieRealtimeRecordReader can only work on HoodieRealtimeFileSplit and not with " + split);
return new HoodieRealtimeRecordReader((HoodieRealtimeFileSplit) split, job,
super.getRecordReader(split, job, reporter));

View File

@@ -41,10 +41,9 @@ public class HoodieRealtimeFileSplit extends FileSplit {
super();
}
public HoodieRealtimeFileSplit(FileSplit baseSplit, String basePath, List<String> deltaLogFiles,
String maxCommitTime) throws IOException {
super(baseSplit.getPath(), baseSplit.getStart(), baseSplit.getLength(),
baseSplit.getLocations());
public HoodieRealtimeFileSplit(FileSplit baseSplit, String basePath, List<String> deltaLogFiles, String maxCommitTime)
throws IOException {
super(baseSplit.getPath(), baseSplit.getStart(), baseSplit.getLength(), baseSplit.getLocations());
this.deltaFilePaths = deltaLogFiles;
this.maxCommitTime = maxCommitTime;
this.basePath = basePath;
@@ -100,11 +99,7 @@ public class HoodieRealtimeFileSplit extends FileSplit {
@Override
public String toString() {
return "HoodieRealtimeFileSplit{"
+ "DataPath=" + getPath()
+ ", deltaFilePaths=" + deltaFilePaths
+ ", maxCommitTime='" + maxCommitTime + '\''
+ ", basePath='" + basePath + '\''
+ '}';
return "HoodieRealtimeFileSplit{" + "DataPath=" + getPath() + ", deltaFilePaths=" + deltaFilePaths
+ ", maxCommitTime='" + maxCommitTime + '\'' + ", basePath='" + basePath + '\'' + '}';
}
}

View File

@@ -28,8 +28,8 @@ import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* Realtime Record Reader which can do compacted (merge-on-read) record reading or
* unmerged reading (parquet and log files read in parallel) based on job configuration.
* Realtime Record Reader which can do compacted (merge-on-read) record reading or unmerged reading (parquet and log
* files read in parallel) based on job configuration.
*/
public class HoodieRealtimeRecordReader implements RecordReader<NullWritable, ArrayWritable> {
@@ -52,8 +52,8 @@ public class HoodieRealtimeRecordReader implements RecordReader<NullWritable, Ar
/**
* Construct record reader based on job configuration
*
* @param split File Split
* @param jobConf Job Configuration
* @param split File Split
* @param jobConf Job Configuration
* @param realReader Parquet Record Reader
* @return Realtime Reader
*/

View File

@@ -35,8 +35,8 @@ import org.apache.hudi.common.util.Option;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader implements
RecordReader<NullWritable, ArrayWritable> {
class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader
implements RecordReader<NullWritable, ArrayWritable> {
private static final Logger LOG = LogManager.getLogger(AbstractRealtimeRecordReader.class);
@@ -51,19 +51,18 @@ class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader impleme
}
/**
* Goes through the log files and populates a map with latest version of each key logged, since
* the base split was written.
* Goes through the log files and populates a map with latest version of each key logged, since the base split was
* written.
*/
private HoodieMergedLogRecordScanner getMergedLogRecordScanner() throws IOException {
// NOTE: HoodieCompactedLogRecordScanner will not return records for an in-flight commit
// but can return records for completed commits > the commit we are trying to read (if using
// readCommit() API)
return new HoodieMergedLogRecordScanner(
FSUtils.getFs(split.getPath().toString(), jobConf), split.getBasePath(),
return new HoodieMergedLogRecordScanner(FSUtils.getFs(split.getPath().toString(), jobConf), split.getBasePath(),
split.getDeltaFilePaths(), usesCustomPayload ? getWriterSchema() : getReaderSchema(), split.getMaxCommitTime(),
getMaxCompactionMemoryInBytes(),
Boolean.valueOf(jobConf.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP,
DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)),
Boolean
.valueOf(jobConf.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)),
false, jobConf.getInt(MAX_DFS_STREAM_BUFFER_SIZE_PROP, DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE),
jobConf.get(SPILLABLE_MAP_BASE_PATH_PROP, DEFAULT_SPILLABLE_MAP_BASE_PATH));
}
@@ -80,8 +79,7 @@ class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader impleme
// TODO(VC): Right now, we assume all records in log, have a matching base record. (which
// would be true until we have a way to index logs too)
// return from delta records map if we have some match.
String key = arrayWritable.get()[HoodieParquetRealtimeInputFormat.HOODIE_RECORD_KEY_COL_POS]
.toString();
String key = arrayWritable.get()[HoodieParquetRealtimeInputFormat.HOODIE_RECORD_KEY_COL_POS].toString();
if (deltaRecordMap.containsKey(key)) {
// TODO(NA): Invoke preCombine here by converting arrayWritable to Avro. This is required since the
// deltaRecord may not be a full record and needs values of columns from the parquet
@@ -106,8 +104,8 @@ class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader impleme
ArrayWritable aWritable = (ArrayWritable) avroToArrayWritable(recordToReturn, getWriterSchema());
Writable[] replaceValue = aWritable.get();
if (LOG.isDebugEnabled()) {
LOG.debug(String.format("key %s, base values: %s, log values: %s", key,
arrayWritableToString(arrayWritable), arrayWritableToString(aWritable)));
LOG.debug(String.format("key %s, base values: %s, log values: %s", key, arrayWritableToString(arrayWritable),
arrayWritableToString(aWritable)));
}
Writable[] originalValue = arrayWritable.get();
try {

View File

@@ -38,8 +38,8 @@ import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer;
import org.apache.hudi.hadoop.RecordReaderValueIterator;
import org.apache.hudi.hadoop.SafeParquetRecordReaderWrapper;
class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader implements
RecordReader<NullWritable, ArrayWritable> {
class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader
implements RecordReader<NullWritable, ArrayWritable> {
// Log Record unmerged scanner
private final HoodieUnMergedLogRecordScanner logRecordScanner;
@@ -60,8 +60,8 @@ class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader implemen
* Construct a Unmerged record reader that parallely consumes both parquet and log records and buffers for upstream
* clients to consume
*
* @param split File split
* @param job Job Configuration
* @param split File split
* @param job Job Configuration
* @param realReader Parquet Reader
*/
public RealtimeUnmergedRecordReader(HoodieRealtimeFileSplit split, JobConf job,
@@ -74,12 +74,11 @@ class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader implemen
Option.empty(), x -> x, new DefaultSizeEstimator<>());
// Consumer of this record reader
this.iterator = this.executor.getQueue().iterator();
this.logRecordScanner = new HoodieUnMergedLogRecordScanner(
FSUtils.getFs(split.getPath().toString(), jobConf), split.getBasePath(),
split.getDeltaFilePaths(), getReaderSchema(), split.getMaxCommitTime(), Boolean.valueOf(jobConf
.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)),
false, jobConf.getInt(MAX_DFS_STREAM_BUFFER_SIZE_PROP, DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE),
record -> {
this.logRecordScanner = new HoodieUnMergedLogRecordScanner(FSUtils.getFs(split.getPath().toString(), jobConf),
split.getBasePath(), split.getDeltaFilePaths(), getReaderSchema(), split.getMaxCommitTime(),
Boolean
.valueOf(jobConf.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)),
false, jobConf.getInt(MAX_DFS_STREAM_BUFFER_SIZE_PROP, DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE), record -> {
// convert Hoodie log record to Hadoop AvroWritable and buffer
GenericRecord rec = (GenericRecord) record.getData().getInsertValue(getReaderSchema()).get();
ArrayWritable aWritable = (ArrayWritable) avroToArrayWritable(rec, getWriterSchema());
@@ -125,7 +124,7 @@ class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader implemen
@Override
public long getPos() throws IOException {
//TODO: vb - No logical way to represent parallel stream pos in a single long.
// TODO: vb - No logical way to represent parallel stream pos in a single long.
// Should we just return invalid (-1). Where is it used ?
return 0;
}

View File

@@ -84,18 +84,14 @@ public class HoodieInputFormatTest {
// Before the commit
files = inputFormat.listStatus(jobConf);
assertEquals(10, files.length);
ensureFilesInCommit(
"Commit 200 has not been committed. We should not see files from this commit", files, "200",
0);
ensureFilesInCommit("Commit 200 has not been committed. We should not see files from this commit", files, "200", 0);
InputFormatTestUtil.commit(basePath, "200");
files = inputFormat.listStatus(jobConf);
assertEquals(10, files.length);
ensureFilesInCommit(
"5 files have been updated to commit 200. We should see 5 files from commit 200 and 5 "
+ "files from 100 commit", files, "200", 5);
ensureFilesInCommit(
"5 files have been updated to commit 200. We should see 5 files from commit 100 and 5 "
+ "files from 200 commit", files, "100", 5);
ensureFilesInCommit("5 files have been updated to commit 200. We should see 5 files from commit 200 and 5 "
+ "files from 100 commit", files, "200", 5);
ensureFilesInCommit("5 files have been updated to commit 200. We should see 5 files from commit 100 and 5 "
+ "files from 200 commit", files, "100", 5);
}
@Test
@@ -110,9 +106,8 @@ public class HoodieInputFormatTest {
InputFormatTestUtil.setupIncremental(jobConf, "100", 1);
FileStatus[] files = inputFormat.listStatus(jobConf);
assertEquals(
"We should exclude commit 100 when returning incremental pull with start commit time as "
+ "100", 0, files.length);
assertEquals("We should exclude commit 100 when returning incremental pull with start commit time as " + "100", 0,
files.length);
}
@Test
@@ -140,43 +135,31 @@ public class HoodieInputFormatTest {
InputFormatTestUtil.setupIncremental(jobConf, "100", 1);
FileStatus[] files = inputFormat.listStatus(jobConf);
assertEquals("Pulling 1 commit from 100, should get us the 5 files committed at 200", 5,
files.length);
ensureFilesInCommit("Pulling 1 commit from 100, should get us the 5 files committed at 200",
files, "200", 5);
assertEquals("Pulling 1 commit from 100, should get us the 5 files committed at 200", 5, files.length);
ensureFilesInCommit("Pulling 1 commit from 100, should get us the 5 files committed at 200", files, "200", 5);
InputFormatTestUtil.setupIncremental(jobConf, "100", 3);
files = inputFormat.listStatus(jobConf);
assertEquals(
"Pulling 3 commits from 100, should get us the 3 files from 400 commit, 1 file from 300 "
+ "commit and 1 file from 200 commit", 5, files.length);
ensureFilesInCommit("Pulling 3 commits from 100, should get us the 3 files from 400 commit",
files, "400", 3);
ensureFilesInCommit("Pulling 3 commits from 100, should get us the 1 files from 300 commit",
files, "300", 1);
ensureFilesInCommit("Pulling 3 commits from 100, should get us the 1 files from 200 commit",
files, "200", 1);
assertEquals("Pulling 3 commits from 100, should get us the 3 files from 400 commit, 1 file from 300 "
+ "commit and 1 file from 200 commit", 5, files.length);
ensureFilesInCommit("Pulling 3 commits from 100, should get us the 3 files from 400 commit", files, "400", 3);
ensureFilesInCommit("Pulling 3 commits from 100, should get us the 1 files from 300 commit", files, "300", 1);
ensureFilesInCommit("Pulling 3 commits from 100, should get us the 1 files from 200 commit", files, "200", 1);
InputFormatTestUtil.setupIncremental(jobConf, "100", HoodieHiveUtil.MAX_COMMIT_ALL);
files = inputFormat.listStatus(jobConf);
assertEquals(
"Pulling all commits from 100, should get us the 1 file from each of 200,300,400,500,400 "
+ "commits", 5, files.length);
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 600 commit",
files, "600", 1);
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 500 commit",
files, "500", 1);
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 400 commit",
files, "400", 1);
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 300 commit",
files, "300", 1);
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 200 commit",
files, "200", 1);
assertEquals("Pulling all commits from 100, should get us the 1 file from each of 200,300,400,500,400 " + "commits",
5, files.length);
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 600 commit", files, "600", 1);
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 500 commit", files, "500", 1);
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 400 commit", files, "400", 1);
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 300 commit", files, "300", 1);
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 200 commit", files, "200", 1);
}
//TODO enable this after enabling predicate pushdown
// TODO enable this after enabling predicate pushdown
public void testPredicatePushDown() throws IOException {
// initial commit
Schema schema = InputFormatTestUtil.readSchema("/sample1.avsc");
@@ -186,8 +169,7 @@ public class HoodieInputFormatTest {
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
// check whether we have 10 records at this point
ensureRecordsInCommit("We need to have 10 records at this point for commit " + commit1, commit1,
10, 10);
ensureRecordsInCommit("We need to have 10 records at this point for commit " + commit1, commit1, 10, 10);
// update 2 records in the original parquet file and save it as commit 200
String commit2 = "20160629193623";
@@ -196,27 +178,23 @@ public class HoodieInputFormatTest {
InputFormatTestUtil.setupIncremental(jobConf, commit1, 1);
// check whether we have 2 records at this point
ensureRecordsInCommit(
"We need to have 2 records that was modified at commit " + commit2 + " and no more",
commit2, 2, 2);
ensureRecordsInCommit("We need to have 2 records that was modified at commit " + commit2 + " and no more", commit2,
2, 2);
// Make sure we have the 10 records if we roll back the stattime
InputFormatTestUtil.setupIncremental(jobConf, "0", 2);
ensureRecordsInCommit(
"We need to have 8 records that was modified at commit " + commit1 + " and no more",
commit1, 8, 10);
ensureRecordsInCommit(
"We need to have 2 records that was modified at commit " + commit2 + " and no more",
commit2, 2, 10);
ensureRecordsInCommit("We need to have 8 records that was modified at commit " + commit1 + " and no more", commit1,
8, 10);
ensureRecordsInCommit("We need to have 2 records that was modified at commit " + commit2 + " and no more", commit2,
2, 10);
}
private void ensureRecordsInCommit(String msg, String commit, int expectedNumberOfRecordsInCommit,
int totalExpected) throws IOException {
private void ensureRecordsInCommit(String msg, String commit, int expectedNumberOfRecordsInCommit, int totalExpected)
throws IOException {
int actualCount = 0;
int totalCount = 0;
InputSplit[] splits = inputFormat.getSplits(jobConf, 1);
for (InputSplit split : splits) {
RecordReader<NullWritable, ArrayWritable> recordReader = inputFormat
.getRecordReader(split, jobConf, null);
RecordReader<NullWritable, ArrayWritable> recordReader = inputFormat.getRecordReader(split, jobConf, null);
NullWritable key = recordReader.createKey();
ArrayWritable writable = recordReader.createValue();
@@ -234,8 +212,7 @@ public class HoodieInputFormatTest {
assertEquals(msg, totalExpected, totalCount);
}
public static void ensureFilesInCommit(String msg, FileStatus[] files, String commit,
int expected) {
public static void ensureFilesInCommit(String msg, FileStatus[] files, String commit, int expected) {
int count = 0;
for (FileStatus file : files) {
String commitTs = FSUtils.getCommitTime(file.getPath().getName());

View File

@@ -44,21 +44,20 @@ public class InputFormatTestUtil {
private static String TEST_WRITE_TOKEN = "1-0-1";
public static File prepareDataset(TemporaryFolder basePath, int numberOfFiles,
String commitNumber) throws IOException {
public static File prepareDataset(TemporaryFolder basePath, int numberOfFiles, String commitNumber)
throws IOException {
basePath.create();
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.getRoot().toString());
File partitionPath = basePath.newFolder("2016", "05", "01");
for (int i = 0; i < numberOfFiles; i++) {
File dataFile = new File(partitionPath,
FSUtils.makeDataFileName(commitNumber, TEST_WRITE_TOKEN, "fileid" + i));
File dataFile = new File(partitionPath, FSUtils.makeDataFileName(commitNumber, TEST_WRITE_TOKEN, "fileid" + i));
dataFile.createNewFile();
}
return partitionPath;
}
public static void simulateUpdates(File directory, final String originalCommit,
int numberOfFilesUpdated, String newCommit, boolean randomize) throws IOException {
public static void simulateUpdates(File directory, final String originalCommit, int numberOfFilesUpdated,
String newCommit, boolean randomize) throws IOException {
List<File> dataFiles = Arrays.asList(directory.listFiles(new FilenameFilter() {
@Override
public boolean accept(File dir, String name) {
@@ -69,8 +68,7 @@ public class InputFormatTestUtil {
if (randomize) {
Collections.shuffle(dataFiles);
}
List<File> toUpdateList = dataFiles
.subList(0, Math.min(numberOfFilesUpdated, dataFiles.size()));
List<File> toUpdateList = dataFiles.subList(0, Math.min(numberOfFilesUpdated, dataFiles.size()));
for (File file : toUpdateList) {
String fileId = FSUtils.getFileId(file.getName());
File dataFile = new File(directory, FSUtils.makeDataFileName(newCommit, TEST_WRITE_TOKEN, fileId));
@@ -88,18 +86,17 @@ public class InputFormatTestUtil {
new File(basePath.getRoot().toString() + "/.hoodie/", commitNumber + ".deltacommit").createNewFile();
}
public static void setupIncremental(JobConf jobConf, String startCommit,
int numberOfCommitsToPull) {
String modePropertyName = String
.format(HoodieHiveUtil.HOODIE_CONSUME_MODE_PATTERN, HoodieTestUtils.RAW_TRIPS_TEST_NAME);
public static void setupIncremental(JobConf jobConf, String startCommit, int numberOfCommitsToPull) {
String modePropertyName =
String.format(HoodieHiveUtil.HOODIE_CONSUME_MODE_PATTERN, HoodieTestUtils.RAW_TRIPS_TEST_NAME);
jobConf.set(modePropertyName, HoodieHiveUtil.INCREMENTAL_SCAN_MODE);
String startCommitTimestampName = String
.format(HoodieHiveUtil.HOODIE_START_COMMIT_PATTERN, HoodieTestUtils.RAW_TRIPS_TEST_NAME);
String startCommitTimestampName =
String.format(HoodieHiveUtil.HOODIE_START_COMMIT_PATTERN, HoodieTestUtils.RAW_TRIPS_TEST_NAME);
jobConf.set(startCommitTimestampName, startCommit);
String maxCommitPulls = String
.format(HoodieHiveUtil.HOODIE_MAX_COMMIT_PATTERN, HoodieTestUtils.RAW_TRIPS_TEST_NAME);
String maxCommitPulls =
String.format(HoodieHiveUtil.HOODIE_MAX_COMMIT_PATTERN, HoodieTestUtils.RAW_TRIPS_TEST_NAME);
jobConf.setInt(maxCommitPulls, numberOfCommitsToPull);
}
@@ -107,8 +104,8 @@ public class InputFormatTestUtil {
return new Schema.Parser().parse(InputFormatTestUtil.class.getResourceAsStream(location));
}
public static File prepareParquetDataset(TemporaryFolder basePath, Schema schema,
int numberOfFiles, int numberOfRecords, String commitNumber) throws IOException {
public static File prepareParquetDataset(TemporaryFolder basePath, Schema schema, int numberOfFiles,
int numberOfRecords, String commitNumber) throws IOException {
basePath.create();
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.getRoot().toString());
File partitionPath = basePath.newFolder("2016", "05", "01");
@@ -117,8 +114,8 @@ public class InputFormatTestUtil {
}
public static File prepareSimpleParquetDataset(TemporaryFolder basePath, Schema schema,
int numberOfFiles, int numberOfRecords, String commitNumber) throws Exception {
public static File prepareSimpleParquetDataset(TemporaryFolder basePath, Schema schema, int numberOfFiles,
int numberOfRecords, String commitNumber) throws Exception {
basePath.create();
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.getRoot().toString());
File partitionPath = basePath.newFolder("2016", "05", "01");
@@ -126,8 +123,8 @@ public class InputFormatTestUtil {
return partitionPath;
}
public static File prepareNonPartitionedParquetDataset(TemporaryFolder baseDir, Schema schema,
int numberOfFiles, int numberOfRecords, String commitNumber) throws IOException {
public static File prepareNonPartitionedParquetDataset(TemporaryFolder baseDir, Schema schema, int numberOfFiles,
int numberOfRecords, String commitNumber) throws IOException {
baseDir.create();
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), baseDir.getRoot().toString());
File basePath = baseDir.getRoot();
@@ -135,17 +132,15 @@ public class InputFormatTestUtil {
return basePath;
}
private static void createData(Schema schema,
File partitionPath, int numberOfFiles, int numberOfRecords, String commitNumber)
throws IOException {
private static void createData(Schema schema, File partitionPath, int numberOfFiles, int numberOfRecords,
String commitNumber) throws IOException {
AvroParquetWriter parquetWriter;
for (int i = 0; i < numberOfFiles; i++) {
String fileId = FSUtils.makeDataFileName(commitNumber, TEST_WRITE_TOKEN, "fileid" + i);
File dataFile = new File(partitionPath, fileId);
parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()), schema);
try {
for (GenericRecord record : generateAvroRecords(schema, numberOfRecords, commitNumber,
fileId)) {
for (GenericRecord record : generateAvroRecords(schema, numberOfRecords, commitNumber, fileId)) {
parquetWriter.write(record);
}
} finally {
@@ -154,9 +149,8 @@ public class InputFormatTestUtil {
}
}
private static void createSimpleData(Schema schema,
File partitionPath, int numberOfFiles, int numberOfRecords, String commitNumber)
throws Exception {
private static void createSimpleData(Schema schema, File partitionPath, int numberOfFiles, int numberOfRecords,
String commitNumber) throws Exception {
AvroParquetWriter parquetWriter;
for (int i = 0; i < numberOfFiles; i++) {
String fileId = FSUtils.makeDataFileName(commitNumber, "1", "fileid" + i);
@@ -179,8 +173,8 @@ public class InputFormatTestUtil {
}
}
private static Iterable<? extends GenericRecord> generateAvroRecords(Schema schema,
int numberOfRecords, String commitTime, String fileId) throws IOException {
private static Iterable<? extends GenericRecord> generateAvroRecords(Schema schema, int numberOfRecords,
String commitTime, String fileId) throws IOException {
List<GenericRecord> records = new ArrayList<>(numberOfRecords);
for (int i = 0; i < numberOfRecords; i++) {
records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, commitTime, fileId));
@@ -198,17 +192,14 @@ public class InputFormatTestUtil {
})[0];
String fileId = FSUtils.getFileId(fileToUpdate.getName());
File dataFile = new File(directory, FSUtils.makeDataFileName(newCommit, TEST_WRITE_TOKEN, fileId));
AvroParquetWriter parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()),
schema);
AvroParquetWriter parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()), schema);
try {
for (GenericRecord record : generateAvroRecords(schema, totalNumberOfRecords, originalCommit,
fileId)) {
for (GenericRecord record : generateAvroRecords(schema, totalNumberOfRecords, originalCommit, fileId)) {
if (numberOfRecordsToUpdate > 0) {
// update this record
record.put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, newCommit);
String oldSeqNo = (String) record.get(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD);
record.put(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD,
oldSeqNo.replace(originalCommit, newCommit));
record.put(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, oldSeqNo.replace(originalCommit, newCommit));
numberOfRecordsToUpdate--;
}
parquetWriter.write(record);

View File

@@ -36,12 +36,12 @@ import org.junit.rules.TemporaryFolder;
/**
*/
public class TestHoodieROTablePathFilter extends HoodieCommonTestHarness {
@Before
public void setUp() throws Exception {
initMetaClient();
}
@Test
public void testHoodiePaths() throws IOException {
// Create a temp folder as the base path
@@ -61,28 +61,26 @@ public class TestHoodieROTablePathFilter extends HoodieCommonTestHarness {
Path partitionPath = new Path("file://" + basePath + File.separator + "2017/01/01");
assertTrue("Directories should be accepted", pathFilter.accept(partitionPath));
assertTrue(pathFilter.accept(new Path(
"file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "001", "f1"))));
assertFalse(pathFilter.accept(new Path(
"file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "001", "f2"))));
assertTrue(pathFilter.accept(new Path(
"file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "001", "f3"))));
assertTrue(pathFilter.accept(new Path(
"file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "002", "f2"))));
assertFalse(pathFilter.accept(new Path(
"file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "003", "f3"))));
assertTrue(
pathFilter.accept(new Path("file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "001", "f1"))));
assertFalse(
pathFilter.accept(new Path("file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "001", "f2"))));
assertTrue(
pathFilter.accept(new Path("file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "001", "f3"))));
assertTrue(
pathFilter.accept(new Path("file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "002", "f2"))));
assertFalse(
pathFilter.accept(new Path("file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "003", "f3"))));
assertFalse(pathFilter.accept(new Path("file:///" + HoodieTestUtils.getCommitFilePath(basePath, "001"))));
assertFalse(pathFilter.accept(new Path("file:///" + HoodieTestUtils.getCommitFilePath(basePath, "002"))));
assertFalse(pathFilter.accept(new Path("file:///"
+ HoodieTestUtils.getInflightCommitFilePath(basePath, "003"))));
assertFalse(pathFilter.accept(new Path("file:///"
+ HoodieTestUtils.getRequestedCompactionFilePath(basePath, "004"))));
assertFalse(pathFilter.accept(new Path("file:///" + basePath + "/"
+ HoodieTableMetaClient.METAFOLDER_NAME + "/")));
assertFalse(pathFilter.accept(new Path("file:///" + HoodieTestUtils.getInflightCommitFilePath(basePath, "003"))));
assertFalse(
pathFilter.accept(new Path("file:///" + HoodieTestUtils.getRequestedCompactionFilePath(basePath, "004"))));
assertFalse(pathFilter.accept(new Path("file:///" + basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/")));
assertFalse(pathFilter.accept(new Path("file:///" + basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME)));
assertFalse(pathFilter.accept(new Path(
"file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "003", "f3"))));
assertFalse(
pathFilter.accept(new Path("file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "003", "f3"))));
}

View File

@@ -33,16 +33,9 @@ public class TestRecordReaderValueIterator {
@Test
public void testValueIterator() {
String[] values = new String[]{
"hoodie",
"efficient",
"new project",
"realtime",
"spark",
"dataset",
};
List<Pair<Integer, String>> entries = IntStream.range(0, values.length)
.boxed().map(idx -> Pair.of(idx, values[idx])).collect(Collectors.toList());
String[] values = new String[] {"hoodie", "efficient", "new project", "realtime", "spark", "dataset",};
List<Pair<Integer, String>> entries =
IntStream.range(0, values.length).boxed().map(idx -> Pair.of(idx, values[idx])).collect(Collectors.toList());
TestRecordReader reader = new TestRecordReader(entries);
RecordReaderValueIterator<IntWritable, Text> itr = new RecordReaderValueIterator<IntWritable, Text>(reader);
for (int i = 0; i < values.length; i++) {

View File

@@ -91,28 +91,22 @@ public class HoodieRealtimeRecordReaderTest {
@Rule
public TemporaryFolder basePath = new TemporaryFolder();
private Writer writeLogFile(File partitionDir, Schema schema, String fileId,
String baseCommit, String newCommit, int numberOfRecords)
throws InterruptedException, IOException {
private Writer writeLogFile(File partitionDir, Schema schema, String fileId, String baseCommit, String newCommit,
int numberOfRecords) throws InterruptedException, IOException {
return writeDataBlockToLogFile(partitionDir, schema, fileId, baseCommit, newCommit, numberOfRecords, 0, 0);
}
private Writer writeRollback(File partitionDir, Schema schema, String fileId,
String baseCommit, String newCommit, String rolledBackInstant, int logVersion)
throws InterruptedException, IOException {
Writer writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(new Path(partitionDir.getPath()))
.withFileId(fileId).overBaseCommit(baseCommit)
.withFs(fs)
.withLogVersion(logVersion)
.withLogWriteToken("1-0-1")
private Writer writeRollback(File partitionDir, Schema schema, String fileId, String baseCommit, String newCommit,
String rolledBackInstant, int logVersion) throws InterruptedException, IOException {
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(new Path(partitionDir.getPath())).withFileId(fileId)
.overBaseCommit(baseCommit).withFs(fs).withLogVersion(logVersion).withLogWriteToken("1-0-1")
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
// generate metadata
Map<HeaderMetadataType, String> header = Maps.newHashMap();
header.put(HeaderMetadataType.INSTANT_TIME, newCommit);
header.put(HeaderMetadataType.TARGET_INSTANT_TIME, rolledBackInstant);
header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK
.ordinal()));
header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE,
String.valueOf(HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
// if update belongs to an existing log file
writer = writer.appendBlock(new HoodieCommandBlock(header));
return writer;
@@ -121,12 +115,9 @@ public class HoodieRealtimeRecordReaderTest {
private HoodieLogFormat.Writer writeDataBlockToLogFile(File partitionDir, Schema schema, String fileId,
String baseCommit, String newCommit, int numberOfRecords, int offset, int logVersion)
throws InterruptedException, IOException {
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(new Path(partitionDir.getPath()))
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId)
.withLogVersion(logVersion)
.withLogWriteToken("1-0-1")
.overBaseCommit(baseCommit).withFs(fs).build();
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(new Path(partitionDir.getPath()))
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId).withLogVersion(logVersion)
.withLogWriteToken("1-0-1").overBaseCommit(baseCommit).withFs(fs).build();
List<IndexedRecord> records = new ArrayList<>();
for (int i = offset; i < offset + numberOfRecords; i++) {
records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, newCommit, "fileid0"));
@@ -141,19 +132,17 @@ public class HoodieRealtimeRecordReaderTest {
}
private HoodieLogFormat.Writer writeRollbackBlockToLogFile(File partitionDir, Schema schema, String fileId,
String baseCommit, String newCommit, String oldCommit, int logVersion)
throws InterruptedException, IOException {
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(new Path(partitionDir.getPath()))
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId)
.overBaseCommit(baseCommit).withLogVersion(logVersion).withFs(fs).build();
String baseCommit, String newCommit, String oldCommit, int logVersion) throws InterruptedException, IOException {
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(new Path(partitionDir.getPath()))
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId).overBaseCommit(baseCommit)
.withLogVersion(logVersion).withFs(fs).build();
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, newCommit);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, oldCommit);
header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK
.ordinal()));
header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE,
String.valueOf(HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
HoodieCommandBlock rollbackBlock = new HoodieCommandBlock(header);
writer = writer.appendBlock(rollbackBlock);
return writer;
@@ -172,12 +161,10 @@ public class HoodieRealtimeRecordReaderTest {
public void testReader(boolean partitioned) throws Exception {
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
HoodieTestUtils.init(hadoopConf, basePath.getRoot().getAbsolutePath(),
HoodieTableType.MERGE_ON_READ);
HoodieTestUtils.init(hadoopConf, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
String baseInstant = "100";
File partitionDir =
partitioned ? InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, 100, baseInstant)
: InputFormatTestUtil.prepareNonPartitionedParquetDataset(basePath, schema, 1, 100, baseInstant);
File partitionDir = partitioned ? InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, 100, baseInstant)
: InputFormatTestUtil.prepareNonPartitionedParquetDataset(basePath, schema, 1, 100, baseInstant);
InputFormatTestUtil.commit(basePath, baseInstant);
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
@@ -187,9 +174,9 @@ public class HoodieRealtimeRecordReaderTest {
logVersionsWithAction.add(Pair.of(HoodieTimeline.DELTA_COMMIT_ACTION, 2));
// TODO: HUDI-154 Once Hive 2.x PR (PR-674) is merged, enable this change
// logVersionsWithAction.add(Pair.of(HoodieTimeline.ROLLBACK_ACTION, 3));
FileSlice fileSlice = new FileSlice(partitioned ? FSUtils.getRelativePartitionPath(new Path(
basePath.getRoot().getAbsolutePath()), new Path(partitionDir.getAbsolutePath())) : "default",
baseInstant, "fileid0");
FileSlice fileSlice =
new FileSlice(partitioned ? FSUtils.getRelativePartitionPath(new Path(basePath.getRoot().getAbsolutePath()),
new Path(partitionDir.getAbsolutePath())) : "default", baseInstant, "fileid0");
logVersionsWithAction.stream().forEach(logVersionWithAction -> {
try {
// update files or generate new log file
@@ -197,55 +184,53 @@ public class HoodieRealtimeRecordReaderTest {
String action = logVersionWithAction.getKey();
int baseInstantTs = Integer.parseInt(baseInstant);
String instantTime = String.valueOf(baseInstantTs + logVersion);
String latestInstant = action.equals(HoodieTimeline.ROLLBACK_ACTION)
? String.valueOf(baseInstantTs + logVersion - 2) : instantTime;
String latestInstant =
action.equals(HoodieTimeline.ROLLBACK_ACTION) ? String.valueOf(baseInstantTs + logVersion - 2)
: instantTime;
HoodieLogFormat.Writer writer = null;
if (action.equals(HoodieTimeline.ROLLBACK_ACTION)) {
writer = writeRollback(partitionDir, schema, "fileid0", baseInstant,
instantTime, String.valueOf(baseInstantTs + logVersion - 1), logVersion);
writer = writeRollback(partitionDir, schema, "fileid0", baseInstant, instantTime,
String.valueOf(baseInstantTs + logVersion - 1), logVersion);
} else {
writer = writeDataBlockToLogFile(partitionDir, schema, "fileid0", baseInstant,
instantTime, 100, 0, logVersion);
writer =
writeDataBlockToLogFile(partitionDir, schema, "fileid0", baseInstant, instantTime, 100, 0, logVersion);
}
long size = writer.getCurrentSize();
writer.close();
assertTrue("block - size should be > 0", size > 0);
//create a split with baseFile (parquet file written earlier) and new log file(s)
// create a split with baseFile (parquet file written earlier) and new log file(s)
fileSlice.addLogFile(writer.getLogFile());
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + baseInstant + ".parquet"), 0, 1,
jobConf), basePath.getRoot().getPath(),
fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()).map(h -> h.getPath().toString())
.collect(Collectors.toList()), instantTime);
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + baseInstant + ".parquet"), 0, 1, jobConf),
basePath.getRoot().getPath(), fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator())
.map(h -> h.getPath().toString()).collect(Collectors.toList()),
instantTime);
//create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader =
new MapredParquetInputFormat().getRecordReader(
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null),
jobConf, null);
// create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader = new MapredParquetInputFormat().getRecordReader(
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null), jobConf, null);
JobConf jobConf = new JobConf();
List<Schema.Field> fields = schema.getFields();
String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(","));
String postions = fields.stream().map(f -> String.valueOf(f.pos()))
.collect(Collectors.joining(","));
String postions = fields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(","));
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
if (partitioned) {
jobConf.set("partition_columns", "datestr");
}
//validate record reader compaction
// validate record reader compaction
HoodieRealtimeRecordReader recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader);
//use reader to read base Parquet File and log file, merge in flight and return latest commit
//here all 100 records should be updated, see above
// use reader to read base Parquet File and log file, merge in flight and return latest commit
// here all 100 records should be updated, see above
NullWritable key = recordReader.createKey();
ArrayWritable value = recordReader.createValue();
while (recordReader.next(key, value)) {
Writable[] values = value.get();
//check if the record written is with latest commit, here "101"
// check if the record written is with latest commit, here "101"
Assert.assertEquals(latestInstant, values[0].toString());
key = recordReader.createKey();
value = recordReader.createValue();
@@ -263,53 +248,48 @@ public class HoodieRealtimeRecordReaderTest {
public void testUnMergedReader() throws Exception {
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
HoodieTestUtils.init(hadoopConf, basePath.getRoot().getAbsolutePath(),
HoodieTableType.MERGE_ON_READ);
HoodieTestUtils.init(hadoopConf, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
String commitTime = "100";
final int numRecords = 1000;
final int firstBatchLastRecordKey = numRecords - 1;
final int secondBatchLastRecordKey = 2 * numRecords - 1;
File partitionDir = InputFormatTestUtil
.prepareParquetDataset(basePath, schema, 1, numRecords, commitTime);
File partitionDir = InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, numRecords, commitTime);
InputFormatTestUtil.commit(basePath, commitTime);
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
// insert new records to log file
String newCommitTime = "101";
HoodieLogFormat.Writer writer = writeDataBlockToLogFile(partitionDir, schema, "fileid0", commitTime,
newCommitTime, numRecords, numRecords, 0);
HoodieLogFormat.Writer writer =
writeDataBlockToLogFile(partitionDir, schema, "fileid0", commitTime, newCommitTime, numRecords, numRecords, 0);
long size = writer.getCurrentSize();
writer.close();
assertTrue("block - size should be > 0", size > 0);
//create a split with baseFile (parquet file written earlier) and new log file(s)
// create a split with baseFile (parquet file written earlier) and new log file(s)
String logFilePath = writer.getLogFile().getPath().toString();
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + commitTime + ".parquet"), 0, 1,
jobConf), basePath.getRoot().getPath(), Arrays.asList(logFilePath), newCommitTime);
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + commitTime + ".parquet"), 0, 1, jobConf),
basePath.getRoot().getPath(), Arrays.asList(logFilePath), newCommitTime);
//create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader =
new MapredParquetInputFormat().getRecordReader(
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null),
jobConf, null);
// create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader = new MapredParquetInputFormat().getRecordReader(
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null), jobConf, null);
JobConf jobConf = new JobConf();
List<Schema.Field> fields = schema.getFields();
String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(","));
String postions = fields.stream().map(f -> String.valueOf(f.pos()))
.collect(Collectors.joining(","));
String postions = fields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(","));
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
jobConf.set("partition_columns", "datestr");
// Enable merge skipping.
jobConf.set("hoodie.realtime.merge.skip", "true");
//validate unmerged record reader
// validate unmerged record reader
RealtimeUnmergedRecordReader recordReader = new RealtimeUnmergedRecordReader(split, jobConf, reader);
//use reader to read base Parquet File and log file
//here all records should be present. Also ensure log records are in order.
// use reader to read base Parquet File and log file
// here all records should be present. Also ensure log records are in order.
NullWritable key = recordReader.createKey();
ArrayWritable value = recordReader.createValue();
int numRecordsAtCommit1 = 0;
@@ -347,43 +327,38 @@ public class HoodieRealtimeRecordReaderTest {
public void testReaderWithNestedAndComplexSchema() throws Exception {
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getComplexEvolvedSchema());
HoodieTestUtils.init(hadoopConf, basePath.getRoot().getAbsolutePath(),
HoodieTableType.MERGE_ON_READ);
HoodieTestUtils.init(hadoopConf, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
String commitTime = "100";
int numberOfRecords = 100;
int numberOfLogRecords = numberOfRecords / 2;
File partitionDir = InputFormatTestUtil
.prepareParquetDataset(basePath, schema, 1, numberOfRecords, commitTime);
File partitionDir = InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, numberOfRecords, commitTime);
InputFormatTestUtil.commit(basePath, commitTime);
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
// update files or generate new log file
String newCommitTime = "101";
HoodieLogFormat.Writer writer = writeLogFile(partitionDir, schema, "fileid0", commitTime,
newCommitTime, numberOfLogRecords);
HoodieLogFormat.Writer writer =
writeLogFile(partitionDir, schema, "fileid0", commitTime, newCommitTime, numberOfLogRecords);
long size = writer.getCurrentSize();
writer.close();
assertTrue("block - size should be > 0", size > 0);
InputFormatTestUtil.deltaCommit(basePath, newCommitTime);
//create a split with baseFile (parquet file written earlier) and new log file(s)
// create a split with baseFile (parquet file written earlier) and new log file(s)
String logFilePath = writer.getLogFile().getPath().toString();
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + commitTime + ".parquet"), 0, 1,
jobConf), basePath.getRoot().getPath(), Arrays.asList(logFilePath), newCommitTime);
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + commitTime + ".parquet"), 0, 1, jobConf),
basePath.getRoot().getPath(), Arrays.asList(logFilePath), newCommitTime);
//create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader =
new MapredParquetInputFormat().getRecordReader(
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null),
jobConf, null);
// create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader = new MapredParquetInputFormat().getRecordReader(
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null), jobConf, null);
JobConf jobConf = new JobConf();
List<Schema.Field> fields = schema.getFields();
String names = fields.stream().map(f -> f.name()).collect(Collectors.joining(","));
String positions = fields.stream().map(f -> String.valueOf(f.pos()))
.collect(Collectors.joining(","));
String positions = fields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(","));
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, positions);
jobConf.set("partition_columns", "datestr");
@@ -401,7 +376,7 @@ public class HoodieRealtimeRecordReaderTest {
++numRecordsRead;
Writable[] values = value.get();
String recordCommitTime;
//check if the record written is with latest commit, here "101"
// check if the record written is with latest commit, here "101"
if (numRecordsRead > numberOfLogRecords) {
recordCommitTime = commitTime;
} else {
@@ -414,12 +389,10 @@ public class HoodieRealtimeRecordReaderTest {
value = recordReader.createValue();
// Assert type STRING
Assert.assertEquals("test value for field: field1", values[5].toString(),
"field" + currentRecordNo);
Assert.assertEquals("test value for field: field1", values[5].toString(), "field" + currentRecordNo);
Assert.assertEquals("test value for field: field2", values[6].toString(),
"field" + currentRecordNo + recordCommitTimeSuffix);
Assert.assertEquals("test value for field: name", values[7].toString(),
"name" + currentRecordNo);
Assert.assertEquals("test value for field: name", values[7].toString(), "name" + currentRecordNo);
// Assert type INT
IntWritable intWritable = (IntWritable) values[8];
@@ -459,20 +432,17 @@ public class HoodieRealtimeRecordReaderTest {
Assert.assertEquals("test value for field: tags[\"mapItem2\"].item1",
((ArrayWritable) mapItemValue2value).get()[0].toString(), "item2" + currentRecordNo);
Assert.assertEquals("test value for field: tags[\"mapItem1\"].item2",
((ArrayWritable) mapItemValue1value).get()[1].toString(),
"item" + currentRecordNo + recordCommitTimeSuffix);
((ArrayWritable) mapItemValue1value).get()[1].toString(), "item" + currentRecordNo + recordCommitTimeSuffix);
Assert.assertEquals("test value for field: tags[\"mapItem2\"].item2",
((ArrayWritable) mapItemValue2value).get()[1].toString(),
"item2" + currentRecordNo + recordCommitTimeSuffix);
((ArrayWritable) mapItemValue2value).get()[1].toString(), "item2" + currentRecordNo + recordCommitTimeSuffix);
// Assert type RECORD
ArrayWritable recordItem = (ArrayWritable) values[13];
Writable[] nestedRecord = recordItem.get();
Assert.assertEquals("test value for field: testNestedRecord.isAdmin",
((BooleanWritable) nestedRecord[0]).get(), false);
Assert
.assertEquals("test value for field: testNestedRecord.userId", nestedRecord[1].toString(),
"UserId" + currentRecordNo + recordCommitTimeSuffix);
Assert.assertEquals("test value for field: testNestedRecord.isAdmin", ((BooleanWritable) nestedRecord[0]).get(),
false);
Assert.assertEquals("test value for field: testNestedRecord.userId", nestedRecord[1].toString(),
"UserId" + currentRecordNo + recordCommitTimeSuffix);
// Assert type ARRAY
ArrayWritable arrayValue = (ArrayWritable) values[14];
@@ -489,13 +459,12 @@ public class HoodieRealtimeRecordReaderTest {
// initial commit
List<String> logFilePaths = new ArrayList<>();
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
HoodieTestUtils.init(hadoopConf, basePath.getRoot().getAbsolutePath(),
HoodieTableType.MERGE_ON_READ);
HoodieTestUtils.init(hadoopConf, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
String commitTime = "100";
int numberOfRecords = 100;
int numberOfLogRecords = numberOfRecords / 2;
File partitionDir = InputFormatTestUtil
.prepareSimpleParquetDataset(basePath, schema, 1, numberOfRecords, commitTime);
File partitionDir =
InputFormatTestUtil.prepareSimpleParquetDataset(basePath, schema, 1, numberOfRecords, commitTime);
InputFormatTestUtil.commit(basePath, commitTime);
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
@@ -504,8 +473,8 @@ public class HoodieRealtimeRecordReaderTest {
// update files and generate new log file but don't commit
schema = SchemaTestUtil.getComplexEvolvedSchema();
String newCommitTime = "101";
HoodieLogFormat.Writer writer = writeDataBlockToLogFile(partitionDir, schema, "fileid0", commitTime,
newCommitTime, numberOfLogRecords, 0, 1);
HoodieLogFormat.Writer writer =
writeDataBlockToLogFile(partitionDir, schema, "fileid0", commitTime, newCommitTime, numberOfLogRecords, 0, 1);
long size = writer.getCurrentSize();
logFilePaths.add(writer.getLogFile().getPath().toString());
writer.close();
@@ -513,23 +482,20 @@ public class HoodieRealtimeRecordReaderTest {
// write rollback for the previous block in new log file version
newCommitTime = "102";
writer = writeRollbackBlockToLogFile(partitionDir, schema, "fileid0", commitTime,
newCommitTime, "101", 1);
writer = writeRollbackBlockToLogFile(partitionDir, schema, "fileid0", commitTime, newCommitTime, "101", 1);
logFilePaths.add(writer.getLogFile().getPath().toString());
writer.close();
assertTrue("block - size should be > 0", size > 0);
InputFormatTestUtil.deltaCommit(basePath, newCommitTime);
//create a split with baseFile (parquet file written earlier) and new log file(s)
// create a split with baseFile (parquet file written earlier) and new log file(s)
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
new FileSplit(new Path(partitionDir + "/fileid0_1_" + commitTime + ".parquet"), 0, 1,
jobConf), basePath.getRoot().getPath(), logFilePaths, newCommitTime);
new FileSplit(new Path(partitionDir + "/fileid0_1_" + commitTime + ".parquet"), 0, 1, jobConf),
basePath.getRoot().getPath(), logFilePaths, newCommitTime);
//create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader =
new MapredParquetInputFormat().getRecordReader(
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null),
jobConf, null);
// create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader = new MapredParquetInputFormat().getRecordReader(
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null), jobConf, null);
JobConf jobConf = new JobConf();
List<Schema.Field> fields = schema.getFields();
@@ -537,8 +503,7 @@ public class HoodieRealtimeRecordReaderTest {
// Try to read all the fields passed by the new schema
String names = fields.stream().map(f -> f.name()).collect(Collectors.joining(","));
String positions = fields.stream().map(f -> String.valueOf(f.pos()))
.collect(Collectors.joining(","));
String positions = fields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(","));
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, positions);
jobConf.set("partition_columns", "datestr");
@@ -554,8 +519,7 @@ public class HoodieRealtimeRecordReaderTest {
// Try to read all the fields passed by the new schema
names = firstSchemaFields.stream().map(f -> f.name()).collect(Collectors.joining(","));
positions = firstSchemaFields.stream().map(f -> String.valueOf(f.pos()))
.collect(Collectors.joining(","));
positions = firstSchemaFields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(","));
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, positions);
jobConf.set("partition_columns", "datestr");