-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths #5723
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,9 +20,12 @@ package org.apache.spark.execution.datasources | |
| import org.apache.hadoop.conf.Configuration | ||
| import org.apache.hadoop.fs.{FileStatus, Path, PathFilter} | ||
| import org.apache.hadoop.mapred.{FileInputFormat, JobConf} | ||
| import org.apache.hudi.SparkAdapterSupport | ||
| import org.apache.spark.HoodieHadoopFSUtils | ||
| import org.apache.spark.metrics.source.HiveCatalogMetrics | ||
| import org.apache.spark.sql.SparkSession | ||
| import org.apache.spark.sql.catalyst.{InternalRow, expressions} | ||
| import org.apache.spark.sql.catalyst.expressions.{AttributeReference, BoundReference, Expression, InterpretedPredicate} | ||
| import org.apache.spark.sql.execution.datasources._ | ||
| import org.apache.spark.sql.types.StructType | ||
|
|
||
|
|
@@ -34,7 +37,77 @@ class HoodieInMemoryFileIndex(sparkSession: SparkSession, | |
| parameters: Map[String, String], | ||
| userSpecifiedSchema: Option[StructType], | ||
| fileStatusCache: FileStatusCache = NoopCache) | ||
| extends InMemoryFileIndex(sparkSession, rootPathsSpecified, parameters, userSpecifiedSchema, fileStatusCache) { | ||
| extends InMemoryFileIndex(sparkSession, rootPathsSpecified, parameters, userSpecifiedSchema, fileStatusCache) | ||
| with SparkAdapterSupport { | ||
|
|
||
| /** | ||
| * Returns all valid files grouped into partitions when the data is partitioned. If the data is unpartitioned, | ||
| * this will return a single partition with no partition values | ||
| * | ||
| * NOTE: This method replicates the one it overrides, however it uses custom method | ||
| * that accepts files starting with "." | ||
| */ | ||
| override def listFiles(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = { | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Override this to accept files starting with "." to allow log files.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why is there a need for that? That's exactly what
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is needed, as we'll get partitions in protected def listLatestBaseFiles(globbedPaths: Seq[Path], partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Map[Path, Seq[FileStatus]] = {
val partitionDirs = if (globbedPaths.isEmpty) {
fileIndex.listFiles(partitionFilters, dataFilters)
} else {
val inMemoryFileIndex = HoodieInMemoryFileIndex.create(sparkSession, globbedPaths)
inMemoryFileIndex.listFiles(partitionFilters, dataFilters)
}which will call val selectedPartitions = if (partitionSpec().partitionColumns.isEmpty) {
// Method isDataPath will filter path if it is a log file
PartitionDirectory(InternalRow.empty, allFiles().filter(f => isDataPath(f.getPath))) :: Nil
}As
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It makes sense now. Thanks for clarifying! |
||
| val selectedPartitions = if (partitionSpec().partitionColumns.isEmpty) { | ||
| PartitionDirectory(InternalRow.empty, allFiles().filter(f => isDataPath(f.getPath))) :: Nil | ||
| } else { | ||
| prunePartitions(partitionFilters, partitionSpec()).map { | ||
| case PartitionPath(values, path) => | ||
| val files: Seq[FileStatus] = leafDirToChildrenFiles.get(path) match { | ||
codope marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| case Some(existingDir) => | ||
| // Directory has children files in it, return them | ||
| existingDir.filter(f => isDataPath(f.getPath)) | ||
|
|
||
| case None => | ||
| // Directory does not exist, or has no children files | ||
| Nil | ||
| } | ||
| PartitionDirectory(values, files) | ||
| } | ||
| } | ||
| logTrace("Selected files after partition pruning:\n\t" + selectedPartitions.mkString("\n\t")) | ||
| selectedPartitions | ||
| } | ||
|
|
||
| private def isDataPath(path: Path): Boolean = { | ||
| val name = path.getName | ||
| !(name.startsWith("_") && !name.contains("=")) | ||
| } | ||
|
|
||
| private def prunePartitions( | ||
| predicates: Seq[Expression], | ||
| partitionSpec: PartitionSpec): Seq[PartitionPath] = { | ||
| val PartitionSpec(partitionColumns, partitions) = partitionSpec | ||
| val partitionColumnNames = partitionColumns.map(_.name).toSet | ||
| val partitionPruningPredicates = predicates.filter { | ||
| _.references.map(_.name).toSet.subsetOf(partitionColumnNames) | ||
| } | ||
|
|
||
| if (partitionPruningPredicates.nonEmpty) { | ||
| val predicate = partitionPruningPredicates.reduce(expressions.And) | ||
|
|
||
| val boundPredicate = sparkAdapter.createInterpretedPredicate(predicate.transform { | ||
| case a: AttributeReference => | ||
| val index = partitionColumns.indexWhere(a.name == _.name) | ||
| BoundReference(index, partitionColumns(index).dataType, nullable = true) | ||
| }) | ||
|
|
||
| val selected = partitions.filter { | ||
| case PartitionPath(values, _) => boundPredicate.eval(values) | ||
| } | ||
| logInfo { | ||
| val total = partitions.length | ||
| val selectedSize = selected.length | ||
| val percentPruned = (1 - selectedSize.toDouble / total.toDouble) * 100 | ||
| s"Selected $selectedSize partitions out of $total, " + | ||
| s"pruned ${if (total == 0) "0" else s"$percentPruned%"} partitions." | ||
| } | ||
|
|
||
| selected | ||
| } else { | ||
| partitions | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * List leaf files of given paths. This method will submit a Spark job to do parallel | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@boneanxs appreciate your effort of addressing this TODO.
Although, the better way to refactor this would be to instead change
listLatestBaseFilesto belistLatestFileSlicesand return file-slices instead of just the base-files. Right now we've essentially duplicated this method implementation in here. I'd suggest we avoid the duplication by changing the method as noted above and keeping this code mostly intact.Keep in mind: one of the goals of the refactoring of COW/MOR relations was to bring implementations closer together making them bifurcate only at a points where it's necessary (for ex, in a way they actually read file-slices) and everywhere else keep them mostly identical.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
cc @codope
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'll address this in #5722