nsivabalan commented on issue #4784:
URL: https://github.com/apache/hudi/issues/4784#issuecomment-1036330505


   I could not reproduce the partitioning issue you are facing.
   
   local spark shell
   ```
   
   import java.sql.Timestamp
   import spark.implicits._
   
   import org.apache.hudi.QuickstartUtils._
   import scala.collection.JavaConversions._
   import org.apache.spark.sql.SaveMode._
   import org.apache.hudi.DataSourceReadOptions._
   import org.apache.hudi.DataSourceWriteOptions._
   import org.apache.hudi.config.HoodieWriteConfig._
   
   
   val df1 = Seq(
           ("row1", 1, "part1" ,1578283932000L ),
           ("row2", 1, "part1", 1578283942000L)
         ).toDF("row", "ppath", "preComb","eventTime")
   
   
    df1.write.format("hudi").
           options(getQuickstartWriteConfigs).
           option(PRECOMBINE_FIELD_OPT_KEY, "preComb").
           option(RECORDKEY_FIELD_OPT_KEY, "row").
           option(PARTITIONPATH_FIELD_OPT_KEY, 
"preComb:simple,ppath:timestamp").
           
option("hoodie.datasource.write.keygenerator.class","org.apache.hudi.keygen.CustomKeyGenerator").
           
option("hoodie.deltastreamer.keygen.timebased.timestamp.type","EPOCHMILLISECONDS").
           
option("hoodie.deltastreamer.keygen.timebased.output.dateformat","yyyy-MM-dd").
           option("hoodie.deltastreamer.keygen.timebased.timezone","GMT+8:00").
           option(TABLE_NAME, "timestamp_tbl4").
           mode(Overwrite).
           save("/tmp/hudi_timestamp_tbl4")
   
   
   val hudiDF4 = spark.read.format("hudi").load("/tmp/hudi_timestamp_tbl4")
   hudiDF4.registerTempTable("tbl4")
   spark.sql("describe tbl4").show()
   spark.sql("select * from tbl4 limit 3").show()
   
   ```
   
   Output
   ```
   spark.sql("select * from tbl4 limit 3").show()
   
+-------------------+--------------------+------------------+----------------------+--------------------+----+-------------+-------+-----+
   
|_hoodie_commit_time|_hoodie_commit_seqno|_hoodie_record_key|_hoodie_partition_path|
   _hoodie_file_name| row|    eventTime|preComb|ppath|
   
+-------------------+--------------------+------------------+----------------------+--------------------+----+-------------+-------+-----+
   |  20220211102107283|20220211102107283...|              row1|      
part1/1970-01-01|dfc23d4b-8177-4fa...|row1|1578283932000|  part1|    0|
   |  20220211102107283|20220211102107283...|              row2|      
part1/1970-01-01|dfc23d4b-8177-4fa...|row2|1578283942000|  part1|    0|
   
+-------------------+--------------------+------------------+----------------------+--------------------+----+-------------+-------+-----+
   ```
   
   specifically values for _hoodie_partition_path are 
   part1/1970-01-01
   
   2: if you disable hive style partitioning, you may not see the "fieldname=". 
But if you want to enable it, don't think hudi allows changing the fieldname 
for partition paths. 
   3: I am not sure on how to leverage partition pruning for custom key gen 
based tables. @xushiyan @YannByron @bhasudha : do you folks have any pointers 
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


Reply via email to