wangxianghu commented on a change in pull request #2431:
URL: https://github.com/apache/hudi/pull/2431#discussion_r563537637



##########
File path: 
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala
##########
@@ -181,16 +183,33 @@ object DataSourceWriteOptions {
   @Deprecated
   val DEFAULT_STORAGE_TYPE_OPT_VAL = COW_STORAGE_TYPE_OPT_VAL
 
-  def translateStorageTypeToTableType(optParams: Map[String, String]) : 
Map[String, String] = {
+  def translateOptParams(optParams: Map[String, String]): Map[String, String] 
= {
+    // translate StorageType to TableType
+    var newOptParams = optParams
     if (optParams.contains(STORAGE_TYPE_OPT_KEY) && 
!optParams.contains(TABLE_TYPE_OPT_KEY)) {
       log.warn(STORAGE_TYPE_OPT_KEY + " is deprecated and will be removed in a 
later release; Please use " + TABLE_TYPE_OPT_KEY)
-      optParams ++ Map(TABLE_TYPE_OPT_KEY -> optParams(STORAGE_TYPE_OPT_KEY))
-    } else {
-      optParams
+      newOptParams = optParams ++ Map(TABLE_TYPE_OPT_KEY -> 
optParams(STORAGE_TYPE_OPT_KEY))
     }
+    // translate the api partitionBy of spark DataFrameWriter to 
PARTITIONPATH_FIELD_OPT_KEY
+    if (optParams.contains(SparkDataSourceUtils.PARTITIONING_COLUMNS_KEY) && 
!optParams.contains(PARTITIONPATH_FIELD_OPT_KEY)) {
+      val partitionColumns = 
optParams.get(SparkDataSourceUtils.PARTITIONING_COLUMNS_KEY)
+        .map(SparkDataSourceUtils.decodePartitioningColumns)
+        .getOrElse(Nil)
+
+      val keyGeneratorClass = 
optParams.getOrElse(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY,
+        DataSourceWriteOptions.DEFAULT_KEYGENERATOR_CLASS_OPT_VAL)
+      val partitionPathField =
+        keyGeneratorClass match {
+          case "org.apache.hudi.keygen.CustomKeyGenerator" =>
+            partitionColumns.map(e => s"$e:SIMPLE").mkString(",")

Review comment:
       we can not simply put `SIMPLE` and `partitionBy` field together. Since 
when user use `CustomKeyGenerator ` and the partitionpath field is of timestamp 
type,  the str after the `partitionBy` field should be `TIMESTAMP`

##########
File path: 
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala
##########
@@ -181,16 +183,33 @@ object DataSourceWriteOptions {
   @Deprecated
   val DEFAULT_STORAGE_TYPE_OPT_VAL = COW_STORAGE_TYPE_OPT_VAL
 
-  def translateStorageTypeToTableType(optParams: Map[String, String]) : 
Map[String, String] = {
+  def translateOptParams(optParams: Map[String, String]): Map[String, String] 
= {
+    // translate StorageType to TableType
+    var newOptParams = optParams
     if (optParams.contains(STORAGE_TYPE_OPT_KEY) && 
!optParams.contains(TABLE_TYPE_OPT_KEY)) {
       log.warn(STORAGE_TYPE_OPT_KEY + " is deprecated and will be removed in a 
later release; Please use " + TABLE_TYPE_OPT_KEY)
-      optParams ++ Map(TABLE_TYPE_OPT_KEY -> optParams(STORAGE_TYPE_OPT_KEY))
-    } else {
-      optParams
+      newOptParams = optParams ++ Map(TABLE_TYPE_OPT_KEY -> 
optParams(STORAGE_TYPE_OPT_KEY))
     }
+    // translate the api partitionBy of spark DataFrameWriter to 
PARTITIONPATH_FIELD_OPT_KEY
+    if (optParams.contains(SparkDataSourceUtils.PARTITIONING_COLUMNS_KEY) && 
!optParams.contains(PARTITIONPATH_FIELD_OPT_KEY)) {
+      val partitionColumns = 
optParams.get(SparkDataSourceUtils.PARTITIONING_COLUMNS_KEY)
+        .map(SparkDataSourceUtils.decodePartitioningColumns)
+        .getOrElse(Nil)
+
+      val keyGeneratorClass = 
optParams.getOrElse(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY,
+        DataSourceWriteOptions.DEFAULT_KEYGENERATOR_CLASS_OPT_VAL)
+      val partitionPathField =
+        keyGeneratorClass match {
+          case "org.apache.hudi.keygen.CustomKeyGenerator" =>
+            partitionColumns.map(e => s"$e:SIMPLE").mkString(",")

Review comment:
       > @wangxianghu Thank you for your review. My opinion is this:In 
accordance with the habit of using Spark, the partition field value 
corresponding to partitionBy is the original value, so the default is to use 
SIMPLE. If we automatically infer whether to use TIMESTAMP based on the field 
type, the rules are not easy to determine. For example, if a field is long, we 
Do you need to convert to TIMESTAMP? If you want to convert, but the value is 
not a timestamp, an error will be reported, so SIMPLE is used by default. If 
you want to use TIMESTAMP, users can directly use 
`hoodie.datasource.write.partitionpath. field`Go to specify
   
   yes, I get your point. we'd better support both `SIMPLE` and `TIMESTAMP` 
type patitionpath in a unified way




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

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


Reply via email to