1
0

Reformatting code per Google Code Style all over

This commit is contained in:
Vinoth Chandar
2017-11-12 22:54:56 -08:00
committed by vinoth chandar
parent 5a62480a92
commit e45679f5e2
254 changed files with 21580 additions and 21108 deletions

View File

@@ -22,47 +22,48 @@ import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
public class HoodieHiveUtil {
public static final Logger LOG =
LogManager.getLogger(HoodieHiveUtil.class);
public static final String HOODIE_CONSUME_MODE_PATTERN = "hoodie.%s.consume.mode";
public static final String HOODIE_START_COMMIT_PATTERN = "hoodie.%s.consume.start.timestamp";
public static final String HOODIE_MAX_COMMIT_PATTERN = "hoodie.%s.consume.max.commits";
public static final String INCREMENTAL_SCAN_MODE = "INCREMENTAL";
public static final String LATEST_SCAN_MODE = "LATEST";
public static final String DEFAULT_SCAN_MODE = LATEST_SCAN_MODE;
public static final int DEFAULT_MAX_COMMITS = 1;
public static final int MAX_COMMIT_ALL = -1;
public static final int DEFAULT_LEVELS_TO_BASEPATH = 3;
public static final Logger LOG =
LogManager.getLogger(HoodieHiveUtil.class);
public static Integer readMaxCommits(JobContext job, String tableName) {
String maxCommitName = String.format(HOODIE_MAX_COMMIT_PATTERN, tableName);
int maxCommits = job.getConfiguration().getInt(maxCommitName, DEFAULT_MAX_COMMITS);
if (maxCommits == MAX_COMMIT_ALL) {
maxCommits = Integer.MAX_VALUE;
}
LOG.info("Read max commits - " + maxCommits);
return maxCommits;
public static final String HOODIE_CONSUME_MODE_PATTERN = "hoodie.%s.consume.mode";
public static final String HOODIE_START_COMMIT_PATTERN = "hoodie.%s.consume.start.timestamp";
public static final String HOODIE_MAX_COMMIT_PATTERN = "hoodie.%s.consume.max.commits";
public static final String INCREMENTAL_SCAN_MODE = "INCREMENTAL";
public static final String LATEST_SCAN_MODE = "LATEST";
public static final String DEFAULT_SCAN_MODE = LATEST_SCAN_MODE;
public static final int DEFAULT_MAX_COMMITS = 1;
public static final int MAX_COMMIT_ALL = -1;
public static final int DEFAULT_LEVELS_TO_BASEPATH = 3;
public static Integer readMaxCommits(JobContext job, String tableName) {
String maxCommitName = String.format(HOODIE_MAX_COMMIT_PATTERN, tableName);
int maxCommits = job.getConfiguration().getInt(maxCommitName, DEFAULT_MAX_COMMITS);
if (maxCommits == MAX_COMMIT_ALL) {
maxCommits = Integer.MAX_VALUE;
}
LOG.info("Read max commits - " + maxCommits);
return maxCommits;
}
public static String readStartCommitTime(JobContext job, String tableName) {
String startCommitTimestampName = String.format(HOODIE_START_COMMIT_PATTERN, tableName);
LOG.info("Read start commit time - " + job.getConfiguration().get(startCommitTimestampName));
return job.getConfiguration().get(startCommitTimestampName);
}
public static String readStartCommitTime(JobContext job, String tableName) {
String startCommitTimestampName = String.format(HOODIE_START_COMMIT_PATTERN, tableName);
LOG.info("Read start commit time - " + job.getConfiguration().get(startCommitTimestampName));
return job.getConfiguration().get(startCommitTimestampName);
}
public static String readMode(JobContext job, String tableName) {
String modePropertyName = String.format(HOODIE_CONSUME_MODE_PATTERN, tableName);
String mode =job.getConfiguration().get(modePropertyName, DEFAULT_SCAN_MODE);
LOG.info(modePropertyName + ": " + mode);
return mode;
}
public static String readMode(JobContext job, String tableName) {
String modePropertyName = String.format(HOODIE_CONSUME_MODE_PATTERN, tableName);
String mode = job.getConfiguration().get(modePropertyName, DEFAULT_SCAN_MODE);
LOG.info(modePropertyName + ": " + mode);
return mode;
}
public static Path getNthParent(Path path, int n) {
Path parent = path;
for (int i = 0; i < n; i++) {
parent = parent.getParent();
}
return parent;
public static Path getNthParent(Path path, int n) {
Path parent = path;
for (int i = 0; i < n; i++) {
parent = parent.getParent();
}
return parent;
}
}

View File

@@ -16,6 +16,10 @@
package com.uber.hoodie.hadoop;
import static parquet.filter2.predicate.FilterApi.and;
import static parquet.filter2.predicate.FilterApi.binaryColumn;
import static parquet.filter2.predicate.FilterApi.gt;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
import com.uber.hoodie.common.model.HoodieRecord;
@@ -26,6 +30,12 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.exception.InvalidDatasetException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configurable;
@@ -54,156 +64,151 @@ import parquet.hadoop.metadata.FileMetaData;
import parquet.hadoop.metadata.ParquetMetadata;
import parquet.io.api.Binary;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import static parquet.filter2.predicate.FilterApi.and;
import static parquet.filter2.predicate.FilterApi.binaryColumn;
import static parquet.filter2.predicate.FilterApi.gt;
/**
* 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 HoodieInputFormat extends MapredParquetInputFormat
implements Configurable {
public static final Log LOG = LogFactory.getLog(HoodieInputFormat.class);
public static final Log LOG = LogFactory.getLog(HoodieInputFormat.class);
protected Configuration conf;
protected Configuration conf;
@Override
public FileStatus[] listStatus(JobConf job) throws IOException {
// Get all the file status from FileInputFormat and then do the filter
FileStatus[] fileStatuses = super.listStatus(job);
Map<HoodieTableMetaClient, List<FileStatus>> groupedFileStatus = groupFileStatus(fileStatuses);
LOG.info("Found a total of " + groupedFileStatus.size() + " groups");
List<FileStatus> returns = new ArrayList<>();
for(Map.Entry<HoodieTableMetaClient, List<FileStatus>> entry: groupedFileStatus.entrySet()) {
HoodieTableMetaClient metadata = entry.getKey();
if (metadata == null) {
// Add all the paths which are not hoodie specific
returns.addAll(entry.getValue());
continue;
}
@Override
public FileStatus[] listStatus(JobConf job) throws IOException {
// Get all the file status from FileInputFormat and then do the filter
FileStatus[] fileStatuses = super.listStatus(job);
Map<HoodieTableMetaClient, List<FileStatus>> groupedFileStatus = groupFileStatus(fileStatuses);
LOG.info("Found a total of " + groupedFileStatus.size() + " groups");
List<FileStatus> returns = new ArrayList<>();
for (Map.Entry<HoodieTableMetaClient, List<FileStatus>> entry : groupedFileStatus.entrySet()) {
HoodieTableMetaClient metadata = entry.getKey();
if (metadata == null) {
// Add all the paths which are not hoodie specific
returns.addAll(entry.getValue());
continue;
}
FileStatus[] statuses = entry.getValue().toArray(new FileStatus[entry.getValue().size()]);
if (LOG.isDebugEnabled()) {
LOG.debug("Hoodie Metadata initialized with completed commit Ts as :" + metadata);
}
String tableName = metadata.getTableConfig().getTableName();
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().getCommitsAndCompactionsTimeline().filterCompletedInstants();
TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metadata, timeline, statuses);
FileStatus[] statuses = entry.getValue().toArray(new FileStatus[entry.getValue().size()]);
if (LOG.isDebugEnabled()) {
LOG.debug("Hoodie Metadata initialized with completed commit Ts as :" + metadata);
}
String tableName = metadata.getTableConfig().getTableName();
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().getCommitsAndCompactionsTimeline()
.filterCompletedInstants();
TableFileSystemView.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);
// 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());
for (HoodieDataFile filteredFile : filteredFiles) {
LOG.info("Processing incremental hoodie file - " + filteredFile.getPath());
filteredFile = checkFileStatus(filteredFile);
returns.add(filteredFile.getFileStatus());
}
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());
LOG.info("Total paths to process after hoodie filter " + filteredFiles.size());
for (HoodieDataFile filteredFile : filteredFiles) {
if (LOG.isDebugEnabled()) {
LOG.debug("Processing latest hoodie file - " + filteredFile.getPath());
}
filteredFile = checkFileStatus(filteredFile);
returns.add(filteredFile.getFileStatus());
}
}
if (HoodieHiveUtil.INCREMENTAL_SCAN_MODE.equals(mode)) {
// this is of the form commitTs_partition_sequenceNumber
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());
for (HoodieDataFile filteredFile : filteredFiles) {
LOG.info("Processing incremental hoodie file - " + filteredFile.getPath());
filteredFile = checkFileStatus(filteredFile);
returns.add(filteredFile.getFileStatus());
}
return returns.toArray(new FileStatus[returns.size()]);
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());
LOG.info("Total paths to process after hoodie filter " + filteredFiles.size());
for (HoodieDataFile filteredFile : filteredFiles) {
if (LOG.isDebugEnabled()) {
LOG.debug("Processing latest hoodie file - " + filteredFile.getPath());
}
filteredFile = checkFileStatus(filteredFile);
returns.add(filteredFile.getFileStatus());
}
}
}
return returns.toArray(new FileStatus[returns.size()]);
/**
* 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();
}
/**
* 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();
try {
if (dataFile.getFileSize() == 0) {
FileSystem fs = dataPath.getFileSystem(conf);
LOG.info("Refreshing file status " + dataFile.getPath());
return new HoodieDataFile(fs.getFileStatus(dataPath));
}
return dataFile;
} catch (IOException e) {
throw new HoodieIOException("Could not get FileStatus on path " + dataPath);
}
}
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 with "."
continue;
}
if ((metadata == null && nonHoodieBasePath == null) || (metadata == null && !status.getPath()
.toString()
.contains(nonHoodieBasePath)) || (metadata != null && !status.getPath().toString()
.contains(metadata.getBasePath()))) {
try {
if (dataFile.getFileSize() == 0) {
FileSystem fs = dataPath.getFileSystem(conf);
LOG.info("Refreshing file status " + dataFile.getPath());
return new HoodieDataFile(fs.getFileStatus(dataPath));
}
return dataFile;
} catch (IOException e) {
throw new HoodieIOException("Could not get FileStatus on path " + dataPath);
metadata = getTableMetaClient(status.getPath().getFileSystem(conf),
status.getPath().getParent());
nonHoodieBasePath = null;
} catch (InvalidDatasetException e) {
LOG.info("Handling a non-hoodie path " + status.getPath());
metadata = null;
nonHoodieBasePath =
status.getPath().getParent().toString();
}
}
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 with "."
continue;
}
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());
nonHoodieBasePath = null;
} catch (InvalidDatasetException e) {
LOG.info("Handling a non-hoodie path " + status.getPath());
metadata = null;
nonHoodieBasePath =
status.getPath().getParent().toString();
}
if(!grouped.containsKey(metadata)) {
grouped.put(metadata, new ArrayList<>());
}
}
grouped.get(metadata).add(status);
if (!grouped.containsKey(metadata)) {
grouped.put(metadata, new ArrayList<>());
}
return grouped;
}
grouped.get(metadata).add(status);
}
return grouped;
}
public void setConf(Configuration conf) {
this.conf = conf;
}
public void setConf(Configuration conf) {
this.conf = conf;
}
public Configuration getConf() {
return conf;
}
public Configuration getConf() {
return conf;
}
@Override
public RecordReader<Void, ArrayWritable> getRecordReader(final InputSplit split,
final JobConf job, final Reporter reporter) throws IOException {
// TODO enable automatic predicate pushdown after fixing issues
@Override
public RecordReader<Void, 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();
@@ -213,91 +218,83 @@ public class HoodieInputFormat extends MapredParquetInputFormat
// FilterPredicate predicate = constructHoodiePredicate(job, tableName, split);
// LOG.info("Setting parquet predicate push down as " + predicate);
// ParquetInputFormat.setFilterPredicate(job, predicate);
//clearOutExistingPredicate(job);
//clearOutExistingPredicate(job);
// }
return super.getRecordReader(split, job, reporter);
}
return super.getRecordReader(split, job, reporter);
}
/**
* Clears out the filter expression (if this is not done, then ParquetReader will override the FilterPredicate set)
*
* @param job
*/
private void clearOutExistingPredicate(JobConf job) {
job.unset(TableScanDesc.FILTER_EXPR_CONF_STR);
}
/**
* Clears out the filter expression (if this is not done, then ParquetReader will override the
* FilterPredicate set)
*/
private void clearOutExistingPredicate(JobConf job) {
job.unset(TableScanDesc.FILTER_EXPR_CONF_STR);
}
/**
* Constructs the predicate to push down to parquet storage.
* This creates the predicate for `hoodie_commit_time` > 'start_commit_time' and ANDs with the existing predicate if one is present already.
*
* @param job
* @param tableName
* @return
*/
private FilterPredicate constructHoodiePredicate(JobConf job,
String tableName,
InputSplit split) throws IOException {
FilterPredicate commitTimePushdown = constructCommitTimePushdownPredicate(job, tableName);
LOG.info("Commit time predicate - " + commitTimePushdown.toString());
FilterPredicate existingPushdown = constructHQLPushdownPredicate(job, split);
LOG.info("Existing predicate - " + existingPushdown);
/**
* Constructs the predicate to push down to parquet storage. This creates the predicate for
* `hoodie_commit_time` > 'start_commit_time' and ANDs with the existing predicate if one is
* present already.
*/
private FilterPredicate constructHoodiePredicate(JobConf job,
String tableName,
InputSplit split) throws IOException {
FilterPredicate commitTimePushdown = constructCommitTimePushdownPredicate(job, tableName);
LOG.info("Commit time predicate - " + commitTimePushdown.toString());
FilterPredicate existingPushdown = constructHQLPushdownPredicate(job, split);
LOG.info("Existing predicate - " + existingPushdown);
FilterPredicate hoodiePredicate;
if (existingPushdown != null) {
hoodiePredicate = and(existingPushdown, commitTimePushdown);
} else {
hoodiePredicate = commitTimePushdown;
}
LOG.info("Hoodie Predicate - " + hoodiePredicate);
return hoodiePredicate;
FilterPredicate hoodiePredicate;
if (existingPushdown != null) {
hoodiePredicate = and(existingPushdown, commitTimePushdown);
} else {
hoodiePredicate = commitTimePushdown;
}
LOG.info("Hoodie Predicate - " + hoodiePredicate);
return hoodiePredicate;
}
private FilterPredicate constructHQLPushdownPredicate(JobConf job, InputSplit split)
throws IOException {
String serializedPushdown = job.get(TableScanDesc.FILTER_EXPR_CONF_STR);
String columnNamesString = job.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR);
if (serializedPushdown == null || columnNamesString == null || serializedPushdown.isEmpty()
|| columnNamesString.isEmpty()) {
return null;
} else {
SearchArgument sarg =
SearchArgumentFactory.create(Utilities.deserializeExpression(serializedPushdown));
final Path finalPath = ((FileSplit) split).getPath();
final ParquetMetadata parquetMetadata = ParquetFileReader.readFooter(job, finalPath);
final FileMetaData fileMetaData = parquetMetadata.getFileMetaData();
return ParquetFilterPredicateConverter
.toFilterPredicate(sarg, fileMetaData.getSchema());
}
private FilterPredicate constructHQLPushdownPredicate(JobConf job, InputSplit split)
throws IOException {
String serializedPushdown = job.get(TableScanDesc.FILTER_EXPR_CONF_STR);
String columnNamesString = job.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR);
if (serializedPushdown == null || columnNamesString == null || serializedPushdown.isEmpty()
|| columnNamesString.isEmpty()) {
return null;
} else {
SearchArgument sarg =
SearchArgumentFactory.create(Utilities.deserializeExpression(serializedPushdown));
final Path finalPath = ((FileSplit) split).getPath();
final ParquetMetadata parquetMetadata = ParquetFileReader.readFooter(job, finalPath);
final FileMetaData fileMetaData = parquetMetadata.getFileMetaData();
return ParquetFilterPredicateConverter
.toFilterPredicate(sarg, fileMetaData.getSchema());
}
}
private FilterPredicate constructCommitTimePushdownPredicate(JobConf job, String tableName)
throws IOException {
String lastIncrementalTs = HoodieHiveUtil.readStartCommitTime(Job.getInstance(job), tableName);
Operators.BinaryColumn sequenceColumn =
binaryColumn(HoodieRecord.COMMIT_TIME_METADATA_FIELD);
FilterPredicate p = gt(sequenceColumn, Binary.fromString(lastIncrementalTs));
LOG.info("Setting predicate in InputFormat " + p.toString());
return p;
}
private FilterPredicate constructCommitTimePushdownPredicate(JobConf job, String tableName)
throws IOException {
String lastIncrementalTs = HoodieHiveUtil.readStartCommitTime(Job.getInstance(job), tableName);
Operators.BinaryColumn sequenceColumn =
binaryColumn(HoodieRecord.COMMIT_TIME_METADATA_FIELD);
FilterPredicate p = gt(sequenceColumn, Binary.fromString(lastIncrementalTs));
LOG.info("Setting predicate in InputFormat " + p.toString());
return p;
}
/**
* 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
*
* @param dataPath
* @return
* @throws IOException
*/
protected static HoodieTableMetaClient getTableMetaClient(FileSystem fs, Path dataPath) {
int levels = HoodieHiveUtil.DEFAULT_LEVELS_TO_BASEPATH;
if (HoodiePartitionMetadata.hasPartitionMetadata(fs, dataPath)) {
HoodiePartitionMetadata metadata = new HoodiePartitionMetadata(fs, dataPath);
metadata.readFromFS();
levels = metadata.getPartitionDepth();
}
Path baseDir = HoodieHiveUtil.getNthParent(dataPath, levels);
LOG.info("Reading hoodie metadata from path " + baseDir.toString());
return new HoodieTableMetaClient(fs, baseDir.toString());
/**
* 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) {
int levels = HoodieHiveUtil.DEFAULT_LEVELS_TO_BASEPATH;
if (HoodiePartitionMetadata.hasPartitionMetadata(fs, dataPath)) {
HoodiePartitionMetadata metadata = new HoodiePartitionMetadata(fs, dataPath);
metadata.readFromFS();
levels = metadata.getPartitionDepth();
}
Path baseDir = HoodieHiveUtil.getNthParent(dataPath, levels);
LOG.info("Reading hoodie metadata from path " + baseDir.toString());
return new HoodieTableMetaClient(fs, baseDir.toString());
}
}

View File

@@ -21,7 +21,11 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
import com.uber.hoodie.exception.DatasetNotFoundException;
import com.uber.hoodie.exception.HoodieException;
import java.io.Serializable;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@@ -29,150 +33,142 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import java.io.Serializable;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.stream.Collectors;
/**
* 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
*
* 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.
*
* hadoopConf.setClass("mapreduce.input.pathFilter.class",
* com.uber.hoodie.hadoop.HoodieROTablePathFilter.class,
* org.apache.hadoop.fs.PathFilter.class)
*
* hadoopConf.setClass("mapreduce.input.pathFilter.class", com.uber.hoodie.hadoop.HoodieROTablePathFilter.class,
* org.apache.hadoop.fs.PathFilter.class)
*/
public class HoodieROTablePathFilter implements PathFilter, Serializable {
public static final Log LOG = LogFactory.getLog(HoodieROTablePathFilter.class);
public static final Log LOG = LogFactory.getLog(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
*/
private HashMap<String, HashSet<Path>> hoodiePathCache;
/**
* 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;
/**
* Paths that are known to be non-hoodie datasets.
*/
private HashSet<String> nonHoodiePathCache;
/**
* Paths that are known to be non-hoodie datasets.
*/
private HashSet<String> nonHoodiePathCache;
public HoodieROTablePathFilter() {
hoodiePathCache = new HashMap<>();
nonHoodiePathCache = new HashSet<>();
public HoodieROTablePathFilter() {
hoodiePathCache = new HashMap<>();
nonHoodiePathCache = new HashSet<>();
}
/**
* Obtain the path, two levels from provided path
*
* @return said path if available, null otherwise
*/
private Path safeGetParentsParent(Path path) {
if (path.getParent() != null && path.getParent().getParent() != null
&& path.getParent().getParent().getParent() != null) {
return path.getParent().getParent().getParent();
}
return null;
}
/**
* Obtain the path, two levels from provided path
*
* @return said path if available, null otherwise
*/
private Path safeGetParentsParent(Path path) {
if (path.getParent() != null && path.getParent().getParent() != null && path.getParent().getParent().getParent() != null) {
return path.getParent().getParent().getParent();
}
return null;
@Override
public boolean accept(Path path) {
if (LOG.isDebugEnabled()) {
LOG.debug("Checking acceptance for path " + path);
}
Path folder = null;
try {
FileSystem fs = path.getFileSystem(new Configuration());
if (fs.isDirectory(path)) {
return true;
}
@Override
public boolean accept(Path path) {
// Assumes path is a file
folder = path.getParent(); // get the immediate parent.
// Try to use the caches.
if (nonHoodiePathCache.contains(folder.toString())) {
if (LOG.isDebugEnabled()) {
LOG.debug("Checking acceptance for path " + path);
LOG.debug("Accepting non-hoodie path from cache: " + path);
}
Path folder = null;
return true;
}
if (hoodiePathCache.containsKey(folder.toString())) {
if (LOG.isDebugEnabled()) {
LOG.debug(String.format("%s Hoodie path checked against cache, accept => %s \n",
path,
hoodiePathCache.get(folder.toString()).contains(path)));
}
return hoodiePathCache.get(folder.toString()).contains(path);
}
// Perform actual checking.
Path baseDir;
if (HoodiePartitionMetadata.hasPartitionMetadata(fs, folder)) {
HoodiePartitionMetadata metadata = new HoodiePartitionMetadata(fs, folder);
metadata.readFromFS();
baseDir = HoodieHiveUtil.getNthParent(folder, metadata.getPartitionDepth());
} else {
baseDir = safeGetParentsParent(folder);
}
if (baseDir != null) {
try {
FileSystem fs = path.getFileSystem(new Configuration());
if (fs.isDirectory(path)) {
return true;
}
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, baseDir.toString());
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient,
metaClient.getActiveTimeline().getCommitTimeline()
.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);
for (HoodieDataFile lfile : latestFiles) {
hoodiePathCache.get(folder.toString()).add(new Path(lfile.getPath()));
}
// Assumes path is a file
folder = path.getParent(); // get the immediate parent.
// Try to use the caches.
if (nonHoodiePathCache.contains(folder.toString())) {
if (LOG.isDebugEnabled()) {
LOG.debug("Accepting non-hoodie path from cache: " + path);
}
return true;
}
if (hoodiePathCache.containsKey(folder.toString())) {
if (LOG.isDebugEnabled()) {
LOG.debug(String.format("%s Hoodie path checked against cache, accept => %s \n",
path,
hoodiePathCache.get(folder.toString()).contains(path)));
}
return hoodiePathCache.get(folder.toString()).contains(path);
}
// Perform actual checking.
Path baseDir;
if (HoodiePartitionMetadata.hasPartitionMetadata(fs, folder)) {
HoodiePartitionMetadata metadata = new HoodiePartitionMetadata(fs, folder);
metadata.readFromFS();
baseDir = HoodieHiveUtil.getNthParent(folder, metadata.getPartitionDepth());
} else {
baseDir = safeGetParentsParent(folder);
}
if (baseDir != null) {
try {
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, baseDir.toString());
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient,
metaClient.getActiveTimeline().getCommitTimeline()
.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);
for (HoodieDataFile lfile: latestFiles) {
hoodiePathCache.get(folder.toString()).add(new Path(lfile.getPath()));
}
// accept the path, if its among the latest files.
if (LOG.isDebugEnabled()) {
LOG.debug(String.format("%s checked after cache population, accept => %s \n",
path,
hoodiePathCache.get(folder.toString()).contains(path)));
}
return hoodiePathCache.get(folder.toString()).contains(path);
} catch (DatasetNotFoundException e) {
// Non-hoodie path, accept it.
if (LOG.isDebugEnabled()) {
LOG.debug(String.format("(1) Caching non-hoodie path under %s \n",
folder.toString()));
}
nonHoodiePathCache.add(folder.toString());
return true;
}
} else {
// files is at < 3 level depth in FS tree, can't be hoodie dataset
if (LOG.isDebugEnabled()) {
LOG.debug(String.format("(2) Caching non-hoodie path under %s \n", folder.toString()));
}
nonHoodiePathCache.add(folder.toString());
return true;
}
} catch (Exception e) {
String msg = "Error checking path :" + path +", under folder: "+ folder;
LOG.error(msg, e);
throw new HoodieException(msg, e);
// accept the path, if its among the latest files.
if (LOG.isDebugEnabled()) {
LOG.debug(String.format("%s checked after cache population, accept => %s \n",
path,
hoodiePathCache.get(folder.toString()).contains(path)));
}
return hoodiePathCache.get(folder.toString()).contains(path);
} catch (DatasetNotFoundException e) {
// Non-hoodie path, accept it.
if (LOG.isDebugEnabled()) {
LOG.debug(String.format("(1) Caching non-hoodie path under %s \n",
folder.toString()));
}
nonHoodiePathCache.add(folder.toString());
return true;
}
} else {
// files is at < 3 level depth in FS tree, can't be hoodie dataset
if (LOG.isDebugEnabled()) {
LOG.debug(String.format("(2) Caching non-hoodie path under %s \n", folder.toString()));
}
nonHoodiePathCache.add(folder.toString());
return true;
}
} catch (Exception e) {
String msg = "Error checking path :" + path + ", under folder: " + folder;
LOG.error(msg, e);
throw new HoodieException(msg, e);
}
}
}

View File

@@ -24,13 +24,13 @@ 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
@Target(ElementType.TYPE)
@Retention(RetentionPolicy.RUNTIME)
public @interface UseFileSplitsFromInputFormat {
}

View File

@@ -25,7 +25,7 @@ import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe;
*/
public class HoodieParquetSerde extends ParquetHiveSerDe {
public HoodieParquetSerde() {
super();
}
public HoodieParquetSerde() {
super();
}
}

View File

@@ -18,81 +18,83 @@
package com.uber.hoodie.hadoop.realtime;
import org.apache.hadoop.mapred.FileSplit;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.mapred.FileSplit;
/**
* Filesplit that wraps the base split and a list of log files to merge deltas from.
*/
public class HoodieRealtimeFileSplit extends FileSplit {
private List<String> deltaFilePaths;
private List<String> deltaFilePaths;
private String maxCommitTime;
private String maxCommitTime;
private String basePath;
private String basePath;
public HoodieRealtimeFileSplit() {
super();
public HoodieRealtimeFileSplit() {
super();
}
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;
}
public List<String> getDeltaFilePaths() {
return deltaFilePaths;
}
public String getMaxCommitTime() {
return maxCommitTime;
}
public String getBasePath() {
return basePath;
}
private static void writeString(String str, DataOutput out) throws IOException {
byte[] pathBytes = str.getBytes(StandardCharsets.UTF_8);
out.writeInt(pathBytes.length);
out.write(pathBytes);
}
private static String readString(DataInput in) throws IOException {
byte[] pathBytes = new byte[in.readInt()];
in.readFully(pathBytes);
return new String(pathBytes, StandardCharsets.UTF_8);
}
@Override
public void write(DataOutput out) throws IOException {
super.write(out);
writeString(maxCommitTime, out);
out.writeInt(deltaFilePaths.size());
for (String logFilePath : deltaFilePaths) {
writeString(logFilePath, out);
}
}
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;
}
public List<String> getDeltaFilePaths() {
return deltaFilePaths;
}
public String getMaxCommitTime() {
return maxCommitTime;
}
public String getBasePath() {
return basePath;
}
private static void writeString(String str, DataOutput out) throws IOException {
byte[] pathBytes = str.getBytes(StandardCharsets.UTF_8);
out.writeInt(pathBytes.length);
out.write(pathBytes);
}
private static String readString(DataInput in) throws IOException {
byte[] pathBytes = new byte[in.readInt()];
in.readFully(pathBytes);
return new String(pathBytes, StandardCharsets.UTF_8);
}
@Override
public void write(DataOutput out) throws IOException {
super.write(out);
writeString(maxCommitTime, out);
out.writeInt(deltaFilePaths.size());
for (String logFilePath: deltaFilePaths) {
writeString(logFilePath, out);
}
}
@Override
public void readFields(DataInput in) throws IOException {
super.readFields(in);
maxCommitTime = readString(in);
int totalLogFiles = in.readInt();
deltaFilePaths = new ArrayList<>(totalLogFiles);
for (int i=0; i < totalLogFiles; i++) {
deltaFilePaths.add(readString(in));
}
@Override
public void readFields(DataInput in) throws IOException {
super.readFields(in);
maxCommitTime = readString(in);
int totalLogFiles = in.readInt();
deltaFilePaths = new ArrayList<>(totalLogFiles);
for (int i = 0; i < totalLogFiles; i++) {
deltaFilePaths.add(readString(in));
}
}
}

View File

@@ -19,9 +19,7 @@
package com.uber.hoodie.hadoop.realtime;
import com.google.common.base.Preconditions;
import com.google.common.collect.Sets;
import com.uber.hoodie.common.model.FileSlice;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
@@ -32,7 +30,16 @@ import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.hadoop.HoodieInputFormat;
import com.uber.hoodie.hadoop.UseFileSplitsFromInputFormat;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configurable;
@@ -47,168 +54,168 @@ import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.RecordReader;
import org.apache.hadoop.mapred.Reporter;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* Input Format, that provides a real-time view of data in a Hoodie dataset
*/
@UseFileSplitsFromInputFormat
public class HoodieRealtimeInputFormat extends HoodieInputFormat implements Configurable {
public static final Log LOG = LogFactory.getLog(HoodieRealtimeInputFormat.class);
public static final Log LOG = LogFactory.getLog(HoodieRealtimeInputFormat.class);
// These positions have to be deterministic across all tables
public static final int HOODIE_COMMIT_TIME_COL_POS = 0;
public static final int HOODIE_RECORD_KEY_COL_POS = 2;
public static final int HOODIE_PARTITION_PATH_COL_POS = 3;
// These positions have to be deterministic across all tables
public static final int HOODIE_COMMIT_TIME_COL_POS = 0;
public static final int HOODIE_RECORD_KEY_COL_POS = 2;
public static final int HOODIE_PARTITION_PATH_COL_POS = 3;
@Override
public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
@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()));
// 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 -> {
// find if we have a metaclient already for this partition.
Optional<String> matchingBasePath = metaClientMap.keySet().stream()
.filter(basePath -> p.toString().startsWith(basePath)).findFirst();
if (matchingBasePath.isPresent()) {
return metaClientMap.get(matchingBasePath.get());
}
// obtain all unique parent folders for splits
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 -> {
// find if we have a metaclient already for this partition.
Optional<String> matchingBasePath = metaClientMap.keySet().stream()
.filter(basePath -> p.toString().startsWith(basePath)).findFirst();
if (matchingBasePath.isPresent()) {
return metaClientMap.get(matchingBasePath.get());
}
try {
HoodieTableMetaClient metaClient = getTableMetaClient(p.getFileSystem(conf), p);
metaClientMap.put(metaClient.getBasePath(), metaClient);
return metaClient;
} catch (IOException e) {
throw new HoodieIOException("Error creating hoodie meta client against : " + p, e);
}
}));
try {
HoodieTableMetaClient metaClient = getTableMetaClient(p.getFileSystem(conf), p);
metaClientMap.put(metaClient.getBasePath(), metaClient);
return metaClient;
} catch (IOException e) {
throw new HoodieIOException("Error creating hoodie meta client against : " + p, e);
}
}));
// for all unique split parents, obtain all delta files based on delta commit timeline, grouped on file id
List<HoodieRealtimeFileSplit> rtSplits = new ArrayList<>();
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);
// for all unique split parents, obtain all delta files based on delta commit timeline, grouped on file id
List<HoodieRealtimeFileSplit> rtSplits = new ArrayList<>();
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);
try {
Stream<FileSlice> latestFileSlices = fsView.getLatestFileSlices(relPartitionPath);
// subgroup splits again by file id & match with log files.
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());
dataFileSplits.forEach(split -> {
try {
Stream<FileSlice> latestFileSlices = fsView.getLatestFileSlices(relPartitionPath);
// subgroup splits again by file id & match with log files.
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());
dataFileSplits.forEach(split -> {
try {
List<String> logFilePaths = fileSlice.getLogFiles()
.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.COMPACTION_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);
List<String> logFilePaths = fileSlice.getLogFiles()
.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.COMPACTION_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);
}
});
});
LOG.info("Returning a total splits of " + rtSplits.size());
return rtSplits.toArray(new InputSplit[rtSplits.size()]);
} catch (Exception e) {
throw new HoodieException("Error obtaining data file/log file grouping: " + partitionPath,
e);
}
});
LOG.info("Returning a total splits of " + rtSplits.size());
return rtSplits.toArray(new InputSplit[rtSplits.size()]);
}
@Override
public FileStatus[] listStatus(JobConf job) throws IOException {
// Call the HoodieInputFormat::listStatus to obtain all latest parquet files, based on commit timeline.
return super.listStatus(job);
}
/**
* Add a field to the existing fields projected
*/
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, "");
String readColNamesPrefix = readColNames + ",";
if (readColNames == null || readColNames.isEmpty()) {
readColNamesPrefix = "";
}
String readColIdsPrefix = readColIds + ",";
if (readColIds == null || readColIds.isEmpty()) {
readColIdsPrefix = "";
}
@Override
public FileStatus[] listStatus(JobConf job) throws IOException {
// Call the HoodieInputFormat::listStatus to obtain all latest parquet files, based on commit timeline.
return super.listStatus(job);
if (!readColNames.contains(fieldName)) {
// If not already in the list - then add it
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) ",
conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR),
conf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR)));
}
}
return conf;
}
/**
* Add a field to the existing fields projected
*/
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, "");
private static 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);
return configuration;
}
String readColNamesPrefix = readColNames + ",";
if (readColNames == null || readColNames.isEmpty()) {
readColNamesPrefix = "";
}
String readColIdsPrefix = readColIds + ",";
if (readColIds == null || readColIds.isEmpty()) {
readColIdsPrefix = "";
}
@Override
public RecordReader<Void, ArrayWritable> getRecordReader(final InputSplit split,
final JobConf job,
final Reporter reporter) throws IOException {
LOG.info("Creating record reader with readCols :" + job
.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR));
// sanity check
Preconditions.checkArgument(split instanceof HoodieRealtimeFileSplit,
"HoodieRealtimeRecordReader can only work on HoodieRealtimeFileSplit and not with "
+ split);
return new HoodieRealtimeRecordReader((HoodieRealtimeFileSplit) split, job,
super.getRecordReader(split, job, reporter));
}
if (!readColNames.contains(fieldName)) {
// If not already in the list - then add it
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) ",
conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR),
conf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR)));
}
}
return conf;
}
@Override
public void setConf(Configuration conf) {
this.conf = addRequiredProjectionFields(conf);
}
private static 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);
return configuration;
}
@Override
public RecordReader<Void, ArrayWritable> getRecordReader(final InputSplit split,
final JobConf job,
final Reporter reporter) throws IOException {
LOG.info("Creating record reader with readCols :" + job.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR));
// sanity check
Preconditions.checkArgument(split instanceof HoodieRealtimeFileSplit,
"HoodieRealtimeRecordReader can only work on HoodieRealtimeFileSplit and not with " + split );
return new HoodieRealtimeRecordReader((HoodieRealtimeFileSplit) split, job, super.getRecordReader(split, job, reporter));
}
@Override
public void setConf(Configuration conf) {
this.conf = addRequiredProjectionFields(conf);
}
@Override
public Configuration getConf() {
return conf;
}
@Override
public Configuration getConf() {
return conf;
}
}

View File

@@ -18,14 +18,21 @@
package com.uber.hoodie.hadoop.realtime;
import com.uber.hoodie.common.model.HoodieAvroPayload;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.table.log.HoodieCompactedLogRecordScanner;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.ReflectionUtils;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import java.util.stream.Collectors;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericArray;
import org.apache.avro.generic.GenericFixed;
@@ -51,291 +58,274 @@ import parquet.avro.AvroSchemaConverter;
import parquet.hadoop.ParquetFileReader;
import parquet.schema.MessageType;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import java.util.stream.Collectors;
/**
* 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 class HoodieRealtimeRecordReader implements RecordReader<Void, ArrayWritable> {
private final RecordReader<Void, ArrayWritable> parquetReader;
private final HoodieRealtimeFileSplit split;
private final JobConf jobConf;
private final RecordReader<Void, ArrayWritable> parquetReader;
private final HoodieRealtimeFileSplit split;
private final JobConf jobConf;
public static final Log LOG = LogFactory.getLog(HoodieRealtimeRecordReader.class);
public static final Log LOG = LogFactory.getLog(HoodieRealtimeRecordReader.class);
private final HashMap<String, ArrayWritable> deltaRecordMap;
private final MessageType baseFileSchema;
private final HashMap<String, ArrayWritable> deltaRecordMap;
private final MessageType baseFileSchema;
public HoodieRealtimeRecordReader(HoodieRealtimeFileSplit split,
JobConf job,
RecordReader<Void, ArrayWritable> realReader) {
this.split = split;
this.jobConf = job;
this.parquetReader = realReader;
this.deltaRecordMap = new HashMap<>();
public HoodieRealtimeRecordReader(HoodieRealtimeFileSplit split,
JobConf job,
RecordReader<Void, ArrayWritable> realReader) {
this.split = split;
this.jobConf = job;
this.parquetReader = realReader;
this.deltaRecordMap = new HashMap<>();
LOG.info("cfg ==> " + job.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR));
try {
baseFileSchema = readSchema(jobConf, split.getPath());
readAndCompactLog();
} catch (IOException e) {
throw new HoodieIOException(
"Could not create HoodieRealtimeRecordReader on path " + this.split.getPath(), e);
}
LOG.info("cfg ==> " + job.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR));
try {
baseFileSchema = readSchema(jobConf, split.getPath());
readAndCompactLog();
} catch (IOException e) {
throw new HoodieIOException(
"Could not create HoodieRealtimeRecordReader on path " + this.split.getPath(), e);
}
}
/**
* 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 {
return ParquetFileReader.readFooter(conf, parquetFilePath).getFileMetaData()
.getSchema();
} catch (IOException e) {
throw new HoodieIOException("Failed to read footer for parquet " + parquetFilePath,
e);
}
}
/**
* Goes through the log files and populates a map with latest version of each key logged, since
* the base split was written.
*/
private void readAndCompactLog() throws IOException {
Schema writerSchema = new AvroSchemaConverter().convert(baseFileSchema);
List<String> projectionFields = orderFields(
jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR),
jobConf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR),
jobConf.get("partition_columns", ""));
// TODO(vc): In the future, the reader schema should be updated based on log files & be able to null out fields not present before
Schema readerSchema = generateProjectionSchema(writerSchema, projectionFields);
LOG.info(
String.format("About to read compacted logs %s for base split %s, projecting cols %s",
split.getDeltaFilePaths(), split.getPath(), projectionFields));
HoodieCompactedLogRecordScanner compactedLogRecordScanner =
new HoodieCompactedLogRecordScanner(FSUtils.getFs(), split.getBasePath(),
split.getDeltaFilePaths(),
readerSchema, split.getMaxCommitTime());
// 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)
for (HoodieRecord<? extends HoodieRecordPayload> hoodieRecord : compactedLogRecordScanner) {
GenericRecord rec = (GenericRecord) hoodieRecord.getData().getInsertValue(readerSchema)
.get();
String key = hoodieRecord.getRecordKey();
// we assume, a later safe record in the log, is newer than what we have in the map & replace it.
ArrayWritable aWritable = (ArrayWritable) avroToArrayWritable(rec, writerSchema);
deltaRecordMap.put(key, aWritable);
if (LOG.isDebugEnabled()) {
LOG.debug("Log record : " + arrayWritableToString(aWritable));
}
}
}
private static String arrayWritableToString(ArrayWritable writable) {
if (writable == null) {
return "null";
}
/**
* 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 {
return ParquetFileReader.readFooter(conf, parquetFilePath).getFileMetaData()
.getSchema();
} catch (IOException e) {
throw new HoodieIOException("Failed to read footer for parquet " + parquetFilePath,
e);
}
StringBuilder builder = new StringBuilder();
Writable[] values = writable.get();
builder.append(String.format("Size: %s,", values.length));
for (Writable w : values) {
builder.append(w + " ");
}
return builder.toString();
}
/**
* 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.
*/
public static List<String> orderFields(String fieldNameCsv, String fieldOrderCsv,
String partitioningFieldsCsv) {
String[] fieldOrders = fieldOrderCsv.split(",");
Set<String> partitioningFields = Arrays.stream(partitioningFieldsCsv.split(","))
.collect(Collectors.toSet());
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",
fieldNames.size(), fieldOrders.length));
}
TreeMap<Integer, String> orderedFieldMap = new TreeMap<>();
for (int ox = 0; ox < fieldOrders.length; ox++) {
orderedFieldMap.put(Integer.parseInt(fieldOrders[ox]), fieldNames.get(ox));
}
return new ArrayList<>(orderedFieldMap.values());
}
/**
* Generate a reader schema off the provided writeSchema, to just project out the provided
* columns
*/
public static Schema generateProjectionSchema(Schema writeSchema, List<String> fieldNames) {
List<Schema.Field> projectedFields = new ArrayList<>();
for (String fn : fieldNames) {
Schema.Field field = writeSchema.getField(fn);
if (field == null) {
throw new HoodieException("Field " + fn + " not found log schema. Query cannot proceed!");
}
projectedFields
.add(new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultValue()));
}
return Schema.createRecord(projectedFields);
}
/**
* Goes through the log files and populates a map with latest version of each key logged, since the base split was written.
*/
private void readAndCompactLog() throws IOException {
Schema writerSchema = new AvroSchemaConverter().convert(baseFileSchema);
List<String> projectionFields = orderFields(
jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR),
jobConf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR),
jobConf.get("partition_columns", ""));
// TODO(vc): In the future, the reader schema should be updated based on log files & be able to null out fields not present before
Schema readerSchema = generateProjectionSchema(writerSchema, projectionFields);
/**
* Convert the projected read from delta record into an array writable
*/
public static Writable avroToArrayWritable(Object value, Schema schema) {
LOG.info(
String.format("About to read compacted logs %s for base split %s, projecting cols %s",
split.getDeltaFilePaths(), split.getPath(), projectionFields));
HoodieCompactedLogRecordScanner compactedLogRecordScanner =
new HoodieCompactedLogRecordScanner(FSUtils.getFs(), split.getBasePath(), split.getDeltaFilePaths(),
readerSchema, split.getMaxCommitTime());
// 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)
for (HoodieRecord<? extends HoodieRecordPayload> hoodieRecord : compactedLogRecordScanner) {
GenericRecord rec = (GenericRecord) hoodieRecord.getData().getInsertValue(readerSchema)
.get();
String key = hoodieRecord.getRecordKey();
// we assume, a later safe record in the log, is newer than what we have in the map & replace it.
ArrayWritable aWritable = (ArrayWritable) avroToArrayWritable(rec, writerSchema);
deltaRecordMap.put(key, aWritable);
if (LOG.isDebugEnabled()) {
LOG.debug("Log record : " + arrayWritableToString(aWritable));
}
}
// if value is null, make a NullWritable
if (value == null) {
return NullWritable.get();
}
private static String arrayWritableToString(ArrayWritable writable) {
if (writable == null) {
return "null";
switch (schema.getType()) {
case STRING:
return new Text(value.toString());
case BYTES:
return new BytesWritable((byte[]) value);
case INT:
return new IntWritable((Integer) value);
case LONG:
return new LongWritable((Long) value);
case FLOAT:
return new FloatWritable((Float) value);
case DOUBLE:
return new DoubleWritable((Double) value);
case BOOLEAN:
return new BooleanWritable((Boolean) value);
case NULL:
return NullWritable.get();
case RECORD:
GenericRecord record = (GenericRecord) value;
Writable[] values1 = new Writable[schema.getFields().size()];
int index1 = 0;
for (Schema.Field field : schema.getFields()) {
values1[index1++] = avroToArrayWritable(record.get(field.name()), field.schema());
}
StringBuilder builder = new StringBuilder();
Writable[] values = writable.get();
builder.append(String.format("Size: %s,", values.length));
for (Writable w: values) {
builder.append(w + " ");
return new ArrayWritable(Writable.class, values1);
case ENUM:
return new Text(value.toString());
case ARRAY:
GenericArray arrayValue = (GenericArray) value;
Writable[] values2 = new Writable[arrayValue.size()];
int index2 = 0;
for (Object obj : arrayValue) {
values2[index2++] = avroToArrayWritable(obj, schema.getElementType());
}
return builder.toString();
}
/**
* 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.
*
* @param fieldNameCsv
* @param fieldOrderCsv
* @return
*/
public static List<String> orderFields(String fieldNameCsv, String fieldOrderCsv,
String partitioningFieldsCsv) {
String[] fieldOrders = fieldOrderCsv.split(",");
Set<String> partitioningFields = Arrays.stream(partitioningFieldsCsv.split(","))
.collect(Collectors.toSet());
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",
fieldNames.size(), fieldOrders.length));
return new ArrayWritable(Writable.class, values2);
case MAP:
Map mapValue = (Map) value;
Writable[] values3 = new Writable[mapValue.size()];
int index3 = 0;
for (Object entry : mapValue.entrySet()) {
Map.Entry mapEntry = (Map.Entry) entry;
Writable[] mapValues = new Writable[2];
mapValues[0] = new Text(mapEntry.getKey().toString());
mapValues[1] = avroToArrayWritable(mapEntry.getValue(), schema.getValueType());
values3[index3++] = new ArrayWritable(Writable.class, mapValues);
}
TreeMap<Integer, String> orderedFieldMap = new TreeMap<>();
for (int ox = 0; ox < fieldOrders.length; ox++) {
orderedFieldMap.put(Integer.parseInt(fieldOrders[ox]), fieldNames.get(ox));
return new ArrayWritable(Writable.class, values3);
case UNION:
List<Schema> types = schema.getTypes();
if (types.size() != 2) {
throw new IllegalArgumentException("Only support union with 2 fields");
}
return new ArrayList<>(orderedFieldMap.values());
}
/**
* Generate a reader schema off the provided writeSchema, to just project out
* the provided columns
*
* @param writeSchema
* @param fieldNames
* @return
*/
public static Schema generateProjectionSchema(Schema writeSchema, List<String> fieldNames) {
List<Schema.Field> projectedFields = new ArrayList<>();
for (String fn: fieldNames) {
Schema.Field field = writeSchema.getField(fn);
if (field == null) {
throw new HoodieException("Field "+ fn + " not found log schema. Query cannot proceed!");
}
projectedFields.add(new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultValue()));
}
return Schema.createRecord(projectedFields);
}
/**
* Convert the projected read from delta record into an array writable
*
* @param value
* @param schema
* @return
*/
public static Writable avroToArrayWritable(Object value, Schema schema) {
// if value is null, make a NullWritable
if (value == null) {
return NullWritable.get();
}
switch (schema.getType()) {
case STRING:
return new Text(value.toString());
case BYTES:
return new BytesWritable((byte[]) value);
case INT:
return new IntWritable((Integer) value);
case LONG:
return new LongWritable((Long) value);
case FLOAT:
return new FloatWritable((Float) value);
case DOUBLE:
return new DoubleWritable((Double) value);
case BOOLEAN:
return new BooleanWritable((Boolean) value);
case NULL:
return NullWritable.get();
case RECORD:
GenericRecord record = (GenericRecord) value;
Writable[] values1 = new Writable[schema.getFields().size()];
int index1 = 0;
for (Schema.Field field : schema.getFields()) {
values1[index1++] = avroToArrayWritable(record.get(field.name()), field.schema());
}
return new ArrayWritable(Writable.class, values1);
case ENUM:
return new Text(value.toString());
case ARRAY:
GenericArray arrayValue = (GenericArray) value;
Writable[] values2 = new Writable[arrayValue.size()];
int index2 = 0;
for (Object obj : arrayValue) {
values2[index2++] = avroToArrayWritable(obj, schema.getElementType());
}
return new ArrayWritable(Writable.class, values2);
case MAP:
Map mapValue = (Map) value;
Writable[] values3 = new Writable[mapValue.size()];
int index3 = 0;
for (Object entry : mapValue.entrySet()) {
Map.Entry mapEntry = (Map.Entry) entry;
Writable[] mapValues = new Writable[2];
mapValues[0] = new Text(mapEntry.getKey().toString());
mapValues[1] = avroToArrayWritable(mapEntry.getValue(), schema.getValueType());
values3[index3++] = new ArrayWritable(Writable.class, mapValues);
}
return new ArrayWritable(Writable.class, values3);
case UNION:
List<Schema> types = schema.getTypes();
if (types.size() != 2) {
throw new IllegalArgumentException("Only support union with 2 fields");
}
Schema s1 = types.get(0);
Schema s2 = types.get(1);
if (s1.getType() == Schema.Type.NULL) {
return avroToArrayWritable(value, s2);
} else if (s2.getType() == Schema.Type.NULL) {
return avroToArrayWritable(value, s1);
} else {
throw new IllegalArgumentException("Only support union with null");
}
case FIXED:
return new BytesWritable(((GenericFixed) value).bytes());
}
return null;
}
@Override
public boolean next(Void aVoid, ArrayWritable arrayWritable) throws IOException {
// Call the underlying parquetReader.next - which may replace the passed in ArrayWritable with a new block of values
boolean result = this.parquetReader.next(aVoid, arrayWritable);
if(!result) {
// if the result is false, then there are no more records
return false;
Schema s1 = types.get(0);
Schema s2 = types.get(1);
if (s1.getType() == Schema.Type.NULL) {
return avroToArrayWritable(value, s2);
} else if (s2.getType() == Schema.Type.NULL) {
return avroToArrayWritable(value, s1);
} else {
// 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()[HoodieRealtimeInputFormat.HOODIE_RECORD_KEY_COL_POS].toString();
if (LOG.isDebugEnabled()) {
LOG.debug(String.format("key %s, base values: %s, log values: %s",
key, arrayWritableToString(arrayWritable), arrayWritableToString(deltaRecordMap.get(key))));
}
if (deltaRecordMap.containsKey(key)) {
Writable[] replaceValue = deltaRecordMap.get(key).get();
Writable[] originalValue = arrayWritable.get();
System.arraycopy(replaceValue, 0, originalValue, 0, originalValue.length);
arrayWritable.set(originalValue);
}
return true;
throw new IllegalArgumentException("Only support union with null");
}
case FIXED:
return new BytesWritable(((GenericFixed) value).bytes());
}
return null;
}
@Override
public Void createKey() {
return parquetReader.createKey();
@Override
public boolean next(Void aVoid, ArrayWritable arrayWritable) throws IOException {
// Call the underlying parquetReader.next - which may replace the passed in ArrayWritable with a new block of values
boolean result = this.parquetReader.next(aVoid, arrayWritable);
if (!result) {
// if the result is false, then there are no more records
return false;
} else {
// 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()[HoodieRealtimeInputFormat.HOODIE_RECORD_KEY_COL_POS]
.toString();
if (LOG.isDebugEnabled()) {
LOG.debug(String.format("key %s, base values: %s, log values: %s",
key, arrayWritableToString(arrayWritable),
arrayWritableToString(deltaRecordMap.get(key))));
}
if (deltaRecordMap.containsKey(key)) {
Writable[] replaceValue = deltaRecordMap.get(key).get();
Writable[] originalValue = arrayWritable.get();
System.arraycopy(replaceValue, 0, originalValue, 0, originalValue.length);
arrayWritable.set(originalValue);
}
return true;
}
}
@Override
public ArrayWritable createValue() {
return parquetReader.createValue();
}
@Override
public Void createKey() {
return parquetReader.createKey();
}
@Override
public long getPos() throws IOException {
return parquetReader.getPos();
}
@Override
public ArrayWritable createValue() {
return parquetReader.createValue();
}
@Override
public void close() throws IOException {
parquetReader.close();
}
@Override
public long getPos() throws IOException {
return parquetReader.getPos();
}
@Override
public float getProgress() throws IOException {
return parquetReader.getProgress();
}
@Override
public void close() throws IOException {
parquetReader.close();
}
@Override
public float getProgress() throws IOException {
return parquetReader.getProgress();
}
}

View File

@@ -17,22 +17,23 @@
package com.uber.hoodie.hadoop;
import org.junit.Test;
import static org.junit.Assert.*;
import static org.junit.Assert.assertTrue;
import java.lang.annotation.Annotation;
import org.junit.Test;
public class AnnotationTest {
@Test
public void testAnnotation() {
assertTrue(HoodieInputFormat.class.isAnnotationPresent(UseFileSplitsFromInputFormat.class));
Annotation[] annotations = HoodieInputFormat.class.getAnnotations();
boolean found = false;
for (Annotation annotation : annotations) {
if ("UseFileSplitsFromInputFormat".equals(annotation.annotationType().getSimpleName())){
found = true;
}
}
assertTrue(found);
@Test
public void testAnnotation() {
assertTrue(HoodieInputFormat.class.isAnnotationPresent(UseFileSplitsFromInputFormat.class));
Annotation[] annotations = HoodieInputFormat.class.getAnnotations();
boolean found = false;
for (Annotation annotation : annotations) {
if ("UseFileSplitsFromInputFormat".equals(annotation.annotationType().getSimpleName())) {
found = true;
}
}
assertTrue(found);
}
}

View File

@@ -16,222 +16,235 @@
package com.uber.hoodie.hadoop;
import static org.junit.Assert.assertEquals;
import com.uber.hoodie.common.util.FSUtils;
import java.io.File;
import java.io.IOException;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.io.ArrayWritable;
import org.apache.hadoop.mapred.*;
import org.apache.hadoop.mapred.FileInputFormat;
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.RecordReader;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.IOException;
import static org.junit.Assert.assertEquals;
public class HoodieInputFormatTest {
private HoodieInputFormat inputFormat;
private JobConf jobConf;
@Before public void setUp() {
inputFormat = new HoodieInputFormat();
jobConf = new JobConf();
inputFormat.setConf(jobConf);
}
private HoodieInputFormat inputFormat;
private JobConf jobConf;
@Rule public TemporaryFolder basePath = new TemporaryFolder();
@Before
public void setUp() {
inputFormat = new HoodieInputFormat();
jobConf = new JobConf();
inputFormat.setConf(jobConf);
}
@Test public void testInputFormatLoad() throws IOException {
// initial commit
File partitionDir = InputFormatTestUtil.prepareDataset(basePath, 10, "100");
InputFormatTestUtil.commit(basePath, "100");
@Rule
public TemporaryFolder basePath = new TemporaryFolder();
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
@Test
public void testInputFormatLoad() throws IOException {
// initial commit
File partitionDir = InputFormatTestUtil.prepareDataset(basePath, 10, "100");
InputFormatTestUtil.commit(basePath, "100");
InputSplit[] inputSplits = inputFormat.getSplits(jobConf, 10);
assertEquals(10, inputSplits.length);
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
FileStatus[] files = inputFormat.listStatus(jobConf);
assertEquals(10, files.length);
}
InputSplit[] inputSplits = inputFormat.getSplits(jobConf, 10);
assertEquals(10, inputSplits.length);
@Test public void testInputFormatUpdates() throws IOException {
// initial commit
File partitionDir = InputFormatTestUtil.prepareDataset(basePath, 10, "100");
InputFormatTestUtil.commit(basePath, "100");
FileStatus[] files = inputFormat.listStatus(jobConf);
assertEquals(10, files.length);
}
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
@Test
public void testInputFormatUpdates() throws IOException {
// initial commit
File partitionDir = InputFormatTestUtil.prepareDataset(basePath, 10, "100");
InputFormatTestUtil.commit(basePath, "100");
FileStatus[] files = inputFormat.listStatus(jobConf);
assertEquals(10, files.length);
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
// update files
InputFormatTestUtil.simulateUpdates(partitionDir, "100", 5, "200", true);
// 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);
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);
}
FileStatus[] files = inputFormat.listStatus(jobConf);
assertEquals(10, files.length);
@Test public void testIncrementalSimple() throws IOException {
// initial commit
File partitionDir = InputFormatTestUtil.prepareDataset(basePath, 10, "100");
InputFormatTestUtil.commit(basePath, "100");
// update files
InputFormatTestUtil.simulateUpdates(partitionDir, "100", 5, "200", true);
// 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);
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);
}
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
@Test
public void testIncrementalSimple() throws IOException {
// initial commit
File partitionDir = InputFormatTestUtil.prepareDataset(basePath, 10, "100");
InputFormatTestUtil.commit(basePath, "100");
InputFormatTestUtil.setupIncremental(jobConf, "100", 1);
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
FileStatus[] files = inputFormat.listStatus(jobConf);
assertEquals(
"We should exclude commit 100 when returning incremental pull with start commit time as 100",
0, files.length);
}
InputFormatTestUtil.setupIncremental(jobConf, "100", 1);
@Test public void testIncrementalWithMultipleCommits() throws IOException {
// initial commit
File partitionDir = InputFormatTestUtil.prepareDataset(basePath, 10, "100");
InputFormatTestUtil.commit(basePath, "100");
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
// update files
InputFormatTestUtil.simulateUpdates(partitionDir, "100", 5, "200", false);
InputFormatTestUtil.commit(basePath, "200");
FileStatus[] files = inputFormat.listStatus(jobConf);
assertEquals(
"We should exclude commit 100 when returning incremental pull with start commit time as 100",
0, files.length);
}
InputFormatTestUtil.simulateUpdates(partitionDir, "100", 4, "300", false);
InputFormatTestUtil.commit(basePath, "300");
@Test
public void testIncrementalWithMultipleCommits() throws IOException {
// initial commit
File partitionDir = InputFormatTestUtil.prepareDataset(basePath, 10, "100");
InputFormatTestUtil.commit(basePath, "100");
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
// update files
InputFormatTestUtil.simulateUpdates(partitionDir, "100", 5, "200", false);
InputFormatTestUtil.commit(basePath, "200");
InputFormatTestUtil.simulateUpdates(partitionDir, "100", 3, "400", false);
InputFormatTestUtil.commit(basePath, "400");
InputFormatTestUtil.simulateUpdates(partitionDir, "100", 4, "300", false);
InputFormatTestUtil.commit(basePath, "300");
InputFormatTestUtil.simulateUpdates(partitionDir, "100", 2, "500", false);
InputFormatTestUtil.commit(basePath, "500");
InputFormatTestUtil.simulateUpdates(partitionDir, "100", 3, "400", false);
InputFormatTestUtil.commit(basePath, "400");
InputFormatTestUtil.simulateUpdates(partitionDir, "100", 1, "600", false);
InputFormatTestUtil.commit(basePath, "600");
InputFormatTestUtil.simulateUpdates(partitionDir, "100", 2, "500", false);
InputFormatTestUtil.commit(basePath, "500");
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);
InputFormatTestUtil.simulateUpdates(partitionDir, "100", 1, "600", false);
InputFormatTestUtil.commit(basePath, "600");
InputFormatTestUtil.setupIncremental(jobConf, "100", 3);
files = inputFormat.listStatus(jobConf);
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 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", 3);
files = inputFormat.listStatus(jobConf);
InputFormatTestUtil.setupIncremental(jobConf, "100", HoodieHiveUtil.MAX_COMMIT_ALL);
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 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);
}
InputFormatTestUtil.setupIncremental(jobConf, "100", HoodieHiveUtil.MAX_COMMIT_ALL);
files = inputFormat.listStatus(jobConf);
//TODO enable this after enabling predicate pushdown
public void testPredicatePushDown() throws IOException {
// initial commit
Schema schema = InputFormatTestUtil.readSchema("/sample1.avro");
String commit1 = "20160628071126";
File partitionDir =
InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, 10, commit1);
InputFormatTestUtil.commit(basePath, commit1);
// 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);
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);
}
// update 2 records in the original parquet file and save it as commit 200
String commit2 = "20160629193623";
InputFormatTestUtil.simulateParquetUpdates(partitionDir, schema, commit1, 10, 2, commit2);
InputFormatTestUtil.commit(basePath, commit2);
//TODO enable this after enabling predicate pushdown
public void testPredicatePushDown() throws IOException {
// initial commit
Schema schema = InputFormatTestUtil.readSchema("/sample1.avro");
String commit1 = "20160628071126";
File partitionDir =
InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, 10, commit1);
InputFormatTestUtil.commit(basePath, commit1);
// 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);
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);
// 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);
}
// update 2 records in the original parquet file and save it as commit 200
String commit2 = "20160629193623";
InputFormatTestUtil.simulateParquetUpdates(partitionDir, schema, commit1, 10, 2, commit2);
InputFormatTestUtil.commit(basePath, commit2);
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<Void, ArrayWritable>
recordReader = inputFormat.getRecordReader(split, jobConf, null);
Void key = recordReader.createKey();
ArrayWritable writable = recordReader.createValue();
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);
// 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);
}
while(recordReader.next(key, writable)) {
// writable returns an array with [field1, field2, _hoodie_commit_time, _hoodie_commit_seqno]
// Take the commit time and compare with the one we are interested in
if(commit.equals((writable.get()[2]).toString())) {
actualCount++;
}
totalCount++;
}
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<Void, ArrayWritable>
recordReader = inputFormat.getRecordReader(split, jobConf, null);
Void key = recordReader.createKey();
ArrayWritable writable = recordReader.createValue();
while (recordReader.next(key, writable)) {
// writable returns an array with [field1, field2, _hoodie_commit_time, _hoodie_commit_seqno]
// Take the commit time and compare with the one we are interested in
if (commit.equals((writable.get()[2]).toString())) {
actualCount++;
}
assertEquals(msg, expectedNumberOfRecordsInCommit, actualCount);
assertEquals(msg, totalExpected, totalCount);
totalCount++;
}
}
assertEquals(msg, expectedNumberOfRecordsInCommit, actualCount);
assertEquals(msg, totalExpected, totalCount);
}
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());
if (commit.equals(commitTs)) {
count++;
}
}
assertEquals(msg, expected, count);
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());
if (commit.equals(commitTs)) {
count++;
}
}
assertEquals(msg, expected, count);
}
}

View File

@@ -16,20 +16,10 @@
package com.uber.hoodie.hadoop;
import com.uber.hoodie.avro.MercifulJsonConverter;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.SchemaTestUtil;
import com.uber.hoodie.common.util.TestRecord;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.JobConf;
import org.apache.parquet.avro.AvroParquetWriter;
import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.FilenameFilter;
import java.io.IOException;
@@ -37,123 +27,139 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.JobConf;
import org.apache.parquet.avro.AvroParquetWriter;
import org.junit.rules.TemporaryFolder;
public class InputFormatTestUtil {
public static File prepareDataset(TemporaryFolder basePath, int numberOfFiles,
String commitNumber) throws IOException {
basePath.create();
HoodieTestUtils.init(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, 1, "fileid" + i));
dataFile.createNewFile();
public static File prepareDataset(TemporaryFolder basePath, int numberOfFiles,
String commitNumber) throws IOException {
basePath.create();
HoodieTestUtils.init(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, 1, "fileid" + i));
dataFile.createNewFile();
}
return partitionPath;
}
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) {
String commitTs = FSUtils.getCommitTime(name);
return originalCommit.equals(commitTs);
}
}));
if (randomize) {
Collections.shuffle(dataFiles);
}
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, 1, fileId));
dataFile.createNewFile();
}
}
public static void commit(TemporaryFolder basePath, String commitNumber) throws IOException {
// create the commit
new File(basePath.getRoot().toString() + "/.hoodie/", commitNumber + ".commit").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);
jobConf.set(modePropertyName, HoodieHiveUtil.INCREMENTAL_SCAN_MODE);
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);
jobConf.setInt(maxCommitPulls, numberOfCommitsToPull);
}
public static Schema readSchema(String location) throws IOException {
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 {
basePath.create();
HoodieTestUtils.init(basePath.getRoot().toString());
File partitionPath = basePath.newFolder("2016", "05", "01");
AvroParquetWriter parquetWriter;
for (int i = 0; i < numberOfFiles; i++) {
String fileId = FSUtils.makeDataFileName(commitNumber, 1, "fileid" + i);
File dataFile =
new File(partitionPath, fileId);
// dataFile.createNewFile();
parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()),
schema);
try {
for (GenericRecord record : generateAvroRecords(schema, numberOfRecords, commitNumber,
fileId)) {
parquetWriter.write(record);
}
return partitionPath;
} finally {
parquetWriter.close();
}
}
return partitionPath;
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) {
String commitTs = FSUtils.getCommitTime(name);
return originalCommit.equals(commitTs);
}
}));
if(randomize) {
Collections.shuffle(dataFiles);
}
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, 1, fileId));
dataFile.createNewFile();
}
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));
}
return records;
}
public static void simulateParquetUpdates(File directory, Schema schema, String originalCommit,
int totalNumberOfRecords, int numberOfRecordsToUpdate,
String newCommit) throws IOException {
File fileToUpdate = directory.listFiles(new FilenameFilter() {
@Override
public boolean accept(File dir, String name) {
return name.endsWith("parquet");
}
})[0];
String fileId = FSUtils.getFileId(fileToUpdate.getName());
File dataFile = new File(directory, FSUtils.makeDataFileName(newCommit, 1, fileId));
AvroParquetWriter parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()),
schema);
try {
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));
numberOfRecordsToUpdate--;
}
parquetWriter.write(record);
}
} finally {
parquetWriter.close();
}
public static void commit(TemporaryFolder basePath, String commitNumber) throws IOException {
// create the commit
new File(basePath.getRoot().toString() + "/.hoodie/", commitNumber + ".commit").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);
jobConf.set(modePropertyName, HoodieHiveUtil.INCREMENTAL_SCAN_MODE);
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);
jobConf.setInt(maxCommitPulls, numberOfCommitsToPull);
}
public static Schema readSchema(String location) throws IOException {
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 {
basePath.create();
HoodieTestUtils.init(basePath.getRoot().toString());
File partitionPath = basePath.newFolder("2016", "05", "01");
AvroParquetWriter parquetWriter;
for (int i = 0; i < numberOfFiles; i++) {
String fileId = FSUtils.makeDataFileName(commitNumber, 1, "fileid" + i);
File dataFile =
new File(partitionPath, fileId);
// dataFile.createNewFile();
parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()),
schema);
try {
for (GenericRecord record : generateAvroRecords(schema, numberOfRecords, commitNumber, fileId)) {
parquetWriter.write(record);
}
} finally {
parquetWriter.close();
}
}
return partitionPath;
}
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));
}
return records;
}
public static void simulateParquetUpdates(File directory, Schema schema, String originalCommit,
int totalNumberOfRecords, int numberOfRecordsToUpdate,
String newCommit) throws IOException {
File fileToUpdate = directory.listFiles(new FilenameFilter() {
@Override public boolean accept(File dir, String name) {
return name.endsWith("parquet");
}
})[0];
String fileId = FSUtils.getFileId(fileToUpdate.getName());
File dataFile = new File(directory, FSUtils.makeDataFileName(newCommit, 1, fileId));
AvroParquetWriter parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()),
schema);
try {
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));
numberOfRecordsToUpdate--;
}
parquetWriter.write(record);
}
} finally {
parquetWriter.close();
}
}
}
}

View File

@@ -15,63 +15,66 @@
*/
package com.uber.hoodie.hadoop;
import com.uber.hoodie.common.model.HoodieTestUtils;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import java.io.File;
import java.io.IOException;
import org.apache.hadoop.fs.Path;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.IOException;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
/**
*/
public class TestHoodieROTablePathFilter {
@Test
public void testHoodiePaths() throws IOException {
// Create a temp folder as the base path
HoodieTableMetaClient metaClient = HoodieTestUtils.initOnTemp();
String basePath = metaClient.getBasePath();
@Test
public void testHoodiePaths() throws IOException {
// Create a temp folder as the base path
HoodieTableMetaClient metaClient = HoodieTestUtils.initOnTemp();
String basePath = metaClient.getBasePath();
HoodieTestUtils.createCommitFiles(basePath, "001", "002");
HoodieTestUtils.createInflightCommitFiles(basePath, "003");
HoodieTestUtils.createCommitFiles(basePath, "001", "002");
HoodieTestUtils.createInflightCommitFiles(basePath, "003");
HoodieTestUtils.createDataFile(basePath, "2017/01/01", "001", "f1");
HoodieTestUtils.createDataFile(basePath, "2017/01/01", "001", "f2");
HoodieTestUtils.createDataFile(basePath, "2017/01/01", "001", "f3");
HoodieTestUtils.createDataFile(basePath, "2017/01/01", "002", "f2");
HoodieTestUtils.createDataFile(basePath, "2017/01/01", "003", "f3");
HoodieTestUtils.createDataFile(basePath, "2017/01/01", "001", "f1");
HoodieTestUtils.createDataFile(basePath, "2017/01/01", "001", "f2");
HoodieTestUtils.createDataFile(basePath, "2017/01/01", "001", "f3");
HoodieTestUtils.createDataFile(basePath, "2017/01/01", "002", "f2");
HoodieTestUtils.createDataFile(basePath, "2017/01/01", "003", "f3");
HoodieROTablePathFilter pathFilter = new HoodieROTablePathFilter();
Path partitionPath = new Path("file://" + basePath + File.separator + "2017/01/01");
assertTrue("Directories should be accepted", pathFilter.accept(partitionPath));
HoodieROTablePathFilter pathFilter = new HoodieROTablePathFilter();
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"))));
}
@Test
public void testNonHoodiePaths() throws IOException {
TemporaryFolder folder = new TemporaryFolder();
folder.create();
String basePath = folder.getRoot().getAbsolutePath();
HoodieROTablePathFilter pathFilter = new HoodieROTablePathFilter();
@Test
public void testNonHoodiePaths() throws IOException {
TemporaryFolder folder = new TemporaryFolder();
folder.create();
String basePath = folder.getRoot().getAbsolutePath();
HoodieROTablePathFilter pathFilter = new HoodieROTablePathFilter();
String path = basePath + File.separator + "nonhoodiefolder";
new File(path).mkdirs();
assertTrue(pathFilter.accept(new Path("file:///" + path)));
String path = basePath + File.separator + "nonhoodiefolder";
new File(path).mkdirs();
assertTrue(pathFilter.accept(new Path("file:///" + path)));
path = basePath + File.separator + "nonhoodiefolder/somefile";
new File(path).createNewFile();
assertTrue(pathFilter.accept(new Path("file:///" + path)));
}
path = basePath + File.separator + "nonhoodiefolder/somefile";
new File(path).createNewFile();
assertTrue(pathFilter.accept(new Path("file:///" + path)));
}
}

View File

@@ -19,6 +19,8 @@
package com.uber.hoodie.hadoop.realtime;
import static org.junit.Assert.assertTrue;
import com.google.common.collect.Maps;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.model.HoodieTableType;
@@ -30,6 +32,13 @@ import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.SchemaTestUtil;
import com.uber.hoodie.hadoop.InputFormatTestUtil;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.Path;
@@ -52,211 +61,234 @@ import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import static org.junit.Assert.assertTrue;
public class HoodieRealtimeRecordReaderTest {
private JobConf jobConf;
private JobConf jobConf;
@Before
public void setUp() {
jobConf = new JobConf();
@Before
public void setUp() {
jobConf = new JobConf();
}
@Rule
public TemporaryFolder basePath = new TemporaryFolder();
private HoodieLogFormat.Writer writeLogFile(File partitionDir, Schema schema, String fileId,
String baseCommit, String newCommit, int numberOfRecords)
throws InterruptedException, IOException {
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(new Path(partitionDir.getPath()))
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId)
.overBaseCommit(baseCommit).withFs(FSUtils.getFs()).build();
List<IndexedRecord> records = new ArrayList<>();
for (int i = 0; i < numberOfRecords; i++) {
records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, newCommit, "fileid0"));
}
Schema writeSchema = records.get(0).getSchema();
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, newCommit);
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, writeSchema, metadata);
writer = writer.appendBlock(dataBlock);
long size = writer.getCurrentSize();
return writer;
}
@Rule
public TemporaryFolder basePath = new TemporaryFolder();
@Test
public void testReader() throws Exception {
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
HoodieTestUtils
.initTableType(basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
String commitTime = "100";
File partitionDir = InputFormatTestUtil
.prepareParquetDataset(basePath, schema, 1, 100, commitTime);
InputFormatTestUtil.commit(basePath, commitTime);
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
private HoodieLogFormat.Writer writeLogFile(File partitionDir, Schema schema, String fileId,
String baseCommit, String newCommit, int numberOfRecords) throws InterruptedException,IOException {
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(new Path(partitionDir.getPath()))
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId)
.overBaseCommit(baseCommit).withFs(FSUtils.getFs()).build();
List<IndexedRecord> records = new ArrayList<>();
for(int i=0; i < numberOfRecords; i++) {
records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, newCommit, "fileid0"));
}
Schema writeSchema = records.get(0).getSchema();
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, newCommit);
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, writeSchema, metadata);
writer = writer.appendBlock(dataBlock);
long size = writer.getCurrentSize();
return writer;
}
// update files or generate new log file
String newCommitTime = "101";
HoodieLogFormat.Writer writer = writeLogFile(partitionDir, schema, "fileid0", commitTime,
newCommitTime, 100);
long size = writer.getCurrentSize();
writer.close();
assertTrue("block - size should be > 0", size > 0);
@Test
public void testReader() throws Exception {
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
HoodieTestUtils.initTableType(basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
String commitTime = "100";
File partitionDir = InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, 100, commitTime);
InputFormatTestUtil.commit(basePath, commitTime);
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
//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_" + commitTime + ".parquet"), 0, 1, jobConf), basePath.getRoot().getPath(),
Arrays.asList(logFilePath), newCommitTime);
// update files or generate new log file
String newCommitTime = "101";
HoodieLogFormat.Writer writer = writeLogFile(partitionDir, schema, "fileid0", commitTime, newCommitTime, 100);
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)
String logFilePath = writer.getLogFile().getPath().toString();
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(new FileSplit(new Path(partitionDir
+ "/fileid0_1_" + commitTime + ".parquet"),0,1,jobConf), basePath.getRoot().getPath(), Arrays.asList(logFilePath), newCommitTime);
//create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<Void, ArrayWritable> reader =
new MapredParquetInputFormat().
getRecordReader(new FileSplit(split.getPath(), 0,
FSUtils.getFs().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(","));
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
jobConf.set("partition_columns", "datestr");
//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
Void 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"
Assert.assertEquals(values[0].toString(), newCommitTime);
key = recordReader.createKey();
value = recordReader.createValue();
}
}
@Test
public void testReaderWithNestedAndComplexSchema() throws Exception {
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getComplexEvolvedSchema());
HoodieTestUtils.initTableType(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);
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);
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)
String logFilePath = writer.getLogFile().getPath().toString();
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(new FileSplit(new Path(partitionDir
+ "/fileid0_1_" + commitTime + ".parquet"),0,1,jobConf), basePath.getRoot().getPath(), Arrays.asList(logFilePath), newCommitTime);
//create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<Void, ArrayWritable> reader =
new MapredParquetInputFormat().
//create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<Void, ArrayWritable> reader =
new MapredParquetInputFormat().
getRecordReader(new FileSplit(split.getPath(), 0,
FSUtils.getFs().getLength(split.getPath()), (String[]) null), jobConf, null);
JobConf jobConf = new JobConf();
List<Schema.Field> fields = schema.getFields();
FSUtils.getFs().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(","));
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
jobConf.set("partition_columns", "datestr");
String names = fields.stream().map(f -> f.name()).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");
//validate record reader compaction
HoodieRealtimeRecordReader recordReader = new HoodieRealtimeRecordReader(split, jobConf,
reader);
// 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 the first 50 records should be updated, see above
Void key = recordReader.createKey();
ArrayWritable value = recordReader.createValue();
int numRecordsRead = 0;
while (recordReader.next(key, value)) {
int currentRecordNo = numRecordsRead;
++numRecordsRead;
Writable[] values = value.get();
String recordCommitTime;
//check if the record written is with latest commit, here "101"
if (numRecordsRead > numberOfLogRecords) {
recordCommitTime = commitTime;
} else {
recordCommitTime = newCommitTime;
}
String recordCommitTimeSuffix = "@" + recordCommitTime;
Assert.assertEquals(values[0].toString(), recordCommitTime);
key = recordReader.createKey();
value = recordReader.createValue();
// Assert type STRING
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 type INT
IntWritable intWritable = (IntWritable)values[8];
Assert.assertEquals("test value for field: favoriteIntNumber", intWritable.get(), currentRecordNo + recordCommitTime.hashCode());
// Assert type LONG
LongWritable longWritable = (LongWritable)values[9];
Assert.assertEquals("test value for field: favoriteNumber", longWritable.get(), currentRecordNo + recordCommitTime.hashCode());
// Assert type FLOAT
FloatWritable floatWritable = (FloatWritable)values[10];
Assert.assertEquals("test value for field: favoriteFloatNumber", floatWritable.get(), (float)((currentRecordNo + recordCommitTime.hashCode()) / 1024.0), 0);
// Assert type DOUBLE
DoubleWritable doubleWritable = (DoubleWritable)values[11];
Assert.assertEquals("test value for field: favoriteDoubleNumber", doubleWritable.get(), (currentRecordNo + recordCommitTime.hashCode()) / 1024.0, 0);
// Assert type MAP
ArrayWritable mapItem = (ArrayWritable)values[12];
Writable[] mapItemValues = mapItem.get();
ArrayWritable mapItemValue1 = (ArrayWritable)mapItemValues[0];
ArrayWritable mapItemValue2 = (ArrayWritable)mapItemValues[1];
Assert.assertEquals("test value for field: tags", mapItemValue1.get()[0].toString(), "mapItem1");
Assert.assertEquals("test value for field: tags", mapItemValue2.get()[0].toString(), "mapItem2");
ArrayWritable mapItemValue1value = (ArrayWritable)mapItemValue1.get()[1];
ArrayWritable mapItemValue2value = (ArrayWritable)mapItemValue2.get()[1];
Assert.assertEquals("test value for field: tags", mapItemValue1value.get().length, 2);
Assert.assertEquals("test value for field: tags", mapItemValue2value.get().length, 2);
Assert.assertEquals("test value for field: tags[\"mapItem1\"].item1", mapItemValue1value.get()[0].toString(), "item" + currentRecordNo);
Assert.assertEquals("test value for field: tags[\"mapItem2\"].item1", mapItemValue2value.get()[0].toString(), "item2" + currentRecordNo);
Assert.assertEquals("test value for field: tags[\"mapItem1\"].item2", mapItemValue1value.get()[1].toString(), "item" + currentRecordNo + recordCommitTimeSuffix);
Assert.assertEquals("test value for field: tags[\"mapItem2\"].item2", 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 type ARRAY
ArrayWritable arrayValue = (ArrayWritable)values[14];
Writable[] arrayValues = arrayValue.get();
for (int i = 0; i < arrayValues.length; i++) {
Assert.assertEquals("test value for field: stringArray", arrayValues[i].toString(), "stringArray" + i + recordCommitTimeSuffix);
}
}
//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
Void 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"
Assert.assertEquals(values[0].toString(), newCommitTime);
key = recordReader.createKey();
value = recordReader.createValue();
}
}
@Test
public void testReaderWithNestedAndComplexSchema() throws Exception {
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getComplexEvolvedSchema());
HoodieTestUtils
.initTableType(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);
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);
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)
String logFilePath = writer.getLogFile().getPath().toString();
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(new FileSplit(new Path(partitionDir
+ "/fileid0_1_" + commitTime + ".parquet"), 0, 1, jobConf), basePath.getRoot().getPath(),
Arrays.asList(logFilePath), newCommitTime);
//create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<Void, ArrayWritable> reader =
new MapredParquetInputFormat().
getRecordReader(new FileSplit(split.getPath(), 0,
FSUtils.getFs().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(","));
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, positions);
jobConf.set("partition_columns", "datestr");
// 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 the first 50 records should be updated, see above
Void key = recordReader.createKey();
ArrayWritable value = recordReader.createValue();
int numRecordsRead = 0;
while (recordReader.next(key, value)) {
int currentRecordNo = numRecordsRead;
++numRecordsRead;
Writable[] values = value.get();
String recordCommitTime;
//check if the record written is with latest commit, here "101"
if (numRecordsRead > numberOfLogRecords) {
recordCommitTime = commitTime;
} else {
recordCommitTime = newCommitTime;
}
String recordCommitTimeSuffix = "@" + recordCommitTime;
Assert.assertEquals(values[0].toString(), recordCommitTime);
key = recordReader.createKey();
value = recordReader.createValue();
// Assert type STRING
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 type INT
IntWritable intWritable = (IntWritable) values[8];
Assert.assertEquals("test value for field: favoriteIntNumber", intWritable.get(),
currentRecordNo + recordCommitTime.hashCode());
// Assert type LONG
LongWritable longWritable = (LongWritable) values[9];
Assert.assertEquals("test value for field: favoriteNumber", longWritable.get(),
currentRecordNo + recordCommitTime.hashCode());
// Assert type FLOAT
FloatWritable floatWritable = (FloatWritable) values[10];
Assert.assertEquals("test value for field: favoriteFloatNumber", floatWritable.get(),
(float) ((currentRecordNo + recordCommitTime.hashCode()) / 1024.0), 0);
// Assert type DOUBLE
DoubleWritable doubleWritable = (DoubleWritable) values[11];
Assert.assertEquals("test value for field: favoriteDoubleNumber", doubleWritable.get(),
(currentRecordNo + recordCommitTime.hashCode()) / 1024.0, 0);
// Assert type MAP
ArrayWritable mapItem = (ArrayWritable) values[12];
Writable[] mapItemValues = mapItem.get();
ArrayWritable mapItemValue1 = (ArrayWritable) mapItemValues[0];
ArrayWritable mapItemValue2 = (ArrayWritable) mapItemValues[1];
Assert.assertEquals("test value for field: tags", mapItemValue1.get()[0].toString(),
"mapItem1");
Assert.assertEquals("test value for field: tags", mapItemValue2.get()[0].toString(),
"mapItem2");
ArrayWritable mapItemValue1value = (ArrayWritable) mapItemValue1.get()[1];
ArrayWritable mapItemValue2value = (ArrayWritable) mapItemValue2.get()[1];
Assert.assertEquals("test value for field: tags", mapItemValue1value.get().length, 2);
Assert.assertEquals("test value for field: tags", mapItemValue2value.get().length, 2);
Assert.assertEquals("test value for field: tags[\"mapItem1\"].item1",
mapItemValue1value.get()[0].toString(), "item" + currentRecordNo);
Assert.assertEquals("test value for field: tags[\"mapItem2\"].item1",
mapItemValue2value.get()[0].toString(), "item2" + currentRecordNo);
Assert.assertEquals("test value for field: tags[\"mapItem1\"].item2",
mapItemValue1value.get()[1].toString(),
"item" + currentRecordNo + recordCommitTimeSuffix);
Assert.assertEquals("test value for field: tags[\"mapItem2\"].item2",
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 type ARRAY
ArrayWritable arrayValue = (ArrayWritable) values[14];
Writable[] arrayValues = arrayValue.get();
for (int i = 0; i < arrayValues.length; i++) {
Assert.assertEquals("test value for field: stringArray", arrayValues[i].toString(),
"stringArray" + i + recordCommitTimeSuffix);
}
}
}
}

View File

@@ -13,11 +13,9 @@
# See the License for the specific language governing permissions and
# limitations under the License.
#
log4j.rootLogger=WARN, A1
log4j.category.com.uber=INFO
log4j.category.org.apache.parquet.hadoop=WARN
# A1 is set to be a ConsoleAppender.
log4j.appender.A1=org.apache.log4j.ConsoleAppender
# A1 uses PatternLayout.