parisni opened a new issue, #11599:
URL: https://github.com/apache/hudi/issues/11599

   hudi 0.14.1
   spark 3.3.x
   ------------
   I did not find any mention of hudi promotion type in the documentation
   
   apparently promotion type is not supported #5519 : 
   >  there's no way this could be reliably supported w/in Hudi currently
   
   Now from my tests below:
   - hudi supports several promotion types with no hive-sync involved
   - [hive/glue-sync only supports int->bigint and 
float->double](https://github.com/apache/hudi/blob/dbfe8b23c0b4f160b26379053873cfc2a46acef4/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/HiveSchemaUtil.java#L430-L436)
   
   
   So:
   1. what is the state of promotion type support ?
   2. is it safe ?
   
   
   ```
   tableName = "hudi_promotion"
   basePath = "/tmp/hudi_nested"
   from pyspark.sql.functions import expr
   
   NB_RECORDS=10_000 # generate enough records to get multiple parquet files
   hudi_options = {
       "hoodie.table.name": tableName,
       "hoodie.datasource.write.recordkey.field": "event_id",
       "hoodie.datasource.write.partitionpath.field": "event_date",
       "hoodie.datasource.write.table.name": tableName,
       "hoodie.datasource.write.operation": "bulk_insert",
       "hoodie.datasource.write.precombine.field": "version",
       "hoodie.upsert.shuffle.parallelism": 1,
       "hoodie.insert.shuffle.parallelism": 1,
       "hoodie.delete.shuffle.parallelism": 1,
       "hoodie.datasource.write.keygenerator.class": 
"org.apache.hudi.keygen.ComplexKeyGenerator",
       "hoodie.datasource.write.hive_style_partitioning": "true",
       "hoodie.datasource.hive_sync.database": "default",
       "hoodie.datasource.hive_sync.table": tableName,
       "hoodie.datasource.hive_sync.mode": "hms",
       "hoodie.datasource.hive_sync.enable": "false",
       "hoodie.datasource.hive_sync.partition_fields": "event_date",
       "hoodie.datasource.write.keygenerator.class": 
"org.apache.hudi.keygen.ComplexKeyGenerator",
       "hoodie.datasource.hive_sync.partition_extractor_class": 
"org.apache.hudi.hive.MultiPartKeysValueExtractor",
       "hoodie.metadata.enable": "true",
       "hoodie.parquet.max.file.size": 1 * 1024 * 1024,
   }
   spark.sql("drop table if exists hudi_promotion").show()
   
   # INIT TABLE w/ INT, FLOAT and INT
   df = (
   spark.range(1,NB_RECORDS).withColumn("event_id", expr("id"))
       .withColumn("event_date", expr("current_date()"))
       .withColumn("version", expr("current_date()"))
       .withColumn("int_to_bigint", expr("cast(1 as int)"))
       .withColumn("float_to_double", expr("cast(1.1 as float)"))
       .withColumn("int_to_float", expr("cast(1 as int)"))
   )
   
(df.write.format("hudi").options(**hudi_options).mode("overwrite").save(basePath))
   spark.read.format("hudi").load(basePath).printSchema()
   #spark.sql("desc table hudi_promotion").show()
   
   # NOW TURN to BIGINT, DOUBLE and FLOAT in an other partition
   df = (
   spark.range(1,2).withColumn("event_id", expr("id"))
       .withColumn("event_date", expr("current_date() + 1"))
       .withColumn("version", expr("current_date()"))
       .withColumn("int_to_bigint", expr("cast(1 as bigint)"))
       .withColumn("float_to_double", expr("cast(1.1 as double)"))
       .withColumn("int_to_float", expr("cast(1.1 as float)"))
   )
   
(df.write.format("hudi").options(**hudi_options).mode("append").save(basePath))
   spark.read.format("hudi").load(basePath).printSchema()
   #spark.sql("desc table hudi_promotion").show()
   
   
   # NOW upsert the previous partition
   df = (
   spark.range(1,2).withColumn("event_id", expr("id"))
       .withColumn("event_date", expr("current_date()"))
       .withColumn("version", expr("current_date()"))
       .withColumn("int_to_bigint", expr("cast(1 as bigint)"))
       .withColumn("float_to_double", expr("cast(1.1 as double)"))
       .withColumn("int_to_float", expr("cast(1.1 as float)"))
   )
   
   
(df.write.format("hudi").options(**hudi_options).mode("append").save(basePath))
   spark.read.format("hudi").load(basePath).printSchema()
   #spark.sql("desc table hudi_promotion").show()
   ```


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

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

Reply via email to