nsivabalan commented on code in PR #9345:
URL: https://github.com/apache/hudi/pull/9345#discussion_r1285089880
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala:
##########
@@ -106,14 +109,30 @@ case class HoodieFileIndex(spark: SparkSession,
*
* @return List of FileStatus for base files
*/
- def allFiles: Seq[FileStatus] = {
+ def allBaseFiles: Seq[FileStatus] = {
Review Comment:
private?
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala:
##########
@@ -122,86 +141,132 @@ case class HoodieFileIndex(spark: SparkSession,
* @return list of PartitionDirectory containing partition to base files
mapping
*/
override def listFiles(partitionFilters: Seq[Expression], dataFilters:
Seq[Expression]): Seq[PartitionDirectory] = {
- // Look up candidate files names in the col-stats index, if all of the
following conditions are true
- // - Data-skipping is enabled
- // - Col-Stats Index is present
- // - List of predicates (filters) is present
- val candidateFilesNamesOpt: Option[Set[String]] =
+ // Prune the partition path by the partition filters
+ // NOTE: Non-partitioned tables are assumed to consist from a single
partition
+ // encompassing the whole table
+ val partitionsAndFileSlices =
getFileSlicesForPrunedPartitions(partitionFilters)
+ val partitionsAndFilteredFileSlices = filterFileSlices(dataFilters,
partitionsAndFileSlices).map {
+ case (partitionOpt, fileSlices) =>
+ val allCandidateFiles: Seq[FileStatus] = fileSlices.flatMap(fs => {
+ val baseFileStatusOpt =
getBaseFileStatus(Option.apply(fs.getBaseFile.orElse(null)))
+ val logFilesStatus = if (includeLogFiles) {
+
fs.getLogFiles.map[FileStatus](JFunction.toJavaFunction[HoodieLogFile,
FileStatus](lf => lf.getFileStatus))
+ } else {
+ java.util.stream.Stream.empty()
+ }
+ val files =
logFilesStatus.collect(Collectors.toList[FileStatus]).asScala
+ baseFileStatusOpt.foreach(f => files.append(f))
+ files
+ })
+
+ PartitionDirectory(InternalRow.fromSeq(partitionOpt.get.values),
allCandidateFiles)
+ }
+
+ hasPushedDownPartitionPredicates = true
+
+ if (shouldReadAsPartitionedTable()) {
+ partitionsAndFilteredFileSlices
+ } else {
+ Seq(PartitionDirectory(InternalRow.empty,
partitionsAndFilteredFileSlices.flatMap(_.files)))
+ }
+ }
+
+ def filterFileSlices(dataFilters: Seq[Expression], partitionAndFileSlices:
Seq[(Option[BaseHoodieTableFileIndex.PartitionPath], Seq[FileSlice])])
+ : Seq[(Option[BaseHoodieTableFileIndex.PartitionPath], Seq[FileSlice])] = {
+ // If there are no data filters, return all the file slices.
+ // If there are no file slices, return empty list.
+ if (partitionAndFileSlices.isEmpty || dataFilters.isEmpty) {
+ partitionAndFileSlices
+ } else {
+ // Look up candidate files names in the col-stats index, if all of the
following conditions are true
+ // - Data-skipping is enabled
+ // - Col-Stats Index is present
+ // - List of predicates (filters) is present
+ val candidateFilesNamesOpt: Option[Set[String]] =
Review Comment:
lets file a follow up ticket.
looks like we are not leveraging the pruned partition filters while polling
col stats only.
Anyways, after we get the list from col stats, we will compare against
allFils and trim it down properly. So, its not very bad.
but the amount of data fetched from col stats could be trimmed
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala:
##########
@@ -69,6 +71,8 @@ abstract class BaseMergeOnReadSnapshotRelation(sqlContext:
SQLContext,
prunedDataSchema:
Option[StructType])
extends HoodieBaseRelation(sqlContext, metaClient, optParams, userSchema,
prunedDataSchema) {
+ fileIndex.setIncludeLogFiles(true)
Review Comment:
I feel this is not elegant and is left open and the caller can invoke
anytime.
Can we introduce a protected method called shouldIncludeLogFiles() to
HoodieBaseRelation and set the default value to false.
and we can override that in MergeOnReadSnapshotRelation or
BaseMergeOnReadSnapshotRelation and set the value to true.
we can change the constructor or HoodieFileIndex to take an additional arg
called shouldIncludeLogFiles, and the callers are expected to set it when
invoking the constructor
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala:
##########
@@ -122,86 +141,132 @@ case class HoodieFileIndex(spark: SparkSession,
* @return list of PartitionDirectory containing partition to base files
mapping
*/
override def listFiles(partitionFilters: Seq[Expression], dataFilters:
Seq[Expression]): Seq[PartitionDirectory] = {
- // Look up candidate files names in the col-stats index, if all of the
following conditions are true
- // - Data-skipping is enabled
- // - Col-Stats Index is present
- // - List of predicates (filters) is present
- val candidateFilesNamesOpt: Option[Set[String]] =
+ // Prune the partition path by the partition filters
+ // NOTE: Non-partitioned tables are assumed to consist from a single
partition
+ // encompassing the whole table
+ val partitionsAndFileSlices =
getFileSlicesForPrunedPartitions(partitionFilters)
+ val partitionsAndFilteredFileSlices = filterFileSlices(dataFilters,
partitionsAndFileSlices).map {
+ case (partitionOpt, fileSlices) =>
+ val allCandidateFiles: Seq[FileStatus] = fileSlices.flatMap(fs => {
+ val baseFileStatusOpt =
getBaseFileStatus(Option.apply(fs.getBaseFile.orElse(null)))
+ val logFilesStatus = if (includeLogFiles) {
+
fs.getLogFiles.map[FileStatus](JFunction.toJavaFunction[HoodieLogFile,
FileStatus](lf => lf.getFileStatus))
+ } else {
+ java.util.stream.Stream.empty()
+ }
+ val files =
logFilesStatus.collect(Collectors.toList[FileStatus]).asScala
+ baseFileStatusOpt.foreach(f => files.append(f))
+ files
+ })
+
+ PartitionDirectory(InternalRow.fromSeq(partitionOpt.get.values),
allCandidateFiles)
+ }
+
+ hasPushedDownPartitionPredicates = true
+
+ if (shouldReadAsPartitionedTable()) {
+ partitionsAndFilteredFileSlices
+ } else {
+ Seq(PartitionDirectory(InternalRow.empty,
partitionsAndFilteredFileSlices.flatMap(_.files)))
+ }
+ }
+
+ def filterFileSlices(dataFilters: Seq[Expression], partitionAndFileSlices:
Seq[(Option[BaseHoodieTableFileIndex.PartitionPath], Seq[FileSlice])])
+ : Seq[(Option[BaseHoodieTableFileIndex.PartitionPath], Seq[FileSlice])] = {
+ // If there are no data filters, return all the file slices.
+ // If there are no file slices, return empty list.
+ if (partitionAndFileSlices.isEmpty || dataFilters.isEmpty) {
+ partitionAndFileSlices
+ } else {
+ // Look up candidate files names in the col-stats index, if all of the
following conditions are true
+ // - Data-skipping is enabled
+ // - Col-Stats Index is present
+ // - List of predicates (filters) is present
+ val candidateFilesNamesOpt: Option[Set[String]] =
lookupCandidateFilesInMetadataTable(dataFilters) match {
case Success(opt) => opt
case Failure(e) =>
logError("Failed to lookup candidate files in File Index", e)
spark.sqlContext.getConf(DataSkippingFailureMode.configName,
DataSkippingFailureMode.Fallback.value) match {
case DataSkippingFailureMode.Fallback.value => Option.empty
- case DataSkippingFailureMode.Strict.value => throw new
HoodieException(e);
+ case DataSkippingFailureMode.Strict.value => throw new
HoodieException(e);
}
}
- logDebug(s"Overlapping candidate files from Column Stats Index:
${candidateFilesNamesOpt.getOrElse(Set.empty)}")
-
- var totalFileSize = 0
- var candidateFileSize = 0
+ logDebug(s"Overlapping candidate files from Column Stats Index:
${candidateFilesNamesOpt.getOrElse(Set.empty)}")
+
+ var totalFileSliceSize = 0
+ var candidateFileSliceSize = 0
+
+ val partitionsAndFilteredFileSlices = partitionAndFileSlices.map {
+ case (partitionOpt, fileSlices) =>
+ // Filter in candidate files based on the col-stats index lookup
+ val candidateFileSlices: Seq[FileSlice] = {
+ fileSlices.filter(fs => {
+ val baseFileStatusOpt =
getBaseFileStatus(Option.apply(fs.getBaseFile.orElse(null)))
+ val logFiles =
fs.getLogFiles.collect(Collectors.toSet[HoodieLogFile]).asScala.toSet[HoodieLogFile]
+ // NOTE: This predicate is true when {@code Option} is empty
+ if (candidateFilesNamesOpt.forall(files =>
baseFileStatusOpt.exists(f => files.contains(f.getPath.getName))
+ || files.intersect(logFiles.map(f =>
f.getPath.getName)).nonEmpty)) {
Review Comment:
if both base file and log files are present, are we ensuring all files are
present in candidateFilesNamesOpt? I see "||" here.
we can probably simplify this.
collect all file names to a single Seq (both base file name and log file
name for a file slice).
and then you can check for match w/ col stats. you don't need to deal w/
Option.empty or empty list.
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala:
##########
@@ -122,86 +141,132 @@ case class HoodieFileIndex(spark: SparkSession,
* @return list of PartitionDirectory containing partition to base files
mapping
*/
override def listFiles(partitionFilters: Seq[Expression], dataFilters:
Seq[Expression]): Seq[PartitionDirectory] = {
- // Look up candidate files names in the col-stats index, if all of the
following conditions are true
- // - Data-skipping is enabled
- // - Col-Stats Index is present
- // - List of predicates (filters) is present
- val candidateFilesNamesOpt: Option[Set[String]] =
+ // Prune the partition path by the partition filters
+ // NOTE: Non-partitioned tables are assumed to consist from a single
partition
+ // encompassing the whole table
+ val partitionsAndFileSlices =
getFileSlicesForPrunedPartitions(partitionFilters)
+ val partitionsAndFilteredFileSlices = filterFileSlices(dataFilters,
partitionsAndFileSlices).map {
+ case (partitionOpt, fileSlices) =>
+ val allCandidateFiles: Seq[FileStatus] = fileSlices.flatMap(fs => {
+ val baseFileStatusOpt =
getBaseFileStatus(Option.apply(fs.getBaseFile.orElse(null)))
+ val logFilesStatus = if (includeLogFiles) {
+
fs.getLogFiles.map[FileStatus](JFunction.toJavaFunction[HoodieLogFile,
FileStatus](lf => lf.getFileStatus))
+ } else {
+ java.util.stream.Stream.empty()
+ }
+ val files =
logFilesStatus.collect(Collectors.toList[FileStatus]).asScala
+ baseFileStatusOpt.foreach(f => files.append(f))
+ files
+ })
+
+ PartitionDirectory(InternalRow.fromSeq(partitionOpt.get.values),
allCandidateFiles)
+ }
+
+ hasPushedDownPartitionPredicates = true
+
+ if (shouldReadAsPartitionedTable()) {
+ partitionsAndFilteredFileSlices
+ } else {
+ Seq(PartitionDirectory(InternalRow.empty,
partitionsAndFilteredFileSlices.flatMap(_.files)))
+ }
+ }
+
+ def filterFileSlices(dataFilters: Seq[Expression], partitionAndFileSlices:
Seq[(Option[BaseHoodieTableFileIndex.PartitionPath], Seq[FileSlice])])
Review Comment:
lets call the 2nd arg as prunedPartitionsAndFileSlices
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala:
##########
@@ -122,86 +141,132 @@ case class HoodieFileIndex(spark: SparkSession,
* @return list of PartitionDirectory containing partition to base files
mapping
*/
override def listFiles(partitionFilters: Seq[Expression], dataFilters:
Seq[Expression]): Seq[PartitionDirectory] = {
- // Look up candidate files names in the col-stats index, if all of the
following conditions are true
- // - Data-skipping is enabled
- // - Col-Stats Index is present
- // - List of predicates (filters) is present
- val candidateFilesNamesOpt: Option[Set[String]] =
+ // Prune the partition path by the partition filters
+ // NOTE: Non-partitioned tables are assumed to consist from a single
partition
+ // encompassing the whole table
+ val partitionsAndFileSlices =
getFileSlicesForPrunedPartitions(partitionFilters)
+ val partitionsAndFilteredFileSlices = filterFileSlices(dataFilters,
partitionsAndFileSlices).map {
+ case (partitionOpt, fileSlices) =>
+ val allCandidateFiles: Seq[FileStatus] = fileSlices.flatMap(fs => {
+ val baseFileStatusOpt =
getBaseFileStatus(Option.apply(fs.getBaseFile.orElse(null)))
+ val logFilesStatus = if (includeLogFiles) {
+
fs.getLogFiles.map[FileStatus](JFunction.toJavaFunction[HoodieLogFile,
FileStatus](lf => lf.getFileStatus))
+ } else {
+ java.util.stream.Stream.empty()
+ }
+ val files =
logFilesStatus.collect(Collectors.toList[FileStatus]).asScala
+ baseFileStatusOpt.foreach(f => files.append(f))
+ files
+ })
+
+ PartitionDirectory(InternalRow.fromSeq(partitionOpt.get.values),
allCandidateFiles)
+ }
+
+ hasPushedDownPartitionPredicates = true
+
+ if (shouldReadAsPartitionedTable()) {
+ partitionsAndFilteredFileSlices
+ } else {
+ Seq(PartitionDirectory(InternalRow.empty,
partitionsAndFilteredFileSlices.flatMap(_.files)))
+ }
+ }
+
+ def filterFileSlices(dataFilters: Seq[Expression], partitionAndFileSlices:
Seq[(Option[BaseHoodieTableFileIndex.PartitionPath], Seq[FileSlice])])
+ : Seq[(Option[BaseHoodieTableFileIndex.PartitionPath], Seq[FileSlice])] = {
+ // If there are no data filters, return all the file slices.
+ // If there are no file slices, return empty list.
+ if (partitionAndFileSlices.isEmpty || dataFilters.isEmpty) {
+ partitionAndFileSlices
+ } else {
+ // Look up candidate files names in the col-stats index, if all of the
following conditions are true
+ // - Data-skipping is enabled
+ // - Col-Stats Index is present
+ // - List of predicates (filters) is present
+ val candidateFilesNamesOpt: Option[Set[String]] =
lookupCandidateFilesInMetadataTable(dataFilters) match {
case Success(opt) => opt
case Failure(e) =>
logError("Failed to lookup candidate files in File Index", e)
spark.sqlContext.getConf(DataSkippingFailureMode.configName,
DataSkippingFailureMode.Fallback.value) match {
case DataSkippingFailureMode.Fallback.value => Option.empty
- case DataSkippingFailureMode.Strict.value => throw new
HoodieException(e);
+ case DataSkippingFailureMode.Strict.value => throw new
HoodieException(e);
}
}
- logDebug(s"Overlapping candidate files from Column Stats Index:
${candidateFilesNamesOpt.getOrElse(Set.empty)}")
-
- var totalFileSize = 0
- var candidateFileSize = 0
+ logDebug(s"Overlapping candidate files from Column Stats Index:
${candidateFilesNamesOpt.getOrElse(Set.empty)}")
+
+ var totalFileSliceSize = 0
+ var candidateFileSliceSize = 0
+
+ val partitionsAndFilteredFileSlices = partitionAndFileSlices.map {
Review Comment:
prunedPartitionsAndFilteredFileSlices
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala:
##########
@@ -215,8 +219,8 @@ abstract class BaseMergeOnReadSnapshotRelation(sqlContext:
SQLContext,
HoodieFileIndex.convertFilterForTimestampKeyGenerator(metaClient,
partitionFilters)
if (globPaths.isEmpty) {
- val fileSlices = fileIndex.listFileSlices(convertedPartitionFilters)
- buildSplits(fileSlices.values.flatten.toSeq)
+ val fileSlices = fileIndex.filterFileSlices(dataFilters,
fileIndex.getFileSlicesForPrunedPartitions(convertedPartitionFilters)).flatMap(s
=> s._2)
Review Comment:
I see both, MOR snapshot relation and base file relation
(HoodieFileIndex.listFiles()), are invoking this. So, might as well move
fileIndex.getFileSlicesForPrunedPartitions into fileIndex.filterFileSlices if
all the callers are anyway going to invoke the other method
##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndexWithSQL.scala:
##########
@@ -0,0 +1,379 @@
+/*
+ * 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.functional
+
+import org.apache.hudi.DataSourceWriteOptions.{DELETE_OPERATION_OPT_VAL,
PRECOMBINE_FIELD, RECORDKEY_FIELD}
+import org.apache.hudi.async.SparkAsyncCompactService
+import org.apache.hudi.client.SparkRDDWriteClient
+import org.apache.hudi.client.common.HoodieSparkEngineContext
+import org.apache.hudi.client.utils.MetadataConversionUtils
+import org.apache.hudi.common.config.HoodieMetadataConfig
+import org.apache.hudi.common.model.{FileSlice, HoodieCommitMetadata,
HoodieTableType, WriteOperationType}
+import org.apache.hudi.common.table.HoodieTableConfig
+import org.apache.hudi.common.table.timeline.HoodieInstant
+import org.apache.hudi.config.{HoodieCompactionConfig, HoodieIndexConfig,
HoodieWriteConfig}
+import org.apache.hudi.functional.ColumnStatIndexTestBase.ColumnStatsTestCase
+import org.apache.hudi.index.HoodieIndex.IndexType.INMEMORY
+import org.apache.hudi.metadata.HoodieMetadataFileSystemView
+import org.apache.hudi.util.{JFunction, JavaConversions}
+import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions,
HoodieFileIndex}
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference,
Expression, GreaterThan, Literal}
+import org.apache.spark.sql.types.StringType
+import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue}
+import org.junit.jupiter.api.Disabled
+import org.junit.jupiter.params.ParameterizedTest
+import org.junit.jupiter.params.provider.MethodSource
+
+import java.util.Properties
+import scala.collection.JavaConverters
+
+class TestColumnStatsIndexWithSQL extends ColumnStatIndexTestBase {
+
+ @ParameterizedTest
+ @MethodSource(Array("testMetadataColumnStatsIndexParams"))
+ def testMetadataColumnStatsIndexWithSQL(testCase: ColumnStatsTestCase): Unit
= {
+ val metadataOpts = Map(
+ HoodieMetadataConfig.ENABLE.key -> "true",
+ HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key -> "true"
+ )
+
+ val commonOpts = Map(
+ "hoodie.insert.shuffle.parallelism" -> "4",
+ "hoodie.upsert.shuffle.parallelism" -> "4",
+ HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
+ DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
+ RECORDKEY_FIELD.key -> "c1",
+ PRECOMBINE_FIELD.key -> "c1",
+ HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
+ DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "true",
+ DataSourceReadOptions.QUERY_TYPE.key ->
DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL
+ ) ++ metadataOpts
+ setupTable(testCase, metadataOpts, commonOpts, shouldValidate = true)
+ verifyFileIndexAndSQLQueries(commonOpts)
+ }
+
+ @ParameterizedTest
+ @MethodSource(Array("testMetadataColumnStatsIndexParamsForMOR"))
+ def testMetadataColumnStatsIndexSQLWithInMemoryIndex(testCase:
ColumnStatsTestCase): Unit = {
+ val metadataOpts = Map(
+ HoodieMetadataConfig.ENABLE.key -> "true",
+ HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key -> "true"
+ )
+
+ val commonOpts = Map(
+ "hoodie.insert.shuffle.parallelism" -> "4",
+ "hoodie.upsert.shuffle.parallelism" -> "4",
+ HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
+ DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
+ RECORDKEY_FIELD.key -> "c1",
+ PRECOMBINE_FIELD.key -> "c1",
+ HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
+ DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "true",
+ DataSourceReadOptions.QUERY_TYPE.key ->
DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL,
+ HoodieIndexConfig.INDEX_TYPE.key() -> INMEMORY.name()
+ ) ++ metadataOpts
+
+ doWriteAndValidateColumnStats(testCase, metadataOpts, commonOpts,
+ dataSourcePath = "index/colstats/input-table-json",
+ expectedColStatsSourcePath =
"index/colstats/column-stats-index-table.json",
+ operation = DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
+ saveMode = SaveMode.Overwrite,
+ shouldValidate = false)
+
+ assertEquals(4, getLatestDataFilesCount(commonOpts))
+ assertEquals(0, getLatestDataFilesCount(commonOpts, includeLogFiles =
false))
+ var dataFilter = GreaterThan(attribute("c5"), literal("90"))
+ verifyPruningFileCount(commonOpts, dataFilter, 3)
+ dataFilter = GreaterThan(attribute("c5"), literal("95"))
+ verifyPruningFileCount(commonOpts, dataFilter, 1)
+ }
+
+ @ParameterizedTest
+ @MethodSource(Array("testMetadataColumnStatsIndexParams"))
+ def testMetadataColumnStatsIndexDeletionWithSQL(testCase:
ColumnStatsTestCase): Unit = {
+ val metadataOpts = Map(
+ HoodieMetadataConfig.ENABLE.key -> "true",
+ HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key -> "true"
+ )
+
+ val commonOpts = Map(
+ "hoodie.insert.shuffle.parallelism" -> "4",
+ "hoodie.upsert.shuffle.parallelism" -> "4",
+ HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
+ DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
+ RECORDKEY_FIELD.key -> "c1",
+ PRECOMBINE_FIELD.key -> "c1",
+ HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
+ DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "true",
+ DataSourceReadOptions.QUERY_TYPE.key ->
DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL
+ ) ++ metadataOpts
+ setupTable(testCase, metadataOpts, commonOpts, shouldValidate = true)
+ val lastDf = dfList.last
+
+ lastDf.write.format("org.apache.hudi")
+ .options(commonOpts)
+ .option(DataSourceWriteOptions.OPERATION.key, DELETE_OPERATION_OPT_VAL)
+ .mode(SaveMode.Append)
+ .save(basePath)
+ verifyFileIndexAndSQLQueries(commonOpts,
isTableDataSameAsAfterSecondInstant = true)
+
+ // Add the last df back and verify the queries
+ doWriteAndValidateColumnStats(testCase, metadataOpts, commonOpts,
+ dataSourcePath = "index/colstats/update-input-table-json",
+ expectedColStatsSourcePath = "",
+ operation = DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL,
+ saveMode = SaveMode.Append,
+ shouldValidate = false)
+ verifyFileIndexAndSQLQueries(commonOpts, verifyFileCount = false)
+ }
+
+ @ParameterizedTest
+ @MethodSource(Array("testMetadataColumnStatsIndexParamsForMOR"))
+ def testMetadataColumnStatsIndexCompactionWithSQL(testCase:
ColumnStatsTestCase): Unit = {
+ val metadataOpts = Map(
+ HoodieMetadataConfig.ENABLE.key -> "true",
+ HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key -> "true"
+ )
+
+ val commonOpts = Map(
+ "hoodie.insert.shuffle.parallelism" -> "4",
+ "hoodie.upsert.shuffle.parallelism" -> "4",
+ HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
+ DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
+ RECORDKEY_FIELD.key -> "c1",
+ PRECOMBINE_FIELD.key -> "c1",
+ HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
+ DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "true",
+ DataSourceReadOptions.QUERY_TYPE.key ->
DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL,
+ HoodieCompactionConfig.INLINE_COMPACT.key() -> "true",
+ HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS.key() -> "1"
+ ) ++ metadataOpts
+ setupTable(testCase, metadataOpts, commonOpts, shouldValidate = false)
+
+ assertFalse(hasLogFiles())
+ verifyFileIndexAndSQLQueries(commonOpts)
+ }
+
+ @Disabled("Needs more work")
+ @ParameterizedTest
+ @MethodSource(Array("testMetadataColumnStatsIndexParamsForMOR"))
+ def testMetadataColumnStatsIndexAsyncCompactionWithSQL(testCase:
ColumnStatsTestCase): Unit = {
+ val metadataOpts = Map(
+ HoodieMetadataConfig.ENABLE.key -> "true",
+ HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key -> "true"
+ )
+
+ val commonOpts = Map(
+ "hoodie.insert.shuffle.parallelism" -> "4",
+ "hoodie.upsert.shuffle.parallelism" -> "4",
+ HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
+ DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
+ RECORDKEY_FIELD.key -> "c1",
+ PRECOMBINE_FIELD.key -> "c1",
+ HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
+ DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "true",
+ DataSourceReadOptions.QUERY_TYPE.key ->
DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL,
+ HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS.key() -> "1"
+ ) ++ metadataOpts
+ setupTable(testCase, metadataOpts, commonOpts, shouldValidate = false)
+
+ val writeClient = new SparkRDDWriteClient(new
HoodieSparkEngineContext(jsc), getWriteConfig(commonOpts))
+ writeClient.scheduleCompaction(org.apache.hudi.common.util.Option.empty())
Review Comment:
you were asking about async compaction right.
just after this schedule call, if you were to ingest new batch of data to
the table, that will mimic async compaction.
While reading, a logical file slice will include the base file from last but
one file slice, plus all log files from last but one file slice and log files
from latest file slice too.
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala:
##########
@@ -108,7 +110,9 @@ case class MergeOnReadIncrementalRelation(override val
sqlContext: SQLContext,
}.toSeq
}
- buildSplits(filterFileSlices(fileSlices, globPattern))
+ var filteredFileSlices = filterFileSlices(fileSlices, globPattern)
Review Comment:
yes. from what I understand, we wrote a new test and its failing. likely it
will fail even w/ master code. So, lets de-couple and not block this patch for
the test failure. lets file a follow up ticket and track it.
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala:
##########
@@ -122,86 +141,132 @@ case class HoodieFileIndex(spark: SparkSession,
* @return list of PartitionDirectory containing partition to base files
mapping
*/
override def listFiles(partitionFilters: Seq[Expression], dataFilters:
Seq[Expression]): Seq[PartitionDirectory] = {
- // Look up candidate files names in the col-stats index, if all of the
following conditions are true
- // - Data-skipping is enabled
- // - Col-Stats Index is present
- // - List of predicates (filters) is present
- val candidateFilesNamesOpt: Option[Set[String]] =
+ // Prune the partition path by the partition filters
+ // NOTE: Non-partitioned tables are assumed to consist from a single
partition
+ // encompassing the whole table
+ val partitionsAndFileSlices =
getFileSlicesForPrunedPartitions(partitionFilters)
+ val partitionsAndFilteredFileSlices = filterFileSlices(dataFilters,
partitionsAndFileSlices).map {
+ case (partitionOpt, fileSlices) =>
+ val allCandidateFiles: Seq[FileStatus] = fileSlices.flatMap(fs => {
+ val baseFileStatusOpt =
getBaseFileStatus(Option.apply(fs.getBaseFile.orElse(null)))
+ val logFilesStatus = if (includeLogFiles) {
+
fs.getLogFiles.map[FileStatus](JFunction.toJavaFunction[HoodieLogFile,
FileStatus](lf => lf.getFileStatus))
+ } else {
+ java.util.stream.Stream.empty()
+ }
+ val files =
logFilesStatus.collect(Collectors.toList[FileStatus]).asScala
+ baseFileStatusOpt.foreach(f => files.append(f))
+ files
+ })
+
+ PartitionDirectory(InternalRow.fromSeq(partitionOpt.get.values),
allCandidateFiles)
+ }
+
+ hasPushedDownPartitionPredicates = true
+
+ if (shouldReadAsPartitionedTable()) {
+ partitionsAndFilteredFileSlices
+ } else {
+ Seq(PartitionDirectory(InternalRow.empty,
partitionsAndFilteredFileSlices.flatMap(_.files)))
+ }
+ }
+
+ def filterFileSlices(dataFilters: Seq[Expression], partitionAndFileSlices:
Seq[(Option[BaseHoodieTableFileIndex.PartitionPath], Seq[FileSlice])])
Review Comment:
lets add java docs that this is the core method which will be invoked across
all relations (base file only, incr, mor snapshot). and describe what the
method is expected to do
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala:
##########
@@ -122,86 +141,132 @@ case class HoodieFileIndex(spark: SparkSession,
* @return list of PartitionDirectory containing partition to base files
mapping
*/
override def listFiles(partitionFilters: Seq[Expression], dataFilters:
Seq[Expression]): Seq[PartitionDirectory] = {
- // Look up candidate files names in the col-stats index, if all of the
following conditions are true
- // - Data-skipping is enabled
- // - Col-Stats Index is present
- // - List of predicates (filters) is present
- val candidateFilesNamesOpt: Option[Set[String]] =
+ // Prune the partition path by the partition filters
+ // NOTE: Non-partitioned tables are assumed to consist from a single
partition
+ // encompassing the whole table
+ val partitionsAndFileSlices =
getFileSlicesForPrunedPartitions(partitionFilters)
+ val partitionsAndFilteredFileSlices = filterFileSlices(dataFilters,
partitionsAndFileSlices).map {
+ case (partitionOpt, fileSlices) =>
+ val allCandidateFiles: Seq[FileStatus] = fileSlices.flatMap(fs => {
+ val baseFileStatusOpt =
getBaseFileStatus(Option.apply(fs.getBaseFile.orElse(null)))
+ val logFilesStatus = if (includeLogFiles) {
+
fs.getLogFiles.map[FileStatus](JFunction.toJavaFunction[HoodieLogFile,
FileStatus](lf => lf.getFileStatus))
+ } else {
+ java.util.stream.Stream.empty()
+ }
+ val files =
logFilesStatus.collect(Collectors.toList[FileStatus]).asScala
+ baseFileStatusOpt.foreach(f => files.append(f))
+ files
+ })
+
+ PartitionDirectory(InternalRow.fromSeq(partitionOpt.get.values),
allCandidateFiles)
+ }
+
+ hasPushedDownPartitionPredicates = true
+
+ if (shouldReadAsPartitionedTable()) {
+ partitionsAndFilteredFileSlices
+ } else {
+ Seq(PartitionDirectory(InternalRow.empty,
partitionsAndFilteredFileSlices.flatMap(_.files)))
+ }
+ }
+
+ def filterFileSlices(dataFilters: Seq[Expression], partitionAndFileSlices:
Seq[(Option[BaseHoodieTableFileIndex.PartitionPath], Seq[FileSlice])])
+ : Seq[(Option[BaseHoodieTableFileIndex.PartitionPath], Seq[FileSlice])] = {
+ // If there are no data filters, return all the file slices.
+ // If there are no file slices, return empty list.
+ if (partitionAndFileSlices.isEmpty || dataFilters.isEmpty) {
+ partitionAndFileSlices
+ } else {
+ // Look up candidate files names in the col-stats index, if all of the
following conditions are true
+ // - Data-skipping is enabled
+ // - Col-Stats Index is present
+ // - List of predicates (filters) is present
+ val candidateFilesNamesOpt: Option[Set[String]] =
lookupCandidateFilesInMetadataTable(dataFilters) match {
case Success(opt) => opt
case Failure(e) =>
logError("Failed to lookup candidate files in File Index", e)
spark.sqlContext.getConf(DataSkippingFailureMode.configName,
DataSkippingFailureMode.Fallback.value) match {
case DataSkippingFailureMode.Fallback.value => Option.empty
- case DataSkippingFailureMode.Strict.value => throw new
HoodieException(e);
+ case DataSkippingFailureMode.Strict.value => throw new
HoodieException(e);
}
}
- logDebug(s"Overlapping candidate files from Column Stats Index:
${candidateFilesNamesOpt.getOrElse(Set.empty)}")
-
- var totalFileSize = 0
- var candidateFileSize = 0
+ logDebug(s"Overlapping candidate files from Column Stats Index:
${candidateFilesNamesOpt.getOrElse(Set.empty)}")
+
+ var totalFileSliceSize = 0
+ var candidateFileSliceSize = 0
+
+ val partitionsAndFilteredFileSlices = partitionAndFileSlices.map {
+ case (partitionOpt, fileSlices) =>
+ // Filter in candidate files based on the col-stats index lookup
+ val candidateFileSlices: Seq[FileSlice] = {
+ fileSlices.filter(fs => {
+ val baseFileStatusOpt =
getBaseFileStatus(Option.apply(fs.getBaseFile.orElse(null)))
+ val logFiles =
fs.getLogFiles.collect(Collectors.toSet[HoodieLogFile]).asScala.toSet[HoodieLogFile]
+ // NOTE: This predicate is true when {@code Option} is empty
+ if (candidateFilesNamesOpt.forall(files =>
baseFileStatusOpt.exists(f => files.contains(f.getPath.getName))
+ || files.intersect(logFiles.map(f =>
f.getPath.getName)).nonEmpty)) {
Review Comment:
```
val fileSlicesFiles : Seq[String] = // collect all files from
the file slice to this seq
candidateFilesNamesOpt.forall(files =>
if (fileSlicesFiles.filter(file =>
!files.contains(file)).count == fileSlicesFiles.size){
// if every file in the file slice did not match stats
from cols stats
false
} else {
true
} )
````
##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndexWithSQL.scala:
##########
@@ -0,0 +1,379 @@
+/*
+ * 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.functional
+
+import org.apache.hudi.DataSourceWriteOptions.{DELETE_OPERATION_OPT_VAL,
PRECOMBINE_FIELD, RECORDKEY_FIELD}
+import org.apache.hudi.async.SparkAsyncCompactService
+import org.apache.hudi.client.SparkRDDWriteClient
+import org.apache.hudi.client.common.HoodieSparkEngineContext
+import org.apache.hudi.client.utils.MetadataConversionUtils
+import org.apache.hudi.common.config.HoodieMetadataConfig
+import org.apache.hudi.common.model.{FileSlice, HoodieCommitMetadata,
HoodieTableType, WriteOperationType}
+import org.apache.hudi.common.table.HoodieTableConfig
+import org.apache.hudi.common.table.timeline.HoodieInstant
+import org.apache.hudi.config.{HoodieCompactionConfig, HoodieIndexConfig,
HoodieWriteConfig}
+import org.apache.hudi.functional.ColumnStatIndexTestBase.ColumnStatsTestCase
+import org.apache.hudi.index.HoodieIndex.IndexType.INMEMORY
+import org.apache.hudi.metadata.HoodieMetadataFileSystemView
+import org.apache.hudi.util.{JFunction, JavaConversions}
+import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions,
HoodieFileIndex}
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference,
Expression, GreaterThan, Literal}
+import org.apache.spark.sql.types.StringType
+import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue}
+import org.junit.jupiter.api.Disabled
+import org.junit.jupiter.params.ParameterizedTest
+import org.junit.jupiter.params.provider.MethodSource
+
+import java.util.Properties
+import scala.collection.JavaConverters
+
+class TestColumnStatsIndexWithSQL extends ColumnStatIndexTestBase {
+
+ @ParameterizedTest
+ @MethodSource(Array("testMetadataColumnStatsIndexParams"))
+ def testMetadataColumnStatsIndexWithSQL(testCase: ColumnStatsTestCase): Unit
= {
+ val metadataOpts = Map(
+ HoodieMetadataConfig.ENABLE.key -> "true",
+ HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key -> "true"
+ )
+
+ val commonOpts = Map(
+ "hoodie.insert.shuffle.parallelism" -> "4",
+ "hoodie.upsert.shuffle.parallelism" -> "4",
+ HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
+ DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
+ RECORDKEY_FIELD.key -> "c1",
+ PRECOMBINE_FIELD.key -> "c1",
+ HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
+ DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "true",
+ DataSourceReadOptions.QUERY_TYPE.key ->
DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL
+ ) ++ metadataOpts
+ setupTable(testCase, metadataOpts, commonOpts, shouldValidate = true)
+ verifyFileIndexAndSQLQueries(commonOpts)
+ }
+
+ @ParameterizedTest
+ @MethodSource(Array("testMetadataColumnStatsIndexParamsForMOR"))
+ def testMetadataColumnStatsIndexSQLWithInMemoryIndex(testCase:
ColumnStatsTestCase): Unit = {
+ val metadataOpts = Map(
+ HoodieMetadataConfig.ENABLE.key -> "true",
+ HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key -> "true"
+ )
+
+ val commonOpts = Map(
+ "hoodie.insert.shuffle.parallelism" -> "4",
+ "hoodie.upsert.shuffle.parallelism" -> "4",
+ HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
+ DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
+ RECORDKEY_FIELD.key -> "c1",
+ PRECOMBINE_FIELD.key -> "c1",
+ HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
+ DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "true",
+ DataSourceReadOptions.QUERY_TYPE.key ->
DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL,
+ HoodieIndexConfig.INDEX_TYPE.key() -> INMEMORY.name()
+ ) ++ metadataOpts
+
+ doWriteAndValidateColumnStats(testCase, metadataOpts, commonOpts,
+ dataSourcePath = "index/colstats/input-table-json",
+ expectedColStatsSourcePath =
"index/colstats/column-stats-index-table.json",
+ operation = DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
+ saveMode = SaveMode.Overwrite,
+ shouldValidate = false)
+
+ assertEquals(4, getLatestDataFilesCount(commonOpts))
+ assertEquals(0, getLatestDataFilesCount(commonOpts, includeLogFiles =
false))
+ var dataFilter = GreaterThan(attribute("c5"), literal("90"))
+ verifyPruningFileCount(commonOpts, dataFilter, 3)
+ dataFilter = GreaterThan(attribute("c5"), literal("95"))
+ verifyPruningFileCount(commonOpts, dataFilter, 1)
+ }
+
+ @ParameterizedTest
+ @MethodSource(Array("testMetadataColumnStatsIndexParams"))
+ def testMetadataColumnStatsIndexDeletionWithSQL(testCase:
ColumnStatsTestCase): Unit = {
+ val metadataOpts = Map(
+ HoodieMetadataConfig.ENABLE.key -> "true",
+ HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key -> "true"
+ )
+
+ val commonOpts = Map(
+ "hoodie.insert.shuffle.parallelism" -> "4",
+ "hoodie.upsert.shuffle.parallelism" -> "4",
+ HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
+ DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
+ RECORDKEY_FIELD.key -> "c1",
+ PRECOMBINE_FIELD.key -> "c1",
+ HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
+ DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "true",
+ DataSourceReadOptions.QUERY_TYPE.key ->
DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL
+ ) ++ metadataOpts
+ setupTable(testCase, metadataOpts, commonOpts, shouldValidate = true)
+ val lastDf = dfList.last
+
+ lastDf.write.format("org.apache.hudi")
+ .options(commonOpts)
+ .option(DataSourceWriteOptions.OPERATION.key, DELETE_OPERATION_OPT_VAL)
+ .mode(SaveMode.Append)
+ .save(basePath)
+ verifyFileIndexAndSQLQueries(commonOpts,
isTableDataSameAsAfterSecondInstant = true)
+
+ // Add the last df back and verify the queries
+ doWriteAndValidateColumnStats(testCase, metadataOpts, commonOpts,
+ dataSourcePath = "index/colstats/update-input-table-json",
+ expectedColStatsSourcePath = "",
+ operation = DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL,
+ saveMode = SaveMode.Append,
+ shouldValidate = false)
+ verifyFileIndexAndSQLQueries(commonOpts, verifyFileCount = false)
+ }
+
+ @ParameterizedTest
+ @MethodSource(Array("testMetadataColumnStatsIndexParamsForMOR"))
+ def testMetadataColumnStatsIndexCompactionWithSQL(testCase:
ColumnStatsTestCase): Unit = {
+ val metadataOpts = Map(
+ HoodieMetadataConfig.ENABLE.key -> "true",
+ HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key -> "true"
+ )
+
+ val commonOpts = Map(
+ "hoodie.insert.shuffle.parallelism" -> "4",
+ "hoodie.upsert.shuffle.parallelism" -> "4",
+ HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
+ DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
+ RECORDKEY_FIELD.key -> "c1",
+ PRECOMBINE_FIELD.key -> "c1",
+ HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
+ DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "true",
+ DataSourceReadOptions.QUERY_TYPE.key ->
DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL,
+ HoodieCompactionConfig.INLINE_COMPACT.key() -> "true",
+ HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS.key() -> "1"
+ ) ++ metadataOpts
+ setupTable(testCase, metadataOpts, commonOpts, shouldValidate = false)
+
+ assertFalse(hasLogFiles())
+ verifyFileIndexAndSQLQueries(commonOpts)
+ }
+
+ @Disabled("Needs more work")
+ @ParameterizedTest
+ @MethodSource(Array("testMetadataColumnStatsIndexParamsForMOR"))
+ def testMetadataColumnStatsIndexAsyncCompactionWithSQL(testCase:
ColumnStatsTestCase): Unit = {
+ val metadataOpts = Map(
+ HoodieMetadataConfig.ENABLE.key -> "true",
+ HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key -> "true"
+ )
+
+ val commonOpts = Map(
+ "hoodie.insert.shuffle.parallelism" -> "4",
+ "hoodie.upsert.shuffle.parallelism" -> "4",
+ HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
+ DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
+ RECORDKEY_FIELD.key -> "c1",
+ PRECOMBINE_FIELD.key -> "c1",
+ HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
+ DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "true",
+ DataSourceReadOptions.QUERY_TYPE.key ->
DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL,
+ HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS.key() -> "1"
+ ) ++ metadataOpts
+ setupTable(testCase, metadataOpts, commonOpts, shouldValidate = false)
+
+ val writeClient = new SparkRDDWriteClient(new
HoodieSparkEngineContext(jsc), getWriteConfig(commonOpts))
+ writeClient.scheduleCompaction(org.apache.hudi.common.util.Option.empty())
+ val compactionService = new SparkAsyncCompactService(new
HoodieSparkEngineContext(jsc), writeClient)
+
compactionService.enqueuePendingAsyncServiceInstant(metaClient.reloadActiveTimeline().lastInstant().get())
+ compactionService.start(JFunction.toJavaFunction(b => true))
+ compactionService.waitTillPendingAsyncServiceInstantsReducesTo(0)
+ assertFalse(hasLogFiles())
+ verifyFileIndexAndSQLQueries(commonOpts)
+ }
+
+ private def setupTable(testCase: ColumnStatsTestCase, metadataOpts:
Map[String, String], commonOpts: Map[String, String], shouldValidate: Boolean):
Unit = {
+ doWriteAndValidateColumnStats(testCase, metadataOpts, commonOpts,
+ dataSourcePath = "index/colstats/input-table-json",
+ expectedColStatsSourcePath =
"index/colstats/column-stats-index-table.json",
+ operation = DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
+ saveMode = SaveMode.Overwrite)
+
+ doWriteAndValidateColumnStats(testCase, metadataOpts, commonOpts,
+ dataSourcePath = "index/colstats/another-input-table-json",
+ expectedColStatsSourcePath =
"index/colstats/updated-column-stats-index-table.json",
+ operation = DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL,
+ saveMode = SaveMode.Append)
+
+ // NOTE: MOR and COW have different fixtures since MOR is bearing
delta-log files (holding
+ // deferred updates), diverging from COW
+ val expectedColStatsSourcePath = if (testCase.tableType ==
HoodieTableType.COPY_ON_WRITE) {
+ "index/colstats/cow-updated2-column-stats-index-table.json"
+ } else {
+ "index/colstats/mor-updated2-column-stats-index-table.json"
+ }
+
+ doWriteAndValidateColumnStats(testCase, metadataOpts, commonOpts,
+ dataSourcePath = "index/colstats/update-input-table-json",
+ expectedColStatsSourcePath = expectedColStatsSourcePath,
+ operation = DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL,
+ saveMode = SaveMode.Append,
+ shouldValidate)
+ }
+
+ def verifyFileIndexAndSQLQueries(opts: Map[String, String],
isTableDataSameAsAfterSecondInstant: Boolean = false, verifyFileCount: Boolean
= true): Unit = {
+ var commonOpts = opts
+ val inputDF1 = spark.read.format("hudi")
+ .options(commonOpts)
+ .option("as.of.instant",
metaClient.getActiveTimeline.getInstants.get(1).getTimestamp)
+ .option(DataSourceReadOptions.QUERY_TYPE.key,
DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
+ .option(DataSourceReadOptions.ENABLE_DATA_SKIPPING.key, "false")
+ .load(basePath)
+ inputDF1.createOrReplaceTempView("tbl")
+ val numRecordsWithC5ColumnGreaterThan70 = spark.sql("select * from tbl
where c5 > 70").count()
+ // verify snapshot query
+ verifySQLQueries(numRecordsWithC5ColumnGreaterThan70,
DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL, commonOpts,
isTableDataSameAsAfterSecondInstant)
+
+ // verify read_optimized query
+ verifySQLQueries(numRecordsWithC5ColumnGreaterThan70,
DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, commonOpts,
isTableDataSameAsAfterSecondInstant)
+
+ // verify incremental query
+ verifySQLQueries(numRecordsWithC5ColumnGreaterThan70,
DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL, commonOpts,
isTableDataSameAsAfterSecondInstant)
+ commonOpts = commonOpts +
(DataSourceReadOptions.INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN_FOR_NON_EXISTING_FILES.key
-> "true")
+ verifySQLQueries(numRecordsWithC5ColumnGreaterThan70,
DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL, commonOpts,
isTableDataSameAsAfterSecondInstant)
+
+ if (verifyFileCount) {
+ // First commit creates 4 parquet files
+ // Second commit creates 4 parquet files
+ // Last commit creates one parquet file and 4 log files - total 5 files
(for MOR)
+ // and 4 parquet files and no log file (for COW)
+ // therefore on deletion all those 5 files create a new log file whereas
COW file count remains same
+ var numFiles = if (isTableMOR && isTableDataSameAsAfterSecondInstant) 17
else if (hasLogFiles()) 12 else 8
+ var dataFilter = GreaterThan(attribute("c5"), literal("70"))
+ verifyPruningFileCount(commonOpts, dataFilter, numFiles)
+
+ dataFilter = GreaterThan(attribute("c5"), literal("90"))
+ numFiles = if (isTableMOR && isTableDataSameAsAfterSecondInstant) 11
else if (hasLogFiles()) 7 else 4
+ verifyPruningFileCount(commonOpts, dataFilter, numFiles)
+ }
+ }
+
+ private def verifyPruningFileCount(opts: Map[String, String], dataFilter:
Expression, numFiles: Int): Unit = {
+ val fileIndex = HoodieFileIndex(spark, metaClient, None, opts + ("path" ->
basePath))
Review Comment:
my2 cents. we can keep all comprehensive tests directly using file index.
but lets add some sanity e2e tests to ensure col stats integration works. what
incase in future, due to some refactoring the col stats integration breaks.
just to catch such scenarios, we can have few sanity tests for both table types
where we use the query plans to validate that col stats kicks in.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]