codope commented on issue #13233:
URL: https://github.com/apache/hudi/issues/13233#issuecomment-2845085241

   I can repro using the following script:
   ```
   wget 
https://repo1.maven.org/maven2/org/apache/hudi/hudi-utilities-bundle_2.12/0.15.0/hudi-utilities-bundle_2.12-0.15.0.jar
   
   mkdir -p /tmp/hudi_ts/{input,schema}
   cat <<EOF > /tmp/hudi_ts/input/data.json
   {"id": 1, "ts": 1720631224939}
   EOF
   
   
   cat <<EOF > /tmp/hudi_ts/schema/source.avsc
   {
     "type":"record","name":"Src",
     "fields":[
       {"name":"id","type":"long"},
       {"name":"ts","type":"long"}
     ]
   }
   EOF
   
   
   cat <<EOF > /tmp/hudi_ts/schema/target.avsc
   {
     "type":"record","name":"Tgt",
     "fields":[
       {"name":"id","type":"long"},
       {
         "name":"ts",
         "type":{
           "type":"long",
           "logicalType":"timestamp-millis"
         }
       }
     ]
   }
   EOF
   
   ./bin/spark-submit --jars 
~/hudi/packaging/hudi-spark-bundle/target/hudi-spark3.5-bundle_2.12-1.1.0-SNAPSHOT.jar
 \
     --class org.apache.hudi.utilities.streamer.HoodieStreamer  
~/hudi/packaging/hudi-utilities-bundle/target/hudi-utilities-bundle_2.12-1.1.0-SNAPSHOT.jar
 \
     --table-type COPY_ON_WRITE \
     --source-class org.apache.hudi.utilities.sources.JsonDFSSource \
     --target-base-path file:///tmp/hudi_ts/output \
     --target-table ts_test \
     --schemaprovider-class 
org.apache.hudi.utilities.schema.FilebasedSchemaProvider \
     --source-ordering-field ts \
     --op UPSERT \
     --hoodie-conf hoodie.datasource.write.recordkey.field=id \
     --hoodie-conf 
hoodie.streamer.schemaprovider.source.schema.file=file:///tmp/hudi_ts/schema/source.avsc
 \
     --hoodie-conf 
hoodie.streamer.schemaprovider.target.schema.file=file:///tmp/hudi_ts/schema/target.avsc
 \
     --hoodie-conf hoodie.streamer.source.dfs.root=file:///tmp/hudi_ts/input
   ```
   
   The results when reading the data:
   ```
   ./bin/spark-shell --jars 
~/hudi/packaging/hudi-spark-bundle/target/hudi-spark3.5-bundle_2.12-1.1.0-SNAPSHOT.jar
 \
   --conf 'spark.serializer=org.apache.spark.serializer.KryoSerializer' \
   --conf 
'spark.sql.catalog.spark_catalog=org.apache.spark.sql.hudi.catalog.HoodieCatalog'
 \
   --conf 
'spark.sql.extensions=org.apache.spark.sql.hudi.HoodieSparkSessionExtension' \
   --conf 'spark.kryo.registrator=org.apache.spark.HoodieSparkKryoRegistrar'
   
   
   val df = spark.read.parquet("file:///tmp/hudi_ts/output/")
   df.show(false)
   
   
+-------------------+---------------------+------------------+----------------------+------------------------------------------------------------------------+---+--------------------------+
   |_hoodie_commit_time|_hoodie_commit_seqno 
|_hoodie_record_key|_hoodie_partition_path|_hoodie_file_name                    
                                   |id |ts                        |
   
+-------------------+---------------------+------------------+----------------------+------------------------------------------------------------------------+---+--------------------------+
   |20250501133107326  |20250501133107326_0_0|1                 |               
       
|e0c06561-5b2a-45f3-ad07-f1d4bf650a56-0_0-26-29_20250501133107326.parquet|1  
|1970-01-21 03:27:11.224939|
   
+-------------------+---------------------+------------------+----------------------+------------------------------------------------------------------------+---+--------------------------+
   
   val hudiDf = spark.read.format("hudi").load("file:///tmp/hudi_ts/output/")
   hudiDf.show(false)
   
   
+-------------------+---------------------+------------------+----------------------+------------------------------------------------------------------------+---+--------------------------+
   |_hoodie_commit_time|_hoodie_commit_seqno 
|_hoodie_record_key|_hoodie_partition_path|_hoodie_file_name                    
                                   |id |ts                        |
   
+-------------------+---------------------+------------------+----------------------+------------------------------------------------------------------------+---+--------------------------+
   |20250501133107326  |20250501133107326_0_0|1                 |               
       
|e0c06561-5b2a-45f3-ad07-f1d4bf650a56-0_0-26-29_20250501133107326.parquet|1  
|1970-01-21 03:27:11.224939|
   
+-------------------+---------------------+------------------+----------------------+------------------------------------------------------------------------+---+--------------------------+
   
   
   scala> df.selectExpr("cast(ts as long) as raw_ts").show(false)
   +-------+
   |raw_ts |
   +-------+
   |1720631|
   +-------+
   ```
   
   Note that the long value in parquet file itself is correct. Schema is 
incorrect.
   ```
   (base) sagars@Sagars-MacBook-Pro spark-3.5.2-bin-hadoop3 % parquet-tools cat 
/tmp/hudi_ts/output/e0c06561-5b2a-45f3-ad07-f1d4bf650a56-0_0-26-29_20250501133107326.parquet
   _hoodie_commit_time = 20250501133107326
   _hoodie_commit_seqno = 20250501133107326_0_0
   _hoodie_record_key = 1
   _hoodie_partition_path =
   _hoodie_file_name = 
e0c06561-5b2a-45f3-ad07-f1d4bf650a56-0_0-26-29_20250501133107326.parquet
   id = 1
   ts = 1720631224939
   
   (base) sagars@Sagars-MacBook-Pro spark-3.5.2-bin-hadoop3 % parquet-tools 
schema 
/tmp/hudi_ts/output/e0c06561-5b2a-45f3-ad07-f1d4bf650a56-0_0-26-29_20250501133107326.parquet
   message Tgt {
     optional binary _hoodie_commit_time (STRING);
     optional binary _hoodie_commit_seqno (STRING);
     optional binary _hoodie_record_key (STRING);
     optional binary _hoodie_partition_path (STRING);
     optional binary _hoodie_file_name (STRING);
     required int64 id;
     required int64 ts (TIMESTAMP(MICROS,true));
   }
   ```
   


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