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

   **_Tips before filing an issue_**
   hudi 0.14.0 hudi-flink-bundle The COW/MOR table type writes timestamp data, 
and the time zone for writing data when read.utc-timezone=false is set is still 
the UTC time zone.
   AvroToRowDataConverters and RowDataToAvroConverters timestamp time zone 
conversion is hardcoded to UTC time zone.
   
   **Describe the problem you faced**
   1. hudi-flink1.13-bundle-0.14.0-rc1 write timestamp does not support 
configuration time zone type
   2. The read.utc-timezone attribute only takes effect when the data is read
   
   
   **To Reproduce**
   
   Steps to reproduce the behavior:
   
   1. ./bin/sql-client.sh embedded -j hudi-flink1.13-bundle-0.14.0-rc1.jar shell
   2. When setting read.utc-timezone=true, it is normal to write query 
timestamp data
   3. When setting read.utc-timezone= false to write data, the query time will 
be -8 hours
   ```sql
   Flink SQL> select LOCALTIMESTAMP as tm, timestamph from 
hudi_mor_all_datatype_2 where inth=44;
   +----+-------------------------+-------------------------+
   | op |                      tm |              timestamph |
   +----+-------------------------+-------------------------+
   | +I | 2023-08-11 10:36:38.793 | 2023-08-11 03:10:17.267 |
   +----+-------------------------+-------------------------+
   ```
   
   **Expected behavior**
   
   hudi-flink1.13-bundle supports writing timestamps in non-UTC time zones in a 
configurable way
   
   **Environment Description**
   
   * Hudi version : 0.14.0
   
   * Spark version : 3.2.0
   
   * Flink version: 1.13.2
   
   * Hive version : 1.11.1
   
   * Hadoop version : 3.x
   
   * Storage (HDFS/S3/GCS..) : HDFS
   
   * Running on Docker? (yes/no) : no
   
   **Related code location**
   ```java
   public class AvroToRowDataConverters {
   // ...
     private static AvroToRowDataConverter createTimestampConverter(int 
precision) {
       // ...
       return avroObject -> {
         final Instant instant;
         if (avroObject instanceof Long) {
           instant = Instant.EPOCH.plus((Long) avroObject, chronoUnit);
         } else if (avroObject instanceof Instant) {
           instant = (Instant) avroObject;
         } else {
           JodaConverter jodaConverter = JodaConverter.getConverter();
           if (jodaConverter != null) {
             // joda time has only millisecond precision
             instant = 
Instant.ofEpochMilli(jodaConverter.convertTimestamp(avroObject));
           } else {
             throw new IllegalArgumentException(
                 "Unexpected object type for TIMESTAMP logical type. Received: 
" + avroObject);
           }
         }
         // TODO:Hardcoded to UTC here
         return TimestampData.fromInstant(instant);
       };
     }
   // ...
   }
   
   public class RowDataToAvroConverters {
   // ...
     public static RowDataToAvroConverter createConverter(LogicalType type) {
     // ...
             case TIMESTAMP_WITHOUT_TIME_ZONE:
           final int precision = DataTypeUtils.precision(type);
           if (precision <= 3) {
             converter =
                 new RowDataToAvroConverter() {
                   private static final long serialVersionUID = 1L;
   
                   @Override
                   public Object convert(Schema schema, Object object) {
                     // TODO:Hardcoded to UTC here
                     return ((TimestampData) object).toInstant().toEpochMilli();
                   }
                 };
           } else if (precision <= 6) {
             converter =
                 new RowDataToAvroConverter() {
                   private static final long serialVersionUID = 1L;
   
                   @Override
                   public Object convert(Schema schema, Object object) {
                     // TODO:Hardcoded to UTC here
                     Instant instant = ((TimestampData) object).toInstant();
                     return  
Math.addExact(Math.multiplyExact(instant.getEpochSecond(), 1000_000), 
instant.getNano() / 1000);
                   }
                 };
           } else {
             throw new UnsupportedOperationException("Unsupported timestamp 
precision: " + precision);
           }
           break;
     // ...
     }
   // ...
   }
   ```


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