[GitHub] [hudi] boneanxs commented on a diff in pull request #8452: [HUDI-6077] Add more partition push down filters

2023-07-23 Thread via GitHub


boneanxs commented on code in PR #8452:
URL: https://github.com/apache/hudi/pull/8452#discussion_r1271800816


##
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:
   Oh, sorry, must be rebase making this missing, added it back.



##
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:
   Yea, we've already have tests cover different `URL_ENCODE_PARTITIONING` and 
`HIVE_STYLE_PARTITIONING`, such as 
`org.apache.hudi.functional.TestMORDataSource#testMORPartitionPrune`, 
`org.apache.hudi.TestHoodieFileIndex#testPartitionPruneWithMultiplePartitionColumns`,
 they share the same code paths.



##
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:
   Since method 
`org.apache.hudi.metadata.HoodieTableMetadata#createFSBackedTableMetadata` 
doesn't has metaClient, we have to instantiating it here.
   
   For other callers having metaClient, added a new construct method to pass 
tableConfig.



##
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:
   Like I've said before, we can't see any difference in the physical plan 
since all partition filters are pushed to HUDI side, just some filters can't 
take effect before. In order to make sure partition pruning take effect, I 
added `testPartitionPruneWithMultiplePartitionColumnsWithComplexExpression` and 
check `fileIndex.areAllPartitionPathsCached`, before this pr, the complex 
expression cannot be pushed, so `fileIndex.areAllPartitionPathsCached` return 
true, after this, it should return false.



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



[GitHub] [hudi] boneanxs commented on a diff in pull request #8452: [HUDI-6077] Add more partition push down filters

2023-06-08 Thread via GitHub


boneanxs commented on code in PR #8452:
URL: https://github.com/apache/hudi/pull/8452#discussion_r1223761168


##
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:
   Move creating `HoodieTableConfig` only for `FileSystemBackedTableMetadata`, 
in case `HoodieBackedTableMetadata` creating it twice



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



[GitHub] [hudi] boneanxs commented on a diff in pull request #8452: [HUDI-6077] Add more partition push down filters

2023-06-08 Thread via GitHub


boneanxs commented on code in PR #8452:
URL: https://github.com/apache/hudi/pull/8452#discussion_r1222992789


##
hudi-common/src/main/java/org/apache/hudi/metadata/AbstractHoodieTableMetadata.java:
##
@@ -0,0 +1,111 @@
+/*
+ * 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.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.PartitionPathEncodeUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.exception.TableNotFoundException;
+import org.apache.hudi.expression.ArrayData;
+import org.apache.hudi.hadoop.CachingPath;
+import org.apache.hudi.hadoop.SerializablePath;
+import org.apache.hudi.internal.schema.Types;
+import org.apache.hudi.internal.schema.utils.Conversions;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+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 final transient HoodieEngineContext engineContext;
+
+  protected final SerializableConfiguration hadoopConf;
+  protected final SerializablePath dataBasePath;
+
+  protected final boolean hiveStylePartitioningEnabled;
+  protected final boolean urlEncodePartitioningEnabled;
+
+  // 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));
+FileSystem fs = FSUtils.getFs(dataBasePath, conf.get());
+Path metaPath = new Path(dataBasePath, 
HoodieTableMetaClient.METAFOLDER_NAME);
+TableNotFoundException.checkTableValidity(fs, this.dataBasePath.get(), 
metaPath);
+HoodieTableConfig tableConfig = new HoodieTableConfig(fs, 
metaPath.toString(), null, null);

Review Comment:
   Is it possible that constructing `Configuration` could be the slowness? It 
needs to copy all properties  and synchronize the whole class `Configuration` 
(running into lock contention?).
   
   ```java
   public Configuration(Configuration other) {
   this.resources = (ArrayList) other.resources.clone();
   synchronized(other) {
 if (other.properties != null) {
   this.properties = (Properties)other.properties.clone();
 }
   
 if (other.overlay!=null) {
   this.overlay = (Properties)other.overlay.clone();
 }
   
 this.restrictSystemProps = other.restrictSystemProps;
 if (other.updatingResource != null) {
   this.updatingResource = new ConcurrentHashMap(
  other.updatingResource);
 }
 this.finalParameters = Collections.newSetFromMap(
 new ConcurrentHashMap());
 this.finalParameters.addAll(other.finalParameters);
   }
   
   synchronized(Configuration.class) {
 REGISTRY.put(this, null);
   }
   this.classLoader = other.classLoader;
   this.loadDefaults = other.loadDefaults;
   setQuietMode(other.getQuietMode());
 }
   ```
   
   `HoodieTableMetaClient` could call `new Configuration(configuration` 2 
times, `this.hadoopConf = new SerializableConfiguration(conf);` and `getFs()`.
   
   I do trace the construct method of `AbstractHoodieTableMetadata`:
   
   1. Create HoodieTableMetaClient(1ms):
   
   ![Screenshot 2023-06-08 at 17 09 
29](https://github.com/apache/hudi/assets/10115332/212b9c4e-9d8c-4452-ab17-9106e8b3bc91)
   
   2. Create HoodieTableConfig(0.5ms including getFs and checkTableValidity):
   ![Screenshot 2023-06-08 at 17 07 
37](https://github.com/apache/hudi/assets/10115332/e5275538-080b-4aa9-b120-7c5485ca323d)
   
   and trace the construct method of `HoodieTableMetaClient`
   ![Screensh

[GitHub] [hudi] boneanxs commented on a diff in pull request #8452: [HUDI-6077] Add more partition push down filters

2023-06-08 Thread via GitHub


boneanxs commented on code in PR #8452:
URL: https://github.com/apache/hudi/pull/8452#discussion_r1222992789


##
hudi-common/src/main/java/org/apache/hudi/metadata/AbstractHoodieTableMetadata.java:
##
@@ -0,0 +1,111 @@
+/*
+ * 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.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.PartitionPathEncodeUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.exception.TableNotFoundException;
+import org.apache.hudi.expression.ArrayData;
+import org.apache.hudi.hadoop.CachingPath;
+import org.apache.hudi.hadoop.SerializablePath;
+import org.apache.hudi.internal.schema.Types;
+import org.apache.hudi.internal.schema.utils.Conversions;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+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 final transient HoodieEngineContext engineContext;
+
+  protected final SerializableConfiguration hadoopConf;
+  protected final SerializablePath dataBasePath;
+
+  protected final boolean hiveStylePartitioningEnabled;
+  protected final boolean urlEncodePartitioningEnabled;
+
+  // 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));
+FileSystem fs = FSUtils.getFs(dataBasePath, conf.get());
+Path metaPath = new Path(dataBasePath, 
HoodieTableMetaClient.METAFOLDER_NAME);
+TableNotFoundException.checkTableValidity(fs, this.dataBasePath.get(), 
metaPath);
+HoodieTableConfig tableConfig = new HoodieTableConfig(fs, 
metaPath.toString(), null, null);

Review Comment:
   Is it possible that constructing `Configuration` could be the slowness? It 
needs to copy all properties  and synchronize the whole class `Configuration` 
(running into lock contention?).
   
   ```java
   public Configuration(Configuration other) {
   this.resources = (ArrayList) other.resources.clone();
   synchronized(other) {
 if (other.properties != null) {
   this.properties = (Properties)other.properties.clone();
 }
   
 if (other.overlay!=null) {
   this.overlay = (Properties)other.overlay.clone();
 }
   
 this.restrictSystemProps = other.restrictSystemProps;
 if (other.updatingResource != null) {
   this.updatingResource = new ConcurrentHashMap(
  other.updatingResource);
 }
 this.finalParameters = Collections.newSetFromMap(
 new ConcurrentHashMap());
 this.finalParameters.addAll(other.finalParameters);
   }
   
   synchronized(Configuration.class) {
 REGISTRY.put(this, null);
   }
   this.classLoader = other.classLoader;
   this.loadDefaults = other.loadDefaults;
   setQuietMode(other.getQuietMode());
 }
   ```
   
   `HoodieTableMetaClient` could call `new Configuration(configuration` 2 
times, `this.hadoopConf = new SerializableConfiguration(conf);` and `getFs()`.
   
   I do trace the construct method of `AbstractHoodieTableMetadata`:
   
   1. Create HoodieTableMetaClient(1ms):
   
   ![Screenshot 2023-06-08 at 17 09 
29](https://github.com/apache/hudi/assets/10115332/212b9c4e-9d8c-4452-ab17-9106e8b3bc91)
   
   2. Create HoodieTableConfig(0.5ms including getFs and checkTableValidity):
   ![Screenshot 2023-06-08 at 17 07 
37](https://github.com/apache/hudi/assets/10115332/e5275538-080b-4aa9-b120-7c5485ca323d)
   
   and trace the construct method of `HoodieTableMetaClient`
   ![Screensh

[GitHub] [hudi] boneanxs commented on a diff in pull request #8452: [HUDI-6077] Add more partition push down filters

2023-06-07 Thread via GitHub


boneanxs commented on code in PR #8452:
URL: https://github.com/apache/hudi/pull/8452#discussion_r1222499227


##
hudi-common/src/main/java/org/apache/hudi/metadata/AbstractHoodieTableMetadata.java:
##
@@ -0,0 +1,111 @@
+/*
+ * 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.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.PartitionPathEncodeUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.exception.TableNotFoundException;
+import org.apache.hudi.expression.ArrayData;
+import org.apache.hudi.hadoop.CachingPath;
+import org.apache.hudi.hadoop.SerializablePath;
+import org.apache.hudi.internal.schema.Types;
+import org.apache.hudi.internal.schema.utils.Conversions;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+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 final transient HoodieEngineContext engineContext;
+
+  protected final SerializableConfiguration hadoopConf;
+  protected final SerializablePath dataBasePath;
+
+  protected final boolean hiveStylePartitioningEnabled;
+  protected final boolean urlEncodePartitioningEnabled;
+
+  // 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));
+FileSystem fs = FSUtils.getFs(dataBasePath, conf.get());
+Path metaPath = new Path(dataBasePath, 
HoodieTableMetaClient.METAFOLDER_NAME);
+TableNotFoundException.checkTableValidity(fs, this.dataBasePath.get(), 
metaPath);
+HoodieTableConfig tableConfig = new HoodieTableConfig(fs, 
metaPath.toString(), null, null);

Review Comment:
   I think it's not relate to `loadActiveTimelineOnLoad`, since it's default 
false, let me investigate why metaClient could be slow here.



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



[GitHub] [hudi] boneanxs commented on a diff in pull request #8452: [HUDI-6077] Add more partition push down filters

2023-06-07 Thread via GitHub


boneanxs commented on code in PR #8452:
URL: https://github.com/apache/hudi/pull/8452#discussion_r1222426029


##
hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/Conversions.java:
##
@@ -0,0 +1,87 @@
+/*
+ * 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.internal.schema.utils;
+
+import org.apache.hudi.internal.schema.Type;
+import org.apache.hudi.internal.schema.Types;
+
+import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.UUID;
+
+public class Conversions {
+
+  private static final OffsetDateTime EPOCH = 
Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  private static final HashSet SUPPORTED_PARTITION_TYPES = new 
HashSet<>(
+  Arrays.asList(Type.TypeID.INT,
+  Type.TypeID.LONG,
+  Type.TypeID.BOOLEAN,
+  Type.TypeID.FLOAT,
+  Type.TypeID.DECIMAL,
+  Type.TypeID.DOUBLE,
+  Type.TypeID.UUID,
+  Type.TypeID.DATE,
+  Type.TypeID.STRING));
+
+  public static boolean isSchemaSupportedConversion(Types.RecordType schema) {
+for (Types.Field field: schema.fields()) {
+  if (!SUPPORTED_PARTITION_TYPES.contains(field.type().typeId())) {
+return false;
+  }
+}
+return true;
+  }
+
+  public static Object fromPartitionString(String partitionValue, Type type) {

Review Comment:
   This is specially for partition values(more like a util method), but TypeId 
looks more generic. Should we move it there?



##
hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java:
##
@@ -523,18 +524,31 @@ public List fields() {
   return Arrays.asList(fields);
 }
 
-public Field field(String name) {
+/**
+ * Case-sensitive get field by name
+ */
+public synchronized Field fieldByName(String name) {
   if (nameToFields == null) {
 nameToFields = new HashMap<>();
 for (Field field : fields) {
-  nameToFields.put(field.name().toLowerCase(Locale.ROOT), field);
+  nameToFields.put(field.name(), field);
+}
+  }
+  return nameToFields.get(name);
+}
+
+public synchronized Field fieldByNameCaseInsensitive(String name) {
+  if (lowercaseNameToFields == null) {
+lowercaseNameToFields = new HashMap<>();
+for (Field field : fields) {
+  lowercaseNameToFields.put(field.name().toLowerCase(Locale.ROOT), 
field);
 }
   }
-  return nameToFields.get(name.toLowerCase(Locale.ROOT));
+  return lowercaseNameToFields.get(name.toLowerCase(Locale.ROOT));
 }
 
 @Override
-public Field field(int id) {
+public synchronized Field field(int id) {

Review Comment:
   Since all spark tests share the same JVM, it's possible that many threads 
running concurrently 
here(`org.apache.hudi.metadata.AbstractHoodieTableMetadata#extractPartitionValues`),
 one thread trying to initialize `idToFields`, while another thread trying to 
fetch field.
   
   Let me try to avoid 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



[GitHub] [hudi] boneanxs commented on a diff in pull request #8452: [HUDI-6077] Add more partition push down filters

2023-06-06 Thread via GitHub


boneanxs commented on code in PR #8452:
URL: https://github.com/apache/hudi/pull/8452#discussion_r1220712391


##
hudi-common/src/main/java/org/apache/hudi/metadata/AbstractHoodieTableMetadata.java:
##
@@ -0,0 +1,111 @@
+/*
+ * 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.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.PartitionPathEncodeUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.exception.TableNotFoundException;
+import org.apache.hudi.expression.ArrayData;
+import org.apache.hudi.hadoop.CachingPath;
+import org.apache.hudi.hadoop.SerializablePath;
+import org.apache.hudi.internal.schema.Types;
+import org.apache.hudi.internal.schema.utils.Conversions;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+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 final transient HoodieEngineContext engineContext;
+
+  protected final SerializableConfiguration hadoopConf;
+  protected final SerializablePath dataBasePath;
+
+  protected final boolean hiveStylePartitioningEnabled;
+  protected final boolean urlEncodePartitioningEnabled;
+
+  // 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));
+FileSystem fs = FSUtils.getFs(dataBasePath, conf.get());
+Path metaPath = new Path(dataBasePath, 
HoodieTableMetaClient.METAFOLDER_NAME);
+TableNotFoundException.checkTableValidity(fs, this.dataBasePath.get(), 
metaPath);
+HoodieTableConfig tableConfig = new HoodieTableConfig(fs, 
metaPath.toString(), null, null);

Review Comment:
   @bvaradar I found that `FileSystemBackedTableMetadata` is hotly used in many 
codes, so creating metaClient to get TableConfig could badly affect the 
performance(Making tests timeout).
   
   ![Screenshot 2023-06-07 at 10 14 
35](https://github.com/apache/hudi/assets/10115332/31436b32-8f99-481a-8484-368f3d70c11b)
   
   So I change here to create TableConfig directly.



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



[GitHub] [hudi] boneanxs commented on a diff in pull request #8452: [HUDI-6077] Add more partition push down filters

2023-06-06 Thread via GitHub


boneanxs commented on code in PR #8452:
URL: https://github.com/apache/hudi/pull/8452#discussion_r1220712391


##
hudi-common/src/main/java/org/apache/hudi/metadata/AbstractHoodieTableMetadata.java:
##
@@ -0,0 +1,111 @@
+/*
+ * 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.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.PartitionPathEncodeUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.exception.TableNotFoundException;
+import org.apache.hudi.expression.ArrayData;
+import org.apache.hudi.hadoop.CachingPath;
+import org.apache.hudi.hadoop.SerializablePath;
+import org.apache.hudi.internal.schema.Types;
+import org.apache.hudi.internal.schema.utils.Conversions;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+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 final transient HoodieEngineContext engineContext;
+
+  protected final SerializableConfiguration hadoopConf;
+  protected final SerializablePath dataBasePath;
+
+  protected final boolean hiveStylePartitioningEnabled;
+  protected final boolean urlEncodePartitioningEnabled;
+
+  // 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));
+FileSystem fs = FSUtils.getFs(dataBasePath, conf.get());
+Path metaPath = new Path(dataBasePath, 
HoodieTableMetaClient.METAFOLDER_NAME);
+TableNotFoundException.checkTableValidity(fs, this.dataBasePath.get(), 
metaPath);
+HoodieTableConfig tableConfig = new HoodieTableConfig(fs, 
metaPath.toString(), null, null);

Review Comment:
   @bvaradar I found that `FileSystemBackedTableMetadata` is hotly used in many 
codes, so creating metaClient to get TableConfig could badly affect the 
performance(Making tests timeout).
   
   So I change here to create TableConfig directly.



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



[GitHub] [hudi] boneanxs commented on a diff in pull request #8452: [HUDI-6077] Add more partition push down filters

2023-05-31 Thread via GitHub


boneanxs commented on code in PR #8452:
URL: https://github.com/apache/hudi/pull/8452#discussion_r1211261314


##
hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java:
##
@@ -96,11 +109,32 @@ public List 
getPartitionPathWithPathPrefixes(List relativePathPr
   }
 
   private List getPartitionPathWithPathPrefix(String 
relativePathPrefix) throws IOException {
+return 
getPartitionPathWithPathPrefixUsingFilterExpression(relativePathPrefix, null, 
null);
+  }
+
+  private List 
getPartitionPathWithPathPrefixUsingFilterExpression(String relativePathPrefix,
+   
Types.RecordType partitionFields,
+   
Expression expression) throws IOException {
 List pathsToList = new CopyOnWriteArrayList<>();
 pathsToList.add(StringUtils.isNullOrEmpty(relativePathPrefix)
-? new Path(datasetBasePath) : new Path(datasetBasePath, 
relativePathPrefix));
+? dataBasePath.get() : new Path(dataBasePath.get(), 
relativePathPrefix));
 List partitionPaths = new CopyOnWriteArrayList<>();
 
+int partitionLevel = -1;
+boolean needPushDownExpressions;
+// Not like `HoodieBackedTableMetadata`, since we don't know the exact 
partition levels here,
+// given it's possible that partition values contains `/`, which could 
affect
+// the result to get right `partitionValue` when listing paths, here we 
have
+// to make it more strict that `urlEncodePartitioningEnabled` must be 
enabled.
+// TODO better enable urlEncodePartitioningEnabled if 
hiveStylePartitioningEnabled is enabled?

Review Comment:
   Hey @bvaradar, all comments are addressed. Please review it again.



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



[GitHub] [hudi] boneanxs commented on a diff in pull request #8452: [HUDI-6077] Add more partition push down filters

2023-05-22 Thread via GitHub


boneanxs commented on code in PR #8452:
URL: https://github.com/apache/hudi/pull/8452#discussion_r1201575964


##
hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java:
##
@@ -96,11 +109,32 @@ public List 
getPartitionPathWithPathPrefixes(List relativePathPr
   }
 
   private List getPartitionPathWithPathPrefix(String 
relativePathPrefix) throws IOException {
+return 
getPartitionPathWithPathPrefixUsingFilterExpression(relativePathPrefix, null, 
null);
+  }
+
+  private List 
getPartitionPathWithPathPrefixUsingFilterExpression(String relativePathPrefix,
+   
Types.RecordType partitionFields,
+   
Expression expression) throws IOException {
 List pathsToList = new CopyOnWriteArrayList<>();
 pathsToList.add(StringUtils.isNullOrEmpty(relativePathPrefix)
-? new Path(datasetBasePath) : new Path(datasetBasePath, 
relativePathPrefix));
+? dataBasePath.get() : new Path(dataBasePath.get(), 
relativePathPrefix));
 List partitionPaths = new CopyOnWriteArrayList<>();
 
+int partitionLevel = -1;
+boolean needPushDownExpressions;
+// Not like `HoodieBackedTableMetadata`, since we don't know the exact 
partition levels here,
+// given it's possible that partition values contains `/`, which could 
affect
+// the result to get right `partitionValue` when listing paths, here we 
have
+// to make it more strict that `urlEncodePartitioningEnabled` must be 
enabled.
+// TODO better enable urlEncodePartitioningEnabled if 
hiveStylePartitioningEnabled is enabled?

Review Comment:
   Checking hudi configure here is difficult, since`HoodieConfig` is not passed 
when construct `FileSystemBackedTableMetadata`, that's why this class lefts 
some TODOs here like `TODO: Get the parallelism from HoodieWriteConfig` and 
`get `caseSensitive` from HoodieConfig`.
   
   For now, I think it's safe if we both check `hiveStylePartitioningEnabled` 
and `urlEncodePartitioningEnabled` when using `FileSystemBackedTableMetadata` 
and enable partition push down filters.
   
   And I'll try to resolve TODOs relate to HoodieConfig in a follow-up 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 unsubscribe, e-mail: commits-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] boneanxs commented on a diff in pull request #8452: [HUDI-6077] Add more partition push down filters

2023-05-22 Thread via GitHub


boneanxs commented on code in PR #8452:
URL: https://github.com/apache/hudi/pull/8452#discussion_r1201575964


##
hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java:
##
@@ -96,11 +109,32 @@ public List 
getPartitionPathWithPathPrefixes(List relativePathPr
   }
 
   private List getPartitionPathWithPathPrefix(String 
relativePathPrefix) throws IOException {
+return 
getPartitionPathWithPathPrefixUsingFilterExpression(relativePathPrefix, null, 
null);
+  }
+
+  private List 
getPartitionPathWithPathPrefixUsingFilterExpression(String relativePathPrefix,
+   
Types.RecordType partitionFields,
+   
Expression expression) throws IOException {
 List pathsToList = new CopyOnWriteArrayList<>();
 pathsToList.add(StringUtils.isNullOrEmpty(relativePathPrefix)
-? new Path(datasetBasePath) : new Path(datasetBasePath, 
relativePathPrefix));
+? dataBasePath.get() : new Path(dataBasePath.get(), 
relativePathPrefix));
 List partitionPaths = new CopyOnWriteArrayList<>();
 
+int partitionLevel = -1;
+boolean needPushDownExpressions;
+// Not like `HoodieBackedTableMetadata`, since we don't know the exact 
partition levels here,
+// given it's possible that partition values contains `/`, which could 
affect
+// the result to get right `partitionValue` when listing paths, here we 
have
+// to make it more strict that `urlEncodePartitioningEnabled` must be 
enabled.
+// TODO better enable urlEncodePartitioningEnabled if 
hiveStylePartitioningEnabled is enabled?

Review Comment:
   Checking hudi configure here is difficult, since`HoodieConfig` is not passed 
when construct `FileSystemBackedTableMetadata`, that's why this class lefts 
some TODOs here like `TODO: Get the parallelism from HoodieWriteConfig` and 
`get `caseSensitive` from HoodieConfig`.
   
   I think it's safe if we both check `hiveStylePartitioningEnabled` and 
`urlEncodePartitioningEnabled` when using `FileSystemBackedTableMetadata` and 
enable partition push down filters.
   
   And I'll try to resolve TODOs relate to HoodieConfig in a follow-up 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 unsubscribe, e-mail: commits-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] boneanxs commented on a diff in pull request #8452: [HUDI-6077] Add more partition push down filters

2023-05-21 Thread via GitHub


boneanxs commented on code in PR #8452:
URL: https://github.com/apache/hudi/pull/8452#discussion_r117268


##
hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java:
##
@@ -96,11 +109,32 @@ public List 
getPartitionPathWithPathPrefixes(List relativePathPr
   }
 
   private List getPartitionPathWithPathPrefix(String 
relativePathPrefix) throws IOException {
+return 
getPartitionPathWithPathPrefixUsingFilterExpression(relativePathPrefix, null, 
null);
+  }
+
+  private List 
getPartitionPathWithPathPrefixUsingFilterExpression(String relativePathPrefix,
+   
Types.RecordType partitionFields,
+   
Expression expression) throws IOException {
 List pathsToList = new CopyOnWriteArrayList<>();
 pathsToList.add(StringUtils.isNullOrEmpty(relativePathPrefix)
-? new Path(datasetBasePath) : new Path(datasetBasePath, 
relativePathPrefix));
+? dataBasePath.get() : new Path(dataBasePath.get(), 
relativePathPrefix));
 List partitionPaths = new CopyOnWriteArrayList<>();
 
+int partitionLevel = -1;
+boolean needPushDownExpressions;
+// Not like `HoodieBackedTableMetadata`, since we don't know the exact 
partition levels here,
+// given it's possible that partition values contains `/`, which could 
affect
+// the result to get right `partitionValue` when listing paths, here we 
have
+// to make it more strict that `urlEncodePartitioningEnabled` must be 
enabled.
+// TODO better enable urlEncodePartitioningEnabled if 
hiveStylePartitioningEnabled is enabled?

Review Comment:
   @bvaradar We have to list at least one partition path to get the partition 
levels for `FileSystemBackedTableMetadata` to ensure the number of partition 
columns and partition levels are same. But it might be time consuming 
especially for object store.
   
   So here I have to make it more strict for `FileSystemBackedTableMetadata` to 
check `urlEncodePartitioningEnabled` also.  While we don't need to check it in 
`HoodieTableBackedTableMetadata`, since we can simply get one partition path to 
check it.
   
   I'm thinking maybe we should enable `urlEncodePartitioningEnabled` if 
`hiveStylePartitioningEnabled` is enabled? Since this is the default behavior 
for spark and hive



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



[GitHub] [hudi] boneanxs commented on a diff in pull request #8452: [HUDI-6077] Add more partition push down filters

2023-05-21 Thread via GitHub


boneanxs commented on code in PR #8452:
URL: https://github.com/apache/hudi/pull/8452#discussion_r117268


##
hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java:
##
@@ -96,11 +109,32 @@ public List 
getPartitionPathWithPathPrefixes(List relativePathPr
   }
 
   private List getPartitionPathWithPathPrefix(String 
relativePathPrefix) throws IOException {
+return 
getPartitionPathWithPathPrefixUsingFilterExpression(relativePathPrefix, null, 
null);
+  }
+
+  private List 
getPartitionPathWithPathPrefixUsingFilterExpression(String relativePathPrefix,
+   
Types.RecordType partitionFields,
+   
Expression expression) throws IOException {
 List pathsToList = new CopyOnWriteArrayList<>();
 pathsToList.add(StringUtils.isNullOrEmpty(relativePathPrefix)
-? new Path(datasetBasePath) : new Path(datasetBasePath, 
relativePathPrefix));
+? dataBasePath.get() : new Path(dataBasePath.get(), 
relativePathPrefix));
 List partitionPaths = new CopyOnWriteArrayList<>();
 
+int partitionLevel = -1;
+boolean needPushDownExpressions;
+// Not like `HoodieBackedTableMetadata`, since we don't know the exact 
partition levels here,
+// given it's possible that partition values contains `/`, which could 
affect
+// the result to get right `partitionValue` when listing paths, here we 
have
+// to make it more strict that `urlEncodePartitioningEnabled` must be 
enabled.
+// TODO better enable urlEncodePartitioningEnabled if 
hiveStylePartitioningEnabled is enabled?

Review Comment:
   We have to list at least one partition path to get the partition levels for 
`FileSystemBackedTableMetadata` to ensure the number of partition columns and 
partition levels are same. But it might be time consuming especially for object 
store.
   
   So here I have to make it more strict for `FileSystemBackedTableMetadata` to 
check `urlEncodePartitioningEnabled` also.  While we don't need to check it in 
`HoodieTableBackedTableMetadata`, since we can simply get one partition path to 
check it.
   
   I'm thinking maybe we should enable `urlEncodePartitioningEnabled` if 
`hiveStylePartitioningEnabled` is enabled? Since this is the default behavior 
for spark and hive



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



[GitHub] [hudi] boneanxs commented on a diff in pull request #8452: [HUDI-6077] Add more partition push down filters

2023-05-21 Thread via GitHub


boneanxs commented on code in PR #8452:
URL: https://github.com/apache/hudi/pull/8452#discussion_r1199888295


##
hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java:
##
@@ -121,14 +184,24 @@ private List 
getPartitionPathWithPathPrefix(String relativePathPrefix) t
 List, Option>> result = 
engineContext.map(dirToFileListing, fileStatus -> {
   FileSystem fileSystem = 
fileStatus.getPath().getFileSystem(hadoopConf.get());
   if (fileStatus.isDirectory()) {
+String relativePartitionPath = 
FSUtils.getRelativePartitionPath(new Path(datasetBasePath), 
fileStatus.getPath());
 if (HoodiePartitionMetadata.hasPartitionMetadata(fileSystem, 
fileStatus.getPath())) {
-  return Pair.of(Option.of(FSUtils.getRelativePartitionPath(new 
Path(datasetBasePath), fileStatus.getPath())), Option.empty());
+  if (boundedExpr instanceof Predicates.TrueExpression || 
(Boolean) boundedExpr.eval(HoodieTableMetadata
+  .extractPartitionValues(partitionFields, 
relativePartitionPath, hiveStylePartitioningEnabled, 
urlEncodePartitioningEnabled))) {
+return Pair.of(Option.of(relativePartitionPath), 
Option.empty());
+  }
 } else if 
(!fileStatus.getPath().getName().equals(HoodieTableMetaClient.METAFOLDER_NAME)) 
{
-  return Pair.of(Option.empty(), Option.of(fileStatus.getPath()));
+  if (boundedExpr instanceof Predicates.TrueExpression || 
(Boolean) boundedExpr.eval(HoodieTableMetadata
+  .extractPartitionValues(partitionFields, 
relativePartitionPath, hiveStylePartitioningEnabled, 
urlEncodePartitioningEnabled))) {
+return Pair.of(Option.empty(), 
Option.of(fileStatus.getPath()));
+  }
 }
   } else if 
(fileStatus.getPath().getName().startsWith(HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE_PREFIX))
 {

Review Comment:
   Yes, I think this is needed for non-partitioned datasets.



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



[GitHub] [hudi] boneanxs commented on a diff in pull request #8452: [HUDI-6077] Add more partition push down filters

2023-05-11 Thread via GitHub


boneanxs commented on code in PR #8452:
URL: https://github.com/apache/hudi/pull/8452#discussion_r1191976171


##
hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/FilterGenVisitor.java:
##
@@ -42,9 +43,10 @@ private String quoteStringLiteral(String value) {
 }
   }
 
-  private String visitAnd(Expression left, Expression right) {
-String leftResult = left.accept(this);
-String rightResult = right.accept(this);
+  @Override
+  public String visitAnd(Predicates.And and) {

Review Comment:
   hive sync is always case insensitive



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



[GitHub] [hudi] boneanxs commented on a diff in pull request #8452: [HUDI-6077] Add more partition push down filters

2023-05-11 Thread via GitHub


boneanxs commented on code in PR #8452:
URL: https://github.com/apache/hudi/pull/8452#discussion_r1191975737


##
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:
   Currently, for partition push down, there shouldn't be any nested columns 
are partition columns, so I fail it both in spark2Adapter and spark3Adapter to 
keep consistent.
   
   But in case this method could be used in other codes, we should only fail it 
if using spark2? Since Spark3 supports `supportNestedPredicatePushdown`



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



[GitHub] [hudi] boneanxs commented on a diff in pull request #8452: [HUDI-6077] Add more partition push down filters

2023-05-11 Thread via GitHub


boneanxs commented on code in PR #8452:
URL: https://github.com/apache/hudi/pull/8452#discussion_r1191973525


##
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala:
##
@@ -307,8 +318,20 @@ class SparkHoodieTableFileIndex(spark: SparkSession,
 Seq(new PartitionPath(relativePartitionPathPrefix, 
staticPartitionColumnNameValuePairs.map(_._2._2.asInstanceOf[AnyRef]).toArray))
   } else {
 // Otherwise, compile extracted partition values (from query 
predicates) into a sub-path which is a prefix
-// of the complete partition path, do listing for this prefix-path only
-
listPartitionPaths(Seq(relativePartitionPathPrefix).toList.asJava).asScala
+// of the complete partition path, do listing for this prefix-path and 
filter them with partitionPredicates
+Try {
+  
SparkFilterHelper.convertDataType(partitionSchema).asInstanceOf[RecordType]
+} match {
+  case Success(partitionRecordType) if 
partitionRecordType.fields().size() == _partitionSchemaFromProperties.size =>
+val convertedFilters = SparkFilterHelper.convertFilters(
+  partitionColumnPredicates.flatMap {
+expr => sparkAdapter.translateFilter(expr)
+  })
+listPartitionPaths(Seq(relativePartitionPathPrefix).toList.asJava, 
partitionRecordType, convertedFilters).asScala

Review Comment:
   Make sense, will add check here



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



[GitHub] [hudi] boneanxs commented on a diff in pull request #8452: [HUDI-6077] Add more partition push down filters

2023-05-11 Thread via GitHub


boneanxs commented on code in PR #8452:
URL: https://github.com/apache/hudi/pull/8452#discussion_r1191878586


##
hudi-common/src/main/java/org/apache/hudi/expression/Expression.java:
##
@@ -40,14 +51,19 @@ public enum Operator {
 }
   }
 
-  private final List children;
+  List getChildren();

Review Comment:
   I change this to Interface, which doesn't allowed specify modifier.



##
hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java:
##
@@ -84,6 +96,19 @@ public List getAllPartitionPaths() throws 
IOException {
 return getPartitionPathWithPathPrefixes(Collections.singletonList(""));
   }
 
+  @Override
+  public List getPartitionPathByExpression(List 
relativePathPrefixes,

Review Comment:
   Sure, will add tests soon



##
hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java:
##
@@ -58,13 +62,21 @@ public class FileSystemBackedTableMetadata implements 
HoodieTableMetadata {
   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;
 this.assumeDatePartitioning = assumeDatePartitioning;
+HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()

Review Comment:
   `FileSystemBackedTableMetadata` directly implement interface 
`HoodieTableMetadata`, while `metaClient` is initialized in Class 
`BaseTableMetadata`.
   
   Maybe I need to move metaClient out instead of local variable, in case 
others may need it also.



##
hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java:
##
@@ -95,11 +120,38 @@ public List 
getPartitionPathWithPathPrefixes(List relativePathPr
 }).collect(Collectors.toList());
   }
 
+  private int getRelativePathPartitionLevel(Types.RecordType partitionFields, 
String relativePathPrefix) {
+if (StringUtils.isNullOrEmpty(relativePathPrefix) || partitionFields == 
null || partitionFields.fields().size() == 1) {
+  return 0;
+}
+
+int level = 0;
+for (int i = 1; i < relativePathPrefix.length() - 1; i++) {

Review Comment:
   partitionFields have all partition columns, while relativePathPrefix only 
contains partial partitions.
   
   For ex. partitionFields could be , while 
relativePathPrefix is `/region=US`, we should return 1 to indicate the start 
partition index.



##
hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java:
##
@@ -95,11 +120,38 @@ public List 
getPartitionPathWithPathPrefixes(List relativePathPr
 }).collect(Collectors.toList());
   }
 
+  private int getRelativePathPartitionLevel(Types.RecordType partitionFields, 
String relativePathPrefix) {
+if (StringUtils.isNullOrEmpty(relativePathPrefix) || partitionFields == 
null || partitionFields.fields().size() == 1) {
+  return 0;
+}
+
+int level = 0;
+for (int i = 1; i < relativePathPrefix.length() - 1; i++) {

Review Comment:
   By the way, is it possible we have more than 1 partition columns, while 
`hoodie.datasource.write.partitionpath.urlencode` is disabled.
   
   For ex. partitionFields is  and some partition values 
are "/US/2023/05/12/10"(which means region=US, date=2023/05/12, hour=10), now 
we have only 3 partitions, but we could get 4 partition levels.
   
   We can handle only one partition column while its values contains `/`(like 
date=2023/05/12), but looks it's difficult to identify which value corresponds 
to which column if we have many columns. Do you have any suggestions?



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



[GitHub] [hudi] boneanxs commented on a diff in pull request #8452: [HUDI-6077] Add more partition push down filters

2023-05-03 Thread via GitHub


boneanxs commented on code in PR #8452:
URL: https://github.com/apache/hudi/pull/8452#discussion_r1184492603


##
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieRowCreateHandle.java:
##
@@ -190,16 +189,8 @@ public void testInstantiationFailure(boolean 
enableMetadataTable) {
   HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
   new HoodieRowCreateHandle(table, cfg, " def", 
UUID.randomUUID().toString(), "001", RANDOM.nextInt(10), RANDOM.nextLong(), 
RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE);
   fail("Should have thrown exception");
-} catch (HoodieInsertException ioe) {

Review Comment:
   If `enableMetadataTable` was true, it will throw `TableNotFoundException` 
when creating `HoodieTableMetaClient` in `HoodieBackedTableMetadata`, now since 
we will also create `HoodieTableMetaClient` in `FileSystemBackedTableMetadata`, 
so now the same exception will be throw.



##
hudi-common/src/main/java/org/apache/hudi/expression/BindVisitor.java:
##
@@ -0,0 +1,179 @@
+/*
+ * 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.expression;
+
+import org.apache.hudi.internal.schema.Types;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+public class BindVisitor implements ExpressionVisitor  {
+
+  protected final Types.RecordType recordType;
+  protected final boolean caseSensitive;
+
+  public BindVisitor(Types.RecordType recordType, boolean caseSensitive) {
+this.recordType = recordType;
+this.caseSensitive = caseSensitive;
+  }
+
+  @Override
+  public Expression alwaysTrue() {
+return Predicates.True.get();
+  }
+
+  @Override
+  public Expression alwaysFalse() {
+return Predicates.False.get();
+  }
+
+  @Override
+  public Expression visitAnd(Predicates.And and) {
+if (and.getLeft() instanceof Predicates.False
+|| and.getRight() instanceof Predicates.False) {
+  return alwaysFalse();
+}
+
+Expression left = and.getLeft().accept(this);
+Expression right = and.getRight().accept(this);
+if (left instanceof Predicates.False
+|| right instanceof Predicates.False) {
+  return alwaysFalse();
+}
+
+if (left instanceof Predicates.True
+&& right instanceof Predicates.True) {
+  return alwaysTrue();
+}
+
+if (left instanceof Predicates.True) {
+  return right;
+}
+
+if (right instanceof Predicates.True) {
+  return left;
+}
+
+return Predicates.and(left, right);
+  }
+
+  @Override
+  public Expression visitOr(Predicates.Or or) {
+if (or.getLeft() instanceof Predicates.True
+|| or.getRight() instanceof Predicates.True) {
+  return alwaysTrue();
+}
+
+Expression left = or.getLeft().accept(this);
+Expression right = or.getRight().accept(this);
+if (left instanceof Predicates.True
+|| right instanceof Predicates.True) {
+  return alwaysTrue();
+}
+
+if (left instanceof Predicates.False
+&& right instanceof Predicates.False) {
+  return alwaysFalse();
+}
+
+if (left instanceof Predicates.False) {
+  return right;
+}
+
+if (right instanceof Predicates.False) {
+  return left;
+}
+
+return Predicates.or(left, right);
+  }
+
+  @Override
+  public Expression visitLiteral(Literal literal) {
+return literal;
+  }
+
+  @Override
+  public Expression visitNameReference(NameReference attribute) {
+// TODO Should consider caseSensitive?

Review Comment:
   Now we get field value using case sensitive resolution, I'm thinking maybe 
we also need to support case insensitive.
   
   Or maybe I should use case insensitive by default, since 
`spark.sql.caseSensitive` is default to false.



##
hudi-common/src/main/java/org/apache/hudi/expression/ArrayData.java:
##
@@ -16,22 +16,25 @@
  * limitations under the License.
  */
 
-package org.apache.hudi.hive.expression;

Review Comment:
   These expressions were first introduced in pr 
https://github.com/apache/hudi/pull/6725 to allow filter partitions when sync 
to HMS, so they were in `hudi-hive-sync` before, and now these expressions 
actually can b