ganczarek commented on issue #8756:
URL: https://github.com/apache/hudi/issues/8756#issuecomment-1943989873

   @ad1happy2go. Thank you for trying to reproduce the problem.
   
   I know this issue is quite old, but I don't have a way to test it with newer 
versions of Hudi or Hive right now. However, I can still reproduce the problem 
with Hudi 0.12.2 and Hive 3.1.3 (AWS EMR 6.10.0).
   
   ```
   $ hive --version
   Hive 3.1.3-amzn-3
   Git 
file:///codebuild/output/src425131955/src/build/hive/rpm/BUILD/apache-hive-3.1.3-amzn-3-src
 -r Unknown
   Compiled by release on Tue Feb 21 19:18:24 UTC 2023
   From source with checksum 9c4721677b95382c13a0fc6adb9dc41c
   ```
   
   What version of Hive did you use?
   
   The full code with all imports:
   
   ```scala
   import org.apache.hudi.config.HoodieWriteConfig
   import org.apache.hudi.hive.{HiveSyncConfig, NonPartitionedExtractor}
   import org.apache.hudi.keygen.constant.KeyGeneratorOptions
   import org.apache.hudi.sync.common.HoodieSyncConfig
   import org.apache.spark.sql.SaveMode
   
   import java.sql.Timestamp
   import java.time.{LocalDateTime, ZoneOffset}
   
   val hiveUrl = "jdbc:hive2://localhost:10000"
   val s3TablePath = "s3://<redacted>/test_table"
   
   case class TestClass(bool_value: Boolean, timestamp_value: Timestamp)
   case class TestClass2(bool_value: String, timestamp_value: String)
   
   Seq(
     ("1", "a", TestClass(true, 
Timestamp.from(LocalDateTime.now().toInstant(ZoneOffset.UTC)))),
     ("1", "b", TestClass(false, 
Timestamp.from(LocalDateTime.now().toInstant(ZoneOffset.UTC))))
   ).toDF("partition", "id", "detail")
     .write
     .format("org.apache.hudi")
     .options(
       Map(
         HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key -> "id",
         KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key -> "id",
         HoodieWriteConfig.TBL_NAME.key -> "test_table",
         HiveSyncConfig.HIVE_SYNC_ENABLED.key -> "true",
         HiveSyncConfig.HIVE_URL.key -> hiveUrl,
         HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.key -> "true",
         HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> "test_db",
         HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> 
classOf[NonPartitionedExtractor].getName,
         HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> "test_table",
         KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key -> "partition",
         KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key -> "true",
       )
     )
     .mode(SaveMode.Overwrite)
     .save(s3TablePath)
   
   spark.read.format("org.apache.hudi").load(s3TablePath).show(false)
   /**
   
+-------------------+---------------------+------------------+----------------------+------------------------------------------------------------------------+---+--------------------------------+---------+
   |_hoodie_commit_time|_hoodie_commit_seqno 
|_hoodie_record_key|_hoodie_partition_path|_hoodie_file_name                    
                                   |id |detail                          
|partition|
   
+-------------------+---------------------+------------------+----------------------+------------------------------------------------------------------------+---+--------------------------------+---------+
   |20240214124653234  |20240214124653234_0_0|a                 |partition=1    
       
|4a12398f-e664-48af-a397-28d5741a0256-0_0-76-74_20240214124653234.parquet|a  
|{true, 2024-02-14 12:46:52.992} |1        |
   |20240214124653234  |20240214124653234_0_1|b                 |partition=1    
       
|4a12398f-e664-48af-a397-28d5741a0256-0_0-76-74_20240214124653234.parquet|b  
|{false, 2024-02-14 12:46:52.992}|1        |
   
+-------------------+---------------------+------------------+----------------------+------------------------------------------------------------------------+---+--------------------------------+---------+
   */
   
   Seq(
     ("2", "c", TestClass2("str1", LocalDateTime.now().toString)),
     ("2", "d", TestClass2("str2", LocalDateTime.now().toString))
   ).toDF("partition", "id", "detail")
     .write
     .format("org.apache.hudi")
     .options(
       Map(
         HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key -> "id",
         KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key -> "id",
         HoodieWriteConfig.TBL_NAME.key -> "test_table",
         HiveSyncConfig.HIVE_SYNC_ENABLED.key -> "true",
         HiveSyncConfig.HIVE_URL.key -> hiveUrl,
         HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.key -> "true",
         HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> "test_db",
         HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> 
classOf[NonPartitionedExtractor].getName,
         HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> "test_table",
         KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key -> "partition",
         KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key -> "true",
       )
     )
     .mode(SaveMode.Append)
     .save(s3TablePath)
   )
   
   spark.read.format("org.apache.hudi").load(s3TablePath).show(false)
   // java.lang.UnsupportedOperationException: 
org.apache.parquet.column.values.dictionary.PlainValuesDictionary$PlainLongDictionary
   ```
   
   Table schema in Hive after executing above snippet is:
   ```text
   hive> SHOW CREATE TABLE test_db.test_table;
   OK
   CREATE EXTERNAL TABLE `test_db.test_table`(
     `_hoodie_commit_time` string,
     `_hoodie_commit_seqno` string,
     `_hoodie_record_key` string,
     `_hoodie_partition_path` string,
     `_hoodie_file_name` string,
     `id` string,
     `detail` struct<bool_value:string,timestamp_value:string>)
   PARTITIONED BY (
     `partition` string)
   ROW FORMAT SERDE
     'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
   WITH SERDEPROPERTIES (
     'hoodie.query.as.ro.table'='false',
     'path'='s3://<redacted>/test_table')
   STORED AS INPUTFORMAT
     'org.apache.hudi.hadoop.HoodieParquetInputFormat'
   OUTPUTFORMAT
     'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
   LOCATION
     's3://<redacted>/test_table'
   TBLPROPERTIES (
     'bucketing_version'='2',
     'last_commit_time_sync'='20240214133801211',
     'last_modified_by'='hive',
     'last_modified_time'='1707917907',
     'spark.sql.create.version'='3.3.1-amzn-0',
     'spark.sql.sources.provider'='hudi',
     'spark.sql.sources.schema.numPartCols'='1',
     'spark.sql.sources.schema.numParts'='1',
     
'spark.sql.sources.schema.part.0'='{"type":"struct","fields":[{"name":"_hoodie_commit_time","type":"string","nullable":true,"metadata":{}},{"name":"_hoodie_commit_seqno","type":"string","nullable":true,"metadata":{}},{"name":"_hoodie_record_key","type":"string","nullable":true,"metadata":{}},{"name":"_hoodie_partition_path","type":"string","nullable":true,"metadata":{}},{"name":"_hoodie_file_name","type":"string","nullable":true,"metadata":{}},{"name":"id","type":"string","nullable":true,"metadata":{}},{"name":"detail","type":{"type":"struct","fields":[{"name":"bool_value","type":"string","nullable":true,"metadata":{}},{"name":"timestamp_value","type":"string","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"partition","type":"string","nullable":true,"metadata":{}}]}',
     'spark.sql.sources.schema.partCol.0'='partition',
     'transient_lastDdlTime'='1707917907')
   ```


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