-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-4170] Make user can use hoodie.datasource.read.paths to read necessary files #5722
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 3 commits
52b0ffb
b8671a6
fbb95e5
46c9b7c
b9420c9
2a08f50
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 |
|---|---|---|
|
|
@@ -68,11 +68,9 @@ class DefaultSource extends RelationProvider | |
| override def createRelation(sqlContext: SQLContext, | ||
| optParams: Map[String, String], | ||
| schema: StructType): BaseRelation = { | ||
| // Add default options for unspecified read options keys. | ||
| val parameters = DataSourceOptionsHelper.parametersWithReadDefaults(optParams) | ||
| val path = optParams.get("path") | ||
| val readPathsStr = optParams.get(DataSourceReadOptions.READ_PATHS.key) | ||
|
|
||
| val path = parameters.get("path") | ||
| val readPathsStr = parameters.get(DataSourceReadOptions.READ_PATHS.key) | ||
| if (path.isEmpty && readPathsStr.isEmpty) { | ||
| throw new HoodieException(s"'path' or '$READ_PATHS' or both must be specified.") | ||
| } | ||
|
|
@@ -87,6 +85,16 @@ class DefaultSource extends RelationProvider | |
| } else { | ||
| Seq.empty | ||
| } | ||
|
|
||
| // Add default options for unspecified read options keys. | ||
| val parameters = if(globPaths.nonEmpty) { | ||
| Map( | ||
| "glob.paths" -> globPaths.mkString(",") | ||
| ) ++ DataSourceOptionsHelper.parametersWithReadDefaults(optParams) | ||
|
||
| } else { | ||
| DataSourceOptionsHelper.parametersWithReadDefaults(optParams) | ||
| } | ||
|
|
||
| // Get the table base path | ||
| val tablePath = if (globPaths.nonEmpty) { | ||
| DataSourceUtils.getTablePath(fs, globPaths.toArray) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -26,9 +26,10 @@ import org.apache.hadoop.mapred.JobConf | |
| import org.apache.hudi.HoodieBaseRelation.{convertToAvroSchema, createHFileReader, generateUnsafeProjection, getPartitionPath} | ||
| import org.apache.hudi.HoodieConversionUtils.toScalaOption | ||
| import org.apache.hudi.avro.HoodieAvroUtils | ||
| import org.apache.hudi.common.config.{HoodieMetadataConfig, SerializableConfiguration} | ||
| import org.apache.hudi.common.config.HoodieMetadataConfig | ||
| import org.apache.hudi.common.fs.FSUtils | ||
| import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord} | ||
| import org.apache.hudi.common.fs.FSUtils.getRelativePartitionPath | ||
| import org.apache.hudi.common.model.{FileSlice, HoodieFileFormat, HoodieRecord} | ||
| import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} | ||
| import org.apache.hudi.common.table.view.HoodieTableFileSystemView | ||
| import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver} | ||
|
|
@@ -340,20 +341,51 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, | |
| */ | ||
| protected def collectFileSplits(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[FileSplit] | ||
|
|
||
| protected def listLatestBaseFiles(globbedPaths: Seq[Path], partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Map[Path, Seq[FileStatus]] = { | ||
| val partitionDirs = if (globbedPaths.isEmpty) { | ||
| /** | ||
| * Construct HoodieTableFileSystemView based on globPaths if specified, otherwise use the table path. | ||
| * Will perform pruning if necessary | ||
| */ | ||
| private def getHoodieFsView(globPaths: Seq[Path], partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): HoodieTableFileSystemView = { | ||
|
||
| val partitionDirs = if (globPaths.isEmpty) { | ||
| fileIndex.listFiles(partitionFilters, dataFilters) | ||
| } else { | ||
| val inMemoryFileIndex = HoodieInMemoryFileIndex.create(sparkSession, globbedPaths) | ||
| val inMemoryFileIndex = HoodieInMemoryFileIndex.create(sparkSession, globPaths) | ||
| inMemoryFileIndex.listFiles(partitionFilters, dataFilters) | ||
| } | ||
|
|
||
| val fsView = new HoodieTableFileSystemView(metaClient, timeline, partitionDirs.flatMap(_.files).toArray) | ||
| new HoodieTableFileSystemView(metaClient, timeline, partitionDirs.flatMap(_.files).toArray) | ||
| } | ||
|
|
||
| /** | ||
| * Get all latest base files with partition paths, if globPaths is empty, will listing files | ||
| * under the table path. | ||
| */ | ||
| protected def listLatestBaseFiles(globPaths: Seq[Path], partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Map[Path, Seq[FileStatus]] = { | ||
| val fsView = getHoodieFsView(globPaths, partitionFilters, dataFilters) | ||
| val latestBaseFiles = fsView.getLatestBaseFiles.iterator().asScala.toList.map(_.getFileStatus) | ||
|
|
||
| latestBaseFiles.groupBy(getPartitionPath) | ||
| } | ||
|
|
||
| /** | ||
| * Get all fileSlices(contains base files and log files if exist) from globPaths if not empty, | ||
| * otherwise will use the table path to do the listing. | ||
| */ | ||
| protected def listLatestFileSlices(globPaths: Seq[Path], partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[FileSlice] = { | ||
| val fsView = getHoodieFsView(globPaths, partitionFilters, dataFilters) | ||
| val partitionPaths = fsView.getPartitionPaths.asScala | ||
|
|
||
| if (partitionPaths.isEmpty || latestInstant.isEmpty) { | ||
|
||
| Seq() | ||
| } else { | ||
| val queryTimestamp = this.queryTimestamp.get | ||
| partitionPaths.flatMap { partitionPath => | ||
| val relativePath = getRelativePartitionPath(new Path(basePath), partitionPath) | ||
| fsView.getLatestMergedFileSlicesBeforeOrOn(relativePath, queryTimestamp).iterator().asScala.toSeq | ||
| } | ||
| } | ||
| } | ||
|
|
||
| protected def convertToExpressions(filters: Array[Filter]): Array[Expression] = { | ||
| val catalystExpressions = HoodieSparkUtils.convertToCatalystExpressions(filters, tableStructSchema) | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -104,24 +104,8 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext, | |
| val fileSlices = fileIndex.listFileSlices(convertedPartitionFilters) | ||
| buildSplits(fileSlices.values.flatten.toSeq) | ||
| } else { | ||
| val inMemoryFileIndex = HoodieInMemoryFileIndex.create(sparkSession, globPaths) | ||
| val partitionDirs = inMemoryFileIndex.listFiles(partitionFilters, dataFilters) | ||
|
|
||
| val fsView = new HoodieTableFileSystemView(metaClient, timeline, partitionDirs.flatMap(_.files).toArray) | ||
| val partitionPaths = fsView.getPartitionPaths.asScala | ||
|
|
||
| if (partitionPaths.isEmpty || latestInstant.isEmpty) { | ||
| // If this an empty table OR it has no completed commits yet, return | ||
| List.empty[HoodieMergeOnReadFileSplit] | ||
| } else { | ||
| val queryTimestamp = this.queryTimestamp.get | ||
|
|
||
| val fileSlices = partitionPaths.flatMap { partitionPath => | ||
| val relativePath = getRelativePartitionPath(new Path(basePath), partitionPath) | ||
| fsView.getLatestMergedFileSlicesBeforeOrOn(relativePath, queryTimestamp).iterator().asScala.toSeq | ||
| } | ||
| buildSplits(fileSlices) | ||
| } | ||
| val fileSlices = listLatestFileSlices(globPaths, partitionFilters, dataFilters) | ||
|
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. Hi @alexeykudinkin move this implementation to the HoodieBaseRelation, could you plz review this?
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. SG, will do
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. Hi @alexeykudinkin Could you plz review it these days, as it blocks the pr: #6046, thanks! |
||
| buildSplits(fileSlices) | ||
| } | ||
| } | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,7 +17,7 @@ | |
|
|
||
| package org.apache.hudi.functional | ||
|
|
||
| import org.apache.hadoop.fs.FileSystem | ||
| import org.apache.hadoop.fs.{FileSystem, Path} | ||
| import org.apache.hudi.HoodieConversionUtils.toJavaOption | ||
| import org.apache.hudi.common.config.HoodieMetadataConfig | ||
| import org.apache.hudi.common.model.HoodieRecord | ||
|
|
@@ -296,6 +296,39 @@ class TestCOWDataSource extends HoodieClientTestBase { | |
| assertEquals("replacecommit", commits(1)) | ||
| } | ||
|
|
||
| @Test | ||
| def testReadPathsOnCopyOnWriteTable(): Unit = { | ||
| val records1 = dataGen.generateInsertsContainsAllPartitions("001", 20) | ||
| val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2)) | ||
| inputDF1.write.format("org.apache.hudi") | ||
| .options(commonOpts) | ||
| .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) | ||
| .mode(SaveMode.Append) | ||
| .save(basePath) | ||
|
|
||
| val baseFilePath = fs.listStatus(new Path(basePath, dataGen.getPartitionPaths.head)) | ||
|
||
| .filter(!_.getPath.getName.contains("hoodie_partition_metadata")) | ||
| .filter(_.getPath.getName.endsWith("parquet")) | ||
| .map(_.getPath.toString) | ||
| .mkString(",") | ||
|
|
||
| val records2 = dataGen.generateInsertsContainsAllPartitions("002", 20) | ||
| val inputDF2 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records2), 2)) | ||
| inputDF2.write.format("org.apache.hudi") | ||
| .options(commonOpts) | ||
| // Use bulk insert here to make sure the files have different file groups. | ||
| .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL) | ||
| .mode(SaveMode.Append) | ||
| .save(basePath) | ||
|
|
||
| val hudiReadPathDF = spark.read.format("org.apache.hudi") | ||
| .option(DataSourceReadOptions.READ_PATHS.key, baseFilePath) | ||
| .load() | ||
|
|
||
| val expectedCount = records1.asScala.count(record => record.getPartitionPath == dataGen.getPartitionPaths.head) | ||
| assertEquals(expectedCount, hudiReadPathDF.count()) | ||
| } | ||
|
|
||
| @Test def testOverWriteTableModeUseReplaceAction(): Unit = { | ||
| val records1 = recordsToStrings(dataGen.generateInserts("001", 5)).toList | ||
| val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) | ||
|
|
||
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.
nit: Spacing after if