[HUDI-4170] Make user can use hoodie.datasource.read.paths to read necessary files (#5722)
* Rebase codes * Move listFileSlices to HoodieBaseRelation * Fix review * Fix style * Fix bug
This commit is contained in:
@@ -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 record1FilePaths = 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, record1FilePaths)
|
||||
.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))
|
||||
|
||||
@@ -704,6 +704,96 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
|
||||
assertEquals(partitionCounts("2021/03/03"), count7)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testReadPathsForMergeOnReadTable(): Unit = {
|
||||
// Paths only baseFiles
|
||||
val records1 = dataGen.generateInserts("001", 100)
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
|
||||
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
||||
val baseFilePath = fs.listStatus(new Path(basePath, dataGen.getPartitionPaths.head))
|
||||
.filter(_.getPath.getName.endsWith("parquet"))
|
||||
.map(_.getPath.toString)
|
||||
.mkString(",")
|
||||
val records2 = dataGen.generateUniqueDeleteRecords("002", 100)
|
||||
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records2), 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiReadPathDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(DataSourceReadOptions.READ_PATHS.key, baseFilePath)
|
||||
.load()
|
||||
|
||||
val expectedCount1 = records1.asScala.count(record => record.getPartitionPath == dataGen.getPartitionPaths.head)
|
||||
assertEquals(expectedCount1, hudiReadPathDF1.count())
|
||||
|
||||
// Paths Contains both baseFile and log files
|
||||
val logFilePath = fs.listStatus(new Path(basePath, dataGen.getPartitionPaths.head))
|
||||
.filter(_.getPath.getName.contains("log"))
|
||||
.map(_.getPath.toString)
|
||||
.mkString(",")
|
||||
|
||||
val readPaths = baseFilePath + "," + logFilePath
|
||||
val hudiReadPathDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(DataSourceReadOptions.READ_PATHS.key, readPaths)
|
||||
.load()
|
||||
|
||||
assertEquals(0, hudiReadPathDF2.count())
|
||||
}
|
||||
|
||||
@Test
|
||||
def testReadPathsForOnlyLogFiles(): Unit = {
|
||||
initMetaClient(HoodieTableType.MERGE_ON_READ)
|
||||
val records1 = dataGen.generateInsertsContainsAllPartitions("000", 20)
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2))
|
||||
inputDF1.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
// Use InMemoryIndex to generate log only mor table.
|
||||
.option(HoodieIndexConfig.INDEX_TYPE.key, IndexType.INMEMORY.toString)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
// There should no base file in the file list.
|
||||
assertTrue(DataSourceTestUtils.isLogFileOnly(basePath))
|
||||
|
||||
val logFilePath = fs.listStatus(new Path(basePath, dataGen.getPartitionPaths.head))
|
||||
.filter(_.getPath.getName.contains("log"))
|
||||
.map(_.getPath.toString)
|
||||
.mkString(",")
|
||||
|
||||
val records2 = dataGen.generateInsertsContainsAllPartitions("000", 20)
|
||||
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records2), 2))
|
||||
inputDF2.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
// Use InMemoryIndex to generate log only mor table.
|
||||
.option(HoodieIndexConfig.INDEX_TYPE.key, IndexType.INMEMORY.toString)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
// There should no base file in the file list.
|
||||
assertTrue(DataSourceTestUtils.isLogFileOnly(basePath))
|
||||
|
||||
val expectedCount1 = records1.asScala.count(record => record.getPartitionPath == dataGen.getPartitionPaths.head)
|
||||
|
||||
val hudiReadPathDF = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(DataSourceReadOptions.READ_PATHS.key, logFilePath)
|
||||
.load()
|
||||
|
||||
assertEquals(expectedCount1, hudiReadPathDF.count())
|
||||
}
|
||||
|
||||
@Test
|
||||
def testReadLogOnlyMergeOnReadTable(): Unit = {
|
||||
initMetaClient(HoodieTableType.MERGE_ON_READ)
|
||||
|
||||
Reference in New Issue
Block a user