CodeStyle formatting to conform to basic Checkstyle rules.
The code-style rules follow google style with some changes: 1. Increase line length from 100 to 120 2. Disable JavaDoc related checkstyles as this needs more manual work. Both source and test code are checked for code-style
This commit is contained in:
committed by
vinoth chandar
parent
987f5d6b96
commit
788e4f2d2e
@@ -23,8 +23,7 @@ import org.apache.log4j.Logger;
|
||||
|
||||
public class HoodieHiveUtil {
|
||||
|
||||
public static final Logger LOG =
|
||||
LogManager.getLogger(HoodieHiveUtil.class);
|
||||
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";
|
||||
|
||||
@@ -71,8 +71,7 @@ import parquet.io.api.Binary;
|
||||
* Hoodie/Non-Hoodie datasets
|
||||
*/
|
||||
@UseFileSplitsFromInputFormat
|
||||
public class HoodieInputFormat extends MapredParquetInputFormat
|
||||
implements Configurable {
|
||||
public class HoodieInputFormat extends MapredParquetInputFormat implements Configurable {
|
||||
|
||||
public static final Log LOG = LogFactory.getLog(HoodieInputFormat.class);
|
||||
|
||||
@@ -99,7 +98,8 @@ public class HoodieInputFormat extends MapredParquetInputFormat
|
||||
}
|
||||
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
|
||||
// Get all commits, delta commits, compactions, as all of them produce a base parquet file
|
||||
// today
|
||||
HoodieTimeline timeline = metadata.getActiveTimeline().getCommitsTimeline()
|
||||
.filterCompletedInstants();
|
||||
TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metadata,
|
||||
@@ -112,19 +112,16 @@ public class HoodieInputFormat extends MapredParquetInputFormat
|
||||
// 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)
|
||||
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());
|
||||
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()
|
||||
@@ -171,13 +168,13 @@ public class HoodieInputFormat extends MapredParquetInputFormat
|
||||
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 "."
|
||||
//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()))) {
|
||||
.toString().contains(nonHoodieBasePath)) || (metadata != null && !status.getPath()
|
||||
.toString().contains(metadata.getBasePath()))) {
|
||||
try {
|
||||
metadata = getTableMetaClient(status.getPath().getFileSystem(conf),
|
||||
status.getPath().getParent());
|
||||
@@ -185,8 +182,7 @@ public class HoodieInputFormat extends MapredParquetInputFormat
|
||||
} catch (InvalidDatasetException e) {
|
||||
LOG.info("Handling a non-hoodie path " + status.getPath());
|
||||
metadata = null;
|
||||
nonHoodieBasePath =
|
||||
status.getPath().getParent().toString();
|
||||
nonHoodieBasePath = status.getPath().getParent().toString();
|
||||
}
|
||||
if (!grouped.containsKey(metadata)) {
|
||||
grouped.put(metadata, new ArrayList<>());
|
||||
@@ -209,17 +205,17 @@ public class HoodieInputFormat extends MapredParquetInputFormat
|
||||
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();
|
||||
// String mode = HoodieHiveUtil.readMode(job, tableName);
|
||||
// FileSplit fileSplit = (FileSplit) split;
|
||||
// HoodieTableMetadata metadata = getTableMetadata(fileSplit.getPath().getParent());
|
||||
// String tableName = metadata.getTableName();
|
||||
// String mode = HoodieHiveUtil.readMode(job, tableName);
|
||||
|
||||
// if (HoodieHiveUtil.INCREMENTAL_SCAN_MODE.equals(mode)) {
|
||||
// FilterPredicate predicate = constructHoodiePredicate(job, tableName, split);
|
||||
// LOG.info("Setting parquet predicate push down as " + predicate);
|
||||
// ParquetInputFormat.setFilterPredicate(job, predicate);
|
||||
// if (HoodieHiveUtil.INCREMENTAL_SCAN_MODE.equals(mode)) {
|
||||
// FilterPredicate predicate = constructHoodiePredicate(job, tableName, split);
|
||||
// LOG.info("Setting parquet predicate push down as " + predicate);
|
||||
// ParquetInputFormat.setFilterPredicate(job, predicate);
|
||||
//clearOutExistingPredicate(job);
|
||||
// }
|
||||
// }
|
||||
return super.getRecordReader(split, job, reporter);
|
||||
}
|
||||
|
||||
@@ -236,9 +232,8 @@ public class HoodieInputFormat extends MapredParquetInputFormat
|
||||
* `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 {
|
||||
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);
|
||||
@@ -262,21 +257,19 @@ public class HoodieInputFormat extends MapredParquetInputFormat
|
||||
|| columnNamesString.isEmpty()) {
|
||||
return null;
|
||||
} else {
|
||||
SearchArgument sarg =
|
||||
SearchArgumentFactory.create(Utilities.deserializeExpression(serializedPushdown));
|
||||
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());
|
||||
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);
|
||||
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;
|
||||
|
||||
@@ -13,6 +13,7 @@
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.hadoop;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
@@ -37,12 +38,12 @@ import org.apache.hadoop.fs.PathFilter;
|
||||
/**
|
||||
* Given a path is a part of - Hoodie dataset = accepts ONLY the latest version of each path -
|
||||
* Non-Hoodie dataset = then always accept
|
||||
*
|
||||
* <p>
|
||||
* We can set this filter, on a query engine's Hadoop Config and if it respects path filters, then
|
||||
* you should be able to query both hoodie and non-hoodie datasets as you would normally do.
|
||||
*
|
||||
* hadoopConf.setClass("mapreduce.input.pathFilter.class", com.uber.hoodie.hadoop.HoodieROTablePathFilter.class,
|
||||
* org.apache.hadoop.fs.PathFilter.class)
|
||||
* <p>
|
||||
* hadoopConf.setClass("mapreduce.input.pathFilter.class", com.uber.hoodie.hadoop
|
||||
* .HoodieROTablePathFilter.class, org.apache.hadoop.fs.PathFilter.class)
|
||||
*/
|
||||
public class HoodieROTablePathFilter implements PathFilter, Serializable {
|
||||
|
||||
@@ -104,8 +105,7 @@ public class HoodieROTablePathFilter implements PathFilter, Serializable {
|
||||
|
||||
if (hoodiePathCache.containsKey(folder.toString())) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(String.format("%s Hoodie path checked against cache, accept => %s \n",
|
||||
path,
|
||||
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);
|
||||
@@ -123,37 +123,33 @@ public class HoodieROTablePathFilter implements PathFilter, Serializable {
|
||||
|
||||
if (baseDir != null) {
|
||||
try {
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs.getConf(), baseDir.toString());
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(),
|
||||
baseDir.toString());
|
||||
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient,
|
||||
metaClient.getActiveTimeline().getCommitTimeline()
|
||||
.filterCompletedInstants(),
|
||||
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants(),
|
||||
fs.listStatus(folder));
|
||||
List<HoodieDataFile> latestFiles = fsView
|
||||
.getLatestDataFiles()
|
||||
List<HoodieDataFile> latestFiles = fsView.getLatestDataFiles()
|
||||
.collect(Collectors.toList());
|
||||
// populate the cache
|
||||
if (!hoodiePathCache.containsKey(folder.toString())) {
|
||||
hoodiePathCache.put(folder.toString(), new HashSet<>());
|
||||
}
|
||||
LOG.info("Based on hoodie metadata from base path: " + baseDir.toString() +
|
||||
", caching " + latestFiles.size() + " files under " + folder);
|
||||
LOG.info("Based on hoodie metadata from base path: " + baseDir.toString() + ", caching "
|
||||
+ latestFiles.size() + " files under " + folder);
|
||||
for (HoodieDataFile lfile : latestFiles) {
|
||||
hoodiePathCache.get(folder.toString()).add(new Path(lfile.getPath()));
|
||||
}
|
||||
|
||||
// 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,
|
||||
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()));
|
||||
LOG.debug(String.format("(1) Caching non-hoodie path under %s \n", folder.toString()));
|
||||
}
|
||||
nonHoodiePathCache.add(folder.toString());
|
||||
return true;
|
||||
|
||||
@@ -79,8 +79,7 @@ public class HoodieRealtimeInputFormat extends HoodieInputFormat implements Conf
|
||||
// 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 -> {
|
||||
.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();
|
||||
@@ -97,7 +96,8 @@ public class HoodieRealtimeInputFormat extends HoodieInputFormat implements Conf
|
||||
}
|
||||
}));
|
||||
|
||||
// for all unique split parents, obtain all delta files based on delta commit timeline, grouped on file id
|
||||
// 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
|
||||
@@ -119,14 +119,13 @@ public class HoodieRealtimeInputFormat extends HoodieInputFormat implements Conf
|
||||
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.ROLLBACK_ACTION,
|
||||
HoodieTimeline.DELTA_COMMIT_ACTION))
|
||||
.filterCompletedInstants().lastInstant().get().getTimestamp();
|
||||
.map(logFile -> logFile.getPath().toString()).collect(Collectors.toList());
|
||||
// Get the maxCommit from the last delta or compaction or commit - when
|
||||
// bootstrapped from COW table
|
||||
String maxCommitTime = metaClient.getActiveTimeline().getTimelineOfActions(
|
||||
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION,
|
||||
HoodieTimeline.DELTA_COMMIT_ACTION)).filterCompletedInstants().lastInstant()
|
||||
.get().getTimestamp();
|
||||
rtSplits.add(
|
||||
new HoodieRealtimeFileSplit(split, metaClient.getBasePath(), logFilePaths,
|
||||
maxCommitTime));
|
||||
@@ -147,7 +146,8 @@ public class HoodieRealtimeInputFormat extends HoodieInputFormat implements Conf
|
||||
|
||||
@Override
|
||||
public FileStatus[] listStatus(JobConf job) throws IOException {
|
||||
// Call the HoodieInputFormat::listStatus to obtain all latest parquet files, based on commit timeline.
|
||||
// Call the HoodieInputFormat::listStatus to obtain all latest parquet files, based on commit
|
||||
// timeline.
|
||||
return super.listStatus(job);
|
||||
}
|
||||
|
||||
@@ -170,12 +170,11 @@ public class HoodieRealtimeInputFormat extends HoodieInputFormat implements Conf
|
||||
|
||||
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_NAMES_CONF_STR, readColNamesPrefix + fieldName);
|
||||
conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, readColIdsPrefix + fieldIndex);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(String.format("Adding extra column " + fieldName
|
||||
+ ", to enable log merging cols (%s) ids (%s) ",
|
||||
LOG.debug(String.format(
|
||||
"Adding extra column " + fieldName + ", to enable log merging cols (%s) ids (%s) ",
|
||||
conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR),
|
||||
conf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR)));
|
||||
}
|
||||
@@ -189,15 +188,14 @@ public class HoodieRealtimeInputFormat extends HoodieInputFormat implements Conf
|
||||
HOODIE_RECORD_KEY_COL_POS);
|
||||
configuration = addProjectionField(configuration, HoodieRecord.COMMIT_TIME_METADATA_FIELD,
|
||||
HOODIE_COMMIT_TIME_COL_POS);
|
||||
configuration = addProjectionField(configuration,
|
||||
HoodieRecord.PARTITION_PATH_METADATA_FIELD, HOODIE_PARTITION_PATH_COL_POS);
|
||||
configuration = addProjectionField(configuration, HoodieRecord.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 {
|
||||
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
|
||||
|
||||
@@ -73,8 +73,10 @@ public class HoodieRealtimeRecordReader implements RecordReader<Void, ArrayWrita
|
||||
public static final String DEFAULT_COMPACTION_MEMORY_FRACTION = "0.75";
|
||||
|
||||
// used to choose a trade off between IO vs Memory when performing compaction process
|
||||
// Depending on outputfile size and memory provided, choose true to avoid OOM for large file size + small memory
|
||||
public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP = "compaction.lazy.block.read.enabled";
|
||||
// Depending on outputfile size and memory provided, choose true to avoid OOM for large file
|
||||
// size + small memory
|
||||
public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP =
|
||||
"compaction.lazy.block" + ".read.enabled";
|
||||
public static final String DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED = "true";
|
||||
|
||||
public static final Log LOG = LogFactory.getLog(HoodieRealtimeRecordReader.class);
|
||||
@@ -82,8 +84,7 @@ public class HoodieRealtimeRecordReader implements RecordReader<Void, ArrayWrita
|
||||
private final HashMap<String, ArrayWritable> deltaRecordMap;
|
||||
private final MessageType baseFileSchema;
|
||||
|
||||
public HoodieRealtimeRecordReader(HoodieRealtimeFileSplit split,
|
||||
JobConf job,
|
||||
public HoodieRealtimeRecordReader(HoodieRealtimeFileSplit split, JobConf job,
|
||||
RecordReader<Void, ArrayWritable> realReader) {
|
||||
this.split = split;
|
||||
this.jobConf = job;
|
||||
@@ -106,11 +107,9 @@ public class HoodieRealtimeRecordReader implements RecordReader<Void, ArrayWrita
|
||||
*/
|
||||
private static MessageType readSchema(Configuration conf, Path parquetFilePath) {
|
||||
try {
|
||||
return ParquetFileReader.readFooter(conf, parquetFilePath).getFileMetaData()
|
||||
.getSchema();
|
||||
return ParquetFileReader.readFooter(conf, parquetFilePath).getFileMetaData().getSchema();
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to read footer for parquet " + parquetFilePath,
|
||||
e);
|
||||
throw new HoodieIOException("Failed to read footer for parquet " + parquetFilePath, e);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -125,27 +124,27 @@ public class HoodieRealtimeRecordReader implements RecordReader<Void, ArrayWrita
|
||||
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
|
||||
// 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.getPath().toString(), jobConf),
|
||||
split.getBasePath(),
|
||||
split.getDeltaFilePaths(),
|
||||
readerSchema, split.getMaxCommitTime(),
|
||||
(long) Math.ceil(Double.valueOf(jobConf.get(COMPACTION_MEMORY_FRACTION_PROP, DEFAULT_COMPACTION_MEMORY_FRACTION))
|
||||
*jobConf.getMemoryForMapTask()),
|
||||
Boolean.valueOf(jobConf.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)), false);
|
||||
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.getPath().toString(), jobConf), split.getBasePath(),
|
||||
split.getDeltaFilePaths(), readerSchema, split.getMaxCommitTime(), (long) Math.ceil(Double
|
||||
.valueOf(jobConf.get(COMPACTION_MEMORY_FRACTION_PROP, DEFAULT_COMPACTION_MEMORY_FRACTION))
|
||||
* jobConf.getMemoryForMapTask()), Boolean.valueOf(jobConf
|
||||
.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)),
|
||||
false);
|
||||
// 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)
|
||||
// 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();
|
||||
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.
|
||||
// we assume, a later safe record in the log, is newer than what we have in the map &
|
||||
// replace it.
|
||||
// TODO : handle deletes here
|
||||
ArrayWritable aWritable = (ArrayWritable) avroToArrayWritable(rec, writerSchema);
|
||||
deltaRecordMap.put(key, aWritable);
|
||||
@@ -180,14 +179,13 @@ public class HoodieRealtimeRecordReader implements RecordReader<Void, ArrayWrita
|
||||
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());
|
||||
.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));
|
||||
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++) {
|
||||
@@ -287,26 +285,28 @@ public class HoodieRealtimeRecordReader implements RecordReader<Void, ArrayWrita
|
||||
}
|
||||
case FIXED:
|
||||
return new BytesWritable(((GenericFixed) value).bytes());
|
||||
default:
|
||||
return null;
|
||||
}
|
||||
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
|
||||
// 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)
|
||||
// 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))));
|
||||
LOG.debug(String.format("key %s, base values: %s, log values: %s", key,
|
||||
arrayWritableToString(arrayWritable), arrayWritableToString(deltaRecordMap.get(key))));
|
||||
}
|
||||
if (deltaRecordMap.containsKey(key)) {
|
||||
// TODO(NA): Invoke preCombine here by converting arrayWritable to Avro ?
|
||||
|
||||
@@ -16,7 +16,6 @@
|
||||
|
||||
package com.uber.hoodie.hadoop;
|
||||
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.lang.annotation.Annotation;
|
||||
|
||||
@@ -82,17 +82,17 @@ public class HoodieInputFormatTest {
|
||||
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);
|
||||
"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);
|
||||
"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);
|
||||
"5 files have been updated to commit 200. We should see 5 files from commit 100 and 5 "
|
||||
+ "files from 200 commit", files, "100", 5);
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -108,8 +108,8 @@ public class HoodieInputFormatTest {
|
||||
|
||||
FileStatus[] files = inputFormat.listStatus(jobConf);
|
||||
assertEquals(
|
||||
"We should exclude commit 100 when returning incremental pull with start commit time as 100",
|
||||
0, files.length);
|
||||
"We should exclude commit 100 when returning incremental pull with start commit time as "
|
||||
+ "100", 0, files.length);
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -146,8 +146,8 @@ public class HoodieInputFormatTest {
|
||||
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);
|
||||
"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",
|
||||
@@ -159,23 +159,18 @@ public class HoodieInputFormatTest {
|
||||
files = inputFormat.listStatus(jobConf);
|
||||
|
||||
assertEquals(
|
||||
"Pulling all commits from 100, should get us the 1 file from each of 200,300,400,500,400 commits",
|
||||
5, files.length);
|
||||
ensureFilesInCommit(
|
||||
"Pulling all commits from 100, should get us the 1 files from 600 commit", files, "600",
|
||||
1);
|
||||
ensureFilesInCommit(
|
||||
"Pulling all commits from 100, should get us the 1 files from 500 commit", files, "500",
|
||||
1);
|
||||
ensureFilesInCommit(
|
||||
"Pulling all commits from 100, should get us the 1 files from 400 commit", files, "400",
|
||||
1);
|
||||
ensureFilesInCommit(
|
||||
"Pulling all commits from 100, should get us the 1 files from 300 commit", files, "300",
|
||||
1);
|
||||
ensureFilesInCommit(
|
||||
"Pulling all commits from 100, should get us the 1 files from 200 commit", files, "200",
|
||||
1);
|
||||
"Pulling all commits from 100, should get us the 1 file from each of 200,300,400,500,400 "
|
||||
+ "commits", 5, files.length);
|
||||
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 600 commit",
|
||||
files, "600", 1);
|
||||
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 500 commit",
|
||||
files, "500", 1);
|
||||
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 400 commit",
|
||||
files, "400", 1);
|
||||
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 300 commit",
|
||||
files, "300", 1);
|
||||
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 200 commit",
|
||||
files, "200", 1);
|
||||
}
|
||||
|
||||
//TODO enable this after enabling predicate pushdown
|
||||
@@ -183,8 +178,7 @@ public class HoodieInputFormatTest {
|
||||
// initial commit
|
||||
Schema schema = InputFormatTestUtil.readSchema("/sample1.avro");
|
||||
String commit1 = "20160628071126";
|
||||
File partitionDir =
|
||||
InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, 10, commit1);
|
||||
File partitionDir = InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, 10, commit1);
|
||||
InputFormatTestUtil.commit(basePath, commit1);
|
||||
// Add the paths
|
||||
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
|
||||
@@ -212,19 +206,20 @@ public class HoodieInputFormatTest {
|
||||
commit2, 2, 10);
|
||||
}
|
||||
|
||||
private void ensureRecordsInCommit(String msg, String commit,
|
||||
int expectedNumberOfRecordsInCommit, int totalExpected) throws IOException {
|
||||
private void ensureRecordsInCommit(String msg, String commit, int expectedNumberOfRecordsInCommit,
|
||||
int totalExpected) throws IOException {
|
||||
int actualCount = 0;
|
||||
int totalCount = 0;
|
||||
InputSplit[] splits = inputFormat.getSplits(jobConf, 1);
|
||||
for (InputSplit split : splits) {
|
||||
RecordReader<Void, ArrayWritable>
|
||||
recordReader = inputFormat.getRecordReader(split, jobConf, null);
|
||||
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]
|
||||
// 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++;
|
||||
|
||||
@@ -42,16 +42,15 @@ public class InputFormatTestUtil {
|
||||
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.getRoot().toString());
|
||||
File partitionPath = basePath.newFolder("2016", "05", "01");
|
||||
for (int i = 0; i < numberOfFiles; i++) {
|
||||
File dataFile =
|
||||
new File(partitionPath, FSUtils.makeDataFileName(commitNumber, 1, "fileid" + 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 {
|
||||
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) {
|
||||
@@ -62,8 +61,8 @@ public class InputFormatTestUtil {
|
||||
if (randomize) {
|
||||
Collections.shuffle(dataFiles);
|
||||
}
|
||||
List<File> toUpdateList =
|
||||
dataFiles.subList(0, Math.min(numberOfFilesUpdated, dataFiles.size()));
|
||||
List<File> toUpdateList = dataFiles
|
||||
.subList(0, Math.min(numberOfFilesUpdated, dataFiles.size()));
|
||||
for (File file : toUpdateList) {
|
||||
String fileId = FSUtils.getFileId(file.getName());
|
||||
File dataFile = new File(directory, FSUtils.makeDataFileName(newCommit, 1, fileId));
|
||||
@@ -78,8 +77,8 @@ public class InputFormatTestUtil {
|
||||
|
||||
public static void setupIncremental(JobConf jobConf, String startCommit,
|
||||
int numberOfCommitsToPull) {
|
||||
String modePropertyName = String.format(HoodieHiveUtil.HOODIE_CONSUME_MODE_PATTERN,
|
||||
HoodieTestUtils.RAW_TRIPS_TEST_NAME);
|
||||
String modePropertyName = String
|
||||
.format(HoodieHiveUtil.HOODIE_CONSUME_MODE_PATTERN, HoodieTestUtils.RAW_TRIPS_TEST_NAME);
|
||||
jobConf.set(modePropertyName, HoodieHiveUtil.INCREMENTAL_SCAN_MODE);
|
||||
|
||||
String startCommitTimestampName = String
|
||||
@@ -96,19 +95,16 @@ public class InputFormatTestUtil {
|
||||
}
|
||||
|
||||
public static File prepareParquetDataset(TemporaryFolder basePath, Schema schema,
|
||||
int numberOfFiles, int numberOfRecords,
|
||||
String commitNumber) throws IOException {
|
||||
int numberOfFiles, int numberOfRecords, String commitNumber) throws IOException {
|
||||
basePath.create();
|
||||
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), 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);
|
||||
File dataFile = new File(partitionPath, fileId);
|
||||
// dataFile.createNewFile();
|
||||
parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()),
|
||||
schema);
|
||||
parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()), schema);
|
||||
try {
|
||||
for (GenericRecord record : generateAvroRecords(schema, numberOfRecords, commitNumber,
|
||||
fileId)) {
|
||||
@@ -132,8 +128,7 @@ public class InputFormatTestUtil {
|
||||
}
|
||||
|
||||
public static void simulateParquetUpdates(File directory, Schema schema, String originalCommit,
|
||||
int totalNumberOfRecords, int numberOfRecordsToUpdate,
|
||||
String newCommit) throws IOException {
|
||||
int totalNumberOfRecords, int numberOfRecordsToUpdate, String newCommit) throws IOException {
|
||||
File fileToUpdate = directory.listFiles(new FilenameFilter() {
|
||||
@Override
|
||||
public boolean accept(File dir, String name) {
|
||||
@@ -145,8 +140,8 @@ public class InputFormatTestUtil {
|
||||
AvroParquetWriter parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()),
|
||||
schema);
|
||||
try {
|
||||
for (GenericRecord record : generateAvroRecords(schema, totalNumberOfRecords,
|
||||
originalCommit, fileId)) {
|
||||
for (GenericRecord record : generateAvroRecords(schema, totalNumberOfRecords, originalCommit,
|
||||
fileId)) {
|
||||
if (numberOfRecordsToUpdate > 0) {
|
||||
// update this record
|
||||
record.put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, newCommit);
|
||||
|
||||
@@ -13,6 +13,7 @@
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.hadoop;
|
||||
|
||||
import static org.junit.Assert.assertFalse;
|
||||
|
||||
@@ -18,7 +18,6 @@
|
||||
|
||||
package com.uber.hoodie.hadoop.realtime;
|
||||
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
@@ -85,9 +84,7 @@ public class HoodieRealtimeRecordReaderTest {
|
||||
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(new Path(partitionDir.getPath()))
|
||||
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId)
|
||||
.overBaseCommit(baseCommit)
|
||||
.withFs(fs)
|
||||
.build();
|
||||
.overBaseCommit(baseCommit).withFs(fs).build();
|
||||
List<IndexedRecord> records = new ArrayList<>();
|
||||
for (int i = 0; i < numberOfRecords; i++) {
|
||||
records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, newCommit, "fileid0"));
|
||||
@@ -106,8 +103,8 @@ public class HoodieRealtimeRecordReaderTest {
|
||||
public void testReader() throws Exception {
|
||||
// initial commit
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
|
||||
HoodieTestUtils
|
||||
.initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
|
||||
HoodieTestUtils.initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(),
|
||||
HoodieTableType.MERGE_ON_READ);
|
||||
String commitTime = "100";
|
||||
File partitionDir = InputFormatTestUtil
|
||||
.prepareParquetDataset(basePath, schema, 1, 100, commitTime);
|
||||
@@ -125,15 +122,15 @@ public class HoodieRealtimeRecordReaderTest {
|
||||
|
||||
//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);
|
||||
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, fs.getLength(split.getPath()),
|
||||
(String[]) null), jobConf, null);
|
||||
new MapredParquetInputFormat().getRecordReader(
|
||||
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null),
|
||||
jobConf, null);
|
||||
JobConf jobConf = new JobConf();
|
||||
List<Schema.Field> fields = schema.getFields();
|
||||
String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(","));
|
||||
@@ -164,8 +161,8 @@ public class HoodieRealtimeRecordReaderTest {
|
||||
public void testReaderWithNestedAndComplexSchema() throws Exception {
|
||||
// initial commit
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getComplexEvolvedSchema());
|
||||
HoodieTestUtils
|
||||
.initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
|
||||
HoodieTestUtils.initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(),
|
||||
HoodieTableType.MERGE_ON_READ);
|
||||
String commitTime = "100";
|
||||
int numberOfRecords = 100;
|
||||
int numberOfLogRecords = numberOfRecords / 2;
|
||||
@@ -185,15 +182,15 @@ public class HoodieRealtimeRecordReaderTest {
|
||||
|
||||
//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);
|
||||
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, fs.getLength(split.getPath()),
|
||||
(String[]) null), jobConf, null);
|
||||
new MapredParquetInputFormat().getRecordReader(
|
||||
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null),
|
||||
jobConf, null);
|
||||
JobConf jobConf = new JobConf();
|
||||
List<Schema.Field> fields = schema.getFields();
|
||||
|
||||
|
||||
Reference in New Issue
Block a user