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
@@ -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 ?
|
||||
|
||||
Reference in New Issue
Block a user