lokeshj1703 commented on code in PR #9345:
URL: https://github.com/apache/hudi/pull/9345#discussion_r1285196978


##########
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:
   Addressed



##########
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:
   Addressed



##########
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:
   Addressed



##########
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:
   Addressed



-- 
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]

Reply via email to