[HUDI-1591] Implement Spark's FileIndex for Hudi to support queries via Hudi DataSource using non-globbed table path and partition pruning (#2651)
This commit is contained in:
@@ -0,0 +1,252 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi
|
||||
|
||||
import java.net.URLEncoder
|
||||
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.keygen.ComplexKeyGenerator
|
||||
import org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator.{Config, TimestampType}
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.{SaveMode, SparkSession}
|
||||
import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, EqualTo, GreaterThanOrEqual, LessThan, Literal}
|
||||
import org.apache.spark.sql.execution.datasources.PartitionDirectory
|
||||
import org.apache.spark.sql.types.StringType
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
import org.junit.jupiter.api.BeforeEach
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.ValueSource
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
class TestHoodieFileIndex extends HoodieClientTestBase {
|
||||
|
||||
var spark: SparkSession = _
|
||||
val commonOpts = Map(
|
||||
"hoodie.insert.shuffle.parallelism" -> "4",
|
||||
"hoodie.upsert.shuffle.parallelism" -> "4",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME -> "hoodie_test"
|
||||
)
|
||||
|
||||
@BeforeEach override def setUp() {
|
||||
initPath()
|
||||
initSparkContexts()
|
||||
spark = sqlContext.sparkSession
|
||||
initTestDataGenerator()
|
||||
initFileSystem()
|
||||
initMetaClient()
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = Array(true, false))
|
||||
def testPartitionSchema(partitionEncode: Boolean): Unit = {
|
||||
val records1 = dataGen.generateInsertsContainsAllPartitions("000", 100)
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2))
|
||||
inputDF1.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY, partitionEncode)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient)
|
||||
val fileIndex = HoodieFileIndex(spark, metaClient, None, Map("path" -> basePath))
|
||||
assertEquals("partition", fileIndex.partitionSchema.fields.map(_.name).mkString(","))
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = Array(
|
||||
"org.apache.hudi.keygen.ComplexKeyGenerator",
|
||||
"org.apache.hudi.keygen.SimpleKeyGenerator",
|
||||
"org.apache.hudi.keygen.TimestampBasedKeyGenerator"))
|
||||
def testPartitionSchemaForBuildInKeyGenerator(keyGenerator: String): Unit = {
|
||||
val records1 = dataGen.generateInsertsContainsAllPartitions("000", 100)
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2))
|
||||
inputDF1.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY, keyGenerator)
|
||||
.option(Config.TIMESTAMP_TYPE_FIELD_PROP, TimestampType.DATE_STRING.name())
|
||||
.option(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP, "yyyy/MM/dd")
|
||||
.option(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, "yyyy-MM-dd")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient)
|
||||
val fileIndex = HoodieFileIndex(spark, metaClient, None, Map("path" -> basePath))
|
||||
assertEquals("partition", fileIndex.partitionSchema.fields.map(_.name).mkString(","))
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = Array(
|
||||
"org.apache.hudi.keygen.CustomKeyGenerator",
|
||||
"org.apache.hudi.keygen.CustomAvroKeyGenerator"))
|
||||
def testPartitionSchemaForCustomKeyGenerator(keyGenerator: String): Unit = {
|
||||
val records1 = dataGen.generateInsertsContainsAllPartitions("000", 100)
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2))
|
||||
inputDF1.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY, keyGenerator)
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partition:simple")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient)
|
||||
val fileIndex = HoodieFileIndex(spark, metaClient, None, Map("path" -> basePath))
|
||||
assertEquals("partition", fileIndex.partitionSchema.fields.map(_.name).mkString(","))
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = Array(true, false))
|
||||
def testPartitionPruneWithPartitionEncode(partitionEncode: Boolean): Unit = {
|
||||
val partitions = Array("2021/03/08", "2021/03/09", "2021/03/10", "2021/03/11", "2021/03/12")
|
||||
val newDataGen = new HoodieTestDataGenerator(partitions)
|
||||
val records1 = newDataGen.generateInsertsContainsAllPartitions("000", 100)
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2))
|
||||
inputDF1.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY, partitionEncode)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient)
|
||||
val fileIndex = HoodieFileIndex(spark, metaClient, None, Map("path" -> basePath))
|
||||
|
||||
val partitionFilter1 = EqualTo(attribute("partition"), literal("2021/03/08"))
|
||||
val partitionName = if (partitionEncode) URLEncoder.encode("2021/03/08") else "2021/03/08"
|
||||
val partitionAndFilesAfterPrune = fileIndex.listFiles(Seq(partitionFilter1), Seq.empty)
|
||||
assertEquals(1, partitionAndFilesAfterPrune.size)
|
||||
|
||||
val PartitionDirectory(partitionValues, filesInPartition) = partitionAndFilesAfterPrune(0)
|
||||
assertEquals(partitionValues.toSeq(Seq(StringType)).mkString(","), "2021/03/08")
|
||||
assertEquals(getFileCountInPartitionPath(partitionName), filesInPartition.size)
|
||||
|
||||
val partitionFilter2 = And(
|
||||
GreaterThanOrEqual(attribute("partition"), literal("2021/03/08")),
|
||||
LessThan(attribute("partition"), literal("2021/03/10"))
|
||||
)
|
||||
val prunedPartitions = fileIndex.listFiles(Seq(partitionFilter2),
|
||||
Seq.empty).map(_.values.toSeq(Seq(StringType)).mkString(",")).toList
|
||||
|
||||
assertEquals(List("2021/03/08", "2021/03/09"), prunedPartitions)
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = Array(true, false))
|
||||
def testPartitionPruneWithMultiPartitionColumns(useMetaFileList: Boolean): Unit = {
|
||||
val _spark = spark
|
||||
import _spark.implicits._
|
||||
// Test the case the partition column size is equal to the partition directory level.
|
||||
val inputDF1 = (for (i <- 0 until 10) yield (i, s"a$i", 10 + i, 10000,
|
||||
s"2021-03-0${i % 2 + 1}", "10")).toDF("id", "name", "price", "version", "dt", "hh")
|
||||
|
||||
inputDF1.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(RECORDKEY_FIELD_OPT_KEY, "id")
|
||||
.option(PRECOMBINE_FIELD_OPT_KEY, "version")
|
||||
.option(PARTITIONPATH_FIELD_OPT_KEY, "dt,hh")
|
||||
.option(KEYGENERATOR_CLASS_OPT_KEY, classOf[ComplexKeyGenerator].getName)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY, "false")
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP, useMetaFileList)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient)
|
||||
val fileIndex = HoodieFileIndex(spark, metaClient, None, Map("path" -> basePath))
|
||||
|
||||
val partitionFilter1 = And(
|
||||
EqualTo(attribute("dt"), literal("2021-03-01")),
|
||||
EqualTo(attribute("hh"), literal("10"))
|
||||
)
|
||||
val partitionAndFilesAfterPrune = fileIndex.listFiles(Seq(partitionFilter1), Seq.empty)
|
||||
assertEquals(1, partitionAndFilesAfterPrune.size)
|
||||
|
||||
val PartitionDirectory(partitionValues, filesAfterPrune) = partitionAndFilesAfterPrune(0)
|
||||
// The partition prune will work for this case.
|
||||
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)
|
||||
assertEquals(10, readDF1.count())
|
||||
assertEquals(5, readDF1.filter("dt = '2021-03-01' and hh = '10'").count())
|
||||
|
||||
// Test the case that partition column size not match the partition directory level and
|
||||
// partition column size is > 1. We will not trait it as partitioned table when read.
|
||||
val inputDF2 = (for (i <- 0 until 10) yield (i, s"a$i", 10 + i, 100 * i + 10000,
|
||||
s"2021/03/0${i % 2 + 1}", "10")).toDF("id", "name", "price", "version", "dt", "hh")
|
||||
inputDF2.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(RECORDKEY_FIELD_OPT_KEY, "id")
|
||||
.option(PRECOMBINE_FIELD_OPT_KEY, "version")
|
||||
.option(PARTITIONPATH_FIELD_OPT_KEY, "dt,hh")
|
||||
.option(KEYGENERATOR_CLASS_OPT_KEY, classOf[ComplexKeyGenerator].getName)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY, "false")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
fileIndex.refresh()
|
||||
val partitionFilter2 = And(
|
||||
EqualTo(attribute("dt"), literal("2021/03/01")),
|
||||
EqualTo(attribute("hh"), literal("10"))
|
||||
)
|
||||
val partitionAndFilesAfterPrune2 = fileIndex.listFiles(Seq(partitionFilter2), Seq.empty)
|
||||
|
||||
assertEquals(1, partitionAndFilesAfterPrune2.size)
|
||||
val PartitionDirectory(partitionValues2, filesAfterPrune2) = partitionAndFilesAfterPrune2(0)
|
||||
// The partition prune would not work for this case, so the partition value it
|
||||
// returns is a InternalRow.empty.
|
||||
assertEquals(partitionValues2, InternalRow.empty)
|
||||
// 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)
|
||||
|
||||
assertEquals(10, readDF2.count())
|
||||
// There are 5 rows in the dt = 2021/03/01 and hh = 10
|
||||
assertEquals(5, readDF2.filter("dt = '2021/03/01' and hh ='10'").count())
|
||||
}
|
||||
|
||||
private def attribute(partition: String): AttributeReference = {
|
||||
AttributeReference(partition, StringType, true)()
|
||||
}
|
||||
|
||||
private def literal(value: String): Literal = {
|
||||
Literal.create(value)
|
||||
}
|
||||
|
||||
private def getFileCountInPartitionPath(partitionPath: String): Int = {
|
||||
metaClient.reloadActiveTimeline()
|
||||
val activeInstants = metaClient.getActiveTimeline.getCommitsTimeline.filterCompletedInstants
|
||||
val fileSystemView = new HoodieTableFileSystemView(metaClient, activeInstants)
|
||||
fileSystemView.getAllBaseFiles(partitionPath).iterator().asScala.toSeq.length
|
||||
}
|
||||
|
||||
private def getFileCountInPartitionPaths(partitionPaths: String*): Int = {
|
||||
partitionPaths.map(getFileCountInPartitionPath).sum
|
||||
}
|
||||
}
|
||||
@@ -18,6 +18,10 @@
|
||||
package org.apache.hudi.functional
|
||||
|
||||
import java.sql.{Date, Timestamp}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant
|
||||
@@ -38,7 +42,6 @@ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.ValueSource
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
|
||||
/**
|
||||
* Basic tests on the spark datasource for COW table.
|
||||
@@ -619,4 +622,51 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
.load(basePath + "/*")
|
||||
assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= lit("")).count() == 0)
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = Array(true, false))
|
||||
def testQueryCOWWithBasePathAndFileIndex(partitionEncode: Boolean): Unit = {
|
||||
val N = 20
|
||||
// Test query with partition prune if URL_ENCODE_PARTITIONING_OPT_KEY has enable
|
||||
val records1 = dataGen.generateInsertsContainsAllPartitions("000", N)
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2))
|
||||
inputDF1.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY, partitionEncode)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
|
||||
val countIn20160315 = records1.asScala.count(record => record.getPartitionPath == "2016/03/15")
|
||||
// query the partition by filter
|
||||
val count1 = spark.read.format("hudi")
|
||||
.load(basePath)
|
||||
.filter("partition = '2016/03/15'")
|
||||
.count()
|
||||
assertEquals(countIn20160315, count1)
|
||||
|
||||
// query the partition by path
|
||||
val partitionPath = if (partitionEncode) "2016%2F03%2F15" else "2016/03/15"
|
||||
val count2 = spark.read.format("hudi")
|
||||
.load(basePath + s"/$partitionPath")
|
||||
.count()
|
||||
assertEquals(countIn20160315, count2)
|
||||
|
||||
// Second write with Append mode
|
||||
val records2 = dataGen.generateInsertsContainsAllPartitions("000", N + 1)
|
||||
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records2), 2))
|
||||
inputDF2.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY, partitionEncode)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
// Incremental query without "*" in path
|
||||
val hoodieIncViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath)
|
||||
assertEquals(N + 1, hoodieIncViewDF1.count())
|
||||
}
|
||||
}
|
||||
|
||||
@@ -130,6 +130,10 @@ class TestDataSourceForBootstrap {
|
||||
hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF1.count())
|
||||
assertEquals(numRecordsUpdate, hoodieROViewDF1.filter(s"timestamp == $updateTimestamp").count())
|
||||
// Read without *
|
||||
val hoodieROViewDF1WithBasePath = spark.read.format("hudi").load(basePath)
|
||||
assertEquals(numRecords, hoodieROViewDF1WithBasePath.count())
|
||||
assertEquals(numRecordsUpdate, hoodieROViewDF1WithBasePath.filter(s"timestamp == $updateTimestamp").count())
|
||||
|
||||
verifyIncrementalViewResult(commitInstantTime1, commitInstantTime2, isPartitioned = false, isHiveStylePartitioned = false)
|
||||
}
|
||||
@@ -149,7 +153,8 @@ class TestDataSourceForBootstrap {
|
||||
.save(srcPath)
|
||||
|
||||
// Perform bootstrap
|
||||
val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
|
||||
val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(
|
||||
DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, Some("datestr"))
|
||||
|
||||
// Read bootstrapped table and verify count
|
||||
val hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*")
|
||||
@@ -201,11 +206,15 @@ class TestDataSourceForBootstrap {
|
||||
})
|
||||
|
||||
// Perform bootstrap
|
||||
val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
|
||||
val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(
|
||||
DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, Some("datestr"))
|
||||
|
||||
// Read bootstrapped table and verify count
|
||||
val hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF1.count())
|
||||
// Read without *
|
||||
val hoodieROViewWithBasePathDF1 = spark.read.format("hudi").load(basePath)
|
||||
assertEquals(numRecords, hoodieROViewWithBasePathDF1.count())
|
||||
|
||||
// Perform upsert based on the written bootstrap table
|
||||
val updateDf1 = hoodieROViewDF1.filter(col("_row_key") === verificationRowKey).withColumn(verificationCol, lit(updatedVerificationVal))
|
||||
@@ -268,7 +277,8 @@ class TestDataSourceForBootstrap {
|
||||
})
|
||||
|
||||
// Perform bootstrap
|
||||
val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(
|
||||
DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL, Some("datestr"))
|
||||
|
||||
// Read bootstrapped table and verify count
|
||||
val hoodieROViewDF1 = spark.read.format("hudi")
|
||||
@@ -304,6 +314,13 @@ class TestDataSourceForBootstrap {
|
||||
.load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF2.count())
|
||||
assertEquals(numRecordsUpdate, hoodieROViewDF2.filter(s"timestamp == $updateTimestamp").count())
|
||||
// Test query without "*" for MOR READ_OPTIMIZED
|
||||
val hoodieROViewDFWithBasePath = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY,
|
||||
DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath)
|
||||
assertEquals(numRecords, hoodieROViewDFWithBasePath.count())
|
||||
assertEquals(numRecordsUpdate, hoodieROViewDFWithBasePath.filter(s"timestamp == $updateTimestamp").count())
|
||||
}
|
||||
|
||||
@Test def testMetadataBootstrapMORPartitioned(): Unit = {
|
||||
@@ -325,7 +342,8 @@ class TestDataSourceForBootstrap {
|
||||
})
|
||||
|
||||
// Perform bootstrap
|
||||
val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(
|
||||
DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL, Some("datestr"))
|
||||
|
||||
// Read bootstrapped table and verify count
|
||||
val hoodieROViewDF1 = spark.read.format("hudi")
|
||||
@@ -333,6 +351,12 @@ class TestDataSourceForBootstrap {
|
||||
DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF1.count())
|
||||
// Read bootstrapped table without "*"
|
||||
val hoodieROViewDFWithBasePath = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY,
|
||||
DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath)
|
||||
assertEquals(numRecords, hoodieROViewDFWithBasePath.count())
|
||||
|
||||
// Perform upsert based on the written bootstrap table
|
||||
val updateDf1 = hoodieROViewDF1.filter(col("_row_key") === verificationRowKey).withColumn(verificationCol, lit(updatedVerificationVal))
|
||||
@@ -420,6 +444,9 @@ class TestDataSourceForBootstrap {
|
||||
val hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF1.count())
|
||||
|
||||
val hoodieROViewDFWithBasePath = spark.read.format("hudi").load(basePath)
|
||||
assertEquals(numRecords, hoodieROViewDFWithBasePath.count())
|
||||
|
||||
// Perform upsert
|
||||
val updateTimestamp = Instant.now.toEpochMilli
|
||||
val updateDF = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate, partitionPaths.asJava,
|
||||
@@ -445,13 +472,15 @@ class TestDataSourceForBootstrap {
|
||||
verifyIncrementalViewResult(commitInstantTime1, commitInstantTime2, isPartitioned = true, isHiveStylePartitioned = false)
|
||||
}
|
||||
|
||||
def runMetadataBootstrapAndVerifyCommit(tableType: String): String = {
|
||||
def runMetadataBootstrapAndVerifyCommit(tableType: String,
|
||||
partitionColumns: Option[String] = None): String = {
|
||||
val bootstrapDF = spark.emptyDataFrame
|
||||
bootstrapDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, tableType)
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, partitionColumns.getOrElse(""))
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, classOf[SimpleKeyGenerator].getName)
|
||||
.mode(SaveMode.Overwrite)
|
||||
|
||||
@@ -17,6 +17,7 @@
|
||||
|
||||
package org.apache.hudi.functional
|
||||
|
||||
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.fs.FSUtils
|
||||
import org.apache.hudi.common.model.DefaultHoodieRecordPayload
|
||||
@@ -31,6 +32,8 @@ import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.ValueSource
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
|
||||
@@ -562,4 +565,53 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
df.show(1)
|
||||
df.select("_hoodie_commit_seqno", "fare.amount", "fare.currency", "tip_history").show(1)
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = Array(true, false))
|
||||
def testQueryMORWithBasePathAndFileIndex(partitionEncode: Boolean): Unit = {
|
||||
val N = 20
|
||||
// Test query with partition prune if URL_ENCODE_PARTITIONING_OPT_KEY has enable
|
||||
val records1 = dataGen.generateInsertsContainsAllPartitions("000", N)
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2))
|
||||
inputDF1.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY, partitionEncode)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
|
||||
val countIn20160315 = records1.asScala.count(record => record.getPartitionPath == "2016/03/15")
|
||||
// query the partition by filter
|
||||
val count1 = spark.read.format("hudi")
|
||||
.load(basePath)
|
||||
.filter("partition = '2016/03/15'")
|
||||
.count()
|
||||
assertEquals(countIn20160315, count1)
|
||||
|
||||
// query the partition by path
|
||||
val partitionPath = if (partitionEncode) "2016%2F03%2F15" else "2016/03/15"
|
||||
val count2 = spark.read.format("hudi")
|
||||
.load(basePath + s"/$partitionPath")
|
||||
.count()
|
||||
assertEquals(countIn20160315, count2)
|
||||
|
||||
// Second write with Append mode
|
||||
val records2 = dataGen.generateInsertsContainsAllPartitions("000", N + 1)
|
||||
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records2), 2))
|
||||
inputDF2.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY, partitionEncode)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
// Incremental query without "*" in path
|
||||
val hoodieIncViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath)
|
||||
assertEquals(N + 1, hoodieIncViewDF1.count())
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user