1
0

[HUDI-1371] [HUDI-1893] Support metadata based listing for Spark DataSource and Spark SQL (#2893)

This commit is contained in:
Udit Mehrotra
2021-08-03 14:47:40 -07:00
committed by GitHub
parent 245e1fd17d
commit 1ff2d3459a
13 changed files with 383 additions and 125 deletions

View File

@@ -53,6 +53,10 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
HoodieWriteConfig.TABLE_NAME.key -> "hoodie_test"
)
var queryOpts = Map(
DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key -> DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL
)
@BeforeEach override def setUp() {
initPath()
initSparkContexts()
@@ -60,6 +64,8 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
initTestDataGenerator()
initFileSystem()
initMetaClient()
queryOpts = queryOpts ++ Map("path" -> basePath)
}
@ParameterizedTest
@@ -74,7 +80,7 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
.mode(SaveMode.Overwrite)
.save(basePath)
metaClient = HoodieTableMetaClient.reload(metaClient)
val fileIndex = HoodieFileIndex(spark, metaClient, None, Map("path" -> basePath))
val fileIndex = HoodieFileIndex(spark, metaClient, None, queryOpts)
assertEquals("partition", fileIndex.partitionSchema.fields.map(_.name).mkString(","))
}
@@ -96,7 +102,7 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
.mode(SaveMode.Overwrite)
.save(basePath)
metaClient = HoodieTableMetaClient.reload(metaClient)
val fileIndex = HoodieFileIndex(spark, metaClient, None, Map("path" -> basePath))
val fileIndex = HoodieFileIndex(spark, metaClient, None, queryOpts)
assertEquals("partition", fileIndex.partitionSchema.fields.map(_.name).mkString(","))
}
@@ -115,7 +121,7 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
.mode(SaveMode.Overwrite)
.save(basePath)
metaClient = HoodieTableMetaClient.reload(metaClient)
val fileIndex = HoodieFileIndex(spark, metaClient, None, Map("path" -> basePath))
val fileIndex = HoodieFileIndex(spark, metaClient, None, queryOpts)
assertEquals("partition", fileIndex.partitionSchema.fields.map(_.name).mkString(","))
}
@@ -133,7 +139,7 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
.mode(SaveMode.Overwrite)
.save(basePath)
metaClient = HoodieTableMetaClient.reload(metaClient)
val fileIndex = HoodieFileIndex(spark, metaClient, None, Map("path" -> basePath))
val fileIndex = HoodieFileIndex(spark, metaClient, None, queryOpts)
val partitionFilter1 = EqualTo(attribute("partition"), literal("2021/03/08"))
val partitionName = if (partitionEncode) PartitionPathEncodeUtils.escapePathName("2021/03/08")
@@ -176,7 +182,8 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
.mode(SaveMode.Overwrite)
.save(basePath)
metaClient = HoodieTableMetaClient.reload(metaClient)
val fileIndex = HoodieFileIndex(spark, metaClient, None, Map("path" -> basePath))
val fileIndex = HoodieFileIndex(spark, metaClient, None,
queryOpts ++ Map(HoodieMetadataConfig.METADATA_ENABLE_PROP.key -> useMetaFileList.toString))
val partitionFilter1 = And(
EqualTo(attribute("dt"), literal("2021-03-01")),
@@ -190,7 +197,9 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
assertEquals(partitionValues.toSeq(Seq(StringType)).mkString(","), "2021-03-01,10")
assertEquals(getFileCountInPartitionPath("2021-03-01/10"), filesAfterPrune.size)
val readDF1 = spark.read.format("hudi").load(basePath)
val readDF1 = spark.read.format("hudi")
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key(), useMetaFileList)
.load(basePath)
assertEquals(10, readDF1.count())
assertEquals(5, readDF1.filter("dt = '2021-03-01' and hh = '10'").count())
@@ -206,6 +215,7 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
.option(PARTITIONPATH_FIELD_OPT_KEY.key, "dt,hh")
.option(KEYGENERATOR_CLASS_OPT_KEY.key, classOf[ComplexKeyGenerator].getName)
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY.key, "false")
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key(), useMetaFileList)
.mode(SaveMode.Overwrite)
.save(basePath)
@@ -224,7 +234,9 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
// The returned file size should equal to the whole file size in all the partition paths.
assertEquals(getFileCountInPartitionPaths("2021/03/01/10", "2021/03/02/10"),
filesAfterPrune2.length)
val readDF2 = spark.read.format("hudi").load(basePath)
val readDF2 = spark.read.format("hudi")
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, useMetaFileList)
.load(basePath)
assertEquals(10, readDF2.count())
// There are 5 rows in the dt = 2021/03/01 and hh = 10

View File

@@ -41,7 +41,7 @@ import org.joda.time.format.DateTimeFormat
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue, fail}
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import org.junit.jupiter.params.provider.{CsvSource, ValueSource}
/**
@@ -160,9 +160,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
@ParameterizedTest
//TODO(metadata): Needs HUDI-1459 to be fixed
//@ValueSource(booleans = Array(true, false))
@ValueSource(booleans = Array(false))
@ValueSource(booleans = Array(true, false))
def testCopyOnWriteStorage(isMetadataEnabled: Boolean) {
// Insert Operation
val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList
@@ -180,7 +178,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
// Snapshot query
val snapshotDF1 = spark.read.format("org.apache.hudi")
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.load(basePath + "/*/*/*/*")
.load(basePath + "/*/*/*")
assertEquals(100, snapshotDF1.count())
// Upsert based on the written table with Hudi metadata columns
@@ -189,11 +187,14 @@ class TestCOWDataSource extends HoodieClientTestBase {
updateDf.write.format("org.apache.hudi")
.options(commonOpts)
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.mode(SaveMode.Append)
.save(basePath)
val commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, basePath)
val snapshotDF2 = spark.read.format("hudi").load(basePath + "/*/*/*/*")
val snapshotDF2 = spark.read.format("hudi")
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.load(basePath + "/*/*/*")
assertEquals(100, snapshotDF2.count())
assertEquals(updatedVerificationVal, snapshotDF2.filter(col("_row_key") === verificationRowKey).select(verificationCol).first.getString(0))
@@ -214,7 +215,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
// Snapshot Query
val snapshotDF3 = spark.read.format("org.apache.hudi")
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.load(basePath + "/*/*/*/*")
.load(basePath + "/*/*/*")
assertEquals(100, snapshotDF3.count()) // still 100, since we only updated
// Read Incremental Query
@@ -666,8 +667,8 @@ class TestCOWDataSource extends HoodieClientTestBase {
}
@ParameterizedTest
@ValueSource(booleans = Array(true, false))
def testQueryCOWWithBasePathAndFileIndex(partitionEncode: Boolean): Unit = {
@CsvSource(Array("true,false", "true,true", "false,true", "false,false"))
def testQueryCOWWithBasePathAndFileIndex(partitionEncode: Boolean, isMetadataEnabled: Boolean): Unit = {
val N = 20
// Test query with partition prune if URL_ENCODE_PARTITIONING_OPT_KEY has enable
val records1 = dataGen.generateInsertsContainsAllPartitions("000", N)
@@ -676,6 +677,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
.options(commonOpts)
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY.key, partitionEncode)
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.mode(SaveMode.Overwrite)
.save(basePath)
val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, basePath)
@@ -683,6 +685,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
val countIn20160315 = records1.asScala.count(record => record.getPartitionPath == "2016/03/15")
// query the partition by filter
val count1 = spark.read.format("hudi")
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.load(basePath)
.filter("partition = '2016/03/15'")
.count()
@@ -691,6 +694,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
// query the partition by path
val partitionPath = if (partitionEncode) "2016%2F03%2F15" else "2016/03/15"
val count2 = spark.read.format("hudi")
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.load(basePath + s"/$partitionPath")
.count()
assertEquals(countIn20160315, count2)
@@ -702,6 +706,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
.options(commonOpts)
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY.key, partitionEncode)
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.mode(SaveMode.Append)
.save(basePath)
// Incremental query without "*" in path

View File

@@ -21,6 +21,7 @@ import org.apache.hadoop.fs.Path
import scala.collection.JavaConverters._
import org.apache.hudi.DataSourceWriteOptions.{KEYGENERATOR_CLASS_OPT_KEY, PARTITIONPATH_FIELD_OPT_KEY, PAYLOAD_CLASS_OPT_KEY, PRECOMBINE_FIELD_OPT_KEY, RECORDKEY_FIELD_OPT_KEY}
import org.apache.hudi.common.config.HoodieMetadataConfig
import org.apache.hudi.common.fs.FSUtils
import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieTableType}
import org.apache.hudi.common.table.HoodieTableMetaClient
@@ -75,7 +76,9 @@ class TestMORDataSource extends HoodieClientTestBase {
cleanupFileSystem()
}
@Test def testMergeOnReadStorage() {
@ParameterizedTest
@ValueSource(booleans = Array(true, false))
def testMergeOnReadStorage(isMetadataEnabled: Boolean) {
val fs = FSUtils.getFs(basePath, spark.sparkContext.hadoopConfiguration)
// Bulk Insert Operation
@@ -86,6 +89,7 @@ class TestMORDataSource extends HoodieClientTestBase {
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.mode(SaveMode.Overwrite)
.save(basePath)
@@ -94,7 +98,9 @@ class TestMORDataSource extends HoodieClientTestBase {
// Read RO View
val hudiRODF1 = spark.read.format("org.apache.hudi")
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
.load(basePath + "/*/*/*/*")
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.load(basePath + "/*/*/*")
assertEquals(100, hudiRODF1.count()) // still 100, since we only updated
val insertCommitTime = HoodieDataSourceHelpers.latestCommit(fs, basePath)
val insertCommitTimes = hudiRODF1.select("_hoodie_commit_time").distinct().collectAsList().map(r => r.getString(0)).toList
@@ -105,6 +111,7 @@ class TestMORDataSource extends HoodieClientTestBase {
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
inputDF2.write.format("org.apache.hudi")
.options(commonOpts)
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.mode(SaveMode.Append)
.save(basePath)
@@ -112,7 +119,9 @@ class TestMORDataSource extends HoodieClientTestBase {
val updateCommitTime = HoodieDataSourceHelpers.latestCommit(fs, basePath)
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
.load(basePath + "/*/*/*/*")
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.load(basePath + "/*/*/*")
val updateCommitTimes = hudiSnapshotDF2.select("_hoodie_commit_time").distinct().collectAsList().map(r => r.getString(0)).toList
assertEquals(List(updateCommitTime), updateCommitTimes)
@@ -122,10 +131,13 @@ class TestMORDataSource extends HoodieClientTestBase {
inputDF3.write.format("org.apache.hudi")
.options(commonOpts)
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.mode(SaveMode.Append)
.save(basePath)
val hudiSnapshotDF3 = spark.read.format("hudi").load(basePath + "/*/*/*/*")
val hudiSnapshotDF3 = spark.read.format("hudi")
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.load(basePath + "/*/*/*")
assertEquals(100, hudiSnapshotDF3.count())
assertEquals(updatedVerificationVal, hudiSnapshotDF3.filter(col("_row_key") === verificationRowKey).select(verificationCol).first.getString(0))
}
@@ -580,8 +592,8 @@ class TestMORDataSource extends HoodieClientTestBase {
}
@ParameterizedTest
@ValueSource(booleans = Array(true, false))
def testQueryMORWithBasePathAndFileIndex(partitionEncode: Boolean): Unit = {
@CsvSource(Array("true,false", "true,true", "false,true", "false,false"))
def testQueryMORWithBasePathAndFileIndex(partitionEncode: Boolean, isMetadataEnabled: Boolean): Unit = {
val N = 20
// Test query with partition prune if URL_ENCODE_PARTITIONING_OPT_KEY has enable
val records1 = dataGen.generateInsertsContainsAllPartitions("000", N)
@@ -591,6 +603,7 @@ class TestMORDataSource extends HoodieClientTestBase {
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY.key, partitionEncode)
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.mode(SaveMode.Overwrite)
.save(basePath)
val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, basePath)
@@ -598,6 +611,7 @@ class TestMORDataSource extends HoodieClientTestBase {
val countIn20160315 = records1.asScala.count(record => record.getPartitionPath == "2016/03/15")
// query the partition by filter
val count1 = spark.read.format("hudi")
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.load(basePath)
.filter("partition = '2016/03/15'")
.count()
@@ -606,6 +620,7 @@ class TestMORDataSource extends HoodieClientTestBase {
// query the partition by path
val partitionPath = if (partitionEncode) "2016%2F03%2F15" else "2016/03/15"
val count2 = spark.read.format("hudi")
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.load(basePath + s"/$partitionPath")
.count()
assertEquals(countIn20160315, count2)
@@ -618,6 +633,7 @@ class TestMORDataSource extends HoodieClientTestBase {
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY.key, partitionEncode)
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.mode(SaveMode.Append)
.save(basePath)
// Incremental query without "*" in path