General enhancements
This commit is contained in:
committed by
vinoth chandar
parent
30c5f8b7bd
commit
6946dd7557
@@ -143,9 +143,8 @@ public class StatsCommand implements CommandMarker {
|
|||||||
Snapshot s = globalHistogram.getSnapshot();
|
Snapshot s = globalHistogram.getSnapshot();
|
||||||
rows.add(printFileSizeHistogram("ALL", s));
|
rows.add(printFileSizeHistogram("ALL", s));
|
||||||
|
|
||||||
Function<Object, String> converterFunction = entry -> {
|
Function<Object, String> converterFunction = entry ->
|
||||||
return NumericUtils.humanReadableByteCount((Double.valueOf(entry.toString())));
|
NumericUtils.humanReadableByteCount((Double.valueOf(entry.toString())));
|
||||||
};
|
|
||||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||||
fieldNameToConverterMap.put("Min", converterFunction);
|
fieldNameToConverterMap.put("Min", converterFunction);
|
||||||
fieldNameToConverterMap.put("10th", converterFunction);
|
fieldNameToConverterMap.put("10th", converterFunction);
|
||||||
|
|||||||
@@ -77,7 +77,7 @@ class DedupeSparkJob(basePath: String,
|
|||||||
|
|
||||||
val metadata = new HoodieTableMetaClient(fs.getConf, basePath)
|
val metadata = new HoodieTableMetaClient(fs.getConf, basePath)
|
||||||
|
|
||||||
val allFiles = fs.listStatus(new org.apache.hadoop.fs.Path(s"${basePath}/${duplicatedPartitionPath}"))
|
val allFiles = fs.listStatus(new org.apache.hadoop.fs.Path(s"$basePath/$duplicatedPartitionPath"))
|
||||||
val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles)
|
val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles)
|
||||||
val latestFiles: java.util.List[HoodieDataFile] = fsView.getLatestDataFiles().collect(Collectors.toList[HoodieDataFile]())
|
val latestFiles: java.util.List[HoodieDataFile] = fsView.getLatestDataFiles().collect(Collectors.toList[HoodieDataFile]())
|
||||||
val filteredStatuses = latestFiles.map(f => f.getPath)
|
val filteredStatuses = latestFiles.map(f => f.getPath)
|
||||||
@@ -92,8 +92,8 @@ class DedupeSparkJob(basePath: String,
|
|||||||
val dupeDataSql =
|
val dupeDataSql =
|
||||||
s"""
|
s"""
|
||||||
SELECT `_hoodie_record_key`, `_hoodie_partition_path`, `_hoodie_file_name`, `_hoodie_commit_time`
|
SELECT `_hoodie_record_key`, `_hoodie_partition_path`, `_hoodie_file_name`, `_hoodie_commit_time`
|
||||||
FROM ${tmpTableName} h
|
FROM $tmpTableName h
|
||||||
JOIN ${dedupeTblName} d
|
JOIN $dedupeTblName d
|
||||||
ON h.`_hoodie_record_key` = d.dupe_key
|
ON h.`_hoodie_record_key` = d.dupe_key
|
||||||
"""
|
"""
|
||||||
val dupeMap = sqlContext.sql(dupeDataSql).collectAsList().groupBy(r => r.getString(0))
|
val dupeMap = sqlContext.sql(dupeDataSql).collectAsList().groupBy(r => r.getString(0))
|
||||||
@@ -101,8 +101,7 @@ class DedupeSparkJob(basePath: String,
|
|||||||
|
|
||||||
// Mark all files except the one with latest commits for deletion
|
// Mark all files except the one with latest commits for deletion
|
||||||
dupeMap.foreach(rt => {
|
dupeMap.foreach(rt => {
|
||||||
val key = rt._1
|
val (key, rows) = rt
|
||||||
val rows = rt._2
|
|
||||||
var maxCommit = -1L
|
var maxCommit = -1L
|
||||||
|
|
||||||
rows.foreach(r => {
|
rows.foreach(r => {
|
||||||
@@ -129,7 +128,7 @@ class DedupeSparkJob(basePath: String,
|
|||||||
def fixDuplicates(dryRun: Boolean = true) = {
|
def fixDuplicates(dryRun: Boolean = true) = {
|
||||||
val metadata = new HoodieTableMetaClient(fs.getConf, basePath)
|
val metadata = new HoodieTableMetaClient(fs.getConf, basePath)
|
||||||
|
|
||||||
val allFiles = fs.listStatus(new Path(s"${basePath}/${duplicatedPartitionPath}"))
|
val allFiles = fs.listStatus(new Path(s"$basePath/$duplicatedPartitionPath"))
|
||||||
val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles)
|
val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles)
|
||||||
|
|
||||||
val latestFiles: java.util.List[HoodieDataFile] = fsView.getLatestDataFiles().collect(Collectors.toList[HoodieDataFile]())
|
val latestFiles: java.util.List[HoodieDataFile] = fsView.getLatestDataFiles().collect(Collectors.toList[HoodieDataFile]())
|
||||||
@@ -138,30 +137,28 @@ class DedupeSparkJob(basePath: String,
|
|||||||
val dupeFixPlan = planDuplicateFix()
|
val dupeFixPlan = planDuplicateFix()
|
||||||
|
|
||||||
// 1. Copy all latest files into the temp fix path
|
// 1. Copy all latest files into the temp fix path
|
||||||
fileNameToPathMap.foreach { case (fileName, filePath) => {
|
fileNameToPathMap.foreach { case (fileName, filePath) =>
|
||||||
val badSuffix = if (dupeFixPlan.contains(fileName)) ".bad" else ""
|
val badSuffix = if (dupeFixPlan.contains(fileName)) ".bad" else ""
|
||||||
val dstPath = new Path(s"${repairOutputPath}/${filePath.getName}${badSuffix}")
|
val dstPath = new Path(s"$repairOutputPath/${filePath.getName}$badSuffix")
|
||||||
LOG.info(s"Copying from ${filePath} to ${dstPath}")
|
LOG.info(s"Copying from $filePath to $dstPath")
|
||||||
FileUtil.copy(fs, filePath, fs, dstPath, false, true, fs.getConf)
|
FileUtil.copy(fs, filePath, fs, dstPath, false, true, fs.getConf)
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
// 2. Remove duplicates from the bad files
|
// 2. Remove duplicates from the bad files
|
||||||
dupeFixPlan.foreach { case (fileName, keysToSkip) => {
|
dupeFixPlan.foreach { case (fileName, keysToSkip) =>
|
||||||
val commitTime = FSUtils.getCommitTime(fileNameToPathMap(fileName).getName)
|
val commitTime = FSUtils.getCommitTime(fileNameToPathMap(fileName).getName)
|
||||||
val badFilePath = new Path(s"${repairOutputPath}/${fileNameToPathMap(fileName).getName}.bad")
|
val badFilePath = new Path(s"$repairOutputPath/${fileNameToPathMap(fileName).getName}.bad")
|
||||||
val newFilePath = new Path(s"${repairOutputPath}/${fileNameToPathMap(fileName).getName}")
|
val newFilePath = new Path(s"$repairOutputPath/${fileNameToPathMap(fileName).getName}")
|
||||||
LOG.info(" Skipping and writing new file for : " + fileName)
|
LOG.info(" Skipping and writing new file for : " + fileName)
|
||||||
SparkHelpers.skipKeysAndWriteNewFile(commitTime, fs, badFilePath, newFilePath, dupeFixPlan(fileName))
|
SparkHelpers.skipKeysAndWriteNewFile(commitTime, fs, badFilePath, newFilePath, dupeFixPlan(fileName))
|
||||||
fs.delete(badFilePath, false)
|
fs.delete(badFilePath, false)
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
// 3. Check that there are no duplicates anymore.
|
// 3. Check that there are no duplicates anymore.
|
||||||
val df = sqlContext.read.parquet(s"${repairOutputPath}/*.parquet")
|
val df = sqlContext.read.parquet(s"$repairOutputPath/*.parquet")
|
||||||
df.registerTempTable("fixedTbl")
|
df.registerTempTable("fixedTbl")
|
||||||
val dupeKeyDF = getDupeKeyDF("fixedTbl")
|
val dupeKeyDF = getDupeKeyDF("fixedTbl")
|
||||||
val dupeCnt = dupeKeyDF.count();
|
val dupeCnt = dupeKeyDF.count()
|
||||||
if (dupeCnt != 0) {
|
if (dupeCnt != 0) {
|
||||||
dupeKeyDF.show()
|
dupeKeyDF.show()
|
||||||
throw new HoodieException("Still found some duplicates!!.. Inspect output")
|
throw new HoodieException("Still found some duplicates!!.. Inspect output")
|
||||||
@@ -169,7 +166,7 @@ class DedupeSparkJob(basePath: String,
|
|||||||
|
|
||||||
// 4. Additionally ensure no record keys are left behind.
|
// 4. Additionally ensure no record keys are left behind.
|
||||||
val sourceDF = sparkHelper.getDistinctKeyDF(fileNameToPathMap.map(t => t._2.toString).toList)
|
val sourceDF = sparkHelper.getDistinctKeyDF(fileNameToPathMap.map(t => t._2.toString).toList)
|
||||||
val fixedDF = sparkHelper.getDistinctKeyDF(fileNameToPathMap.map(t => s"${repairOutputPath}/${t._2.getName}").toList)
|
val fixedDF = sparkHelper.getDistinctKeyDF(fileNameToPathMap.map(t => s"$repairOutputPath/${t._2.getName}").toList)
|
||||||
val missedRecordKeysDF = sourceDF.except(fixedDF)
|
val missedRecordKeysDF = sourceDF.except(fixedDF)
|
||||||
val missedCnt = missedRecordKeysDF.count()
|
val missedCnt = missedRecordKeysDF.count()
|
||||||
if (missedCnt != 0) {
|
if (missedCnt != 0) {
|
||||||
@@ -180,17 +177,16 @@ class DedupeSparkJob(basePath: String,
|
|||||||
|
|
||||||
println("No duplicates found & counts are in check!!!! ")
|
println("No duplicates found & counts are in check!!!! ")
|
||||||
// 4. Prepare to copy the fixed files back.
|
// 4. Prepare to copy the fixed files back.
|
||||||
fileNameToPathMap.foreach { case (fileName, filePath) => {
|
fileNameToPathMap.foreach { case (_, filePath) =>
|
||||||
val srcPath = new Path(s"${repairOutputPath}/${filePath.getName}")
|
val srcPath = new Path(s"$repairOutputPath/${filePath.getName}")
|
||||||
val dstPath = new Path(s"${basePath}/${duplicatedPartitionPath}/${filePath.getName}")
|
val dstPath = new Path(s"$basePath/$duplicatedPartitionPath/${filePath.getName}")
|
||||||
if (dryRun) {
|
if (dryRun) {
|
||||||
LOG.info(s"[JUST KIDDING!!!] Copying from ${srcPath} to ${dstPath}")
|
LOG.info(s"[JUST KIDDING!!!] Copying from $srcPath to $dstPath")
|
||||||
} else {
|
} else {
|
||||||
// for real
|
// for real
|
||||||
LOG.info(s"[FOR REAL!!!] Copying from ${srcPath} to ${dstPath}")
|
LOG.info(s"[FOR REAL!!!] Copying from $srcPath to $dstPath")
|
||||||
FileUtil.copy(fs, srcPath, fs, dstPath, false, true, fs.getConf)
|
FileUtil.copy(fs, srcPath, fs, dstPath, false, true, fs.getConf)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -94,12 +94,9 @@ class SparkHelper(sqlContext: SQLContext, fs: FileSystem) {
|
|||||||
* @return
|
* @return
|
||||||
*/
|
*/
|
||||||
def isFileContainsKey(rowKey: String, file: String): Boolean = {
|
def isFileContainsKey(rowKey: String, file: String): Boolean = {
|
||||||
println(s"Checking ${file} for key ${rowKey}")
|
println(s"Checking $file for key $rowKey")
|
||||||
val ff = getRowKeyDF(file).filter(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}` = '${rowKey}'")
|
val ff = getRowKeyDF(file).filter(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}` = '$rowKey'")
|
||||||
if (ff.count() > 0)
|
if (ff.count() > 0) true else false
|
||||||
return true
|
|
||||||
else
|
|
||||||
return false
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -109,7 +106,7 @@ class SparkHelper(sqlContext: SQLContext, fs: FileSystem) {
|
|||||||
* @param sqlContext
|
* @param sqlContext
|
||||||
*/
|
*/
|
||||||
def getKeyCount(file: String, sqlContext: org.apache.spark.sql.SQLContext) = {
|
def getKeyCount(file: String, sqlContext: org.apache.spark.sql.SQLContext) = {
|
||||||
println(getRowKeyDF(file).collect().size)
|
println(getRowKeyDF(file).collect().length)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@@ -128,8 +125,7 @@ class SparkHelper(sqlContext: SQLContext, fs: FileSystem) {
|
|||||||
val bf = new com.uber.hoodie.common.BloomFilter(bfStr)
|
val bf = new com.uber.hoodie.common.BloomFilter(bfStr)
|
||||||
val foundCount = sqlContext.parquetFile(file)
|
val foundCount = sqlContext.parquetFile(file)
|
||||||
.select(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}`")
|
.select(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}`")
|
||||||
.collect().
|
.collect().count(r => !bf.mightContain(r.getString(0)))
|
||||||
filter(r => !bf.mightContain(r.getString(0))).size
|
|
||||||
val totalCount = getKeyCount(file, sqlContext)
|
val totalCount = getKeyCount(file, sqlContext)
|
||||||
s"totalCount: ${totalCount}, foundCount: ${foundCount}"
|
s"totalCount: ${totalCount}, foundCount: ${foundCount}"
|
||||||
totalCount == foundCount
|
totalCount == foundCount
|
||||||
|
|||||||
@@ -239,7 +239,7 @@ public class CompactionAdminClient implements Serializable {
|
|||||||
.filter(fs -> fs.getFileId().equals(op.getFileId())).findFirst().get();
|
.filter(fs -> fs.getFileId().equals(op.getFileId())).findFirst().get();
|
||||||
final int maxVersion =
|
final int maxVersion =
|
||||||
op.getDeltaFilePaths().stream().map(lf -> FSUtils.getFileVersionFromLog(new Path(lf)))
|
op.getDeltaFilePaths().stream().map(lf -> FSUtils.getFileVersionFromLog(new Path(lf)))
|
||||||
.reduce((x, y) -> x > y ? x : y).map(x -> x).orElse(0);
|
.reduce((x, y) -> x > y ? x : y).orElse(0);
|
||||||
List<HoodieLogFile> logFilesToBeMoved =
|
List<HoodieLogFile> logFilesToBeMoved =
|
||||||
merged.getLogFiles().filter(lf -> lf.getLogVersion() > maxVersion).collect(Collectors.toList());
|
merged.getLogFiles().filter(lf -> lf.getLogVersion() > maxVersion).collect(Collectors.toList());
|
||||||
return logFilesToBeMoved.stream().map(lf -> {
|
return logFilesToBeMoved.stream().map(lf -> {
|
||||||
@@ -322,8 +322,7 @@ public class CompactionAdminClient implements Serializable {
|
|||||||
Set<HoodieLogFile> diff =
|
Set<HoodieLogFile> diff =
|
||||||
logFilesInFileSlice.stream().filter(lf -> !logFilesInCompactionOp.contains(lf))
|
logFilesInFileSlice.stream().filter(lf -> !logFilesInCompactionOp.contains(lf))
|
||||||
.collect(Collectors.toSet());
|
.collect(Collectors.toSet());
|
||||||
Preconditions.checkArgument(diff.stream()
|
Preconditions.checkArgument(diff.stream().allMatch(lf -> lf.getBaseCommitTime().equals(compactionInstant)),
|
||||||
.filter(lf -> !lf.getBaseCommitTime().equals(compactionInstant)).count() == 0,
|
|
||||||
"There are some log-files which are neither specified in compaction plan "
|
"There are some log-files which are neither specified in compaction plan "
|
||||||
+ "nor present after compaction request instant. Some of these :" + diff);
|
+ "nor present after compaction request instant. Some of these :" + diff);
|
||||||
} else {
|
} else {
|
||||||
@@ -439,13 +438,13 @@ public class CompactionAdminClient implements Serializable {
|
|||||||
.filter(fs -> fs.getFileId().equals(operation.getFileId())).findFirst().get();
|
.filter(fs -> fs.getFileId().equals(operation.getFileId())).findFirst().get();
|
||||||
List<HoodieLogFile> logFilesToRepair =
|
List<HoodieLogFile> logFilesToRepair =
|
||||||
merged.getLogFiles().filter(lf -> lf.getBaseCommitTime().equals(compactionInstant))
|
merged.getLogFiles().filter(lf -> lf.getBaseCommitTime().equals(compactionInstant))
|
||||||
|
.sorted(HoodieLogFile.getBaseInstantAndLogVersionComparator().reversed())
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
logFilesToRepair.sort(HoodieLogFile.getBaseInstantAndLogVersionComparator().reversed());
|
|
||||||
FileSlice fileSliceForCompaction =
|
FileSlice fileSliceForCompaction =
|
||||||
fileSystemView.getLatestFileSlicesBeforeOrOn(operation.getPartitionPath(), operation.getBaseInstantTime())
|
fileSystemView.getLatestFileSlicesBeforeOrOn(operation.getPartitionPath(), operation.getBaseInstantTime())
|
||||||
.filter(fs -> fs.getFileId().equals(operation.getFileId())).findFirst().get();
|
.filter(fs -> fs.getFileId().equals(operation.getFileId())).findFirst().get();
|
||||||
int maxUsedVersion =
|
int maxUsedVersion =
|
||||||
fileSliceForCompaction.getLogFiles().findFirst().map(lf -> lf.getLogVersion())
|
fileSliceForCompaction.getLogFiles().findFirst().map(HoodieLogFile::getLogVersion)
|
||||||
.orElse(HoodieLogFile.LOGFILE_BASE_VERSION - 1);
|
.orElse(HoodieLogFile.LOGFILE_BASE_VERSION - 1);
|
||||||
String logExtn = fileSliceForCompaction.getLogFiles().findFirst().map(lf -> "." + lf.getFileExtension())
|
String logExtn = fileSliceForCompaction.getLogFiles().findFirst().map(lf -> "." + lf.getFileExtension())
|
||||||
.orElse(HoodieLogFile.DELTA_EXTENSION);
|
.orElse(HoodieLogFile.DELTA_EXTENSION);
|
||||||
|
|||||||
@@ -440,7 +440,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
} else {
|
} else {
|
||||||
return hoodieTable.handleInsertPartition(commitTime, partition, recordItr, partitioner);
|
return hoodieTable.handleInsertPartition(commitTime, partition, recordItr, partitioner);
|
||||||
}
|
}
|
||||||
}, true).flatMap(writeStatuses -> writeStatuses.iterator());
|
}, true).flatMap(List::iterator);
|
||||||
|
|
||||||
return updateIndexAndCommitIfNeeded(writeStatusRDD, hoodieTable, commitTime);
|
return updateIndexAndCommitIfNeeded(writeStatusRDD, hoodieTable, commitTime);
|
||||||
}
|
}
|
||||||
@@ -469,7 +469,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
Partitioner partitioner) {
|
Partitioner partitioner) {
|
||||||
return dedupedRecords.mapToPair(record -> new Tuple2<>(
|
return dedupedRecords.mapToPair(record -> new Tuple2<>(
|
||||||
new Tuple2<>(record.getKey(), Option.apply(record.getCurrentLocation())), record))
|
new Tuple2<>(record.getKey(), Option.apply(record.getCurrentLocation())), record))
|
||||||
.partitionBy(partitioner).map(tuple -> tuple._2());
|
.partitionBy(partitioner).map(Tuple2::_2);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -499,7 +499,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||||
|
|
||||||
List<HoodieWriteStat> stats = writeStatuses.map(status -> status.getStat()).collect();
|
List<HoodieWriteStat> stats = writeStatuses.map(WriteStatus::getStat).collect();
|
||||||
|
|
||||||
updateMetadataAndRollingStats(actionType, metadata, stats);
|
updateMetadataAndRollingStats(actionType, metadata, stats);
|
||||||
|
|
||||||
@@ -522,7 +522,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
|
|
||||||
// add in extra metadata
|
// add in extra metadata
|
||||||
if (extraMetadata.isPresent()) {
|
if (extraMetadata.isPresent()) {
|
||||||
extraMetadata.get().forEach((k, v) -> metadata.addMetadata(k, v));
|
extraMetadata.get().forEach(metadata::addMetadata);
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
@@ -806,7 +806,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
});
|
});
|
||||||
|
|
||||||
List<String> pendingCompactionToRollback =
|
List<String> pendingCompactionToRollback =
|
||||||
commits.stream().filter(c -> pendingCompactions.contains(c)).collect(Collectors.toList());
|
commits.stream().filter(pendingCompactions::contains).collect(Collectors.toList());
|
||||||
List<String> commitsToRollback =
|
List<String> commitsToRollback =
|
||||||
commits.stream().filter(c -> !pendingCompactions.contains(c)).collect(Collectors.toList());
|
commits.stream().filter(c -> !pendingCompactions.contains(c)).collect(Collectors.toList());
|
||||||
|
|
||||||
@@ -837,12 +837,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Remove interleaving pending compactions before rolling back commits
|
// Remove interleaving pending compactions before rolling back commits
|
||||||
pendingCompactionToRollback.stream().forEach(this::deletePendingCompaction);
|
pendingCompactionToRollback.forEach(this::deletePendingCompaction);
|
||||||
|
|
||||||
List<HoodieRollbackStat> stats = table.rollback(jsc, commitsToRollback);
|
List<HoodieRollbackStat> stats = table.rollback(jsc, commitsToRollback);
|
||||||
|
|
||||||
// cleanup index entries
|
// cleanup index entries
|
||||||
commitsToRollback.stream().forEach(s -> {
|
commitsToRollback.forEach(s -> {
|
||||||
if (!index.rollbackCommit(s)) {
|
if (!index.rollbackCommit(s)) {
|
||||||
throw new HoodieRollbackException("Rollback index changes failed, for time :" + s);
|
throw new HoodieRollbackException("Rollback index changes failed, for time :" + s);
|
||||||
}
|
}
|
||||||
@@ -1076,7 +1076,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
// everything
|
// everything
|
||||||
// so pick it from one of the records.
|
// so pick it from one of the records.
|
||||||
return new HoodieRecord<T>(rec1.getKey(), reducedData);
|
return new HoodieRecord<T>(rec1.getKey(), reducedData);
|
||||||
}, parallelism).map(recordTuple -> recordTuple._2());
|
}, parallelism).map(Tuple2::_2);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -1099,7 +1099,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
// Create a Hoodie table which encapsulated the commits and files visible
|
// Create a Hoodie table which encapsulated the commits and files visible
|
||||||
HoodieTable table = HoodieTable.getHoodieTable(
|
HoodieTable table = HoodieTable.getHoodieTable(
|
||||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
|
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
|
||||||
if (table.getMetaClient().getCommitActionType() == HoodieTimeline.COMMIT_ACTION) {
|
if (table.getMetaClient().getCommitActionType().equals(HoodieTimeline.COMMIT_ACTION)) {
|
||||||
writeContext = metrics.getCommitCtx();
|
writeContext = metrics.getCommitCtx();
|
||||||
} else {
|
} else {
|
||||||
writeContext = metrics.getDeltaCommitCtx();
|
writeContext = metrics.getDeltaCommitCtx();
|
||||||
@@ -1214,7 +1214,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
|
|
||||||
private HoodieCommitMetadata doCompactionCommit(JavaRDD<WriteStatus> writeStatuses,
|
private HoodieCommitMetadata doCompactionCommit(JavaRDD<WriteStatus> writeStatuses,
|
||||||
HoodieTableMetaClient metaClient, String compactionCommitTime, Optional<Map<String, String>> extraMetadata) {
|
HoodieTableMetaClient metaClient, String compactionCommitTime, Optional<Map<String, String>> extraMetadata) {
|
||||||
List<HoodieWriteStat> updateStatusMap = writeStatuses.map(writeStatus -> writeStatus.getStat())
|
List<HoodieWriteStat> updateStatusMap = writeStatuses.map(WriteStatus::getStat)
|
||||||
.collect();
|
.collect();
|
||||||
|
|
||||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata(true);
|
HoodieCommitMetadata metadata = new HoodieCommitMetadata(true);
|
||||||
|
|||||||
@@ -117,10 +117,9 @@ public class HoodieMemoryConfig extends DefaultHoodieConfig {
|
|||||||
|
|
||||||
if (SparkEnv.get() != null) {
|
if (SparkEnv.get() != null) {
|
||||||
// 1 GB is the default conf used by Spark, look at SparkContext.scala
|
// 1 GB is the default conf used by Spark, look at SparkContext.scala
|
||||||
long executorMemoryInBytes = Long.valueOf(
|
long executorMemoryInBytes = Utils.memoryStringToMb(SparkEnv.get().conf().get(SPARK_EXECUTOR_MEMORY_PROP,
|
||||||
Utils.memoryStringToMb(SparkEnv.get().conf().get(SPARK_EXECUTOR_MEMORY_PROP,
|
|
||||||
DEFAULT_SPARK_EXECUTOR_MEMORY_MB)) * 1024
|
DEFAULT_SPARK_EXECUTOR_MEMORY_MB)) * 1024
|
||||||
* 1024L);
|
* 1024L;
|
||||||
// 0.6 is the default value used by Spark,
|
// 0.6 is the default value used by Spark,
|
||||||
// look at {@link
|
// look at {@link
|
||||||
// https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkConf.scala#L507}
|
// https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkConf.scala#L507}
|
||||||
|
|||||||
@@ -187,13 +187,11 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
|||||||
for (String partitionPath : recordsPerPartition.keySet()) {
|
for (String partitionPath : recordsPerPartition.keySet()) {
|
||||||
long numRecords = recordsPerPartition.get(partitionPath);
|
long numRecords = recordsPerPartition.get(partitionPath);
|
||||||
long numFiles =
|
long numFiles =
|
||||||
filesPerPartition.containsKey(partitionPath) ? filesPerPartition.get(partitionPath)
|
filesPerPartition.getOrDefault(partitionPath, 1L);
|
||||||
: 1L;
|
|
||||||
|
|
||||||
totalComparisons += numFiles * numRecords;
|
totalComparisons += numFiles * numRecords;
|
||||||
totalFiles +=
|
totalFiles +=
|
||||||
filesPerPartition.containsKey(partitionPath) ? filesPerPartition.get(partitionPath)
|
filesPerPartition.getOrDefault(partitionPath, 0L);
|
||||||
: 0L;
|
|
||||||
totalRecords += numRecords;
|
totalRecords += numRecords;
|
||||||
}
|
}
|
||||||
logger.info("TotalRecords: " + totalRecords + ", TotalFiles: " + totalFiles
|
logger.info("TotalRecords: " + totalRecords + ", TotalFiles: " + totalFiles
|
||||||
@@ -340,7 +338,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
return recordComparisons;
|
return recordComparisons;
|
||||||
}).flatMapToPair(t -> t.iterator());
|
}).flatMapToPair(List::iterator);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -369,7 +367,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
|||||||
|
|
||||||
return fileSortedTripletRDD.mapPartitionsWithIndex(
|
return fileSortedTripletRDD.mapPartitionsWithIndex(
|
||||||
new HoodieBloomIndexCheckFunction(metaClient, config.getBasePath()), true)
|
new HoodieBloomIndexCheckFunction(metaClient, config.getBasePath()), true)
|
||||||
.flatMap(indexLookupResults -> indexLookupResults.iterator())
|
.flatMap(List::iterator)
|
||||||
.filter(lookupResult -> lookupResult.getMatchingRecordKeys().size() > 0)
|
.filter(lookupResult -> lookupResult.getMatchingRecordKeys().size() > 0)
|
||||||
.flatMapToPair(lookupResult -> {
|
.flatMapToPair(lookupResult -> {
|
||||||
List<Tuple2<String, String>> vals = new ArrayList<>();
|
List<Tuple2<String, String>> vals = new ArrayList<>();
|
||||||
|
|||||||
@@ -104,7 +104,7 @@ public class HoodieGlobalBloomIndex<T extends HoodieRecordPayload> extends Hoodi
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
return recordComparisons;
|
return recordComparisons;
|
||||||
}).flatMapToPair(t -> t.iterator());
|
}).flatMapToPair(List::iterator);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -97,7 +97,7 @@ public class ConsistencyCheck implements Serializable {
|
|||||||
.filter(p -> !fileNames.contains(new Path(basePath, p).getName()))
|
.filter(p -> !fileNames.contains(new Path(basePath, p).getName()))
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
})
|
})
|
||||||
.flatMap(itr -> itr.iterator()).collect();
|
.flatMap(List::iterator).collect();
|
||||||
if (remainingPaths.size() == 0) {
|
if (remainingPaths.size() == 0) {
|
||||||
break; // we are done.
|
break; // we are done.
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -65,9 +65,9 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
|||||||
private final WriteStatus writeStatus;
|
private final WriteStatus writeStatus;
|
||||||
private final String fileId;
|
private final String fileId;
|
||||||
// Buffer for holding records in memory before they are flushed to disk
|
// Buffer for holding records in memory before they are flushed to disk
|
||||||
List<IndexedRecord> recordList = new ArrayList<>();
|
private List<IndexedRecord> recordList = new ArrayList<>();
|
||||||
// Buffer for holding records (to be deleted) in memory before they are flushed to disk
|
// Buffer for holding records (to be deleted) in memory before they are flushed to disk
|
||||||
List<String> keysToDelete = new ArrayList<>();
|
private List<String> keysToDelete = new ArrayList<>();
|
||||||
private TableFileSystemView.RealtimeView fileSystemView;
|
private TableFileSystemView.RealtimeView fileSystemView;
|
||||||
private String partitionPath;
|
private String partitionPath;
|
||||||
private Iterator<HoodieRecord<T>> recordItr;
|
private Iterator<HoodieRecord<T>> recordItr;
|
||||||
|
|||||||
@@ -168,8 +168,7 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
|||||||
// The window of commit retain == max query run time. So a query could be running which
|
// The window of commit retain == max query run time. So a query could be running which
|
||||||
// still
|
// still
|
||||||
// uses this file.
|
// uses this file.
|
||||||
if (fileCommitTime.equals(lastVersion) || (lastVersionBeforeEarliestCommitToRetain != null
|
if (fileCommitTime.equals(lastVersion) || (fileCommitTime.equals(lastVersionBeforeEarliestCommitToRetain))) {
|
||||||
&& fileCommitTime.equals(lastVersionBeforeEarliestCommitToRetain))) {
|
|
||||||
// move on to the next file
|
// move on to the next file
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
@@ -180,9 +179,7 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
|||||||
.compareTimestamps(earliestCommitToRetain.getTimestamp(), fileCommitTime,
|
.compareTimestamps(earliestCommitToRetain.getTimestamp(), fileCommitTime,
|
||||||
HoodieTimeline.GREATER)) {
|
HoodieTimeline.GREATER)) {
|
||||||
// this is a commit, that should be cleaned.
|
// this is a commit, that should be cleaned.
|
||||||
if (aFile.isPresent()) {
|
aFile.ifPresent(hoodieDataFile -> deletePaths.add(hoodieDataFile.getFileStatus().getPath().toString()));
|
||||||
deletePaths.add(aFile.get().getFileStatus().getPath().toString());
|
|
||||||
}
|
|
||||||
if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) {
|
if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||||
// If merge on read, then clean the log files for the commits as well
|
// If merge on read, then clean the log files for the commits as well
|
||||||
deletePaths.addAll(aSlice.getLogFiles().map(file -> file.getPath().toString())
|
deletePaths.addAll(aSlice.getLogFiles().map(file -> file.getPath().toString())
|
||||||
|
|||||||
@@ -199,8 +199,7 @@ public class HoodieCommitArchiveLog {
|
|||||||
.filter(i -> {
|
.filter(i -> {
|
||||||
return i.isCompleted()
|
return i.isCompleted()
|
||||||
&& (i.getAction().equals(COMMIT_ACTION) || (i.getAction().equals(DELTA_COMMIT_ACTION)));
|
&& (i.getAction().equals(COMMIT_ACTION) || (i.getAction().equals(DELTA_COMMIT_ACTION)));
|
||||||
})
|
}).max(Comparator.comparing(HoodieInstant::getTimestamp));
|
||||||
.sorted(Comparator.comparing(HoodieInstant::getTimestamp).reversed()).findFirst();
|
|
||||||
if (latestCommitted.isPresent()) {
|
if (latestCommitted.isPresent()) {
|
||||||
success &= deleteAllInstantsOlderorEqualsInAuxMetaFolder(latestCommitted.get());
|
success &= deleteAllInstantsOlderorEqualsInAuxMetaFolder(latestCommitted.get());
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -254,9 +254,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
|||||||
public WriteStatus close() {
|
public WriteStatus close() {
|
||||||
try {
|
try {
|
||||||
// write out any pending records (this can happen when inserts are turned into updates)
|
// write out any pending records (this can happen when inserts are turned into updates)
|
||||||
Iterator<String> pendingRecordsItr = keyToNewRecords.keySet().iterator();
|
for (String key : keyToNewRecords.keySet()) {
|
||||||
while (pendingRecordsItr.hasNext()) {
|
|
||||||
String key = pendingRecordsItr.next();
|
|
||||||
if (!writtenRecordKeys.contains(key)) {
|
if (!writtenRecordKeys.contains(key)) {
|
||||||
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key);
|
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key);
|
||||||
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(schema));
|
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(schema));
|
||||||
|
|||||||
@@ -90,7 +90,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
|||||||
log.info("Compactor compacting " + operations + " files");
|
log.info("Compactor compacting " + operations + " files");
|
||||||
return jsc.parallelize(operations, operations.size())
|
return jsc.parallelize(operations, operations.size())
|
||||||
.map(s -> compact(table, metaClient, config, s, compactionInstantTime))
|
.map(s -> compact(table, metaClient, config, s, compactionInstantTime))
|
||||||
.flatMap(writeStatusesItr -> writeStatusesItr.iterator());
|
.flatMap(List::iterator);
|
||||||
}
|
}
|
||||||
|
|
||||||
private List<WriteStatus> compact(HoodieCopyOnWriteTable hoodieCopyOnWriteTable, HoodieTableMetaClient metaClient,
|
private List<WriteStatus> compact(HoodieCopyOnWriteTable hoodieCopyOnWriteTable, HoodieTableMetaClient metaClient,
|
||||||
@@ -141,7 +141,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
|||||||
}
|
}
|
||||||
Iterable<List<WriteStatus>> resultIterable = () -> result;
|
Iterable<List<WriteStatus>> resultIterable = () -> result;
|
||||||
return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream)
|
return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream)
|
||||||
.map(s -> {
|
.peek(s -> {
|
||||||
s.getStat().setTotalUpdatedRecordsCompacted(scanner.getNumMergedRecordsInLog());
|
s.getStat().setTotalUpdatedRecordsCompacted(scanner.getNumMergedRecordsInLog());
|
||||||
s.getStat().setTotalLogFilesCompacted(scanner.getTotalLogFiles());
|
s.getStat().setTotalLogFilesCompacted(scanner.getTotalLogFiles());
|
||||||
s.getStat().setTotalLogRecords(scanner.getTotalLogRecords());
|
s.getStat().setTotalLogRecords(scanner.getTotalLogRecords());
|
||||||
@@ -154,7 +154,6 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
|||||||
RuntimeStats runtimeStats = new RuntimeStats();
|
RuntimeStats runtimeStats = new RuntimeStats();
|
||||||
runtimeStats.setTotalScanTime(scanner.getTotalTimeTakenToReadAndMergeBlocks());
|
runtimeStats.setTotalScanTime(scanner.getTotalTimeTakenToReadAndMergeBlocks());
|
||||||
s.getStat().setRuntimeStats(runtimeStats);
|
s.getStat().setRuntimeStats(runtimeStats);
|
||||||
return s;
|
|
||||||
}).collect(toList());
|
}).collect(toList());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -74,7 +74,7 @@ public abstract class CompactionStrategy implements Serializable {
|
|||||||
metrics.put(TOTAL_IO_WRITE_MB, totalIOWrite.doubleValue());
|
metrics.put(TOTAL_IO_WRITE_MB, totalIOWrite.doubleValue());
|
||||||
metrics.put(TOTAL_IO_MB, totalIO.doubleValue());
|
metrics.put(TOTAL_IO_MB, totalIO.doubleValue());
|
||||||
metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize.doubleValue());
|
metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize.doubleValue());
|
||||||
metrics.put(TOTAL_LOG_FILES, Double.valueOf(logFiles.size()));
|
metrics.put(TOTAL_LOG_FILES, (double) logFiles.size());
|
||||||
return metrics;
|
return metrics;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -139,13 +139,8 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
|||||||
return fsDataOutputStream;
|
return fsDataOutputStream;
|
||||||
}
|
}
|
||||||
|
|
||||||
SizeAwareFSDataOutputStream os = new SizeAwareFSDataOutputStream(fsDataOutputStream,
|
SizeAwareFSDataOutputStream os = new SizeAwareFSDataOutputStream(
|
||||||
new Runnable() {
|
fsDataOutputStream, () -> openStreams.remove(path.getName()));
|
||||||
@Override
|
|
||||||
public void run() {
|
|
||||||
openStreams.remove(path.getName());
|
|
||||||
}
|
|
||||||
});
|
|
||||||
openStreams.put(path.getName(), os);
|
openStreams.put(path.getName(), os);
|
||||||
return os;
|
return os;
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -145,7 +145,7 @@ public class HoodieMetrics {
|
|||||||
public void updateRollbackMetrics(long durationInMs, long numFilesDeleted) {
|
public void updateRollbackMetrics(long durationInMs, long numFilesDeleted) {
|
||||||
if (config.isMetricsOn()) {
|
if (config.isMetricsOn()) {
|
||||||
logger.info(String
|
logger.info(String
|
||||||
.format("Sending rollback metrics (duration=%d, numFilesDeleted=$d)", durationInMs,
|
.format("Sending rollback metrics (duration=%d, numFilesDeleted=%d)", durationInMs,
|
||||||
numFilesDeleted));
|
numFilesDeleted));
|
||||||
registerGauge(getMetricsName("rollback", "duration"), durationInMs);
|
registerGauge(getMetricsName("rollback", "duration"), durationInMs);
|
||||||
registerGauge(getMetricsName("rollback", "numFilesDeleted"), numFilesDeleted);
|
registerGauge(getMetricsName("rollback", "numFilesDeleted"), numFilesDeleted);
|
||||||
@@ -180,12 +180,7 @@ public class HoodieMetrics {
|
|||||||
void registerGauge(String metricName, final long value) {
|
void registerGauge(String metricName, final long value) {
|
||||||
try {
|
try {
|
||||||
MetricRegistry registry = Metrics.getInstance().getRegistry();
|
MetricRegistry registry = Metrics.getInstance().getRegistry();
|
||||||
registry.register(metricName, new Gauge<Long>() {
|
registry.register(metricName, (Gauge<Long>) () -> value);
|
||||||
@Override
|
|
||||||
public Long getValue() {
|
|
||||||
return value;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
// Here we catch all exception, so the major upsert pipeline will not be affected if the
|
// Here we catch all exception, so the major upsert pipeline will not be affected if the
|
||||||
// metrics system
|
// metrics system
|
||||||
|
|||||||
@@ -201,17 +201,15 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
|||||||
"Error in finding the old file path at commit " + commitTime + " for fileId: " + fileId);
|
"Error in finding the old file path at commit " + commitTime + " for fileId: " + fileId);
|
||||||
} else {
|
} else {
|
||||||
AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getSchema());
|
AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getSchema());
|
||||||
ParquetReader<IndexedRecord> reader = AvroParquetReader.<IndexedRecord>builder(upsertHandle.getOldFilePath())
|
|
||||||
.withConf(getHadoopConf()).build();
|
|
||||||
BoundedInMemoryExecutor<GenericRecord, GenericRecord, Void> wrapper = null;
|
BoundedInMemoryExecutor<GenericRecord, GenericRecord, Void> wrapper = null;
|
||||||
try {
|
try (ParquetReader<IndexedRecord> reader = AvroParquetReader.<IndexedRecord>builder(upsertHandle.getOldFilePath())
|
||||||
|
.withConf(getHadoopConf()).build()) {
|
||||||
wrapper = new SparkBoundedInMemoryExecutor(config, new ParquetReaderIterator(reader),
|
wrapper = new SparkBoundedInMemoryExecutor(config, new ParquetReaderIterator(reader),
|
||||||
new UpdateHandler(upsertHandle), x -> x);
|
new UpdateHandler(upsertHandle), x -> x);
|
||||||
wrapper.execute();
|
wrapper.execute();
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
throw new HoodieException(e);
|
throw new HoodieException(e);
|
||||||
} finally {
|
} finally {
|
||||||
reader.close();
|
|
||||||
upsertHandle.close();
|
upsertHandle.close();
|
||||||
if (null != wrapper) {
|
if (null != wrapper) {
|
||||||
wrapper.shutdownNow();
|
wrapper.shutdownNow();
|
||||||
@@ -480,7 +478,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
|||||||
.merge(e2)).collect();
|
.merge(e2)).collect();
|
||||||
|
|
||||||
Map<String, PartitionCleanStat> partitionCleanStatsMap = partitionCleanStats.stream()
|
Map<String, PartitionCleanStat> partitionCleanStatsMap = partitionCleanStats.stream()
|
||||||
.collect(Collectors.toMap(e -> e._1(), e -> e._2()));
|
.collect(Collectors.toMap(Tuple2::_1, Tuple2::_2));
|
||||||
|
|
||||||
HoodieCleanHelper cleaner = new HoodieCleanHelper(this, config);
|
HoodieCleanHelper cleaner = new HoodieCleanHelper(this, config);
|
||||||
// Return PartitionCleanStat for each partition passed.
|
// Return PartitionCleanStat for each partition passed.
|
||||||
|
|||||||
@@ -52,16 +52,7 @@ import com.uber.hoodie.io.HoodieAppendHandle;
|
|||||||
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
|
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.UncheckedIOException;
|
import java.io.UncheckedIOException;
|
||||||
import java.util.ArrayList;
|
import java.util.*;
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.HashSet;
|
|
||||||
import java.util.Iterator;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.Set;
|
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
@@ -185,18 +176,17 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
|||||||
.getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION,
|
.getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION,
|
||||||
HoodieActiveTimeline.DELTA_COMMIT_ACTION, HoodieActiveTimeline.COMPACTION_ACTION)).getInstants()
|
HoodieActiveTimeline.DELTA_COMMIT_ACTION, HoodieActiveTimeline.COMPACTION_ACTION)).getInstants()
|
||||||
.filter(i -> commits.contains(i.getTimestamp()))
|
.filter(i -> commits.contains(i.getTimestamp()))
|
||||||
.collect(Collectors.toMap(i -> i.getTimestamp(), i -> i));
|
.collect(Collectors.toMap(HoodieInstant::getTimestamp, i -> i));
|
||||||
|
|
||||||
// Atomically un-publish all non-inflight commits
|
// Atomically un-publish all non-inflight commits
|
||||||
commitsAndCompactions.entrySet().stream().map(entry -> entry.getValue())
|
commitsAndCompactions.entrySet().stream().map(Map.Entry::getValue)
|
||||||
.filter(i -> !i.isInflight()).forEach(this.getActiveTimeline()::revertToInflight);
|
.filter(i -> !i.isInflight()).forEach(this.getActiveTimeline()::revertToInflight);
|
||||||
logger.info("Unpublished " + commits);
|
logger.info("Unpublished " + commits);
|
||||||
Long startTime = System.currentTimeMillis();
|
Long startTime = System.currentTimeMillis();
|
||||||
List<HoodieRollbackStat> allRollbackStats = jsc.parallelize(FSUtils
|
List<HoodieRollbackStat> allRollbackStats = jsc.parallelize(FSUtils
|
||||||
.getAllPartitionPaths(this.metaClient.getFs(), this.getMetaClient().getBasePath(),
|
.getAllPartitionPaths(this.metaClient.getFs(), this.getMetaClient().getBasePath(),
|
||||||
config.shouldAssumeDatePartitioning()))
|
config.shouldAssumeDatePartitioning()))
|
||||||
.map((Function<String, List<HoodieRollbackStat>>) partitionPath -> {
|
.map((Function<String, List<HoodieRollbackStat>>) partitionPath -> commits.stream().map(commit -> {
|
||||||
return commits.stream().map(commit -> {
|
|
||||||
HoodieInstant instant = commitsAndCompactions.get(commit);
|
HoodieInstant instant = commitsAndCompactions.get(commit);
|
||||||
HoodieRollbackStat hoodieRollbackStats = null;
|
HoodieRollbackStat hoodieRollbackStats = null;
|
||||||
// Need to put the path filter here since Filter is not serializable
|
// Need to put the path filter here since Filter is not serializable
|
||||||
@@ -218,7 +208,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
|||||||
case HoodieTimeline.COMPACTION_ACTION:
|
case HoodieTimeline.COMPACTION_ACTION:
|
||||||
try {
|
try {
|
||||||
Map<FileStatus, Boolean> results = super
|
Map<FileStatus, Boolean> results = super
|
||||||
.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
|
.deleteCleanedFiles(partitionPath, Collections.singletonList(commit));
|
||||||
hoodieRollbackStats = HoodieRollbackStat.newBuilder()
|
hoodieRollbackStats = HoodieRollbackStat.newBuilder()
|
||||||
.withPartitionPath(partitionPath).withDeletedFileResults(results).build();
|
.withPartitionPath(partitionPath).withDeletedFileResults(results).build();
|
||||||
break;
|
break;
|
||||||
@@ -282,8 +272,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
|||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
return hoodieRollbackStats;
|
return hoodieRollbackStats;
|
||||||
}).collect(Collectors.toList());
|
}).collect(Collectors.toList())).flatMap(List::iterator).filter(Objects::nonNull).collect();
|
||||||
}).flatMap(x -> x.iterator()).filter(x -> x != null).collect();
|
|
||||||
|
|
||||||
commitsAndCompactions.entrySet().stream().map(
|
commitsAndCompactions.entrySet().stream().map(
|
||||||
entry -> new HoodieInstant(true, entry.getValue().getAction(),
|
entry -> new HoodieInstant(true, entry.getValue().getAction(),
|
||||||
@@ -312,9 +301,8 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
|||||||
Optional<String> lastRollingStat = Optional.ofNullable(commitMetadata.getExtraMetadata()
|
Optional<String> lastRollingStat = Optional.ofNullable(commitMetadata.getExtraMetadata()
|
||||||
.get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY));
|
.get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY));
|
||||||
if (lastRollingStat.isPresent()) {
|
if (lastRollingStat.isPresent()) {
|
||||||
HoodieRollingStatMetadata rollingStatMetadata = HoodieCommitMetadata
|
return HoodieCommitMetadata
|
||||||
.fromBytes(lastRollingStat.get().getBytes(), HoodieRollingStatMetadata.class);
|
.fromBytes(lastRollingStat.get().getBytes(), HoodieRollingStatMetadata.class);
|
||||||
return rollingStatMetadata;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return null;
|
return null;
|
||||||
@@ -411,8 +399,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
|||||||
HoodieRollingStat rollingStatForFile = partitionRollingStats.get(fileSlice.getFileId());
|
HoodieRollingStat rollingStatForFile = partitionRollingStats.get(fileSlice.getFileId());
|
||||||
if (rollingStatForFile != null) {
|
if (rollingStatForFile != null) {
|
||||||
long inserts = rollingStatForFile.getInserts();
|
long inserts = rollingStatForFile.getInserts();
|
||||||
long totalSize = averageRecordSize * inserts;
|
return averageRecordSize * inserts;
|
||||||
return totalSize;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@@ -427,10 +414,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
|||||||
|
|
||||||
private boolean isSmallFile(String partitionPath, FileSlice fileSlice) {
|
private boolean isSmallFile(String partitionPath, FileSlice fileSlice) {
|
||||||
long totalSize = getTotalFileSize(partitionPath, fileSlice);
|
long totalSize = getTotalFileSize(partitionPath, fileSlice);
|
||||||
if (totalSize < config.getParquetMaxFileSize()) {
|
return totalSize < config.getParquetMaxFileSize();
|
||||||
return true;
|
|
||||||
}
|
|
||||||
return false;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO (NA) : Make this static part of utility
|
// TODO (NA) : Make this static part of utility
|
||||||
@@ -470,11 +454,8 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
|||||||
commitMetadata.getPartitionToWriteStats().get(partitionPath).stream()
|
commitMetadata.getPartitionToWriteStats().get(partitionPath).stream()
|
||||||
.filter(wStat -> {
|
.filter(wStat -> {
|
||||||
// Filter out stats without prevCommit since they are all inserts
|
// Filter out stats without prevCommit since they are all inserts
|
||||||
if (wStat != null && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT && wStat.getPrevCommit() != null
|
return wStat != null && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT && wStat.getPrevCommit() != null
|
||||||
&& !deletedFiles.contains(wStat.getFileId())) {
|
&& !deletedFiles.contains(wStat.getFileId());
|
||||||
return true;
|
|
||||||
}
|
|
||||||
return false;
|
|
||||||
}).forEach(wStat -> {
|
}).forEach(wStat -> {
|
||||||
HoodieLogFormat.Writer writer = null;
|
HoodieLogFormat.Writer writer = null;
|
||||||
String baseCommitTime = wStat.getPrevCommit();
|
String baseCommitTime = wStat.getPrevCommit();
|
||||||
|
|||||||
@@ -21,10 +21,7 @@ package com.uber.hoodie.common.model;
|
|||||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import java.util.Comparator;
|
import java.util.*;
|
||||||
import java.util.List;
|
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.TreeMap;
|
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -33,10 +30,7 @@ import java.util.stream.Stream;
|
|||||||
public class HoodieFileGroup implements Serializable {
|
public class HoodieFileGroup implements Serializable {
|
||||||
|
|
||||||
public static Comparator<String> getReverseCommitTimeComparator() {
|
public static Comparator<String> getReverseCommitTimeComparator() {
|
||||||
return (o1, o2) -> {
|
return Comparator.reverseOrder();
|
||||||
// reverse the order
|
|
||||||
return o2.compareTo(o1);
|
|
||||||
};
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -127,7 +121,7 @@ public class HoodieFileGroup implements Serializable {
|
|||||||
* Get all the the file slices including in-flight ones as seen in underlying file-system
|
* Get all the the file slices including in-flight ones as seen in underlying file-system
|
||||||
*/
|
*/
|
||||||
public Stream<FileSlice> getAllFileSlicesIncludingInflight() {
|
public Stream<FileSlice> getAllFileSlicesIncludingInflight() {
|
||||||
return fileSlices.entrySet().stream().map(sliceEntry -> sliceEntry.getValue());
|
return fileSlices.entrySet().stream().map(Map.Entry::getValue);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -143,8 +137,8 @@ public class HoodieFileGroup implements Serializable {
|
|||||||
public Stream<FileSlice> getAllFileSlices() {
|
public Stream<FileSlice> getAllFileSlices() {
|
||||||
if (!timeline.empty()) {
|
if (!timeline.empty()) {
|
||||||
return fileSlices.entrySet().stream()
|
return fileSlices.entrySet().stream()
|
||||||
.map(sliceEntry -> sliceEntry.getValue())
|
.map(Map.Entry::getValue)
|
||||||
.filter(slice -> isFileSliceCommitted(slice));
|
.filter(this::isFileSliceCommitted);
|
||||||
}
|
}
|
||||||
return Stream.empty();
|
return Stream.empty();
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -103,7 +103,7 @@ public class HoodieLogFile implements Serializable {
|
|||||||
return new Integer(o2.getLogVersion()).compareTo(o1.getLogVersion());
|
return new Integer(o2.getLogVersion()).compareTo(o1.getLogVersion());
|
||||||
}
|
}
|
||||||
// reverse the order by base-commits
|
// reverse the order by base-commits
|
||||||
return new Integer(baseInstantTime2.compareTo(baseInstantTime1));
|
return baseInstantTime2.compareTo(baseInstantTime1);
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -53,14 +53,13 @@ public enum StorageSchemes {
|
|||||||
}
|
}
|
||||||
|
|
||||||
public static boolean isSchemeSupported(String scheme) {
|
public static boolean isSchemeSupported(String scheme) {
|
||||||
return Arrays.stream(values()).filter(s -> s.getScheme().equals(scheme)).count() > 0;
|
return Arrays.stream(values()).anyMatch(s -> s.getScheme().equals(scheme));
|
||||||
}
|
}
|
||||||
|
|
||||||
public static boolean isAppendSupported(String scheme) {
|
public static boolean isAppendSupported(String scheme) {
|
||||||
if (!isSchemeSupported(scheme)) {
|
if (!isSchemeSupported(scheme)) {
|
||||||
throw new IllegalArgumentException("Unsupported scheme :" + scheme);
|
throw new IllegalArgumentException("Unsupported scheme :" + scheme);
|
||||||
}
|
}
|
||||||
return Arrays.stream(StorageSchemes.values())
|
return Arrays.stream(StorageSchemes.values()).anyMatch(s -> s.supportsAppend() && s.scheme.equals(scheme));
|
||||||
.filter(s -> s.supportsAppend() && s.scheme.equals(scheme)).count() > 0;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -95,8 +95,7 @@ public class HoodieTableConfig implements Serializable {
|
|||||||
fs.mkdirs(metadataFolder);
|
fs.mkdirs(metadataFolder);
|
||||||
}
|
}
|
||||||
Path propertyPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE);
|
Path propertyPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE);
|
||||||
FSDataOutputStream outputStream = fs.create(propertyPath);
|
try (FSDataOutputStream outputStream = fs.create(propertyPath)) {
|
||||||
try {
|
|
||||||
if (!properties.containsKey(HOODIE_TABLE_NAME_PROP_NAME)) {
|
if (!properties.containsKey(HOODIE_TABLE_NAME_PROP_NAME)) {
|
||||||
throw new IllegalArgumentException(
|
throw new IllegalArgumentException(
|
||||||
HOODIE_TABLE_NAME_PROP_NAME + " property needs to be specified");
|
HOODIE_TABLE_NAME_PROP_NAME + " property needs to be specified");
|
||||||
@@ -114,8 +113,6 @@ public class HoodieTableConfig implements Serializable {
|
|||||||
}
|
}
|
||||||
properties
|
properties
|
||||||
.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
|
.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
|
||||||
} finally {
|
|
||||||
outputStream.close();
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -57,7 +57,7 @@ public class HoodieCorruptBlock extends HoodieLogBlock {
|
|||||||
long blockSize,
|
long blockSize,
|
||||||
long blockEndPos,
|
long blockEndPos,
|
||||||
Map<HeaderMetadataType, String> header,
|
Map<HeaderMetadataType, String> header,
|
||||||
Map<HeaderMetadataType, String> footer) throws IOException {
|
Map<HeaderMetadataType, String> footer) {
|
||||||
|
|
||||||
return new HoodieCorruptBlock(corruptedBytes, inputStream, readBlockLazily,
|
return new HoodieCorruptBlock(corruptedBytes, inputStream, readBlockLazily,
|
||||||
Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)),
|
Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)),
|
||||||
|
|||||||
@@ -91,10 +91,10 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
|
|||||||
List<HoodieInstant> pendingCompactionInstants =
|
List<HoodieInstant> pendingCompactionInstants =
|
||||||
metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants().collect(Collectors.toList());
|
metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants().collect(Collectors.toList());
|
||||||
this.fileIdToPendingCompaction = ImmutableMap.copyOf(
|
this.fileIdToPendingCompaction = ImmutableMap.copyOf(
|
||||||
CompactionUtils.getAllPendingCompactionOperations(metaClient).entrySet().stream().map(entry -> {
|
CompactionUtils.getAllPendingCompactionOperations(metaClient).entrySet().stream()
|
||||||
return Pair.of(entry.getKey(), Pair.of(entry.getValue().getKey(),
|
.map(entry -> Pair.of(entry.getKey(), Pair.of(entry.getValue().getKey(),
|
||||||
CompactionOperation.convertFromAvroRecordInstance(entry.getValue().getValue())));
|
CompactionOperation.convertFromAvroRecordInstance(entry.getValue().getValue()))))
|
||||||
}).collect(Collectors.toMap(Pair::getKey, Pair::getValue)));
|
.collect(Collectors.toMap(Pair::getKey, Pair::getValue)));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -152,10 +152,10 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
|
|||||||
String fileId = pair.getValue();
|
String fileId = pair.getValue();
|
||||||
HoodieFileGroup group = new HoodieFileGroup(pair.getKey(), fileId, visibleActiveTimeline);
|
HoodieFileGroup group = new HoodieFileGroup(pair.getKey(), fileId, visibleActiveTimeline);
|
||||||
if (dataFiles.containsKey(pair)) {
|
if (dataFiles.containsKey(pair)) {
|
||||||
dataFiles.get(pair).forEach(dataFile -> group.addDataFile(dataFile));
|
dataFiles.get(pair).forEach(group::addDataFile);
|
||||||
}
|
}
|
||||||
if (logFiles.containsKey(pair)) {
|
if (logFiles.containsKey(pair)) {
|
||||||
logFiles.get(pair).forEach(logFile -> group.addLogFile(logFile));
|
logFiles.get(pair).forEach(group::addLogFile);
|
||||||
}
|
}
|
||||||
if (fileIdToPendingCompaction.containsKey(fileId)) {
|
if (fileIdToPendingCompaction.containsKey(fileId)) {
|
||||||
// If there is no delta-commit after compaction request, this step would ensure a new file-slice appears
|
// If there is no delta-commit after compaction request, this step would ensure a new file-slice appears
|
||||||
@@ -219,9 +219,7 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
|
|||||||
@Override
|
@Override
|
||||||
public Stream<HoodieDataFile> getLatestDataFiles() {
|
public Stream<HoodieDataFile> getLatestDataFiles() {
|
||||||
return fileGroupMap.values().stream()
|
return fileGroupMap.values().stream()
|
||||||
.map(fileGroup -> {
|
.map(fileGroup -> fileGroup.getAllDataFiles().filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst())
|
||||||
return fileGroup.getAllDataFiles().filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst();
|
|
||||||
})
|
|
||||||
.filter(Optional::isPresent)
|
.filter(Optional::isPresent)
|
||||||
.map(Optional::get);
|
.map(Optional::get);
|
||||||
}
|
}
|
||||||
@@ -230,15 +228,13 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
|
|||||||
public Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionPath,
|
public Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionPath,
|
||||||
String maxCommitTime) {
|
String maxCommitTime) {
|
||||||
return getAllFileGroups(partitionPath)
|
return getAllFileGroups(partitionPath)
|
||||||
.map(fileGroup -> {
|
.map(fileGroup -> fileGroup.getAllDataFiles()
|
||||||
return fileGroup.getAllDataFiles()
|
|
||||||
.filter(dataFile ->
|
.filter(dataFile ->
|
||||||
HoodieTimeline.compareTimestamps(dataFile.getCommitTime(),
|
HoodieTimeline.compareTimestamps(dataFile.getCommitTime(),
|
||||||
maxCommitTime,
|
maxCommitTime,
|
||||||
HoodieTimeline.LESSER_OR_EQUAL))
|
HoodieTimeline.LESSER_OR_EQUAL))
|
||||||
.filter(df -> !isDataFileDueToPendingCompaction(df))
|
.filter(df -> !isDataFileDueToPendingCompaction(df))
|
||||||
.findFirst();
|
.findFirst())
|
||||||
})
|
|
||||||
.filter(Optional::isPresent)
|
.filter(Optional::isPresent)
|
||||||
.map(Optional::get);
|
.map(Optional::get);
|
||||||
}
|
}
|
||||||
@@ -246,12 +242,10 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
|
|||||||
@Override
|
@Override
|
||||||
public Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn) {
|
public Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn) {
|
||||||
return fileGroupMap.values().stream()
|
return fileGroupMap.values().stream()
|
||||||
.map(fileGroup -> {
|
.map(fileGroup -> fileGroup.getAllDataFiles()
|
||||||
return fileGroup.getAllDataFiles()
|
|
||||||
.filter(dataFile -> commitsToReturn.contains(dataFile.getCommitTime())
|
.filter(dataFile -> commitsToReturn.contains(dataFile.getCommitTime())
|
||||||
&& !isDataFileDueToPendingCompaction(dataFile))
|
&& !isDataFileDueToPendingCompaction(dataFile))
|
||||||
.findFirst();
|
.findFirst())
|
||||||
})
|
|
||||||
.filter(Optional::isPresent)
|
.filter(Optional::isPresent)
|
||||||
.map(Optional::get);
|
.map(Optional::get);
|
||||||
}
|
}
|
||||||
@@ -259,15 +253,13 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
|
|||||||
@Override
|
@Override
|
||||||
public Stream<HoodieDataFile> getLatestDataFilesOn(String partitionPath, String instantTime) {
|
public Stream<HoodieDataFile> getLatestDataFilesOn(String partitionPath, String instantTime) {
|
||||||
return getAllFileGroups(partitionPath)
|
return getAllFileGroups(partitionPath)
|
||||||
.map(fileGroup -> {
|
.map(fileGroup -> fileGroup.getAllDataFiles()
|
||||||
return fileGroup.getAllDataFiles()
|
|
||||||
.filter(dataFile ->
|
.filter(dataFile ->
|
||||||
HoodieTimeline.compareTimestamps(dataFile.getCommitTime(),
|
HoodieTimeline.compareTimestamps(dataFile.getCommitTime(),
|
||||||
instantTime,
|
instantTime,
|
||||||
HoodieTimeline.EQUAL))
|
HoodieTimeline.EQUAL))
|
||||||
.filter(df -> !isDataFileDueToPendingCompaction(df))
|
.filter(df -> !isDataFileDueToPendingCompaction(df))
|
||||||
.findFirst();
|
.findFirst())
|
||||||
})
|
|
||||||
.filter(Optional::isPresent)
|
.filter(Optional::isPresent)
|
||||||
.map(Optional::get);
|
.map(Optional::get);
|
||||||
}
|
}
|
||||||
@@ -275,7 +267,7 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
|
|||||||
@Override
|
@Override
|
||||||
public Stream<HoodieDataFile> getAllDataFiles(String partitionPath) {
|
public Stream<HoodieDataFile> getAllDataFiles(String partitionPath) {
|
||||||
return getAllFileGroups(partitionPath)
|
return getAllFileGroups(partitionPath)
|
||||||
.map(fileGroup -> fileGroup.getAllDataFiles())
|
.map(HoodieFileGroup::getAllDataFiles)
|
||||||
.flatMap(dataFileList -> dataFileList)
|
.flatMap(dataFileList -> dataFileList)
|
||||||
.filter(df -> !isDataFileDueToPendingCompaction(df));
|
.filter(df -> !isDataFileDueToPendingCompaction(df));
|
||||||
}
|
}
|
||||||
@@ -283,7 +275,7 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
|
|||||||
@Override
|
@Override
|
||||||
public Stream<FileSlice> getLatestFileSlices(String partitionPath) {
|
public Stream<FileSlice> getLatestFileSlices(String partitionPath) {
|
||||||
return getAllFileGroups(partitionPath)
|
return getAllFileGroups(partitionPath)
|
||||||
.map(fileGroup -> fileGroup.getLatestFileSlice())
|
.map(HoodieFileGroup::getLatestFileSlice)
|
||||||
.filter(Optional::isPresent)
|
.filter(Optional::isPresent)
|
||||||
.map(Optional::get)
|
.map(Optional::get)
|
||||||
.map(this::filterDataFileAfterPendingCompaction);
|
.map(this::filterDataFileAfterPendingCompaction);
|
||||||
@@ -312,11 +304,8 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
|
|||||||
*/
|
*/
|
||||||
private boolean isFileSliceAfterPendingCompaction(FileSlice fileSlice) {
|
private boolean isFileSliceAfterPendingCompaction(FileSlice fileSlice) {
|
||||||
Pair<String, CompactionOperation> compactionWithInstantTime = fileIdToPendingCompaction.get(fileSlice.getFileId());
|
Pair<String, CompactionOperation> compactionWithInstantTime = fileIdToPendingCompaction.get(fileSlice.getFileId());
|
||||||
if ((null != compactionWithInstantTime)
|
return (null != compactionWithInstantTime)
|
||||||
&& fileSlice.getBaseInstantTime().equals(compactionWithInstantTime.getKey())) {
|
&& fileSlice.getBaseInstantTime().equals(compactionWithInstantTime.getKey());
|
||||||
return true;
|
|
||||||
}
|
|
||||||
return false;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -330,7 +319,7 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
|
|||||||
// Data file is filtered out of the file-slice as the corresponding compaction
|
// Data file is filtered out of the file-slice as the corresponding compaction
|
||||||
// instant not completed yet.
|
// instant not completed yet.
|
||||||
FileSlice transformed = new FileSlice(fileSlice.getBaseInstantTime(), fileSlice.getFileId());
|
FileSlice transformed = new FileSlice(fileSlice.getBaseInstantTime(), fileSlice.getFileId());
|
||||||
fileSlice.getLogFiles().forEach(lf -> transformed.addLogFile(lf));
|
fileSlice.getLogFiles().forEach(transformed::addLogFile);
|
||||||
return transformed;
|
return transformed;
|
||||||
}
|
}
|
||||||
return fileSlice;
|
return fileSlice;
|
||||||
@@ -358,8 +347,8 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
|
|||||||
merged.setDataFile(penultimateSlice.getDataFile().get());
|
merged.setDataFile(penultimateSlice.getDataFile().get());
|
||||||
}
|
}
|
||||||
// Add Log files from penultimate and last slices
|
// Add Log files from penultimate and last slices
|
||||||
penultimateSlice.getLogFiles().forEach(lf -> merged.addLogFile(lf));
|
penultimateSlice.getLogFiles().forEach(merged::addLogFile);
|
||||||
lastSlice.getLogFiles().forEach(lf -> merged.addLogFile(lf));
|
lastSlice.getLogFiles().forEach(merged::addLogFile);
|
||||||
return merged;
|
return merged;
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -409,7 +398,7 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
|
|||||||
@Override
|
@Override
|
||||||
public Stream<FileSlice> getAllFileSlices(String partitionPath) {
|
public Stream<FileSlice> getAllFileSlices(String partitionPath) {
|
||||||
return getAllFileGroups(partitionPath)
|
return getAllFileGroups(partitionPath)
|
||||||
.map(group -> group.getAllFileSlices())
|
.map(HoodieFileGroup::getAllFileSlices)
|
||||||
.flatMap(sliceList -> sliceList);
|
.flatMap(sliceList -> sliceList);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -80,10 +80,8 @@ public class ParquetUtils {
|
|||||||
Schema readSchema = HoodieAvroUtils.getRecordKeySchema();
|
Schema readSchema = HoodieAvroUtils.getRecordKeySchema();
|
||||||
AvroReadSupport.setAvroReadSchema(conf, readSchema);
|
AvroReadSupport.setAvroReadSchema(conf, readSchema);
|
||||||
AvroReadSupport.setRequestedProjection(conf, readSchema);
|
AvroReadSupport.setRequestedProjection(conf, readSchema);
|
||||||
ParquetReader reader = null;
|
|
||||||
Set<String> rowKeys = new HashSet<>();
|
Set<String> rowKeys = new HashSet<>();
|
||||||
try {
|
try (ParquetReader reader = AvroParquetReader.builder(filePath).withConf(conf).build()) {
|
||||||
reader = AvroParquetReader.builder(filePath).withConf(conf).build();
|
|
||||||
Object obj = reader.read();
|
Object obj = reader.read();
|
||||||
while (obj != null) {
|
while (obj != null) {
|
||||||
if (obj instanceof GenericRecord) {
|
if (obj instanceof GenericRecord) {
|
||||||
@@ -97,15 +95,8 @@ public class ParquetUtils {
|
|||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new HoodieIOException("Failed to read row keys from Parquet " + filePath, e);
|
throw new HoodieIOException("Failed to read row keys from Parquet " + filePath, e);
|
||||||
|
|
||||||
} finally {
|
}
|
||||||
if (reader != null) {
|
|
||||||
try {
|
|
||||||
reader.close();
|
|
||||||
} catch (IOException e) {
|
|
||||||
// ignore
|
// ignore
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return rowKeys;
|
return rowKeys;
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -212,10 +203,7 @@ public class ParquetUtils {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Boolean apply(String recordKey) {
|
public Boolean apply(String recordKey) {
|
||||||
if (candidateKeys.contains(recordKey)) {
|
return candidateKeys.contains(recordKey);
|
||||||
return true;
|
|
||||||
}
|
|
||||||
return false;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -83,7 +83,7 @@ public class ReflectionUtils {
|
|||||||
*/
|
*/
|
||||||
public static Object loadClass(String clazz, Object... constructorArgs) {
|
public static Object loadClass(String clazz, Object... constructorArgs) {
|
||||||
Class<?>[] constructorArgTypes = Arrays.stream(constructorArgs)
|
Class<?>[] constructorArgTypes = Arrays.stream(constructorArgs)
|
||||||
.map(arg -> arg.getClass()).toArray(Class<?>[]::new);
|
.map(Object::getClass).toArray(Class<?>[]::new);
|
||||||
return loadClass(clazz, constructorArgTypes, constructorArgs);
|
return loadClass(clazz, constructorArgTypes, constructorArgs);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -131,9 +131,9 @@ public abstract class AbstractRealtimeRecordReader {
|
|||||||
builder.append(String.format("(Size: %s)[", values.length));
|
builder.append(String.format("(Size: %s)[", values.length));
|
||||||
for (Writable w : values) {
|
for (Writable w : values) {
|
||||||
if (w instanceof ArrayWritable) {
|
if (w instanceof ArrayWritable) {
|
||||||
builder.append(arrayWritableToString((ArrayWritable) w) + " ");
|
builder.append(arrayWritableToString((ArrayWritable) w)).append(" ");
|
||||||
} else {
|
} else {
|
||||||
builder.append(w + " ");
|
builder.append(w).append(" ");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
builder.append("]");
|
builder.append("]");
|
||||||
@@ -186,7 +186,7 @@ public abstract class AbstractRealtimeRecordReader {
|
|||||||
if (field == null) {
|
if (field == null) {
|
||||||
throw new HoodieException("Field " + fn + " not found in log schema. Query cannot proceed! "
|
throw new HoodieException("Field " + fn + " not found in log schema. Query cannot proceed! "
|
||||||
+ "Derived Schema Fields: "
|
+ "Derived Schema Fields: "
|
||||||
+ schemaFieldsMap.keySet().stream().collect(Collectors.toList()));
|
+ new ArrayList<>(schemaFieldsMap.keySet()));
|
||||||
}
|
}
|
||||||
projectedFields
|
projectedFields
|
||||||
.add(new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultValue()));
|
.add(new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultValue()));
|
||||||
|
|||||||
@@ -58,7 +58,7 @@ object AvroConversionUtils {
|
|||||||
|
|
||||||
def createConverterToAvro(dataType: DataType,
|
def createConverterToAvro(dataType: DataType,
|
||||||
structName: String,
|
structName: String,
|
||||||
recordNamespace: String): (Any) => Any = {
|
recordNamespace: String): Any => Any = {
|
||||||
dataType match {
|
dataType match {
|
||||||
case BinaryType => (item: Any) =>
|
case BinaryType => (item: Any) =>
|
||||||
item match {
|
item match {
|
||||||
|
|||||||
@@ -79,7 +79,7 @@ object DataSourceWriteOptions {
|
|||||||
val BULK_INSERT_OPERATION_OPT_VAL = "bulk_insert"
|
val BULK_INSERT_OPERATION_OPT_VAL = "bulk_insert"
|
||||||
val INSERT_OPERATION_OPT_VAL = "insert"
|
val INSERT_OPERATION_OPT_VAL = "insert"
|
||||||
val UPSERT_OPERATION_OPT_VAL = "upsert"
|
val UPSERT_OPERATION_OPT_VAL = "upsert"
|
||||||
val DEFAULT_OPERATION_OPT_VAL = UPSERT_OPERATION_OPT_VAL;
|
val DEFAULT_OPERATION_OPT_VAL = UPSERT_OPERATION_OPT_VAL
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The storage type for the underlying data, for this write.
|
* The storage type for the underlying data, for this write.
|
||||||
|
|||||||
@@ -56,7 +56,7 @@ class IncrementalRelation(val sqlContext: SQLContext,
|
|||||||
// TODO : Figure out a valid HoodieWriteConfig
|
// TODO : Figure out a valid HoodieWriteConfig
|
||||||
val hoodieTable = HoodieTable.getHoodieTable(metaClient, HoodieWriteConfig.newBuilder().withPath(basePath).build(),
|
val hoodieTable = HoodieTable.getHoodieTable(metaClient, HoodieWriteConfig.newBuilder().withPath(basePath).build(),
|
||||||
sqlContext.sparkContext)
|
sqlContext.sparkContext)
|
||||||
val commitTimeline = hoodieTable.getMetaClient.getCommitTimeline.filterCompletedInstants();
|
val commitTimeline = hoodieTable.getMetaClient.getCommitTimeline.filterCompletedInstants()
|
||||||
if (commitTimeline.empty()) {
|
if (commitTimeline.empty()) {
|
||||||
throw new HoodieException("No instants to incrementally pull")
|
throw new HoodieException("No instants to incrementally pull")
|
||||||
}
|
}
|
||||||
@@ -90,7 +90,7 @@ class IncrementalRelation(val sqlContext: SQLContext,
|
|||||||
}
|
}
|
||||||
// unset the path filter, otherwise if end_instant_time is not the latest instant, path filter set for RO view
|
// unset the path filter, otherwise if end_instant_time is not the latest instant, path filter set for RO view
|
||||||
// will filter out all the files incorrectly.
|
// will filter out all the files incorrectly.
|
||||||
sqlContext.sparkContext.hadoopConfiguration.unset("mapreduce.input.pathFilter.class");
|
sqlContext.sparkContext.hadoopConfiguration.unset("mapreduce.input.pathFilter.class")
|
||||||
val sOpts = optParams.filter(p => !p._1.equalsIgnoreCase("path"))
|
val sOpts = optParams.filter(p => !p._1.equalsIgnoreCase("path"))
|
||||||
sqlContext.read.options(sOpts)
|
sqlContext.read.options(sOpts)
|
||||||
.schema(latestSchema) // avoid AnalysisException for empty input
|
.schema(latestSchema) // avoid AnalysisException for empty input
|
||||||
|
|||||||
@@ -99,7 +99,7 @@ public class HoodieSnapshotCopier implements Serializable {
|
|||||||
Path outputPath = new Path(outputDir);
|
Path outputPath = new Path(outputDir);
|
||||||
if (fs.exists(outputPath)) {
|
if (fs.exists(outputPath)) {
|
||||||
logger.warn(
|
logger.warn(
|
||||||
String.format("The output path %targetBasePath already exists, deleting", outputPath));
|
String.format("The output path %s targetBasePath already exists, deleting", outputPath));
|
||||||
fs.delete(new Path(outputDir), true);
|
fs.delete(new Path(outputDir), true);
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -155,7 +155,7 @@ public class HoodieSnapshotCopier implements Serializable {
|
|||||||
}
|
}
|
||||||
if (fs.exists(targetFilePath)) {
|
if (fs.exists(targetFilePath)) {
|
||||||
logger.error(String.format(
|
logger.error(String.format(
|
||||||
"The target output commit file (%targetBasePath) already exists.", targetFilePath));
|
"The target output commit file (%s targetBasePath) already exists.", targetFilePath));
|
||||||
}
|
}
|
||||||
FileUtil.copy(fs, commitStatus.getPath(), fs, targetFilePath, false, fs.getConf());
|
FileUtil.copy(fs, commitStatus.getPath(), fs, targetFilePath, false, fs.getConf());
|
||||||
}
|
}
|
||||||
@@ -166,7 +166,8 @@ public class HoodieSnapshotCopier implements Serializable {
|
|||||||
// Create the _SUCCESS tag
|
// Create the _SUCCESS tag
|
||||||
Path successTagPath = new Path(outputDir + "/_SUCCESS");
|
Path successTagPath = new Path(outputDir + "/_SUCCESS");
|
||||||
if (!fs.exists(successTagPath)) {
|
if (!fs.exists(successTagPath)) {
|
||||||
logger.info("Creating _SUCCESS under targetBasePath: " + outputDir);
|
logger.info(String.format(
|
||||||
|
"Creating _SUCCESS under targetBasePath: $s", outputDir));
|
||||||
fs.createNewFile(successTagPath);
|
fs.createNewFile(successTagPath);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@@ -175,7 +176,7 @@ public class HoodieSnapshotCopier implements Serializable {
|
|||||||
// Take input configs
|
// Take input configs
|
||||||
final Config cfg = new Config();
|
final Config cfg = new Config();
|
||||||
new JCommander(cfg, args);
|
new JCommander(cfg, args);
|
||||||
logger.info(String.format("Snapshot hoodie table from %targetBasePath to %targetBasePath",
|
logger.info(String.format("Snapshot hoodie table from %s targetBasePath to %stargetBasePath",
|
||||||
cfg.basePath, cfg.outputPath));
|
cfg.basePath, cfg.outputPath));
|
||||||
|
|
||||||
// Create a spark job to do the snapshot copy
|
// Create a spark job to do the snapshot copy
|
||||||
|
|||||||
@@ -95,14 +95,8 @@ public class UtilHelpers {
|
|||||||
}
|
}
|
||||||
long len = fs.getFileStatus(p).getLen();
|
long len = fs.getFileStatus(p).getLen();
|
||||||
ByteBuffer buf = ByteBuffer.allocate((int) len);
|
ByteBuffer buf = ByteBuffer.allocate((int) len);
|
||||||
FSDataInputStream inputStream = null;
|
try (FSDataInputStream inputStream = fs.open(p)) {
|
||||||
try {
|
|
||||||
inputStream = fs.open(p);
|
|
||||||
inputStream.readFully(0, buf.array(), 0, buf.array().length);
|
inputStream.readFully(0, buf.array(), 0, buf.array().length);
|
||||||
} finally {
|
|
||||||
if (inputStream != null) {
|
|
||||||
inputStream.close();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
return new String(buf.array());
|
return new String(buf.array());
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -28,6 +28,7 @@ import java.io.Serializable;
|
|||||||
import java.text.ParseException;
|
import java.text.ParseException;
|
||||||
import java.text.SimpleDateFormat;
|
import java.text.SimpleDateFormat;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.Date;
|
import java.util.Date;
|
||||||
import java.util.TimeZone;
|
import java.util.TimeZone;
|
||||||
import org.apache.avro.generic.GenericRecord;
|
import org.apache.avro.generic.GenericRecord;
|
||||||
@@ -73,7 +74,7 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
|
|||||||
|
|
||||||
if (timestampType == TimestampType.DATE_STRING || timestampType == TimestampType.MIXED) {
|
if (timestampType == TimestampType.DATE_STRING || timestampType == TimestampType.MIXED) {
|
||||||
DataSourceUtils
|
DataSourceUtils
|
||||||
.checkRequiredProperties(config, Arrays.asList(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP));
|
.checkRequiredProperties(config, Collections.singletonList(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP));
|
||||||
this.inputDateFormat = new SimpleDateFormat(
|
this.inputDateFormat = new SimpleDateFormat(
|
||||||
config.getString(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP));
|
config.getString(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP));
|
||||||
this.inputDateFormat.setTimeZone(TimeZone.getTimeZone("GMT"));
|
this.inputDateFormat.setTimeZone(TimeZone.getTimeZone("GMT"));
|
||||||
|
|||||||
@@ -23,7 +23,8 @@ import com.uber.hoodie.common.util.FSUtils;
|
|||||||
import com.uber.hoodie.common.util.TypedProperties;
|
import com.uber.hoodie.common.util.TypedProperties;
|
||||||
import com.uber.hoodie.exception.HoodieIOException;
|
import com.uber.hoodie.exception.HoodieIOException;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Arrays;
|
import java.util.Collections;
|
||||||
|
|
||||||
import org.apache.avro.Schema;
|
import org.apache.avro.Schema;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
@@ -52,7 +53,7 @@ public class FilebasedSchemaProvider extends SchemaProvider {
|
|||||||
|
|
||||||
public FilebasedSchemaProvider(TypedProperties props, JavaSparkContext jssc) {
|
public FilebasedSchemaProvider(TypedProperties props, JavaSparkContext jssc) {
|
||||||
super(props, jssc);
|
super(props, jssc);
|
||||||
DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.SOURCE_SCHEMA_FILE_PROP));
|
DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.SOURCE_SCHEMA_FILE_PROP));
|
||||||
this.fs = FSUtils.getFs(props.getString(Config.SOURCE_SCHEMA_FILE_PROP), jssc.hadoopConfiguration());
|
this.fs = FSUtils.getFs(props.getString(Config.SOURCE_SCHEMA_FILE_PROP), jssc.hadoopConfiguration());
|
||||||
try {
|
try {
|
||||||
this.sourceSchema = new Schema.Parser().parse(
|
this.sourceSchema = new Schema.Parser().parse(
|
||||||
|
|||||||
@@ -25,7 +25,8 @@ import com.uber.hoodie.common.util.TypedProperties;
|
|||||||
import com.uber.hoodie.exception.HoodieIOException;
|
import com.uber.hoodie.exception.HoodieIOException;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.net.URL;
|
import java.net.URL;
|
||||||
import java.util.Arrays;
|
import java.util.Collections;
|
||||||
|
|
||||||
import org.apache.avro.Schema;
|
import org.apache.avro.Schema;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
|
|
||||||
@@ -55,7 +56,7 @@ public class SchemaRegistryProvider extends SchemaProvider {
|
|||||||
|
|
||||||
public SchemaRegistryProvider(TypedProperties props, JavaSparkContext jssc) {
|
public SchemaRegistryProvider(TypedProperties props, JavaSparkContext jssc) {
|
||||||
super(props, jssc);
|
super(props, jssc);
|
||||||
DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.SCHEMA_REGISTRY_URL_PROP));
|
DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.SCHEMA_REGISTRY_URL_PROP));
|
||||||
String registryUrl = props.getString(Config.SCHEMA_REGISTRY_URL_PROP);
|
String registryUrl = props.getString(Config.SCHEMA_REGISTRY_URL_PROP);
|
||||||
try {
|
try {
|
||||||
this.schema = new Schema.Parser().parse(fetchSchemaFromRegistry(registryUrl));
|
this.schema = new Schema.Parser().parse(fetchSchemaFromRegistry(registryUrl));
|
||||||
|
|||||||
@@ -87,7 +87,7 @@ public class AvroConvertor implements Serializable {
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
public GenericRecord fromAvroBinary(byte[] avroBinary) throws IOException {
|
public GenericRecord fromAvroBinary(byte[] avroBinary) {
|
||||||
initSchema();
|
initSchema();
|
||||||
initInjection();
|
initInjection();
|
||||||
return recordInjection.invert(avroBinary).get();
|
return recordInjection.invert(avroBinary).get();
|
||||||
|
|||||||
@@ -39,9 +39,8 @@ public class AvroKafkaSource extends KafkaSource {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected JavaRDD<GenericRecord> toAvroRDD(OffsetRange[] offsetRanges, AvroConvertor avroConvertor) {
|
protected JavaRDD<GenericRecord> toAvroRDD(OffsetRange[] offsetRanges, AvroConvertor avroConvertor) {
|
||||||
JavaRDD<GenericRecord> recordRDD = KafkaUtils
|
return KafkaUtils
|
||||||
.createRDD(sparkContext, String.class, Object.class, StringDecoder.class, KafkaAvroDecoder.class, kafkaParams,
|
.createRDD(sparkContext, String.class, Object.class, StringDecoder.class, KafkaAvroDecoder.class, kafkaParams,
|
||||||
offsetRanges).values().map(obj -> (GenericRecord) obj);
|
offsetRanges).values().map(obj -> (GenericRecord) obj);
|
||||||
return recordRDD;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -26,10 +26,7 @@ import com.uber.hoodie.common.util.collection.Pair;
|
|||||||
import com.uber.hoodie.exception.HoodieIOException;
|
import com.uber.hoodie.exception.HoodieIOException;
|
||||||
import com.uber.hoodie.utilities.schema.SchemaProvider;
|
import com.uber.hoodie.utilities.schema.SchemaProvider;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.*;
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import org.apache.avro.generic.GenericRecord;
|
import org.apache.avro.generic.GenericRecord;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
@@ -58,7 +55,7 @@ public abstract class DFSSource extends Source {
|
|||||||
|
|
||||||
public DFSSource(TypedProperties props, JavaSparkContext sparkContext, SchemaProvider schemaProvider) {
|
public DFSSource(TypedProperties props, JavaSparkContext sparkContext, SchemaProvider schemaProvider) {
|
||||||
super(props, sparkContext, schemaProvider);
|
super(props, sparkContext, schemaProvider);
|
||||||
DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.ROOT_INPUT_PATH_PROP));
|
DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.ROOT_INPUT_PATH_PROP));
|
||||||
this.fs = FSUtils.getFs(props.getString(Config.ROOT_INPUT_PATH_PROP), sparkContext.hadoopConfiguration());
|
this.fs = FSUtils.getFs(props.getString(Config.ROOT_INPUT_PATH_PROP), sparkContext.hadoopConfiguration());
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -75,16 +72,14 @@ public abstract class DFSSource extends Source {
|
|||||||
new Path(props.getString(Config.ROOT_INPUT_PATH_PROP)), true);
|
new Path(props.getString(Config.ROOT_INPUT_PATH_PROP)), true);
|
||||||
while (fitr.hasNext()) {
|
while (fitr.hasNext()) {
|
||||||
LocatedFileStatus fileStatus = fitr.next();
|
LocatedFileStatus fileStatus = fitr.next();
|
||||||
if (fileStatus.isDirectory() || IGNORE_FILEPREFIX_LIST.stream().filter(
|
if (fileStatus.isDirectory() || IGNORE_FILEPREFIX_LIST.stream()
|
||||||
pfx -> fileStatus.getPath().getName().startsWith(pfx)).count() > 0) {
|
.anyMatch(pfx -> fileStatus.getPath().getName().startsWith(pfx))) {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
eligibleFiles.add(fileStatus);
|
eligibleFiles.add(fileStatus);
|
||||||
}
|
}
|
||||||
// sort them by modification time.
|
// sort them by modification time.
|
||||||
eligibleFiles.sort((FileStatus f1, FileStatus f2) -> Long.valueOf(f1.getModificationTime())
|
eligibleFiles.sort(Comparator.comparingLong(FileStatus::getModificationTime));
|
||||||
.compareTo(Long.valueOf(
|
|
||||||
f2.getModificationTime())));
|
|
||||||
|
|
||||||
// Filter based on checkpoint & input size, if needed
|
// Filter based on checkpoint & input size, if needed
|
||||||
long currentBytes = 0;
|
long currentBytes = 0;
|
||||||
@@ -110,8 +105,7 @@ public abstract class DFSSource extends Source {
|
|||||||
// no data to read
|
// no data to read
|
||||||
if (filteredFiles.size() == 0) {
|
if (filteredFiles.size() == 0) {
|
||||||
return new ImmutablePair<>(Optional.empty(),
|
return new ImmutablePair<>(Optional.empty(),
|
||||||
lastCheckpointStr.isPresent() ? lastCheckpointStr.get()
|
lastCheckpointStr.orElseGet(() -> String.valueOf(Long.MIN_VALUE)));
|
||||||
: String.valueOf(Long.MIN_VALUE));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// read the files out.
|
// read the files out.
|
||||||
|
|||||||
@@ -76,7 +76,7 @@ public class HiveIncrPullSource extends Source {
|
|||||||
public HiveIncrPullSource(TypedProperties props, JavaSparkContext sparkContext,
|
public HiveIncrPullSource(TypedProperties props, JavaSparkContext sparkContext,
|
||||||
SchemaProvider schemaProvider) {
|
SchemaProvider schemaProvider) {
|
||||||
super(props, sparkContext, schemaProvider);
|
super(props, sparkContext, schemaProvider);
|
||||||
DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.ROOT_INPUT_PATH_PROP));
|
DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.ROOT_INPUT_PATH_PROP));
|
||||||
this.incrPullRootPath = props.getString(Config.ROOT_INPUT_PATH_PROP);
|
this.incrPullRootPath = props.getString(Config.ROOT_INPUT_PATH_PROP);
|
||||||
this.fs = FSUtils.getFs(incrPullRootPath, sparkContext.hadoopConfiguration());
|
this.fs = FSUtils.getFs(incrPullRootPath, sparkContext.hadoopConfiguration());
|
||||||
}
|
}
|
||||||
@@ -121,7 +121,7 @@ public class HiveIncrPullSource extends Source {
|
|||||||
|
|
||||||
if (!commitToPull.isPresent()) {
|
if (!commitToPull.isPresent()) {
|
||||||
return new ImmutablePair<>(Optional.empty(),
|
return new ImmutablePair<>(Optional.empty(),
|
||||||
lastCheckpointStr.isPresent() ? lastCheckpointStr.get() : "");
|
lastCheckpointStr.orElse(""));
|
||||||
}
|
}
|
||||||
|
|
||||||
// read the files out.
|
// read the files out.
|
||||||
|
|||||||
@@ -35,6 +35,6 @@ public class JsonDFSSource extends DFSSource {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected JavaRDD<GenericRecord> fromFiles(AvroConvertor convertor, String pathStr) {
|
protected JavaRDD<GenericRecord> fromFiles(AvroConvertor convertor, String pathStr) {
|
||||||
return sparkContext.textFile(pathStr).map((String j) -> convertor.fromJson(j));
|
return sparkContext.textFile(pathStr).map(convertor::fromJson);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -40,6 +40,6 @@ public class JsonKafkaSource extends KafkaSource {
|
|||||||
protected JavaRDD<GenericRecord> toAvroRDD(OffsetRange[] offsetRanges, AvroConvertor avroConvertor) {
|
protected JavaRDD<GenericRecord> toAvroRDD(OffsetRange[] offsetRanges, AvroConvertor avroConvertor) {
|
||||||
return KafkaUtils.createRDD(sparkContext, String.class, String.class, StringDecoder.class, StringDecoder.class,
|
return KafkaUtils.createRDD(sparkContext, String.class, String.class, StringDecoder.class, StringDecoder.class,
|
||||||
kafkaParams, offsetRanges)
|
kafkaParams, offsetRanges)
|
||||||
.values().map(jsonStr -> avroConvertor.fromJson(jsonStr));
|
.values().map(avroConvertor::fromJson);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -25,11 +25,8 @@ import com.uber.hoodie.common.util.collection.Pair;
|
|||||||
import com.uber.hoodie.exception.HoodieNotSupportedException;
|
import com.uber.hoodie.exception.HoodieNotSupportedException;
|
||||||
import com.uber.hoodie.utilities.exception.HoodieDeltaStreamerException;
|
import com.uber.hoodie.utilities.exception.HoodieDeltaStreamerException;
|
||||||
import com.uber.hoodie.utilities.schema.SchemaProvider;
|
import com.uber.hoodie.utilities.schema.SchemaProvider;
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.Comparator;
|
import java.util.*;
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.HashSet;
|
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import kafka.common.TopicAndPartition;
|
import kafka.common.TopicAndPartition;
|
||||||
import org.apache.avro.generic.GenericRecord;
|
import org.apache.avro.generic.GenericRecord;
|
||||||
@@ -85,7 +82,7 @@ public abstract class KafkaSource extends Source {
|
|||||||
*/
|
*/
|
||||||
public static String offsetsToStr(OffsetRange[] ranges) {
|
public static String offsetsToStr(OffsetRange[] ranges) {
|
||||||
StringBuilder sb = new StringBuilder();
|
StringBuilder sb = new StringBuilder();
|
||||||
// atleast 1 partition will be present.
|
// at least 1 partition will be present.
|
||||||
sb.append(ranges[0].topic() + ",");
|
sb.append(ranges[0].topic() + ",");
|
||||||
sb.append(Arrays.stream(ranges)
|
sb.append(Arrays.stream(ranges)
|
||||||
.map(r -> String.format("%s:%d", r.partition(), r.untilOffset()))
|
.map(r -> String.format("%s:%d", r.partition(), r.untilOffset()))
|
||||||
@@ -106,8 +103,7 @@ public abstract class KafkaSource extends Source {
|
|||||||
HashMap<TopicAndPartition, LeaderOffset> toOffsetMap,
|
HashMap<TopicAndPartition, LeaderOffset> toOffsetMap,
|
||||||
long numEvents) {
|
long numEvents) {
|
||||||
|
|
||||||
Comparator<OffsetRange> byPartition = (OffsetRange o1, OffsetRange o2) ->
|
Comparator<OffsetRange> byPartition = Comparator.comparing(OffsetRange::partition);
|
||||||
Integer.valueOf(o1.partition()).compareTo(Integer.valueOf(o2.partition()));
|
|
||||||
|
|
||||||
// Create initial offset ranges for each 'to' partition, with from = to offsets.
|
// Create initial offset ranges for each 'to' partition, with from = to offsets.
|
||||||
OffsetRange[] ranges = new OffsetRange[toOffsetMap.size()];
|
OffsetRange[] ranges = new OffsetRange[toOffsetMap.size()];
|
||||||
@@ -144,7 +140,7 @@ public abstract class KafkaSource extends Source {
|
|||||||
}
|
}
|
||||||
|
|
||||||
public static long totalNewMessages(OffsetRange[] ranges) {
|
public static long totalNewMessages(OffsetRange[] ranges) {
|
||||||
return Arrays.asList(ranges).stream().mapToLong(r -> r.count()).sum();
|
return Arrays.stream(ranges).mapToLong(OffsetRange::count).sum();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -166,13 +162,20 @@ public abstract class KafkaSource extends Source {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Kafka reset offset strategies
|
||||||
|
*/
|
||||||
|
enum KafkaResetOffsetStrategies {
|
||||||
|
LARGEST,
|
||||||
|
SMALLEST
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Configs to be passed for this source. All standard Kafka consumer configs are also respected
|
* Configs to be passed for this source. All standard Kafka consumer configs are also respected
|
||||||
*/
|
*/
|
||||||
static class Config {
|
static class Config {
|
||||||
private static final String KAFKA_TOPIC_NAME = "hoodie.deltastreamer.source.kafka.topic";
|
private static final String KAFKA_TOPIC_NAME = "hoodie.deltastreamer.source.kafka.topic";
|
||||||
private static final String DEFAULT_AUTO_RESET_OFFSET = "largest";
|
private static final KafkaResetOffsetStrategies DEFAULT_AUTO_RESET_OFFSET = KafkaResetOffsetStrategies.LARGEST;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@@ -187,7 +190,7 @@ public abstract class KafkaSource extends Source {
|
|||||||
for (Object prop : props.keySet()) {
|
for (Object prop : props.keySet()) {
|
||||||
kafkaParams.put(prop.toString(), props.getString(prop.toString()));
|
kafkaParams.put(prop.toString(), props.getString(prop.toString()));
|
||||||
}
|
}
|
||||||
DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.KAFKA_TOPIC_NAME));
|
DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.KAFKA_TOPIC_NAME));
|
||||||
topicName = props.getString(Config.KAFKA_TOPIC_NAME);
|
topicName = props.getString(Config.KAFKA_TOPIC_NAME);
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -200,7 +203,7 @@ public abstract class KafkaSource extends Source {
|
|||||||
// Obtain current metadata for the topic
|
// Obtain current metadata for the topic
|
||||||
KafkaCluster cluster = new KafkaCluster(ScalaHelpers.toScalaMap(kafkaParams));
|
KafkaCluster cluster = new KafkaCluster(ScalaHelpers.toScalaMap(kafkaParams));
|
||||||
Either<ArrayBuffer<Throwable>, Set<TopicAndPartition>> either = cluster.getPartitions(
|
Either<ArrayBuffer<Throwable>, Set<TopicAndPartition>> either = cluster.getPartitions(
|
||||||
ScalaHelpers.toScalaSet(new HashSet<>(Arrays.asList(topicName))));
|
ScalaHelpers.toScalaSet(new HashSet<>(Collections.singletonList(topicName))));
|
||||||
if (either.isLeft()) {
|
if (either.isLeft()) {
|
||||||
// log errors. and bail out.
|
// log errors. and bail out.
|
||||||
throw new HoodieDeltaStreamerException("Error obtaining partition metadata",
|
throw new HoodieDeltaStreamerException("Error obtaining partition metadata",
|
||||||
@@ -213,15 +216,18 @@ public abstract class KafkaSource extends Source {
|
|||||||
if (lastCheckpointStr.isPresent()) {
|
if (lastCheckpointStr.isPresent()) {
|
||||||
fromOffsets = CheckpointUtils.strToOffsets(lastCheckpointStr.get());
|
fromOffsets = CheckpointUtils.strToOffsets(lastCheckpointStr.get());
|
||||||
} else {
|
} else {
|
||||||
String autoResetValue = props
|
KafkaResetOffsetStrategies autoResetValue = KafkaResetOffsetStrategies.valueOf(
|
||||||
.getString("auto.offset.reset", Config.DEFAULT_AUTO_RESET_OFFSET);
|
props.getString("auto.offset.reset", Config.DEFAULT_AUTO_RESET_OFFSET.toString()).toUpperCase());
|
||||||
if (autoResetValue.equals("smallest")) {
|
switch (autoResetValue) {
|
||||||
|
case SMALLEST:
|
||||||
fromOffsets = new HashMap(ScalaHelpers.toJavaMap(
|
fromOffsets = new HashMap(ScalaHelpers.toJavaMap(
|
||||||
cluster.getEarliestLeaderOffsets(topicPartitions).right().get()));
|
cluster.getEarliestLeaderOffsets(topicPartitions).right().get()));
|
||||||
} else if (autoResetValue.equals("largest")) {
|
break;
|
||||||
|
case LARGEST:
|
||||||
fromOffsets = new HashMap(
|
fromOffsets = new HashMap(
|
||||||
ScalaHelpers.toJavaMap(cluster.getLatestLeaderOffsets(topicPartitions).right().get()));
|
ScalaHelpers.toJavaMap(cluster.getLatestLeaderOffsets(topicPartitions).right().get()));
|
||||||
} else {
|
break;
|
||||||
|
default:
|
||||||
throw new HoodieNotSupportedException(
|
throw new HoodieNotSupportedException(
|
||||||
"Auto reset value must be one of 'smallest' or 'largest' ");
|
"Auto reset value must be one of 'smallest' or 'largest' ");
|
||||||
}
|
}
|
||||||
@@ -236,7 +242,7 @@ public abstract class KafkaSource extends Source {
|
|||||||
OffsetRange[] offsetRanges = CheckpointUtils.computeOffsetRanges(fromOffsets, toOffsets, numEvents);
|
OffsetRange[] offsetRanges = CheckpointUtils.computeOffsetRanges(fromOffsets, toOffsets, numEvents);
|
||||||
long totalNewMsgs = CheckpointUtils.totalNewMessages(offsetRanges);
|
long totalNewMsgs = CheckpointUtils.totalNewMessages(offsetRanges);
|
||||||
if (totalNewMsgs <= 0) {
|
if (totalNewMsgs <= 0) {
|
||||||
return new ImmutablePair<>(Optional.empty(), lastCheckpointStr.isPresent() ? lastCheckpointStr.get() : "");
|
return new ImmutablePair<>(Optional.empty(), lastCheckpointStr.orElse(""));
|
||||||
} else {
|
} else {
|
||||||
log.info("About to read " + totalNewMsgs + " from Kafka for topic :" + topicName);
|
log.info("About to read " + totalNewMsgs + " from Kafka for topic :" + topicName);
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -71,7 +71,7 @@ public class TestDataSource extends Source {
|
|||||||
public Pair<Optional<JavaRDD<GenericRecord>>, String> fetchNewData(Optional<String> lastCheckpointStr,
|
public Pair<Optional<JavaRDD<GenericRecord>>, String> fetchNewData(Optional<String> lastCheckpointStr,
|
||||||
long sourceLimit) {
|
long sourceLimit) {
|
||||||
|
|
||||||
int nextCommitNum = lastCheckpointStr.isPresent() ? Integer.parseInt(lastCheckpointStr.get()) + 1 : 0;
|
int nextCommitNum = lastCheckpointStr.map(s -> Integer.parseInt(s) + 1).orElse(0);
|
||||||
String commitTime = String.format("%05d", nextCommitNum);
|
String commitTime = String.format("%05d", nextCommitNum);
|
||||||
// No new data.
|
// No new data.
|
||||||
if (sourceLimit <= 0) {
|
if (sourceLimit <= 0) {
|
||||||
|
|||||||
28
pom.xml
28
pom.xml
@@ -197,6 +197,29 @@
|
|||||||
</execution>
|
</execution>
|
||||||
</executions>
|
</executions>
|
||||||
</plugin>
|
</plugin>
|
||||||
|
<plugin>
|
||||||
|
<groupId>org.scalastyle</groupId>
|
||||||
|
<artifactId>scalastyle-maven-plugin</artifactId>
|
||||||
|
<version>1.0.0</version>
|
||||||
|
<configuration>
|
||||||
|
<verbose>false</verbose>
|
||||||
|
<failOnViolation>true</failOnViolation>
|
||||||
|
<includeTestSourceDirectory>true</includeTestSourceDirectory>
|
||||||
|
<failOnWarning>false</failOnWarning>
|
||||||
|
<sourceDirectory>${project.basedir}/src/main/scala</sourceDirectory>
|
||||||
|
<testSourceDirectory>${project.basedir}/src/test/scala</testSourceDirectory>
|
||||||
|
<configLocation>style/scalastyle-config.xml</configLocation>
|
||||||
|
<outputEncoding>UTF-8</outputEncoding>
|
||||||
|
</configuration>
|
||||||
|
<executions>
|
||||||
|
<execution>
|
||||||
|
<phase>compile</phase>
|
||||||
|
<goals>
|
||||||
|
<goal>check</goal>
|
||||||
|
</goals>
|
||||||
|
</execution>
|
||||||
|
</executions>
|
||||||
|
</plugin>
|
||||||
<plugin>
|
<plugin>
|
||||||
<groupId>org.apache.maven.plugins</groupId>
|
<groupId>org.apache.maven.plugins</groupId>
|
||||||
<artifactId>maven-compiler-plugin</artifactId>
|
<artifactId>maven-compiler-plugin</artifactId>
|
||||||
@@ -634,11 +657,6 @@
|
|||||||
<version>1.9.13</version>
|
<version>1.9.13</version>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
|
||||||
<dependency>
|
|
||||||
<groupId>junit</groupId>
|
|
||||||
<artifactId>junit</artifactId>
|
|
||||||
<version>${junit.version}</version>
|
|
||||||
</dependency>
|
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.hadoop</groupId>
|
<groupId>org.apache.hadoop</groupId>
|
||||||
<artifactId>hadoop-hdfs</artifactId>
|
<artifactId>hadoop-hdfs</artifactId>
|
||||||
|
|||||||
98
style/scalastyle-config.xml
Normal file
98
style/scalastyle-config.xml
Normal file
@@ -0,0 +1,98 @@
|
|||||||
|
<scalastyle>
|
||||||
|
<name>Scalastyle standard configuration</name>
|
||||||
|
<check level="warning" class="org.scalastyle.file.FileTabChecker" enabled="true"></check>
|
||||||
|
<check level="warning" class="org.scalastyle.file.FileLengthChecker" enabled="true">
|
||||||
|
<parameters>
|
||||||
|
<parameter name="maxFileLength"><![CDATA[800]]></parameter>
|
||||||
|
</parameters>
|
||||||
|
</check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.SpacesAfterPlusChecker" enabled="true"></check>
|
||||||
|
<check level="warning" class="org.scalastyle.file.WhitespaceEndOfLineChecker" enabled="true"></check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.SpacesBeforePlusChecker" enabled="true"></check>
|
||||||
|
<check level="warning" class="org.scalastyle.file.FileLineLengthChecker" enabled="true">
|
||||||
|
<parameters>
|
||||||
|
<parameter name="maxLineLength"><![CDATA[160]]></parameter>
|
||||||
|
<parameter name="tabSize"><![CDATA[4]]></parameter>
|
||||||
|
</parameters>
|
||||||
|
</check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.ClassNamesChecker" enabled="true">
|
||||||
|
<parameters>
|
||||||
|
<parameter name="regex"><![CDATA[[A-Z][A-Za-z]*]]></parameter>
|
||||||
|
</parameters>
|
||||||
|
</check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.ObjectNamesChecker" enabled="true">
|
||||||
|
<parameters>
|
||||||
|
<parameter name="regex"><![CDATA[[A-Z][A-Za-z]*]]></parameter>
|
||||||
|
</parameters>
|
||||||
|
</check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.PackageObjectNamesChecker" enabled="true">
|
||||||
|
<parameters>
|
||||||
|
<parameter name="regex"><![CDATA[^[a-z][A-Za-z]*$]]></parameter>
|
||||||
|
</parameters>
|
||||||
|
</check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.EqualsHashCodeChecker" enabled="true"></check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.IllegalImportsChecker" enabled="true">
|
||||||
|
<parameters>
|
||||||
|
<parameter name="illegalImports"><![CDATA[sun._,java.awt._]]></parameter>
|
||||||
|
</parameters>
|
||||||
|
</check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.ParameterNumberChecker" enabled="true">
|
||||||
|
<parameters>
|
||||||
|
<parameter name="maxParameters"><![CDATA[8]]></parameter>
|
||||||
|
</parameters>
|
||||||
|
</check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.MagicNumberChecker" enabled="true">
|
||||||
|
<parameters>
|
||||||
|
<parameter name="ignore"><![CDATA[-1,0,1,2,3]]></parameter>
|
||||||
|
</parameters>
|
||||||
|
</check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.NoWhitespaceBeforeLeftBracketChecker" enabled="true"></check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.NoWhitespaceAfterLeftBracketChecker" enabled="true"></check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.ReturnChecker" enabled="true"></check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.NullChecker" enabled="true"></check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.NoCloneChecker" enabled="true"></check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.NoFinalizeChecker" enabled="true"></check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.CovariantEqualsChecker" enabled="true"></check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.StructuralTypeChecker" enabled="true"></check>
|
||||||
|
<check level="warning" class="org.scalastyle.file.RegexChecker" enabled="true">
|
||||||
|
<parameters>
|
||||||
|
<parameter name="regex"><![CDATA[println]]></parameter>
|
||||||
|
</parameters>
|
||||||
|
</check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.NumberOfTypesChecker" enabled="true">
|
||||||
|
<parameters>
|
||||||
|
<parameter name="maxTypes"><![CDATA[30]]></parameter>
|
||||||
|
</parameters>
|
||||||
|
</check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.CyclomaticComplexityChecker" enabled="true">
|
||||||
|
<parameters>
|
||||||
|
<parameter name="maximum"><![CDATA[10]]></parameter>
|
||||||
|
</parameters>
|
||||||
|
</check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.UppercaseLChecker" enabled="true"></check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.SimplifyBooleanExpressionChecker" enabled="true"></check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.IfBraceChecker" enabled="true">
|
||||||
|
<parameters>
|
||||||
|
<parameter name="singleLineAllowed"><![CDATA[true]]></parameter>
|
||||||
|
<parameter name="doubleLineAllowed"><![CDATA[false]]></parameter>
|
||||||
|
</parameters>
|
||||||
|
</check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.MethodLengthChecker" enabled="true">
|
||||||
|
<parameters>
|
||||||
|
<parameter name="maxLength"><![CDATA[50]]></parameter>
|
||||||
|
</parameters>
|
||||||
|
</check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.MethodNamesChecker" enabled="true">
|
||||||
|
<parameters>
|
||||||
|
<parameter name="regex"><![CDATA[^[a-z][A-Za-z0-9]*$]]></parameter>
|
||||||
|
</parameters>
|
||||||
|
</check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.NumberOfMethodsInTypeChecker" enabled="true">
|
||||||
|
<parameters>
|
||||||
|
<parameter name="maxMethods"><![CDATA[30]]></parameter>
|
||||||
|
</parameters>
|
||||||
|
</check>
|
||||||
|
<check level="warning" class="org.scalastyle.scalariform.PublicMethodsHaveTypeChecker" enabled="true"></check>
|
||||||
|
<check level="warning" class="org.scalastyle.file.NewLineAtEofChecker" enabled="true"></check>
|
||||||
|
<check level="warning" class="org.scalastyle.file.NoNewLineAtEofChecker" enabled="false"></check>
|
||||||
|
</scalastyle>
|
||||||
Reference in New Issue
Block a user