suryaprasanna commented on code in PR #17850:
URL: https://github.com/apache/hudi/pull/17850#discussion_r2854714278


##########
hudi-aws/src/test/java/org/apache/hudi/aws/sync/TestGluePartitionPushdown.java:
##########
@@ -91,6 +95,8 @@ public void testPushDownFilters() {
   @Test
   public void testPushDownFilterIfExceedLimit() {
     Properties props = new Properties();
+    props.put(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),
+        MultiPartKeysValueExtractor.class.getName());

Review Comment:
   If we default NonPartitionedExtractor it is needed. So, removing this code 
change. So, it wont be required anymore.



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -291,6 +291,8 @@ class HoodieSparkSqlWriterInternal {
           if 
(StringUtils.nonEmpty(hoodieConfig.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME)))
             
hoodieConfig.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME)
           else KeyGeneratorType.getKeyGeneratorClassName(hoodieConfig)
+        val partitionValueExtractorClassName = hoodieConfig
+          
.getStringOrDefault(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), 
null)

Review Comment:
   True, let me just use DataSourceWriteOptions configs alone.



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java:
##########
@@ -452,7 +453,8 @@ HoodieTableMetaClient 
initializeEmptyTable(HoodieTableMetaClient.TableBuilder ta
       payloadClass = overridingMergeConfigs.get().getMiddle();
       mergeStrategyId = overridingMergeConfigs.get().getRight();
     }
-
+    String partitionValueExtractorClassName = 
props.getString(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),
+        HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.defaultValue());

Review Comment:
   DataSourceWriteOptions.PARTITION_EXTRACTOR_CLASS is actually same as 
HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS. Let me use 
DataSourceWriteOptions.PARTITION_EXTRACTOR_CLASS here.



##########
hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/util/TestPartitionFilterGenerator.java:
##########
@@ -36,6 +38,7 @@ public class TestPartitionFilterGenerator {
   @Test
   public void testPushDownFilters() {
     Properties props = new Properties();
+    props.put(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), 
MultiPartKeysValueExtractor.class.getName());

Review Comment:
   If we default NonPartitionedExtractor it is needed. So, removing this code 
change. So, it wont be required anymore.



##########
hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java:
##########
@@ -119,40 +118,10 @@ public class HoodieSyncConfig extends HoodieConfig {
   public static final ConfigProperty<String> 
META_SYNC_PARTITION_EXTRACTOR_CLASS = ConfigProperty
       .key("hoodie.datasource.hive_sync.partition_extractor_class")
       .defaultValue("org.apache.hudi.hive.MultiPartKeysValueExtractor")
-      .withInferFunction(cfg -> {
-        Option<String> partitionFieldsOpt;
-        if (StringUtils.nonEmpty(cfg.getString(META_SYNC_PARTITION_FIELDS))) {
-          partitionFieldsOpt = 
Option.ofNullable(cfg.getString(META_SYNC_PARTITION_FIELDS));
-        } else {
-          partitionFieldsOpt = HoodieTableConfig.getPartitionFieldProp(cfg)
-              .or(() -> 
Option.ofNullable(cfg.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME)));
-        }
-        if (!partitionFieldsOpt.isPresent()) {
-          return Option.empty();
-        }
-        String partitionFields = partitionFieldsOpt.get();
-        if (StringUtils.nonEmpty(partitionFields)) {
-          int numOfPartFields = partitionFields.split(",").length;
-          if (numOfPartFields == 1) {
-            if (cfg.contains(HIVE_STYLE_PARTITIONING_ENABLE)
-                && 
cfg.getString(HIVE_STYLE_PARTITIONING_ENABLE).equals("true")) {
-              return 
Option.of("org.apache.hudi.hive.HiveStylePartitionValueExtractor");
-            } else if (cfg.contains(SLASH_SEPARATED_DATE_PARTITIONING)
-                && 
cfg.getString(SLASH_SEPARATED_DATE_PARTITIONING).equals("true")) {
-              return 
Option.of("org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor");
-            } else {
-              return 
Option.of("org.apache.hudi.hive.SinglePartPartitionValueExtractor");
-            }
-          } else {
-            return 
Option.of("org.apache.hudi.hive.MultiPartKeysValueExtractor");
-          }
-        } else {
-          return Option.of("org.apache.hudi.hive.NonPartitionedExtractor");
-        }
-      })
+      
.withInferFunction(HoodieTableConfigUtils::inferPartitionValueExtractorClass)

Review Comment:
   1. Lets check if META_SYNC_PARTITION_FIELDS is explicitly configured. if 
yes, lets take that as first choice.
   > This cannot be done as the meta sync configs are not part of hudi-common 
module they are of hudi-sync module.
   2. Missing SLASH_SEPARATED_DATE_PARTITIONING Check
   > My bad, fixed it now.
   3. Different Config Key for Hive-Style Partitioning
   > public static final ConfigProperty<String> HIVE_STYLE_PARTITIONING_ENABLE 
= KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE;
   > So, either of them can be used. Let me use KeyGeneratorOptions.
   4. Changed Logic for Empty Partition Fields
   > Reverting this change. We can make this change as part of a separate PR.



##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala:
##########
@@ -224,11 +226,13 @@ object HoodieSparkUtils extends SparkAdapterSupport with 
SparkVersionsSupport wi
                                  partitionPath: String,
                                  tableBasePath: StoragePath,
                                  tableSchema: StructType,
-                                 tableConfig: java.util.Map[String, String],
+                                 tableConfig: HoodieTableConfig,
                                  timeZoneId: String,
-                                 shouldValidatePartitionColumns: Boolean): 
Array[Object] = {
+                                 shouldValidatePartitionColumns: Boolean,
+                                 usePartitionValueExtractorOnRead: Boolean): 
Array[Object] = {
     val keyGeneratorClass = 
KeyGeneratorType.getKeyGeneratorClassName(tableConfig)
-    val timestampKeyGeneratorType = 
tableConfig.get(TimestampKeyGeneratorConfig.TIMESTAMP_TYPE_FIELD.key())
+    val timestampKeyGeneratorType = 
tableConfig.propsMap().get(TimestampKeyGeneratorConfig.TIMESTAMP_TYPE_FIELD.key())
+    val partitionValueExtractorClass = 
tableConfig.getPartitionValueExtractorClass

Review Comment:
   Yes, then the user need to set usePartitionValueExtractorOnRead to be false. 
So, we dont rely on that field. That is why by default 
usePartitionValueExtractorOnRead is set to false.



##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##########
@@ -1208,6 +1217,10 @@ public String getKeyGeneratorClassName() {
     return KeyGeneratorType.getKeyGeneratorClassName(this);
   }
 
+  public String getPartitionValueExtractorClass() {

Review Comment:
   Good idea, yes doing that.



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -749,7 +752,8 @@ class HoodieSparkSqlWriterInternal {
           
String.valueOf(HoodieTableConfig.PARTITION_METAFILE_USE_BASE_FORMAT.defaultValue())
         ))
         val tableFormat = 
hoodieConfig.getStringOrDefault(HoodieTableConfig.TABLE_FORMAT)
-
+        val partitionValueExtractorClassName = hoodieConfig
+          
.getStringOrDefault(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), 
null)

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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to