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();
|
||||
rows.add(printFileSizeHistogram("ALL", s));
|
||||
|
||||
Function<Object, String> converterFunction = entry -> {
|
||||
return NumericUtils.humanReadableByteCount((Double.valueOf(entry.toString())));
|
||||
};
|
||||
Function<Object, String> converterFunction = entry ->
|
||||
NumericUtils.humanReadableByteCount((Double.valueOf(entry.toString())));
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
fieldNameToConverterMap.put("Min", converterFunction);
|
||||
fieldNameToConverterMap.put("10th", converterFunction);
|
||||
|
||||
@@ -77,7 +77,7 @@ class DedupeSparkJob(basePath: String,
|
||||
|
||||
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 latestFiles: java.util.List[HoodieDataFile] = fsView.getLatestDataFiles().collect(Collectors.toList[HoodieDataFile]())
|
||||
val filteredStatuses = latestFiles.map(f => f.getPath)
|
||||
@@ -92,8 +92,8 @@ class DedupeSparkJob(basePath: String,
|
||||
val dupeDataSql =
|
||||
s"""
|
||||
SELECT `_hoodie_record_key`, `_hoodie_partition_path`, `_hoodie_file_name`, `_hoodie_commit_time`
|
||||
FROM ${tmpTableName} h
|
||||
JOIN ${dedupeTblName} d
|
||||
FROM $tmpTableName h
|
||||
JOIN $dedupeTblName d
|
||||
ON h.`_hoodie_record_key` = d.dupe_key
|
||||
"""
|
||||
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
|
||||
dupeMap.foreach(rt => {
|
||||
val key = rt._1
|
||||
val rows = rt._2
|
||||
val (key, rows) = rt
|
||||
var maxCommit = -1L
|
||||
|
||||
rows.foreach(r => {
|
||||
@@ -129,7 +128,7 @@ class DedupeSparkJob(basePath: String,
|
||||
def fixDuplicates(dryRun: Boolean = true) = {
|
||||
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 latestFiles: java.util.List[HoodieDataFile] = fsView.getLatestDataFiles().collect(Collectors.toList[HoodieDataFile]())
|
||||
@@ -138,30 +137,28 @@ class DedupeSparkJob(basePath: String,
|
||||
val dupeFixPlan = planDuplicateFix()
|
||||
|
||||
// 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 dstPath = new Path(s"${repairOutputPath}/${filePath.getName}${badSuffix}")
|
||||
LOG.info(s"Copying from ${filePath} to ${dstPath}")
|
||||
val dstPath = new Path(s"$repairOutputPath/${filePath.getName}$badSuffix")
|
||||
LOG.info(s"Copying from $filePath to $dstPath")
|
||||
FileUtil.copy(fs, filePath, fs, dstPath, false, true, fs.getConf)
|
||||
}
|
||||
}
|
||||
|
||||
// 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 badFilePath = new Path(s"${repairOutputPath}/${fileNameToPathMap(fileName).getName}.bad")
|
||||
val newFilePath = new Path(s"${repairOutputPath}/${fileNameToPathMap(fileName).getName}")
|
||||
val badFilePath = new Path(s"$repairOutputPath/${fileNameToPathMap(fileName).getName}.bad")
|
||||
val newFilePath = new Path(s"$repairOutputPath/${fileNameToPathMap(fileName).getName}")
|
||||
LOG.info(" Skipping and writing new file for : " + fileName)
|
||||
SparkHelpers.skipKeysAndWriteNewFile(commitTime, fs, badFilePath, newFilePath, dupeFixPlan(fileName))
|
||||
fs.delete(badFilePath, false)
|
||||
}
|
||||
}
|
||||
|
||||
// 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")
|
||||
val dupeKeyDF = getDupeKeyDF("fixedTbl")
|
||||
val dupeCnt = dupeKeyDF.count();
|
||||
val dupeCnt = dupeKeyDF.count()
|
||||
if (dupeCnt != 0) {
|
||||
dupeKeyDF.show()
|
||||
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.
|
||||
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 missedCnt = missedRecordKeysDF.count()
|
||||
if (missedCnt != 0) {
|
||||
@@ -180,17 +177,16 @@ class DedupeSparkJob(basePath: String,
|
||||
|
||||
println("No duplicates found & counts are in check!!!! ")
|
||||
// 4. Prepare to copy the fixed files back.
|
||||
fileNameToPathMap.foreach { case (fileName, filePath) => {
|
||||
val srcPath = new Path(s"${repairOutputPath}/${filePath.getName}")
|
||||
val dstPath = new Path(s"${basePath}/${duplicatedPartitionPath}/${filePath.getName}")
|
||||
fileNameToPathMap.foreach { case (_, filePath) =>
|
||||
val srcPath = new Path(s"$repairOutputPath/${filePath.getName}")
|
||||
val dstPath = new Path(s"$basePath/$duplicatedPartitionPath/${filePath.getName}")
|
||||
if (dryRun) {
|
||||
LOG.info(s"[JUST KIDDING!!!] Copying from ${srcPath} to ${dstPath}")
|
||||
LOG.info(s"[JUST KIDDING!!!] Copying from $srcPath to $dstPath")
|
||||
} else {
|
||||
// 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)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -94,12 +94,9 @@ class SparkHelper(sqlContext: SQLContext, fs: FileSystem) {
|
||||
* @return
|
||||
*/
|
||||
def isFileContainsKey(rowKey: String, file: String): Boolean = {
|
||||
println(s"Checking ${file} for key ${rowKey}")
|
||||
val ff = getRowKeyDF(file).filter(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}` = '${rowKey}'")
|
||||
if (ff.count() > 0)
|
||||
return true
|
||||
else
|
||||
return false
|
||||
println(s"Checking $file for key $rowKey")
|
||||
val ff = getRowKeyDF(file).filter(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}` = '$rowKey'")
|
||||
if (ff.count() > 0) true else false
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -109,7 +106,7 @@ class SparkHelper(sqlContext: SQLContext, fs: FileSystem) {
|
||||
* @param 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 foundCount = sqlContext.parquetFile(file)
|
||||
.select(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}`")
|
||||
.collect().
|
||||
filter(r => !bf.mightContain(r.getString(0))).size
|
||||
.collect().count(r => !bf.mightContain(r.getString(0)))
|
||||
val totalCount = getKeyCount(file, sqlContext)
|
||||
s"totalCount: ${totalCount}, foundCount: ${foundCount}"
|
||||
totalCount == foundCount
|
||||
|
||||
Reference in New Issue
Block a user