yihua commented on code in PR #10352:
URL: https://github.com/apache/hudi/pull/10352#discussion_r1584146799


##########
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##########
@@ -1872,4 +1883,175 @@ public HoodieRecord next() {
       }
     };
   }
+
+  public static HoodieData<HoodieRecord> 
convertFilesToPartitionStatsRecords(HoodieEngineContext engineContext,
+                                                                             
List<DirectoryInfo> partitionInfoList,

Review Comment:
   Does this contain log files in MOR tables?



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionStatsIndex.scala:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.PARTITIONPATH_FIELD
+import org.apache.hudi.common.model.{FileSlice, HoodieTableType}
+import org.apache.hudi.common.table.HoodieTableMetaClient
+import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions
+import org.apache.hudi.metadata.HoodieMetadataFileSystemView
+import org.apache.hudi.util.JFunction
+import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, 
HoodieFileIndex}
+import org.apache.spark.sql.SaveMode
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, 
Expression, Literal}
+import org.apache.spark.sql.types.StringType
+import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
+import org.junit.jupiter.api.{Tag, Test}
+import org.junit.jupiter.params.ParameterizedTest
+import org.junit.jupiter.params.provider.EnumSource
+
+import scala.collection.JavaConversions._
+
+/**
+ * Test cases on partition stats index with Spark datasource.
+ */
+@Tag("functional")
+class TestPartitionStatsIndex extends PartitionStatsIndexTestBase {

Review Comment:
   Could you add docs on each test scenario so it's easy to read?



##########
hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestDataSkippingWithMORColstats.java:
##########
@@ -318,6 +321,7 @@ private Map<String, String> getOptions() {
     options.put(HoodieMetadataConfig.ENABLE.key(), "true");
     options.put(HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key(), 
"true");
     options.put(HoodieMetadataConfig.COLUMN_STATS_INDEX_FOR_COLUMNS.key(), 
"trip_type");
+    // 
options.put(HoodieMetadataConfig.ENABLE_METADATA_INDEX_PARTITION_STATS.key(), 
"false");

Review Comment:
   nit: remove?



##########
hudi-common/src/main/java/org/apache/hudi/common/util/BaseFileUtils.java:
##########
@@ -67,6 +70,61 @@ public static BaseFileUtils getInstance(HoodieFileFormat 
fileFormat) {
     throw new UnsupportedOperationException(fileFormat.name() + " format not 
supported yet.");
   }
 
+  /**
+   * Aggregate column range statistics across files in a partition.
+   *
+   * @param fileColumnRanges List of column range statistics for each file in 
a partition
+   */
+  public static <T extends Comparable<T>> HoodieColumnRangeMetadata<T> 
getColumnRangeInPartition(@Nonnull List<HoodieColumnRangeMetadata<T>> 
fileColumnRanges) {
+    if (fileColumnRanges.size() == 1) {
+      // Only one parquet file, we can just return that range.
+      return fileColumnRanges.get(0);
+    }
+    // There are multiple files. Compute min(file_mins) and max(file_maxs)
+    return fileColumnRanges.stream()
+        .sequential()
+        .reduce(BaseFileUtils::mergeRanges).get();
+  }
+
+  private static <T extends Comparable<T>> HoodieColumnRangeMetadata<T> 
mergeRanges(HoodieColumnRangeMetadata<T> one,

Review Comment:
   Do we merge column ranges between the base and log files (e.g., log file 
contains updates and deletes)?  Is that covered in this PR or by existing 
column stats logic (how does that work)?



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionStatsIndexWithSql.scala:
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.common.model.WriteOperationType
+import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
+import org.apache.spark.sql.hudi.HoodieSparkSqlTestBase
+import org.apache.spark.sql.hudi.HoodieSparkSqlTestBase.getLastCommitMetadata
+import org.junit.jupiter.api.Tag
+
+import scala.collection.JavaConverters._
+
+@Tag("functional")
+class TestPartitionStatsIndexWithSql extends HoodieSparkSqlTestBase {
+
+  val sqlTempTable = "hudi_tbl"
+
+  test("Test Create Table As Select") {
+    withTempDir { tmp =>
+      val tableName = generateTableName
+      val tablePath = s"${tmp.getCanonicalPath}/$tableName"
+      // Create table with date type partition
+      spark.sql(
+        s"""
+           | create table $tableName using hudi
+           | partitioned by (dt)
+           | tblproperties(
+           |    primaryKey = 'id',
+           |    preCombineField = 'ts',
+           |    'hoodie.metadata.index.partition.stats.enable' = 'true'
+           | )
+           | location '$tablePath'
+           | AS
+           | select 1 as id, 'a1' as name, 10 as price, 1000 as ts, 
cast('2021-05-06' as date) as dt
+         """.stripMargin
+      )
+
+      assertResult(WriteOperationType.BULK_INSERT) {
+        getLastCommitMetadata(spark, tablePath).getOperationType
+      }
+      checkAnswer(s"select id, name, price, ts, cast(dt as string) from 
$tableName")(

Review Comment:
   @codope is this done?  I don't seem to see the validation on the partition 
prunning in the tests in this class.



##########
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##########
@@ -1872,4 +1883,175 @@ public HoodieRecord next() {
       }
     };
   }
+
+  public static HoodieData<HoodieRecord> 
convertFilesToPartitionStatsRecords(HoodieEngineContext engineContext,
+                                                                             
List<DirectoryInfo> partitionInfoList,
+                                                                             
HoodieMetadataConfig metadataConfig,
+                                                                             
HoodieTableMetaClient dataTableMetaClient) {
+    final List<String> columnsToIndex = 
metadataConfig.getColumnsEnabledForColumnStatsIndex();
+    if (columnsToIndex.isEmpty()) {
+      return engineContext.emptyHoodieData();
+    }
+    LOG.debug("Indexing following columns for partition stats index: {}", 
columnsToIndex);
+    // Create records for MDT
+    int parallelism = Math.max(Math.min(partitionInfoList.size(), 
metadataConfig.getPartitionStatsIndexParallelism()), 1);
+    return engineContext.parallelize(partitionInfoList, 
parallelism).flatMap(partitionInfo -> {
+      final String partitionPath = partitionInfo.getRelativePath();
+      // Step 1: Collect Column Metadata for Each File (Your existing code 
does this)
+      List<List<HoodieColumnRangeMetadata<Comparable>>> fileColumnMetadata = 
partitionInfo.getFileNameToSizeMap().keySet().stream()
+          .map(fileName -> getFileStatsRangeMetadata(partitionPath, 
partitionPath + "/" + fileName, dataTableMetaClient, columnsToIndex, false))
+          .collect(toList());
+      // Step 2: Flatten and Group by Column Name
+      Map<String, List<HoodieColumnRangeMetadata<Comparable>>> 
columnMetadataMap = fileColumnMetadata.stream()
+          .flatMap(List::stream) // Flatten the list
+          
.collect(Collectors.groupingBy(HoodieColumnRangeMetadata::getColumnName, 
toList())); // Group by column name
+      // Step 3: Aggregate Column Ranges
+      Stream<HoodieColumnRangeMetadata<Comparable>> 
partitionStatsRangeMetadata = columnMetadataMap.entrySet().stream()
+          .map(entry -> 
BaseFileUtils.getColumnRangeInPartition(entry.getValue()));
+      return HoodieMetadataPayload.createPartitionStatsRecords(partitionPath, 
partitionStatsRangeMetadata.collect(toList()), false).iterator();
+    });
+  }
+
+  private static List<HoodieColumnRangeMetadata<Comparable>> 
getFileStatsRangeMetadata(String partitionPath,
+                                                                               
        String filePath,
+                                                                               
        HoodieTableMetaClient datasetMetaClient,
+                                                                               
        List<String> columnsToIndex,
+                                                                               
        boolean isDeleted) {
+    String filePartitionPath = filePath.startsWith("/") ? 
filePath.substring(1) : filePath;
+    String fileName = FSUtils.getFileName(filePath, partitionPath);
+    if (isDeleted) {
+      return columnsToIndex.stream()
+          .map(entry -> HoodieColumnRangeMetadata.stub(fileName, entry))
+          .collect(Collectors.toList());
+    }
+    return readColumnRangeMetadataFrom(filePartitionPath, datasetMetaClient, 
columnsToIndex);
+  }
+
+  public static HoodieData<HoodieRecord> 
convertMetadataToPartitionStatsRecords(HoodieCommitMetadata commitMetadata,

Review Comment:
   Can this method be consolidated with `convertFilesToPartitionStatsRecords`? 
They share common code.



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