askldjd opened a new issue #4233:
URL: https://github.com/apache/hudi/issues/4233


   **Describe the problem you faced**
   
   Our source parquet file has an INT16 column that contains a mixture of 
positive and negative values. When we convert the parquet file to the hudi 
format, the INT16 column in the hudi parquet loses its 
[LogicalType](https://github.com/apache/parquet-format/blob/master/LogicalTypes.md)
 information and is interpreted as an INT32. That indirectly causes all 
negative values to be interpreted as unsigned value. (E.g. -2 becomes 65534).
   
   **To Reproduce**
   
   Here's the python code that demonstrate this behavior. I created a Pandas 
dataframe with a INT16 column called "short_column" and converted it to hudi 
through pyspark.
   
   ```py
   import pandas as pd
   from pyspark.sql import SparkSession
   
   spark = SparkSession.builder.master("local").getOrCreate()
   parquet_dir = "/tmp/test.parquet"
   
   data = pd.DataFrame({
       '_ts':pd.Series([100], dtype='int32'),
       'id':pd.Series([999], dtype='int32'),
       'short_column':pd.Series([-2], dtype='int16'),
   })
   
   data.to_parquet(parquet_dir)
   
   hudi_dir =  "/tmp/output_hudi"
   
   read_df = spark.read.parquet(parquet_dir)
   
   hudi_options = {
       "hoodie.table.name": 'test_table',
       "hoodie.datasource.write.recordkey.field": "id",
       "hoodie.datasource.write.precombine.field": "_ts",
       "hoodie.datasource.write.partitionpath.field": "",
       "hoodie.datasource.write.keygenerator.class": 
"org.apache.hudi.keygen.CustomKeyGenerator",
       "hoodie.upsert.shuffle.parallelism": 1500,
       "hoodie.insert.shuffle.parallelism": 1500,
       "hoodie.consistency.check.enabled": True,
       "hoodie.index.type": "BLOOM",
       "hoodie.index.bloom.num_entries": 60000,
       "hoodie.index.bloom.fpp": 0.000000001,
       "hoodie.cleaner.commits.retained": 2,
   }
   (       
       read_df.write.format("org.apache.hudi")
           .options(**hudi_options)
           .mode("overwrite")
           .save(hudi_dir)
   )
   ```
   
   **Expected behavior**
   Using **[parquet-tools](https://pypi.org/project/parquet-tools/),** I can 
extract the schema from the original and hudi parquet.
   
   The original "short_column" has the following definition:
   ```
   ############ Column(short_column) ############
   name: short_column
   path: short_column
   max_definition_level: 1
   max_repetition_level: 0
   physical_type: INT32
   logical_type: Int(bitWidth=16, isSigned=true)
   converted_type (legacy): INT_16
   ```
   The original "short_column" has the value -2.
   
   ```
   +-------+------+----------------+
   |   _ts |   id |   short_column |
   |-------+------+----------------|
   |   100 |  999 |             -2 |
   +-------+------+----------------+
   ```
   
   The hudi version has the following definition. You can see that the 
`logical_type` info has been lost.
   ```
   ############ Column(short_column) ############
   name: short_column
   path: short_column
   max_definition_level: 1
   max_repetition_level: 0
   physical_type: INT32
   logical_type: None
   converted_type (legacy): NONE
   ```
   
   The hudi parquet dump shows the following output, which demonstrate that 
"short_column" is mistranslated to 65534.
   ```
   parquet-tools show /tmp/output_hudi/*.parquet
   
+-----------------------+------------------------+----------------------+--------------------------+-------------------------------------------------------------------------+-------+------+----------------+
   |   _hoodie_commit_time |   _hoodie_commit_seqno |   _hoodie_record_key | 
_hoodie_partition_path   | _hoodie_file_name                                    
                   |   _ts |   id |   short_column |
   
|-----------------------+------------------------+----------------------+--------------------------+-------------------------------------------------------------------------+-------+------+----------------|
   |        20211206215931 |     20211206215931_0_1 |                  999 |    
                      | 
7665aecb-3efc-4e83-851b-002d9abfcd59-0_0-21-4510_20211206215931.parquet |   100 
|  999 |          65534 |
   
+-----------------------+------------------------+----------------------+--------------------------+-------------------------------------------------------------------------+-------+------+----------------+
   ```
   
   A clear and concise description of what you expected to happen.
   
   **Environment Description**
   
   * Hudi version : 0.9.0
   
   * Spark version : 
   - hudi-spark3-bundle_2.12
   - spark-avro_2.12
   
   * Running on Docker? (yes/no) : yes
   


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