Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-06-05 Thread via GitHub


KnightChess commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2149214884

   creaete a issue to track it https://issues.apache.org/jira/browse/HUDI-7829


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-30 Thread via GitHub


codope merged PR #10352:
URL: https://github.com/apache/hudi/pull/10352


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-30 Thread via GitHub


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


##
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.hudi.common.testutils.HoodieTestUtils
+import org.apache.spark.sql.hudi.common.HoodieSparkSqlTestBase
+import org.junit.jupiter.api.Tag
+
+import scala.collection.JavaConverters._
+
+@Tag("functional")
+class TestPartitionStatsIndexWithSql extends HoodieSparkSqlTestBase {
+
+  val sqlTempTable = "hudi_tbl"
+
+  test("Test partition stats index following insert, merge into, update and 
delete") {
+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) {
+HoodieSparkSqlTestBase.getLastCommitMetadata(spark, 
tablePath).getOperationType
+  }
+  checkAnswer(s"select id, name, price, ts, cast(dt as string) from 
$tableName")(
+Seq(1, "a1", 10, 1000, "2021-05-06")
+  )
+
+  val partitionValue = "2021-05-06"
+
+  // Check the missing properties for spark sql
+  val metaClient = HoodieTableMetaClient.builder()
+.setBasePath(tablePath)
+.setConf(HoodieTestUtils.getDefaultStorageConf)
+.build()
+  val properties = metaClient.getTableConfig.getProps.asScala.toMap
+  
assertResult(true)(properties.contains(HoodieTableConfig.CREATE_SCHEMA.key))
+  assertResult("dt")(properties(HoodieTableConfig.PARTITION_FIELDS.key))
+  assertResult("ts")(properties(HoodieTableConfig.PRECOMBINE_FIELD.key))
+  assertResult(tableName)(metaClient.getTableConfig.getTableName)
+
+  // Test insert into
+  spark.sql(s"insert into $tableName values(2, 'a2', 10, 1000, 
cast('$partitionValue' as date))")
+  checkAnswer(s"select _hoodie_record_key, _hoodie_partition_path, id, 
name, price, ts, cast(dt as string) from $tableName order by id")(

Review Comment:
   I have it in two seprate tests - `testPartitionStatsWithPartitionBy` and 
`testPartitionStatsIndexFilePruning`



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-30 Thread via GitHub


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


##
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 > HoodieColumnRangeMetadata 
getColumnRangeInPartition(@Nonnull List> 
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 > HoodieColumnRangeMetadata 
mergeRanges(HoodieColumnRangeMetadata one,

Review Comment:
   We follow the same approach as for colstats - 
https://github.com/apache/hudi/blob/f99b181a04e2ee549009d38deca7597219c4ce46/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java#L372
   If the new record is for delete, then we ignore the previous one. If 
previous record is for delete, then we take the newer one, otherwise just merge 
the stats. I don't think we do event-time based merging for MDT.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-30 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##
@@ -1872,4 +1883,175 @@ public HoodieRecord next() {
   }
 };
   }
+
+  public static HoodieData 
convertFilesToPartitionStatsRecords(HoodieEngineContext engineContext,
+ 
List partitionInfoList,
+ 
HoodieMetadataConfig metadataConfig,
+ 
HoodieTableMetaClient dataTableMetaClient) {
+final List 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>> 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>> 
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> 
partitionStatsRangeMetadata = columnMetadataMap.entrySet().stream()
+  .map(entry -> 
BaseFileUtils.getColumnRangeInPartition(entry.getValue()));
+  return HoodieMetadataPayload.createPartitionStatsRecords(partitionPath, 
partitionStatsRangeMetadata.collect(toList()), false).iterator();
+});
+  }
+
+  private static List> 
getFileStatsRangeMetadata(String partitionPath,
+   
String filePath,
+   
HoodieTableMetaClient datasetMetaClient,
+   
List 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 
convertMetadataToPartitionStatsRecords(HoodieCommitMetadata commitMetadata,

Review Comment:
   Ack, HUDI-7696 to track.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-30 Thread via GitHub


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


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

Review Comment:
   Yes, this is just for initialization. We just log a warning and continue for 
other filegroups. This is what colstats is also doing. However, given that 
partition stats will be enabled by default, I think it makes sense to throw 
exception. Will do it.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-30 Thread via GitHub


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


##
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.hudi.common.testutils.HoodieTestUtils
+import org.apache.spark.sql.hudi.common.HoodieSparkSqlTestBase
+import org.junit.jupiter.api.Tag
+
+import scala.collection.JavaConverters._
+
+@Tag("functional")
+class TestPartitionStatsIndexWithSql extends HoodieSparkSqlTestBase {
+
+  val sqlTempTable = "hudi_tbl"
+
+  test("Test partition stats index following insert, merge into, update and 
delete") {
+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) {
+HoodieSparkSqlTestBase.getLastCommitMetadata(spark, 
tablePath).getOperationType
+  }
+  checkAnswer(s"select id, name, price, ts, cast(dt as string) from 
$tableName")(
+Seq(1, "a1", 10, 1000, "2021-05-06")
+  )
+
+  val partitionValue = "2021-05-06"
+
+  // Check the missing properties for spark sql
+  val metaClient = HoodieTableMetaClient.builder()
+.setBasePath(tablePath)
+.setConf(HoodieTestUtils.getDefaultStorageConf)
+.build()
+  val properties = metaClient.getTableConfig.getProps.asScala.toMap
+  
assertResult(true)(properties.contains(HoodieTableConfig.CREATE_SCHEMA.key))
+  assertResult("dt")(properties(HoodieTableConfig.PARTITION_FIELDS.key))
+  assertResult("ts")(properties(HoodieTableConfig.PRECOMBINE_FIELD.key))
+  assertResult(tableName)(metaClient.getTableConfig.getTableName)
+
+  // Test insert into
+  spark.sql(s"insert into $tableName values(2, 'a2', 10, 1000, 
cast('$partitionValue' as date))")
+  checkAnswer(s"select _hoodie_record_key, _hoodie_partition_path, id, 
name, price, ts, cast(dt as string) from $tableName order by id")(

Review Comment:
   I don't seem to see the validation on the partition pruning, e.g., in the 
logical plan, in the tests in this class.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-30 Thread via GitHub


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


##
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 > HoodieColumnRangeMetadata 
getColumnRangeInPartition(@Nonnull List> 
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 > HoodieColumnRangeMetadata 
mergeRanges(HoodieColumnRangeMetadata one,

Review Comment:
   Got it.  I'm trying to understand how col stats are merged between the base 
and log files, e.g., for custom payload, because we may not be able to simply 
take the minimum of all minimum values or the maximum of all maximum values 
from the base and log files, as there can be deletes and there can be 
event-time based merging.  We can discuss this in a separate thread.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-30 Thread via GitHub


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


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

Review Comment:
   I see. Is this for initializing the partition stats only?  Do we have a 
guard to throw an exception if the colstats or partition stats partition is 
initialized on the latest file slices with log files?



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-30 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##
@@ -1872,4 +1883,175 @@ public HoodieRecord next() {
   }
 };
   }
+
+  public static HoodieData 
convertFilesToPartitionStatsRecords(HoodieEngineContext engineContext,
+ 
List partitionInfoList,
+ 
HoodieMetadataConfig metadataConfig,
+ 
HoodieTableMetaClient dataTableMetaClient) {
+final List 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>> 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>> 
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> 
partitionStatsRangeMetadata = columnMetadataMap.entrySet().stream()
+  .map(entry -> 
BaseFileUtils.getColumnRangeInPartition(entry.getValue()));
+  return HoodieMetadataPayload.createPartitionStatsRecords(partitionPath, 
partitionStatsRangeMetadata.collect(toList()), false).iterator();
+});
+  }
+
+  private static List> 
getFileStatsRangeMetadata(String partitionPath,
+   
String filePath,
+   
HoodieTableMetaClient datasetMetaClient,
+   
List 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 
convertMetadataToPartitionStatsRecords(HoodieCommitMetadata commitMetadata,

Review Comment:
   Got it, makes sense.  We can revisit later why it needs two types of 
methods/code path for generating partition stats records (track it in JIRA).



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-30 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2085972850

   
   ## CI report:
   
   * f63dbe172cf8dec2603c266396fb7d31d5cb7f60 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23574)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-30 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2085591638

   
   ## CI report:
   
   * 54d5734d6d206c59c29bd8661cdef2f1692cfa49 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23557)
 
   * f63dbe172cf8dec2603c266396fb7d31d5cb7f60 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23574)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-30 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2085573900

   
   ## CI report:
   
   * 54d5734d6d206c59c29bd8661cdef2f1692cfa49 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23557)
 
   * f63dbe172cf8dec2603c266396fb7d31d5cb7f60 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-30 Thread via GitHub


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


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##
@@ -403,6 +400,13 @@ private boolean initializeFromFilesystem(String 
initializationTime, List

Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-30 Thread via GitHub


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


##
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 > HoodieColumnRangeMetadata 
getColumnRangeInPartition(@Nonnull List> 
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()

Review Comment:
   does not matter.. removed it.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-30 Thread via GitHub


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


##
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 > HoodieColumnRangeMetadata 
getColumnRangeInPartition(@Nonnull List> 
fileColumnRanges) {
+if (fileColumnRanges.size() == 1) {

Review Comment:
   `fileColumnRanges` should not be empty. For empty parquet files, we will err 
out before this point. I'll add a validation anyway.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-30 Thread via GitHub


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


##
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 > HoodieColumnRangeMetadata 
getColumnRangeInPartition(@Nonnull List> 
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 > HoodieColumnRangeMetadata 
mergeRanges(HoodieColumnRangeMetadata one,

Review Comment:
   Yes, it's covered by existing colstats logic. For log files, we get the 
column ranges based on write stats and then merge as usual.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-30 Thread via GitHub


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


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

Review Comment:
   No, the assumptions of colstats apply for parition stats too. Log files are 
considered in subsequent updates with the write status. So, just like colstats, 
we will have to compact the table before enabling partition stats.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-30 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##
@@ -372,6 +379,10 @@ public static Map> convertMetada
   dataMetaClient, isColumnStatsIndexEnabled, 
columnStatsIndexParallelism, targetColumnsForColumnStatsIndex);
   partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, 
metadataColumnStatsRDD);
 }
+if (enabledPartitionTypes.contains(MetadataPartitionType.PARTITION_STATS)) 
{
+  final HoodieData partitionStatsRDD = 
convertMetadataToPartitionStatsRecords(commitMetadata, context, dataMetaClient, 
metadataConfig);
+  partitionToRecordsMap.put(MetadataPartitionType.PARTITION_STATS, 
partitionStatsRDD);
+}

Review Comment:
   No. That's a very good point! Typically, we are enabling indexes one at a 
time. But, I agree with you. I will work on it in HUDI-7690. Btw, this 
limitation is only for initialization. In case of update, we don't compute 
sequuentially.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-30 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##
@@ -1872,4 +1883,175 @@ public HoodieRecord next() {
   }
 };
   }
+
+  public static HoodieData 
convertFilesToPartitionStatsRecords(HoodieEngineContext engineContext,
+ 
List partitionInfoList,
+ 
HoodieMetadataConfig metadataConfig,
+ 
HoodieTableMetaClient dataTableMetaClient) {
+final List 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>> 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>> 
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> 
partitionStatsRangeMetadata = columnMetadataMap.entrySet().stream()
+  .map(entry -> 
BaseFileUtils.getColumnRangeInPartition(entry.getValue()));
+  return HoodieMetadataPayload.createPartitionStatsRecords(partitionPath, 
partitionStatsRangeMetadata.collect(toList()), false).iterator();
+});
+  }
+
+  private static List> 
getFileStatsRangeMetadata(String partitionPath,
+   
String filePath,
+   
HoodieTableMetaClient datasetMetaClient,
+   
List 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 
convertMetadataToPartitionStatsRecords(HoodieCommitMetadata commitMetadata,

Review Comment:
   High-level steps are same but the two methods work with different objects. 
This method works with writeStats while `convertFilesToPartitionStatsRecords` 
works with actual file itself. Also, this method has some preprocessing which 
is not required in the other method. I think the code reads better inline.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-29 Thread via GitHub


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 
convertFilesToPartitionStatsRecords(HoodieEngineContext engineContext,
+ 
List 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 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 > HoodieColumnRangeMetadata 
getColumnRangeInPartition(@Nonnull List> 
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 > HoodieColumnRangeMetadata 
mergeRanges(HoodieColumnRangeMetadata 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 

Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-29 Thread via GitHub


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


##
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 > HoodieColumnRangeMetadata 
getColumnRangeInPartition(@Nonnull List> 
fileColumnRanges) {
+if (fileColumnRanges.size() == 1) {

Review Comment:
   This seems unnecessary as the `reduce` should handle size of 1.  Also, can 
`fileColumnRanges` list be empty, e.g., empty parquet file?



##
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 > HoodieColumnRangeMetadata 
getColumnRangeInPartition(@Nonnull List> 
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()

Review Comment:
   Does this have to be sequential?  or it does not matter.



##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##
@@ -403,6 +400,13 @@ private boolean initializeFromFilesystem(String 
initializationTime, List> convertMetada
   dataMetaClient, isColumnStatsIndexEnabled, 
columnStatsIndexParallelism, targetColumnsForColumnStatsIndex);
   partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, 
metadataColumnStatsRDD);
 }
+if (enabledPartitionTypes.contains(MetadataPartitionType.PARTITION_STATS)) 
{
+  final HoodieData partitionStatsRDD = 
convertMetadataToPartitionStatsRecords(commitMetadata, context, dataMetaClient, 
metadataConfig);
+  partitionToRecordsMap.put(MetadataPartitionType.PARTITION_STATS, 
partitionStatsRDD);
+}

Review Comment:
   A side topic, does this compose a sub-graph in DAG without triggering the 
execution?  Ideally, all types of metadata should be computed in parallel 
leveraging the parallelism in Spark, instead of being computed type by type 
sequentially.



##
hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java:
##
@@ -330,6 +330,27 @@ public final class HoodieMetadataConfig extends 
HoodieConfig {
   .sinceVersion("1.0.0")
   .withDocumentation("Parallelism to use, when generating functional 
index.");
 
+  public static final ConfigProperty 
ENABLE_METADATA_INDEX_PARTITION_STATS = ConfigProperty
+  .key(METADATA_PREFIX + ".index.partition.stats.enable")
+  .defaultValue(true)
+  .sinceVersion("1.0.0")
+  .withDocumentation("Enable aggregating stats for each column at the 
storage partition level. ");

Review Comment:
   ```suggestion
 .withDocumentation("Enable aggregating stats for each column at the 
storage partition level.");
   ```



##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##
@@ -403,6 +400,13 @@ private boolean initializeFromFilesystem(String 
initializationTime, Listhttps://issues.apache.org/jira/browse/AVRO-1810
+   * has a bug - https://issues.apache.org/jira/browse/AVRO-1810;>...
*/
-  protected static final int METADATA_TYPE_PARTITION_LIST = 1;
-  protected static final int METADATA_TYPE_FILE_LIST = 2;
-  protected static final int METADATA_TYPE_COLUMN_STATS = 3;
-  protected static final int METADATA_TYPE_BLOOM_FILTER = 4;
+  private static final int METADATA_TYPE_PARTITION_LIST = 1;
+  private static final int METADATA_TYPE_FILE_LIST = 2;
+  private static final int METADATA_TYPE_COLUMN_STATS = 3;
+  private static final int METADATA_TYPE_BLOOM_FILTER = 4;
   private static final int METADATA_TYPE_RECORD_INDEX = 5;
+  private static final int METADATA_TYPE_PARTITION_STATS = 6;

Review Comment:
   Right, no schema change; just code restructuring maintaining the same ID, 
i.e., add `getId` in `MetadataPartitionType`.  You can refactor this part in a 
separate PR.  



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

Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-29 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2083533620

   
   ## CI report:
   
   * 54d5734d6d206c59c29bd8661cdef2f1692cfa49 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23557)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-29 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2083341226

   
   ## CI report:
   
   * 9c26ef9a2ff160ad99668d077dcd71cfca54563f Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23484)
 
   * 54d5734d6d206c59c29bd8661cdef2f1692cfa49 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23557)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-29 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2083321696

   
   ## CI report:
   
   * 9c26ef9a2ff160ad99668d077dcd71cfca54563f Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23484)
 
   * 54d5734d6d206c59c29bd8661cdef2f1692cfa49 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-29 Thread via GitHub


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


##
hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestDataSkippingWithMORColstats.java:
##
@@ -318,6 +318,7 @@ private Map 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:
   this class has some unit tests that deliberately corrupt parquet file by 
replacing a valid file with file of 0 bytes. In that case, keeping partition 
stats enabled throws error. To avoid that, I had disabled. In the latest 
commit, I have disabled only for tests that simulate corrupt parquet file as 
mentioned above, and not the whole test suite.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-29 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/common/util/BaseFileUtils.java:
##
@@ -67,6 +70,50 @@ 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 fileRanges List of column range statistics for each file in a 
partition
+   */
+  public static > HoodieColumnRangeMetadata 
getColumnRangeInPartition(@Nonnull List> 
fileRanges) {
+if (fileRanges.size() == 1) {
+  // Only one parquet file, we can just return that range.

Review Comment:
   done. added more UTs in `TestBaseFileUtils`



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-29 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java:
##
@@ -651,6 +652,36 @@ public static Stream 
createColumnStatsRecords(String partitionName
 });
   }
 
+  public static Stream createPartitionStatsRecords(String 
partitionPath,
+ 
Collection> columnRangeMetadataList,
+ boolean 
isDeleted) {
+return columnRangeMetadataList.stream().map(columnRangeMetadata -> {
+  HoodieKey key = new HoodieKey(getPartitionStatsIndexKey(partitionPath, 
columnRangeMetadata),
+  MetadataPartitionType.PARTITION_STATS.getPartitionPath());
+
+  HoodieMetadataPayload payload = new 
HoodieMetadataPayload(key.getRecordKey(),
+  HoodieMetadataColumnStats.newBuilder()
+  .setFileName(null)
+  .setColumnName(columnRangeMetadata.getColumnName())
+  
.setMinValue(wrapValueIntoAvro(columnRangeMetadata.getMinValue()))
+  
.setMaxValue(wrapValueIntoAvro(columnRangeMetadata.getMaxValue()))
+  .setNullCount(columnRangeMetadata.getNullCount())
+  .setValueCount(columnRangeMetadata.getValueCount())
+  .setTotalSize(columnRangeMetadata.getTotalSize())
+  
.setTotalUncompressedSize(columnRangeMetadata.getTotalUncompressedSize())
+  .setIsDeleted(isDeleted)
+  .build());
+
+  return new HoodieAvroRecord<>(key, payload);
+});
+  }
+
+  public static String getPartitionStatsIndexKey(String partitionPath, 
HoodieColumnRangeMetadata columnRangeMetadata) {

Review Comment:
   this method is covered in new tests added in `TestHoodieMetadataPayload` - 
we test merging two partition stats payloads, with and without delete.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-26 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/common/util/BaseFileUtils.java:
##
@@ -67,6 +70,50 @@ 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 fileRanges List of column range statistics for each file in a 
partition
+   */
+  public static > HoodieColumnRangeMetadata 
getColumnRangeInPartition(@Nonnull List> 
fileRanges) {
+if (fileRanges.size() == 1) {
+  // Only one parquet file, we can just return that range.

Review Comment:
   fixed comments. added tests for null values in mix/max. and also an 
assertion that column names have to be the same across the objects merged.



##
hudi-common/src/main/java/org/apache/hudi/common/util/BaseFileUtils.java:
##
@@ -67,6 +70,50 @@ 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 fileRanges List of column range statistics for each file in a 
partition
+   */
+  public static > HoodieColumnRangeMetadata 
getColumnRangeInPartition(@Nonnull List> 
fileRanges) {

Review Comment:
   rename: fileRanges to fileColumnRanges. done



##
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java:
##
@@ -651,6 +652,36 @@ public static Stream 
createColumnStatsRecords(String partitionName
 });
   }
 
+  public static Stream createPartitionStatsRecords(String 
partitionPath,
+ 
Collection> columnRangeMetadataList,
+ boolean 
isDeleted) {
+return columnRangeMetadataList.stream().map(columnRangeMetadata -> {
+  HoodieKey key = new HoodieKey(getPartitionStatsIndexKey(partitionPath, 
columnRangeMetadata),
+  MetadataPartitionType.PARTITION_STATS.getPartitionPath());
+
+  HoodieMetadataPayload payload = new 
HoodieMetadataPayload(key.getRecordKey(),
+  HoodieMetadataColumnStats.newBuilder()
+  .setFileName(null)
+  .setColumnName(columnRangeMetadata.getColumnName())
+  
.setMinValue(wrapValueIntoAvro(columnRangeMetadata.getMinValue()))
+  
.setMaxValue(wrapValueIntoAvro(columnRangeMetadata.getMaxValue()))
+  .setNullCount(columnRangeMetadata.getNullCount())
+  .setValueCount(columnRangeMetadata.getValueCount())
+  .setTotalSize(columnRangeMetadata.getTotalSize())
+  
.setTotalUncompressedSize(columnRangeMetadata.getTotalUncompressedSize())
+  .setIsDeleted(isDeleted)
+  .build());
+
+  return new HoodieAvroRecord<>(key, payload);
+});
+  }
+
+  public static String getPartitionStatsIndexKey(String partitionPath, 
HoodieColumnRangeMetadata columnRangeMetadata) {

Review Comment:
   need to UT this and other key generation methods. and also tests that assert 
the values in MT storage. We should not be just testing at the query write/read 
level alone



##
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##
@@ -1901,4 +1910,162 @@ private static Path filePath(String basePath, String 
partition, String filename)
   return new Path(basePath, partition + Path.SEPARATOR + filename);
 }
   }
+
+  public static HoodieData 
convertFilesToPartitionStatsRecords(HoodieEngineContext engineContext,

Review Comment:
   Can we do this automatically? As a user, if I am already doing `partitionBy` 
already, then its fair expectation that the index is built automatically and 
partition pruning works without any extra configs



##
hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestDataSkippingWithMORColstats.java:
##
@@ -318,6 +318,7 @@ private Map 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:
   why are we disabling this?



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-25 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2077363189

   
   ## CI report:
   
   * 9c26ef9a2ff160ad99668d077dcd71cfca54563f Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23484)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-25 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2077344871

   
   ## CI report:
   
   * 9c26ef9a2ff160ad99668d077dcd71cfca54563f UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-25 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2076990525

   
   ## CI report:
   
   * 9c26ef9a2ff160ad99668d077dcd71cfca54563f Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23484)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-25 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2076892767

   
   ## CI report:
   
   * dc029010e8e7aac028a660f66bb4fbfb923573aa Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23480)
 
   * 9c26ef9a2ff160ad99668d077dcd71cfca54563f Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23484)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-25 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2076880980

   
   ## CI report:
   
   * dc029010e8e7aac028a660f66bb4fbfb923573aa Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23480)
 
   * 9c26ef9a2ff160ad99668d077dcd71cfca54563f UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-25 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2076653365

   
   ## CI report:
   
   * dc029010e8e7aac028a660f66bb4fbfb923573aa Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23480)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-25 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2076576482

   
   ## CI report:
   
   * 4a813e80ee21fa51babf05714d95211ef2d94870 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23477)
 
   * dc029010e8e7aac028a660f66bb4fbfb923573aa Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23480)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-25 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2076564146

   
   ## CI report:
   
   * 4a813e80ee21fa51babf05714d95211ef2d94870 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23477)
 
   * dc029010e8e7aac028a660f66bb4fbfb923573aa UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-25 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2076551845

   
   ## CI report:
   
   * 4a813e80ee21fa51babf05714d95211ef2d94870 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23477)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-25 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java:
##
@@ -104,13 +104,14 @@ public class HoodieMetadataPayload implements 
HoodieRecordPayloadhttps://issues.apache.org/jira/browse/AVRO-1810
+   * has a bug - https://issues.apache.org/jira/browse/AVRO-1810;>...
*/
-  protected static final int METADATA_TYPE_PARTITION_LIST = 1;
-  protected static final int METADATA_TYPE_FILE_LIST = 2;
-  protected static final int METADATA_TYPE_COLUMN_STATS = 3;
-  protected static final int METADATA_TYPE_BLOOM_FILTER = 4;
+  private static final int METADATA_TYPE_PARTITION_LIST = 1;
+  private static final int METADATA_TYPE_FILE_LIST = 2;
+  private static final int METADATA_TYPE_COLUMN_STATS = 3;
+  private static final int METADATA_TYPE_BLOOM_FILTER = 4;
   private static final int METADATA_TYPE_RECORD_INDEX = 5;
+  private static final int METADATA_TYPE_PARTITION_STATS = 6;

Review Comment:
   Hmm.. yes we could. Just to be clear, you're suggesting no schema change 
right? As in we will still keep the int value in the records, and not enum.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-25 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/common/util/BaseFileUtils.java:
##
@@ -67,6 +70,50 @@ 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 fileRanges List of column range statistics for each file in a 
partition
+   */
+  public static > HoodieColumnRangeMetadata 
getColumnRangeInPartition(@Nonnull List> 
fileRanges) {
+if (fileRanges.size() == 1) {
+  // Only one parquet file, we can just return that range.
+  return fileRanges.get(0);
+}
+// There are multiple files. Compute min(file_mins) and max(file_maxs)
+return fileRanges.stream()
+.sequential()
+.reduce(BaseFileUtils::mergeRanges).get();
+  }
+
+  private static  > HoodieColumnRangeMetadata 
mergeRanges(HoodieColumnRangeMetadata one,

Review Comment:
   Yes in `ParquetUtils`, some portion of this logic is duplicate. I have 
extracted to a common method in `BaseFileUtils` now.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-25 Thread via GitHub


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


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##
@@ -1580,69 +1595,4 @@ public boolean isInitialized() {
   }
 
   protected abstract BaseHoodieWriteClient initializeWriteClient();
-
-  /**
-   * A class which represents a directory and the files and directories inside 
it.
-   * 
-   * A {@code PartitionFileInfo} object saves the name of the partition and 
various properties requires of each file
-   * required for initializing the metadata table. Saving limited properties 
reduces the total memory footprint when
-   * a very large number of files are present in the dataset being initialized.
-   */

Review Comment:
   Needed, but extracted for reuse.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-25 Thread via GitHub


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


##
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala:
##
@@ -351,6 +356,12 @@ case class HoodieFileIndex(spark: SparkSession,
   Option.empty
 } else if (recordKeys.nonEmpty) {
   Option.apply(recordLevelIndex.getCandidateFiles(getAllFiles(), 
recordKeys))
+} else if (partitionStatsIndex.isIndexAvailable && queryFilters.nonEmpty) {
+  val prunedFileNames = getPrunedFileNames(prunedPartitionsAndFileSlices)
+  val shouldReadInMemory = partitionStatsIndex.shouldReadInMemory(this, 
queryReferencedColumns)
+  partitionStatsIndex.loadTransposed(queryReferencedColumns, 
shouldReadInMemory) { transposedColStatsDF =>

Review Comment:
   This logic is now refactored and updated.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-25 Thread via GitHub


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


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##
@@ -224,6 +224,10 @@ private void enablePartitions() {
 if (dataMetaClient.getFunctionalIndexMetadata().isPresent()) {
   this.enabledPartitionTypes.add(FUNCTIONAL_INDEX);
 }
+if ((metadataConfig.isPartitionStatsIndexEnabled() && 
!metadataConfig.getColumnsEnabledForColumnStatsIndex().isEmpty())
+|| 
dataMetaClient.getTableConfig().isMetadataPartitionAvailable(PARTITION_STATS)) {
+  this.enabledPartitionTypes.add(PARTITION_STATS);

Review Comment:
   Done as part of 
https://github.com/apache/hudi/commit/9d689fd68f544cbd2424790770cc2673701be928



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-25 Thread via GitHub


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


##
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java:
##
@@ -557,7 +558,8 @@ public void 
testDowngradeSixToFiveShouldDeleteRecordIndexPartition() throws Exce
 PARTITION_NAME_COLUMN_STATS,
 PARTITION_NAME_BLOOM_FILTERS,
 PARTITION_NAME_RECORD_INDEX,
-PARTITION_NAME_FUNCTIONAL_INDEX_PREFIX
+PARTITION_NAME_FUNCTIONAL_INDEX_PREFIX,

Review Comment:
   I think we will eventually need to upgrade the table version for 1.0. But, 
this PR does not do any incompatible schema changes, so we should be good for 
now. HUDI-7665 to track.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-25 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2076486121

   
   ## CI report:
   
   * ab97c0d984a8336f781cb9a646c0a22421e7ace0 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23283)
 
   * 4a813e80ee21fa51babf05714d95211ef2d94870 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23477)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-25 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2076475604

   
   ## CI report:
   
   * ab97c0d984a8336f781cb9a646c0a22421e7ace0 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23283)
 
   * 4a813e80ee21fa51babf05714d95211ef2d94870 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-17 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java:
##
@@ -651,6 +652,36 @@ public static Stream 
createColumnStatsRecords(String partitionName
 });
   }
 
+  public static Stream createPartitionStatsRecords(String 
partitionPath,
+ 
Collection> columnRangeMetadataList,
+ boolean 
isDeleted) {
+return columnRangeMetadataList.stream().map(columnRangeMetadata -> {
+  HoodieKey key = new HoodieKey(getPartitionStatsIndexKey(partitionPath, 
columnRangeMetadata),
+  MetadataPartitionType.PARTITION_STATS.getPartitionPath());
+
+  HoodieMetadataPayload payload = new 
HoodieMetadataPayload(key.getRecordKey(),
+  HoodieMetadataColumnStats.newBuilder()
+  .setFileName(null)
+  .setColumnName(columnRangeMetadata.getColumnName())
+  
.setMinValue(wrapValueIntoAvro(columnRangeMetadata.getMinValue()))
+  
.setMaxValue(wrapValueIntoAvro(columnRangeMetadata.getMaxValue()))
+  .setNullCount(columnRangeMetadata.getNullCount())
+  .setValueCount(columnRangeMetadata.getValueCount())
+  .setTotalSize(columnRangeMetadata.getTotalSize())
+  
.setTotalUncompressedSize(columnRangeMetadata.getTotalUncompressedSize())
+  .setIsDeleted(isDeleted)
+  .build());
+
+  return new HoodieAvroRecord<>(key, payload);
+});
+  }
+
+  public static String getPartitionStatsIndexKey(String partitionPath, 
HoodieColumnRangeMetadata columnRangeMetadata) {

Review Comment:
   nit: have the column name as the method argument directly to avoid confusion?



##
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala:
##
@@ -351,6 +356,12 @@ case class HoodieFileIndex(spark: SparkSession,
   Option.empty
 } else if (recordKeys.nonEmpty) {
   Option.apply(recordLevelIndex.getCandidateFiles(getAllFiles(), 
recordKeys))
+} else if (partitionStatsIndex.isIndexAvailable && queryFilters.nonEmpty) {
+  val prunedFileNames = getPrunedFileNames(prunedPartitionsAndFileSlices)
+  val shouldReadInMemory = partitionStatsIndex.shouldReadInMemory(this, 
queryReferencedColumns)
+  partitionStatsIndex.loadTransposed(queryReferencedColumns, 
shouldReadInMemory) { transposedColStatsDF =>

Review Comment:
   Could we avoid transposing here as it has overhead and directly leverage the 
index entry for filtering?



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-17 Thread via GitHub


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


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##
@@ -224,6 +224,10 @@ private void enablePartitions() {
 if (dataMetaClient.getFunctionalIndexMetadata().isPresent()) {
   this.enabledPartitionTypes.add(FUNCTIONAL_INDEX);
 }
+if ((metadataConfig.isPartitionStatsIndexEnabled() && 
!metadataConfig.getColumnsEnabledForColumnStatsIndex().isEmpty())
+|| 
dataMetaClient.getTableConfig().isMetadataPartitionAvailable(PARTITION_STATS)) {
+  this.enabledPartitionTypes.add(PARTITION_STATS);

Review Comment:
   Can this part be generalized too, i.e., relying on the defined partition 
type enums to automatically check the corresponding write config and enable the 
index, instead of modifying the method manually?



##
hudi-common/src/main/java/org/apache/hudi/common/util/BaseFileUtils.java:
##
@@ -67,6 +70,50 @@ 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 fileRanges List of column range statistics for each file in a 
partition
+   */
+  public static > HoodieColumnRangeMetadata 
getColumnRangeInPartition(@Nonnull List> 
fileRanges) {
+if (fileRanges.size() == 1) {
+  // Only one parquet file, we can just return that range.
+  return fileRanges.get(0);
+}
+// There are multiple files. Compute min(file_mins) and max(file_maxs)
+return fileRanges.stream()
+.sequential()
+.reduce(BaseFileUtils::mergeRanges).get();
+  }
+
+  private static  > HoodieColumnRangeMetadata 
mergeRanges(HoodieColumnRangeMetadata one,

Review Comment:
   ```suggestion
 private static > HoodieColumnRangeMetadata 
mergeRanges(HoodieColumnRangeMetadata one,
   ```



##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##
@@ -1580,69 +1595,4 @@ public boolean isInitialized() {
   }
 
   protected abstract BaseHoodieWriteClient initializeWriteClient();
-
-  /**
-   * A class which represents a directory and the files and directories inside 
it.
-   * 
-   * A {@code PartitionFileInfo} object saves the name of the partition and 
various properties requires of each file
-   * required for initializing the metadata table. Saving limited properties 
reduces the total memory footprint when
-   * a very large number of files are present in the dataset being initialized.
-   */

Review Comment:
   So this is no longer needed?



##
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java:
##
@@ -557,7 +558,8 @@ public void 
testDowngradeSixToFiveShouldDeleteRecordIndexPartition() throws Exce
 PARTITION_NAME_COLUMN_STATS,
 PARTITION_NAME_BLOOM_FILTERS,
 PARTITION_NAME_RECORD_INDEX,
-PARTITION_NAME_FUNCTIONAL_INDEX_PREFIX
+PARTITION_NAME_FUNCTIONAL_INDEX_PREFIX,

Review Comment:
   Should this list be generated from the production code, i.e., list of 
supported MDT partitions?  Also, do we need to upgrade the table version?  I 
think master branch is still on table version 6, the same as 0.14.0 release.



##
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java:
##
@@ -104,13 +104,14 @@ public class HoodieMetadataPayload implements 
HoodieRecordPayloadhttps://issues.apache.org/jira/browse/AVRO-1810
+   * has a bug - https://issues.apache.org/jira/browse/AVRO-1810;>...
*/
-  protected static final int METADATA_TYPE_PARTITION_LIST = 1;
-  protected static final int METADATA_TYPE_FILE_LIST = 2;
-  protected static final int METADATA_TYPE_COLUMN_STATS = 3;
-  protected static final int METADATA_TYPE_BLOOM_FILTER = 4;
+  private static final int METADATA_TYPE_PARTITION_LIST = 1;
+  private static final int METADATA_TYPE_FILE_LIST = 2;
+  private static final int METADATA_TYPE_COLUMN_STATS = 3;
+  private static final int METADATA_TYPE_BLOOM_FILTER = 4;
   private static final int METADATA_TYPE_RECORD_INDEX = 5;
+  private static final int METADATA_TYPE_PARTITION_STATS = 6;

Review Comment:
   Maybe we should add enum and guarantee the ordering, and automatically 
assign the type ID?



##
hudi-common/src/main/java/org/apache/hudi/common/util/BaseFileUtils.java:
##
@@ -67,6 +70,50 @@ 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 fileRanges List of column range statistics for each file in a 
partition
+   */
+  public 

Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-16 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2059026878

   
   ## CI report:
   
   * ab97c0d984a8336f781cb9a646c0a22421e7ace0 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23283)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-16 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2058836550

   
   ## CI report:
   
   * 77a9f8ef47d7a5897ee9863665d3490456415f94 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23191)
 
   * ab97c0d984a8336f781cb9a646c0a22421e7ace0 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23283)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-16 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2058824936

   
   ## CI report:
   
   * 77a9f8ef47d7a5897ee9863665d3490456415f94 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23191)
 
   * ab97c0d984a8336f781cb9a646c0a22421e7ace0 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-11 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2049628291

   
   ## CI report:
   
   * 77a9f8ef47d7a5897ee9863665d3490456415f94 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23191)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-11 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2049527246

   
   ## CI report:
   
   * 53e141b4edb11cfc803af29bf90a8583aeb991f3 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23175)
 
   * 77a9f8ef47d7a5897ee9863665d3490456415f94 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23191)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-11 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2049516138

   
   ## CI report:
   
   * 53e141b4edb11cfc803af29bf90a8583aeb991f3 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23175)
 
   * 77a9f8ef47d7a5897ee9863665d3490456415f94 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-10 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2048292729

   
   ## CI report:
   
   * 53e141b4edb11cfc803af29bf90a8583aeb991f3 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23175)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-10 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2048137978

   
   ## CI report:
   
   * 2f0f7115ae2321ad0de852643ecda4fe2fd0ce42 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23105)
 
   * 53e141b4edb11cfc803af29bf90a8583aeb991f3 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23175)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-10 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2048125079

   
   ## CI report:
   
   * 2f0f7115ae2321ad0de852643ecda4fe2fd0ce42 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23105)
 
   * 53e141b4edb11cfc803af29bf90a8583aeb991f3 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-10 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/common/util/BaseFileUtils.java:
##
@@ -64,6 +67,51 @@ 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 fileRanges List of column range statistics for each file in a 
partition
+   */
+  public static > HoodieColumnRangeMetadata 
getColumnRangeInPartition(@Nonnull List> 
fileRanges) {
+if (fileRanges.size() == 1) {
+  // Only one parquet file, we can just return that range.
+  return fileRanges.get(0);
+}
+// There are multiple files. Compute min(file_mins) and max(file_maxs)
+return fileRanges.stream()
+.sequential()
+.reduce(BaseFileUtils::mergeRanges).get();
+  }
+
+  private static  > HoodieColumnRangeMetadata 
mergeRanges(HoodieColumnRangeMetadata one,
+   
  HoodieColumnRangeMetadata another) {
+final T minValue;
+final T maxValue;
+if (one.getMinValue() != null && another.getMinValue() != null) {
+  minValue = 
one.getMinValue().toString().compareTo(another.getMinValue().toString()) < 0 ? 
one.getMinValue() : another.getMinValue();

Review Comment:
   No, lexicographical ordering isn't correct for all types. Since the type 
already extends Comparable, i've removed `toString`.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-10 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/common/util/BaseFileUtils.java:
##
@@ -67,6 +70,50 @@ 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 fileRanges List of column range statistics for each file in a 
partition
+   */
+  public static > HoodieColumnRangeMetadata 
getColumnRangeInPartition(@Nonnull List> 
fileRanges) {
+if (fileRanges.size() == 1) {
+  // Only one parquet file, we can just return that range.
+  return fileRanges.get(0);
+}
+// There are multiple files. Compute min(file_mins) and max(file_maxs)
+return fileRanges.stream()
+.sequential()
+.reduce(BaseFileUtils::mergeRanges).get();

Review Comment:
   fixed



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-10 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/common/util/BaseFileUtils.java:
##
@@ -67,6 +70,50 @@ 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 fileRanges List of column range statistics for each file in a 
partition
+   */
+  public static > HoodieColumnRangeMetadata 
getColumnRangeInPartition(@Nonnull List> 
fileRanges) {
+if (fileRanges.size() == 1) {

Review Comment:
   Yes, fixed the logic and added ut in `testGetColumnRangeInPartition`



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-10 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##
@@ -1915,4 +1924,162 @@ private static Path filePath(String basePath, String 
partition, String filename)
   return new Path(basePath, partition + StoragePath.SEPARATOR + filename);
 }
   }
+
+  public static HoodieData 
convertFilesToPartitionStatsRecords(HoodieEngineContext engineContext,
+ 
List partitionInfoList,
+ 
MetadataRecordsGenerationParams recordsGenerationParams) {
+// Find the columns to index
+HoodieTableMetaClient dataTableMetaClient = 
recordsGenerationParams.getDataMetaClient();
+final List columnsToIndex = getColumnsToIndex(
+recordsGenerationParams,
+Lazy.lazily(() -> tryResolveSchemaForTable(dataTableMetaClient)));
+if (columnsToIndex.isEmpty()) {
+  return engineContext.emptyHoodieData();
+}
+LOG.debug(String.format("Indexing %d columns for partition stats index", 
columnsToIndex.size()));
+// Create records for MDT
+int parallelism = Math.max(Math.min(partitionInfoList.size(), 
recordsGenerationParams.getPartitionStatsIndexParallelism()), 1);
+return engineContext.parallelize(partitionInfoList, 
parallelism).flatMap(partitionFiles -> {
+  final String partitionName = partitionFiles.getRelativePath();
+  Stream> 
partitionStatsRangeMetadata = 
partitionFiles.getFileNameToSizeMap().keySet().stream()
+  .map(fileName -> getFileStatsRangeMetadata(partitionName, 
partitionName + "/" + fileName, dataTableMetaClient, columnsToIndex, false))
+  .map(BaseFileUtils::getColumnRangeInPartition);

Review Comment:
   This is fixed now and added a test `testConvertFilesToPartitionStatsRecords`



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-10 Thread via GitHub


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


##
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionStatsIndex.scala:
##
@@ -0,0 +1,188 @@
+/*
+ * 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.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 {
+
+  val sqlTempTable = "hudi_tbl"
+
+  @ParameterizedTest
+  @EnumSource(classOf[HoodieTableType])
+  def testIndexInitialization(tableType: HoodieTableType): Unit = {
+val hudiOpts = commonOpts + (DataSourceWriteOptions.TABLE_TYPE.key -> 
tableType.name())
+doWriteAndValidateDataAndPartitionStats(
+  hudiOpts,
+  operation = DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
+  saveMode = SaveMode.Overwrite)
+  }
+
+  @ParameterizedTest
+  @EnumSource(classOf[HoodieTableType])
+  def testIndexWithUpsert(tableType: HoodieTableType): Unit = {
+val hudiOpts = commonOpts + (DataSourceWriteOptions.TABLE_TYPE.key -> 
tableType.name())
+doWriteAndValidateDataAndPartitionStats(hudiOpts,
+  operation = DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
+  saveMode = SaveMode.Overwrite)
+doWriteAndValidateDataAndPartitionStats(
+  hudiOpts,
+  operation = DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL,
+  saveMode = SaveMode.Append)
+  }
+
+  @ParameterizedTest
+  @EnumSource(classOf[HoodieTableType])
+  def testIndexWithUpsertNonPartitioned(tableType: HoodieTableType): Unit = {
+val hudiOpts = commonOpts - PARTITIONPATH_FIELD.key + 
(DataSourceWriteOptions.TABLE_TYPE.key -> tableType.name())
+doWriteAndValidateDataAndPartitionStats(
+  hudiOpts,
+  operation = DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
+  saveMode = SaveMode.Overwrite)
+doWriteAndValidateDataAndPartitionStats(
+  hudiOpts,
+  operation = DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL,
+  saveMode = SaveMode.Append)
+  }
+
+  @ParameterizedTest
+  @EnumSource(classOf[HoodieTableType])
+  def testIndexUpsertAndRollback(tableType: HoodieTableType): Unit = {
+val hudiOpts = commonOpts + (DataSourceWriteOptions.TABLE_TYPE.key -> 
tableType.name())
+doWriteAndValidateDataAndPartitionStats(
+  hudiOpts,
+  operation = DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
+  saveMode = SaveMode.Overwrite)
+doWriteAndValidateDataAndPartitionStats(
+  hudiOpts,
+  operation = DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL,
+  saveMode = SaveMode.Append)
+doWriteAndValidateDataAndPartitionStats(
+  hudiOpts,
+  operation = DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL,
+  saveMode = SaveMode.Append)
+rollbackLastInstant(hudiOpts)
+validateDataAndPartitionStats()
+  }
+
+  @Test
+  def testPartitionStatsIndexWithSQL(): Unit = {
+var hudiOpts = commonOpts
+hudiOpts = hudiOpts + (
+  DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "true")
+
+doWriteAndValidateDataAndPartitionStats(
+  hudiOpts,
+  operation = DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
+  saveMode = SaveMode.Overwrite,
+  validate = false)
+

Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-10 Thread via GitHub


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


##
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/PartitionStatsIndexSupport.scala:
##
@@ -0,0 +1,71 @@
+/*
+ * 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 org.apache.hudi.HoodieConversionUtils.toScalaOption
+import org.apache.hudi.avro.model.{HoodieMetadataColumnStats, 
HoodieMetadataRecord}
+import org.apache.hudi.client.common.HoodieSparkEngineContext
+import org.apache.hudi.common.config.HoodieMetadataConfig
+import org.apache.hudi.common.data.HoodieData
+import org.apache.hudi.common.model.HoodieRecord
+import org.apache.hudi.common.table.HoodieTableMetaClient
+import org.apache.hudi.common.util.ValidationUtils.checkState
+import org.apache.hudi.common.util.hash.ColumnIndexID
+import org.apache.hudi.metadata.{HoodieMetadataPayload, HoodieTableMetadata, 
HoodieTableMetadataUtil}
+import org.apache.hudi.util.JFunction
+import org.apache.spark.api.java.JavaSparkContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.types.StructType
+
+import scala.collection.JavaConverters._
+
+class PartitionStatsIndexSupport(spark: SparkSession,
+ tableSchema: StructType,
+ @transient metadataConfig: 
HoodieMetadataConfig,
+ @transient metaClient: HoodieTableMetaClient,
+ allowCaching: Boolean = false)
+  extends ColumnStatsIndexSupport(spark, tableSchema, metadataConfig, 
metaClient, allowCaching) {
+
+  @transient private lazy val engineCtx = new HoodieSparkEngineContext(new 
JavaSparkContext(spark.sparkContext))
+  @transient private lazy val metadataTable: HoodieTableMetadata =
+HoodieTableMetadata.create(engineCtx, metadataConfig, 
metaClient.getBasePathV2.toString)
+
+  override def isIndexAvailable: Boolean = {
+checkState(metadataConfig.enabled, "Metadata Table support has to be 
enabled")

Review Comment:
   not necessary as we are checking whether table config contains partition 
stats in the list of of available indexes.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-10 Thread via GitHub


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


##
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/PartitionStatsIndexSupport.scala:
##
@@ -0,0 +1,71 @@
+/*
+ * 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 org.apache.hudi.HoodieConversionUtils.toScalaOption
+import org.apache.hudi.avro.model.{HoodieMetadataColumnStats, 
HoodieMetadataRecord}
+import org.apache.hudi.client.common.HoodieSparkEngineContext
+import org.apache.hudi.common.config.HoodieMetadataConfig
+import org.apache.hudi.common.data.HoodieData
+import org.apache.hudi.common.model.HoodieRecord
+import org.apache.hudi.common.table.HoodieTableMetaClient
+import org.apache.hudi.common.util.ValidationUtils.checkState
+import org.apache.hudi.common.util.hash.ColumnIndexID
+import org.apache.hudi.metadata.{HoodieMetadataPayload, HoodieTableMetadata, 
HoodieTableMetadataUtil}
+import org.apache.hudi.util.JFunction
+import org.apache.spark.api.java.JavaSparkContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.types.StructType
+
+import scala.collection.JavaConverters._
+
+class PartitionStatsIndexSupport(spark: SparkSession,
+ tableSchema: StructType,
+ @transient metadataConfig: 
HoodieMetadataConfig,
+ @transient metaClient: HoodieTableMetaClient,
+ allowCaching: Boolean = false)
+  extends ColumnStatsIndexSupport(spark, tableSchema, metadataConfig, 
metaClient, allowCaching) {
+
+  @transient private lazy val engineCtx = new HoodieSparkEngineContext(new 
JavaSparkContext(spark.sparkContext))
+  @transient private lazy val metadataTable: HoodieTableMetadata =

Review Comment:
   Refactored as per suggestion



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-10 Thread via GitHub


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


##
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala:
##
@@ -346,6 +352,12 @@ case class HoodieFileIndex(spark: SparkSession,
   Option.empty
 } else if (recordKeys.nonEmpty) {
   Option.apply(recordLevelIndex.getCandidateFiles(getAllFiles(), 
recordKeys))
+} else if (recordKeys.nonEmpty && partitionStatsIndex.isIndexAvailable && 
!queryFilters.isEmpty) {

Review Comment:
   not necessary.. removed.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-10 Thread via GitHub


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


##
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionStatsIndex.scala:
##
@@ -0,0 +1,188 @@
+/*
+ * 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.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 {
+
+  val sqlTempTable = "hudi_tbl"
+
+  @ParameterizedTest
+  @EnumSource(classOf[HoodieTableType])
+  def testIndexInitialization(tableType: HoodieTableType): Unit = {
+val hudiOpts = commonOpts + (DataSourceWriteOptions.TABLE_TYPE.key -> 
tableType.name())
+doWriteAndValidateDataAndPartitionStats(
+  hudiOpts,
+  operation = DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
+  saveMode = SaveMode.Overwrite)
+  }
+
+  @ParameterizedTest
+  @EnumSource(classOf[HoodieTableType])
+  def testIndexWithUpsert(tableType: HoodieTableType): Unit = {
+val hudiOpts = commonOpts + (DataSourceWriteOptions.TABLE_TYPE.key -> 
tableType.name())
+doWriteAndValidateDataAndPartitionStats(hudiOpts,
+  operation = DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
+  saveMode = SaveMode.Overwrite)
+doWriteAndValidateDataAndPartitionStats(
+  hudiOpts,
+  operation = DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL,
+  saveMode = SaveMode.Append)
+  }
+
+  @ParameterizedTest
+  @EnumSource(classOf[HoodieTableType])
+  def testIndexWithUpsertNonPartitioned(tableType: HoodieTableType): Unit = {
+val hudiOpts = commonOpts - PARTITIONPATH_FIELD.key + 
(DataSourceWriteOptions.TABLE_TYPE.key -> tableType.name())
+doWriteAndValidateDataAndPartitionStats(
+  hudiOpts,
+  operation = DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
+  saveMode = SaveMode.Overwrite)
+doWriteAndValidateDataAndPartitionStats(
+  hudiOpts,
+  operation = DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL,
+  saveMode = SaveMode.Append)
+  }
+
+  @ParameterizedTest
+  @EnumSource(classOf[HoodieTableType])
+  def testIndexUpsertAndRollback(tableType: HoodieTableType): Unit = {
+val hudiOpts = commonOpts + (DataSourceWriteOptions.TABLE_TYPE.key -> 
tableType.name())
+doWriteAndValidateDataAndPartitionStats(
+  hudiOpts,
+  operation = DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
+  saveMode = SaveMode.Overwrite)
+doWriteAndValidateDataAndPartitionStats(
+  hudiOpts,
+  operation = DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL,
+  saveMode = SaveMode.Append)
+doWriteAndValidateDataAndPartitionStats(
+  hudiOpts,
+  operation = DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL,
+  saveMode = SaveMode.Append)
+rollbackLastInstant(hudiOpts)
+validateDataAndPartitionStats()
+  }
+
+  @Test
+  def testPartitionStatsIndexWithSQL(): Unit = {
+var hudiOpts = commonOpts
+hudiOpts = hudiOpts + (
+  DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "true")
+
+doWriteAndValidateDataAndPartitionStats(
+  hudiOpts,
+  operation = DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
+  saveMode = SaveMode.Overwrite,
+  validate = false)
+

Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-10 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##
@@ -1901,4 +1910,162 @@ private static Path filePath(String basePath, String 
partition, String filename)
   return new Path(basePath, partition + Path.SEPARATOR + filename);
 }
   }
+
+  public static HoodieData 
convertFilesToPartitionStatsRecords(HoodieEngineContext engineContext,

Review Comment:
   We can avoid this extra step though. 



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-10 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##
@@ -1901,4 +1910,162 @@ private static Path filePath(String basePath, String 
partition, String filename)
   return new Path(basePath, partition + Path.SEPARATOR + filename);
 }
   }
+
+  public static HoodieData 
convertFilesToPartitionStatsRecords(HoodieEngineContext engineContext,

Review Comment:
   no, we need to specify the partition field in target columns config.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-10 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##
@@ -1901,4 +1910,162 @@ private static Path filePath(String basePath, String 
partition, String filename)
   return new Path(basePath, partition + Path.SEPARATOR + filename);
 }
   }
+
+  public static HoodieData 
convertFilesToPartitionStatsRecords(HoodieEngineContext engineContext,
+ 
List partitionInfoList,
+ 
MetadataRecordsGenerationParams recordsGenerationParams) {

Review Comment:
   Removed usage of `MetadataRecordsGenerationParams`. Eventually, we should 
just get rid of the pojo.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-10 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java:
##
@@ -641,6 +642,36 @@ public static Stream 
createColumnStatsRecords(String partitionName
 });
   }
 
+  public static Stream createPartitionStatsRecords(String 
partitionName,

Review Comment:
   Changed to partitionPath



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-10 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/common/util/BaseFileUtils.java:
##
@@ -64,6 +67,51 @@ 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 fileRanges List of column range statistics for each file in a 
partition
+   */
+  public static > HoodieColumnRangeMetadata 
getColumnRangeInPartition(@Nonnull List> 
fileRanges) {
+if (fileRanges.size() == 1) {
+  // Only one parquet file, we can just return that range.
+  return fileRanges.get(0);
+}
+// There are multiple files. Compute min(file_mins) and max(file_maxs)
+return fileRanges.stream()
+.sequential()
+.reduce(BaseFileUtils::mergeRanges).get();
+  }
+
+  private static  > HoodieColumnRangeMetadata 
mergeRanges(HoodieColumnRangeMetadata one,
+   
  HoodieColumnRangeMetadata another) {
+final T minValue;
+final T maxValue;
+if (one.getMinValue() != null && another.getMinValue() != null) {
+  minValue = 
one.getMinValue().toString().compareTo(another.getMinValue().toString()) < 0 ? 
one.getMinValue() : another.getMinValue();
+} else if (one.getMinValue() == null) {
+  minValue = another.getMinValue();
+} else {
+  minValue = one.getMinValue();
+}
+
+if (one.getMaxValue() != null && another.getMaxValue() != null) {
+  maxValue = 
one.getMaxValue().toString().compareTo(another.getMaxValue().toString()) < 0 ? 
another.getMaxValue() : one.getMaxValue();
+} else if (one.getMaxValue() == null) {
+  maxValue = another.getMaxValue();
+} else {
+  maxValue = one.getMaxValue();
+}
+
+return HoodieColumnRangeMetadata.create(
+one.getFilePath(),

Review Comment:
   done



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-04 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2038006439

   
   ## CI report:
   
   * 2f0f7115ae2321ad0de852643ecda4fe2fd0ce42 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23105)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-04 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2037783636

   
   ## CI report:
   
   * 1430466484c43ee011c1a2c27ea1f068fc1b750b Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22888)
 
   * 2f0f7115ae2321ad0de852643ecda4fe2fd0ce42 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=23105)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-04-04 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-2037766469

   
   ## CI report:
   
   * 1430466484c43ee011c1a2c27ea1f068fc1b750b Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22888)
 
   * 2f0f7115ae2321ad0de852643ecda4fe2fd0ce42 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-03-15 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-1999303872

   
   ## CI report:
   
   * 1430466484c43ee011c1a2c27ea1f068fc1b750b Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22888)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-03-15 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-1999290027

   
   ## CI report:
   
   * 1430466484c43ee011c1a2c27ea1f068fc1b750b UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-03-13 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-1994585065

   
   ## CI report:
   
   * 1430466484c43ee011c1a2c27ea1f068fc1b750b Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22888)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-03-13 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-1994463864

   
   ## CI report:
   
   * 8510ace1d4ef0621057b292f1e09e8bfb0c071a5 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22876)
 
   * 1430466484c43ee011c1a2c27ea1f068fc1b750b Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22888)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-03-13 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-1994445979

   
   ## CI report:
   
   * 8510ace1d4ef0621057b292f1e09e8bfb0c071a5 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22876)
 
   * 1430466484c43ee011c1a2c27ea1f068fc1b750b UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-03-11 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-1987885457

   
   ## CI report:
   
   * 8510ace1d4ef0621057b292f1e09e8bfb0c071a5 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22876)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-03-11 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-1987789140

   
   ## CI report:
   
   * 3ce4c6f5bcf61d177d72d6a135773c3707ce6329 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22777)
 
   * 8510ace1d4ef0621057b292f1e09e8bfb0c071a5 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22876)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-03-11 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-1987779742

   
   ## CI report:
   
   * 3ce4c6f5bcf61d177d72d6a135773c3707ce6329 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22777)
 
   * 8510ace1d4ef0621057b292f1e09e8bfb0c071a5 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-03-11 Thread via GitHub


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


##
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala:
##
@@ -346,6 +352,12 @@ case class HoodieFileIndex(spark: SparkSession,
   Option.empty
 } else if (recordKeys.nonEmpty) {
   Option.apply(recordLevelIndex.getCandidateFiles(getAllFiles(), 
recordKeys))
+} else if (recordKeys.nonEmpty && partitionStatsIndex.isIndexAvailable && 
!queryFilters.isEmpty) {
+  val prunedFileNames = getPrunedFileNames(prunedPartitionsAndFileSlices)
+  val shouldReadInMemory = partitionStatsIndex.shouldReadInMemory(this, 
queryReferencedColumns)

Review Comment:
   First thing I want to clarify is that partition stats are collected only 
when column stats is enabled, and for only those columns for which column stats 
is enabled.
   
   Write path: When user will do `.partitionBy("a,b,c")`, then the logic is 
similar to column stats. We use the commit metadata  and convert that to 
partition stats. This happens in 
`HoodieTableMetadataUtil.convertMetadataToPartitionStatsRecords`. The 
difference from column stats is that the stats are aggregated by partition 
value in `BaseFileUtils.getColumnRangeInPartition`.
   
   Read path: `queryReferencedColumns` here contain data filters. Partition 
pruning based on partition filters has already happened one level above.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-03-04 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-1977270151

   
   ## CI report:
   
   * 3ce4c6f5bcf61d177d72d6a135773c3707ce6329 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22777)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-03-04 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-1977183356

   
   ## CI report:
   
   * 58c72d0694d4c9fb1b7cf96bdbc57bec2aa7e9f9 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22775)
 
   * 3ce4c6f5bcf61d177d72d6a135773c3707ce6329 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22777)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-03-04 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-1977169944

   
   ## CI report:
   
   * 58c72d0694d4c9fb1b7cf96bdbc57bec2aa7e9f9 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22775)
 
   * 3ce4c6f5bcf61d177d72d6a135773c3707ce6329 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-03-04 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-1977064792

   
   ## CI report:
   
   * 58c72d0694d4c9fb1b7cf96bdbc57bec2aa7e9f9 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22775)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-03-04 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-1976833855

   
   ## CI report:
   
   * 933b215be746a8c96d144fc870c05855eadc4323 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22640)
 
   * 58c72d0694d4c9fb1b7cf96bdbc57bec2aa7e9f9 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22775)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-03-04 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-1976816878

   
   ## CI report:
   
   * 933b215be746a8c96d144fc870c05855eadc4323 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22640)
 
   * 58c72d0694d4c9fb1b7cf96bdbc57bec2aa7e9f9 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-02-26 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-1964748613

   
   ## CI report:
   
   * 933b215be746a8c96d144fc870c05855eadc4323 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22640)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-02-26 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-1964629101

   
   ## CI report:
   
   * a592ad1361583635a55ead7b634d2b20a92c239f Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22586)
 
   * 933b215be746a8c96d144fc870c05855eadc4323 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22640)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-02-26 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-1964608480

   
   ## CI report:
   
   * a592ad1361583635a55ead7b634d2b20a92c239f Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22586)
 
   * 933b215be746a8c96d144fc870c05855eadc4323 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-02-23 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-1961878815

   
   ## CI report:
   
   * a592ad1361583635a55ead7b634d2b20a92c239f Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22586)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-02-23 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-1961869252

   
   ## CI report:
   
   * d0faf8850bf513fb1d610831b3459680c244 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22552)
 
   * a592ad1361583635a55ead7b634d2b20a92c239f Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22586)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-02-23 Thread via GitHub


hudi-bot commented on PR #10352:
URL: https://github.com/apache/hudi/pull/10352#issuecomment-1961815729

   
   ## CI report:
   
   * d0faf8850bf513fb1d610831b3459680c244 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22552)
 
   * a592ad1361583635a55ead7b634d2b20a92c239f UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-02-22 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##
@@ -1915,4 +1924,162 @@ private static Path filePath(String basePath, String 
partition, String filename)
   return new Path(basePath, partition + StoragePath.SEPARATOR + filename);
 }
   }
+
+  public static HoodieData 
convertFilesToPartitionStatsRecords(HoodieEngineContext engineContext,
+ 
List partitionInfoList,
+ 
MetadataRecordsGenerationParams recordsGenerationParams) {
+// Find the columns to index
+HoodieTableMetaClient dataTableMetaClient = 
recordsGenerationParams.getDataMetaClient();
+final List columnsToIndex = getColumnsToIndex(
+recordsGenerationParams,
+Lazy.lazily(() -> tryResolveSchemaForTable(dataTableMetaClient)));
+if (columnsToIndex.isEmpty()) {
+  return engineContext.emptyHoodieData();
+}
+LOG.debug(String.format("Indexing %d columns for partition stats index", 
columnsToIndex.size()));
+// Create records for MDT
+int parallelism = Math.max(Math.min(partitionInfoList.size(), 
recordsGenerationParams.getPartitionStatsIndexParallelism()), 1);
+return engineContext.parallelize(partitionInfoList, 
parallelism).flatMap(partitionFiles -> {
+  final String partitionName = partitionFiles.getRelativePath();
+  Stream> 
partitionStatsRangeMetadata = 
partitionFiles.getFileNameToSizeMap().keySet().stream()
+  .map(fileName -> getFileStatsRangeMetadata(partitionName, 
partitionName + "/" + fileName, dataTableMetaClient, columnsToIndex, false))

Review Comment:
   This is called one time at the time of intialization of partiiton stats. 
Nevertheless, we can avoid extra work with minimal branching.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-02-22 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##
@@ -1915,4 +1924,162 @@ private static Path filePath(String basePath, String 
partition, String filename)
   return new Path(basePath, partition + StoragePath.SEPARATOR + filename);
 }
   }
+
+  public static HoodieData 
convertFilesToPartitionStatsRecords(HoodieEngineContext engineContext,
+ 
List partitionInfoList,
+ 
MetadataRecordsGenerationParams recordsGenerationParams) {
+// Find the columns to index
+HoodieTableMetaClient dataTableMetaClient = 
recordsGenerationParams.getDataMetaClient();
+final List columnsToIndex = getColumnsToIndex(
+recordsGenerationParams,
+Lazy.lazily(() -> tryResolveSchemaForTable(dataTableMetaClient)));
+if (columnsToIndex.isEmpty()) {
+  return engineContext.emptyHoodieData();
+}
+LOG.debug(String.format("Indexing %d columns for partition stats index", 
columnsToIndex.size()));
+// Create records for MDT
+int parallelism = Math.max(Math.min(partitionInfoList.size(), 
recordsGenerationParams.getPartitionStatsIndexParallelism()), 1);
+return engineContext.parallelize(partitionInfoList, 
parallelism).flatMap(partitionFiles -> {
+  final String partitionName = partitionFiles.getRelativePath();
+  Stream> 
partitionStatsRangeMetadata = 
partitionFiles.getFileNameToSizeMap().keySet().stream()
+  .map(fileName -> getFileStatsRangeMetadata(partitionName, 
partitionName + "/" + fileName, dataTableMetaClient, columnsToIndex, false))

Review Comment:
   This is called one time at the time of intialization of partiiton stats. I 
thought if users enabled partition stats only (no column stats) then we need to 
make this call. Nevertheless, we can avoid extra work in most cases with 
minimal branching.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-02-22 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/common/util/BaseFileUtils.java:
##
@@ -67,6 +70,50 @@ 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 fileRanges List of column range statistics for each file in a 
partition
+   */
+  public static > HoodieColumnRangeMetadata 
getColumnRangeInPartition(@Nonnull List> 
fileRanges) {
+if (fileRanges.size() == 1) {
+  // Only one parquet file, we can just return that range.
+  return fileRanges.get(0);
+}
+// There are multiple files. Compute min(file_mins) and max(file_maxs)
+return fileRanges.stream()
+.sequential()
+.reduce(BaseFileUtils::mergeRanges).get();

Review Comment:
   yea, apologies, current logic is incorrect. i'm going to fix it.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-02-22 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##
@@ -1915,4 +1924,162 @@ private static Path filePath(String basePath, String 
partition, String filename)
   return new Path(basePath, partition + StoragePath.SEPARATOR + filename);
 }
   }
+
+  public static HoodieData 
convertFilesToPartitionStatsRecords(HoodieEngineContext engineContext,
+ 
List partitionInfoList,
+ 
MetadataRecordsGenerationParams recordsGenerationParams) {
+// Find the columns to index
+HoodieTableMetaClient dataTableMetaClient = 
recordsGenerationParams.getDataMetaClient();
+final List columnsToIndex = getColumnsToIndex(
+recordsGenerationParams,
+Lazy.lazily(() -> tryResolveSchemaForTable(dataTableMetaClient)));
+if (columnsToIndex.isEmpty()) {
+  return engineContext.emptyHoodieData();
+}
+LOG.debug(String.format("Indexing %d columns for partition stats index", 
columnsToIndex.size()));
+// Create records for MDT
+int parallelism = Math.max(Math.min(partitionInfoList.size(), 
recordsGenerationParams.getPartitionStatsIndexParallelism()), 1);
+return engineContext.parallelize(partitionInfoList, 
parallelism).flatMap(partitionFiles -> {
+  final String partitionName = partitionFiles.getRelativePath();
+  Stream> 
partitionStatsRangeMetadata = 
partitionFiles.getFileNameToSizeMap().keySet().stream()
+  .map(fileName -> getFileStatsRangeMetadata(partitionName, 
partitionName + "/" + fileName, dataTableMetaClient, columnsToIndex, false))
+  .map(BaseFileUtils::getColumnRangeInPartition);

Review Comment:
   You're right. This is incorrect. I am going to fix it.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [HUDI-7144] Build storage partition stats index and use it for data skipping [hudi]

2024-02-22 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##
@@ -1901,4 +1910,162 @@ private static Path filePath(String basePath, String 
partition, String filename)
   return new Path(basePath, partition + Path.SEPARATOR + filename);
 }
   }
+
+  public static HoodieData 
convertFilesToPartitionStatsRecords(HoodieEngineContext engineContext,
+ 
List partitionInfoList,
+ 
MetadataRecordsGenerationParams recordsGenerationParams) {
+// Find the columns to index
+HoodieTableMetaClient dataTableMetaClient = 
recordsGenerationParams.getDataMetaClient();
+final List columnsToIndex = getColumnsToIndex(
+recordsGenerationParams,
+Lazy.lazily(() -> tryResolveSchemaForTable(dataTableMetaClient)));
+if (columnsToIndex.isEmpty()) {
+  return engineContext.emptyHoodieData();
+}
+LOG.debug(String.format("Indexing %d columns for partition stats index", 
columnsToIndex.size()));
+// Create records for MDT
+int parallelism = Math.max(Math.min(partitionInfoList.size(), 
recordsGenerationParams.getPartitionStatsIndexParallelism()), 1);
+return engineContext.parallelize(partitionInfoList, 
parallelism).flatMap(partitionFiles -> {
+  final String partitionName = partitionFiles.getRelativePath();
+  Stream> 
partitionStatsRangeMetadata = 
partitionFiles.getFileNameToSizeMap().keySet().stream()
+  .map(fileName -> getFileStatsRangeMetadata(partitionName, 
partitionName + "/" + fileName, dataTableMetaClient, columnsToIndex, false))
+  .map(BaseFileUtils::getColumnRangeInPartition);
+  return HoodieMetadataPayload.createPartitionStatsRecords(partitionName, 
partitionStatsRangeMetadata.collect(toList()), false).iterator();
+});
+  }
+
+  private static List> 
getFileStatsRangeMetadata(String partitionPath,
+   
String filePath,
+   
HoodieTableMetaClient datasetMetaClient,
+   
List columnsToIndex,
+   
boolean isDeleted) {
+String filePartitionPath = filePath.startsWith("/") ? 
filePath.substring(1) : filePath;

Review Comment:
   This `filePartitionPath` is a combination of partition name + "/" + file 
name. For non-partitioned tables, we can have "/" in the beginning.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



  1   2   >