codope commented on code in PR #8452: URL: https://github.com/apache/hudi/pull/8452#discussion_r1270853762
########## hudi-common/src/main/java/org/apache/hudi/metadata/AbstractHoodieTableMetadata.java: ########## @@ -0,0 +1,96 @@ +/* + * 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.metadata; + +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.util.PartitionPathEncodeUtils; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.expression.ArrayData; +import org.apache.hudi.hadoop.CachingPath; +import org.apache.hudi.hadoop.SerializablePath; +import org.apache.hudi.internal.schema.Type; +import org.apache.hudi.internal.schema.Types; + +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +public abstract class AbstractHoodieTableMetadata implements HoodieTableMetadata { + + protected transient HoodieEngineContext engineContext; + + protected final SerializableConfiguration hadoopConf; + protected final SerializablePath dataBasePath; + + // TODO get this from HoodieConfig + protected final boolean caseSensitive = false; + + public AbstractHoodieTableMetadata(HoodieEngineContext engineContext, SerializableConfiguration conf, String dataBasePath) { + this.engineContext = engineContext; + this.hadoopConf = conf; + this.dataBasePath = new SerializablePath(new CachingPath(dataBasePath)); + } + + protected static int getPathPartitionLevel(Types.RecordType partitionFields, String path) { + if (StringUtils.isNullOrEmpty(path) || partitionFields == null) { + return 0; + } + + int level = 1; + for (int i = 1; i < path.length() - 1; i++) { + if (path.charAt(i) == Path.SEPARATOR_CHAR) { + level++; + } + } + if (path.startsWith(Path.SEPARATOR)) { + level--; + } + if (path.endsWith(Path.SEPARATOR)) { + level--; + } + return level; + } + + protected static ArrayData extractPartitionValues(Types.RecordType partitionFields, + String relativePartitionPath, + boolean urlEncodePartitioningEnabled) { + if (partitionFields.fields().size() == 1) { + // SinglePartPartitionValue, which might contain slashes. + String partitionValue = relativePartitionPath.split("=")[1]; Review Comment: This assumes table is hive-style partitioned. What happens if it is not hive-style partitioned? Also, if the partition path is url- encoded, then we won't find `=` right? ########## hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestLazyPartitionPathFetching.scala: ########## @@ -55,6 +55,36 @@ class TestLazyPartitionPathFetching extends HoodieSparkSqlTestBase { } } + test("Test querying with date column + partition pruning") { Review Comment: How do you ensure partition pruning kicked in? Can you print the physical plan and paste here? If we can assert partition filters in the plan then it's even better. This is to safeguard any future change does not break the pruning. ########## hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala: ########## @@ -383,6 +383,66 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS } } + /** + * This test mainly ensures all non-partition-prefix filter can be pushed successfully + */ + @ParameterizedTest + @CsvSource(value = Array("true, false", "false, false", "true, true", "false, true")) + def testPartitionPruneWithMultiplePartitionColumnsWithComplexExpression(useMetadataTable: Boolean, + complexExpressionPushDown: Boolean): Unit = { Review Comment: It covers the scenario when either both `URL_ENCODE_PARTITIONING` and `HIVE_STYLE_PARTITIONING` are true or both are false. What if one is true and the other is false? It should fallback to regular listing by prefix right? Is it being tested somehwere? ########## hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestPartitionPushDownWhenListingPaths.scala: ########## @@ -0,0 +1,109 @@ +/* + * 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.spark.sql.hudi + +import org.apache.hudi.common.config.HoodieMetadataConfig + +class TestPartitionPushDownWhenListingPaths extends HoodieSparkSqlTestBase { + + test("Test push down different partitions") { + Seq("true", "false").foreach { enableMetadata => + withSQLConf(HoodieMetadataConfig.ENABLE.key -> enableMetadata) { + Seq("cow", "mor").foreach { tableType => + withTempDir { tmp => + val tableName = generateTableName + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long, + | date_par date, + | country string, + | hour int, + | longValue long + |) using hudi + | location '${tmp.getCanonicalPath}' + | tblproperties ( + | primaryKey ='id', + | type = '$tableType', + | preCombineField = 'ts', + | hoodie.datasource.write.hive_style_partitioning = 'true', + | hoodie.datasource.write.partitionpath.urlencode = 'true' + | ) + | PARTITIONED BY (date_par, country, hour, longValue)""".stripMargin) + spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000, date '2023-02-27', 'ID', 1, 102345L)") + spark.sql(s"insert into $tableName values(2, 'a2', 10, 1000, date '2023-02-28', 'US', 4, 102346L)") + spark.sql(s"insert into $tableName values(3, 'a3', 10, 1000, date '2023-03-01', 'CN', 10, 102347L)") + + // Only filter one partition column + checkAnswer(s"select id, name, price, ts from $tableName where date_par = date'2023-03-01' order by id")( + Seq(3, "a3", 10.0, 1000) + ) + + // Filter with And operation + checkAnswer(s"select id, name, price, ts from $tableName where date_par = date'2023-02-28' and hour = 4 order by id")( + Seq(2, "a2", 10.0, 1000) + ) + + // Filter with Or operation + checkAnswer(s"select id, name, price, ts from $tableName where date_par = date'2023-02-28' or country = 'CN' order by id")( + Seq(2, "a2", 10.0, 1000), + Seq(3, "a3", 10.0, 1000) + ) + + // Filter with GT + checkAnswer(s"select id, name, price, ts from $tableName where date_par > date'2023-02-27' order by id")( Review Comment: same for these sql queries.. if we can assert partition filters in the physical plan that would be great. ########## hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java: ########## @@ -50,20 +56,25 @@ /** * Implementation of {@link HoodieTableMetadata} based file-system-backed table metadata. */ -public class FileSystemBackedTableMetadata implements HoodieTableMetadata { +public class FileSystemBackedTableMetadata extends AbstractHoodieTableMetadata { private static final int DEFAULT_LISTING_PARALLELISM = 1500; - private final transient HoodieEngineContext engineContext; - private final SerializableConfiguration hadoopConf; - private final String datasetBasePath; private final boolean assumeDatePartitioning; + private final boolean hiveStylePartitioningEnabled; + private final boolean urlEncodePartitioningEnabled; + public FileSystemBackedTableMetadata(HoodieEngineContext engineContext, SerializableConfiguration conf, String datasetBasePath, boolean assumeDatePartitioning) { - this.engineContext = engineContext; - this.hadoopConf = conf; - this.datasetBasePath = datasetBasePath; + super(engineContext, conf, datasetBasePath); + + FileSystem fs = FSUtils.getFs(dataBasePath.get(), conf.get()); + Path metaPath = new Path(dataBasePath.get(), HoodieTableMetaClient.METAFOLDER_NAME); + TableNotFoundException.checkTableValidity(fs, this.dataBasePath.get(), metaPath); + HoodieTableConfig tableConfig = new HoodieTableConfig(fs, metaPath.toString(), null, null); Review Comment: Instantiating table cofig can be costly. Don't we have metaClient upstream so that we can set the below parameters from the caller itself? ########## hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala: ########## @@ -186,4 +186,13 @@ class Spark2Adapter extends SparkAdapter { case OFF_HEAP => "OFF_HEAP" case _ => throw new IllegalArgumentException(s"Invalid StorageLevel: $level") } + + override def translateFilter(predicate: Expression, + supportNestedPredicatePushdown: Boolean = false): Option[Filter] = { + if (supportNestedPredicatePushdown) { Review Comment: I don't see this in javadoc here or in SparkAdapter. In general, please add as much javadoc/scaladoc as possible for public APIs for future reference. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
