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

   SparkSQL can not read the latest change data without execute "refresh table 
xxx" after write the data in datasource mode
   
   **To Reproduce**
   
   Steps to reproduce the behavior:
   
   1. run spark-shell and import class 
   ``` import org.apache.spark.sql.SaveMode ```
   2. create like this:
   ```
   spark.sql(
       s"""|CREATE TABLE IF NOT EXISTS `malawi`.`hudi_0_12_1_spark_test` (
           |     `id` INT
           |    ,`name` STRING
           |    ,`age` INT
           |    ,`sync_time` TIMESTAMP
           |) USING HUDI
           |TBLPROPERTIES (
           |     type = 'mor'
           |    ,primaryKey = 'id'
           |    ,preCombineField = 'sync_time'
           |    ,`hoodie.bucket.index.hash.field` = ''
           |    ,`hoodie.datasource.write.hive_style_partitioning` = 'false'
           |    
,`hoodie.table.keygenerator.class`='org.apache.hudi.keygen.ComplexKeyGenerator'
           |)
           |COMMENT 'hudi_0.12.1_test'""".stripMargin
   )
   
   spark.sql(
       s"""|create table `malawi`.`hudi_0_12_1_spark_test_rt`
           |using hudi
           |options(`hoodie.query.as.ro.table` = 'false')
           |location 
'hdfs://NameNodeService1/hoodie/leqee/malawi/hudi_0_12_1_spark_test';
           |""".stripMargin
   )
   ```
   3. make test data
   ```
   var dfData = spark.sql(
       s"""|select 1 as id,'name1' as name, 18 as age, now() as sync_time 
           | union all 
           |select 2 as id,'name2' as name, 22 as age, now() as sync_time 
           | union all 
           |select 3 as id,'name3' as name, 23 as age, now() as sync_time
           |""".stripMargin
   )
   
   var dfData2 = spark.sql(
       s"""|select 4 as id,'name1' as name, 18 as age, now() as sync_time
           |""".stripMargin
   )
   ```
   4. make hudi datasource options
   ```
   var hoodieProp = Map("hoodie.table.name" -> "hudi_0_12_1_spark_test", 
"hoodie.datasource.write.operation"->"upsert", 
"hoodie.datasource.write.recordkey.field" -> "id", 
"hoodie.datasource.write.keygenerator.class"->"org.apache.hudi.keygen.ComplexKeyGenerator",
 "hoodie.datasource.write.partitionpath.field"->"", 
"hoodie.datasource.write.precombine.field"->"sync_time", 
"hoodie.metadata.enable"->"true", "hoodie.upsert.shuffle.parallelism"->"10", 
"hoodie.embed.timeline.server"->"false")
   ```
   5. write data fist time
   ```
   
dfData.write.format("org.apache.hudi").options(hoodieProp).mode(SaveMode.Append).save("hdfs://xxx/malawi/hudi_0_12_1_spark_test")
   ```
   6. query in spark sql
   ```
   spark.sql(s"""|select *
                 | from (
                 |         select 'ori' as flag,a.* from 
`malawi`.`hudi_0_12_1_spark_test` a
                 |         union all
                 |         select '_rt' as flag,b.* from 
`malawi`.`hudi_0_12_1_spark_test_rt` b
                 |      ) t
                 |order by t.id asc, t.flag asc""".stripMargin
   ).show(false)
   ```
   
![image](https://user-images.githubusercontent.com/62231347/204456307-04f29a4e-6d4c-4c97-9429-32ebeb92f0c1.png)
   
   7. write data second time
   ```
   
dfData2.write.format("org.apache.hudi").options(hoodieProp).mode(SaveMode.Append).save("hdfs://xxx/malawi/hudi_0_12_1_spark_test")
   ```
   8. repeat step 6
   the data(id = 4) should be queried out in _rt table but it is not
   
![image](https://user-images.githubusercontent.com/62231347/204456452-4cf23bad-af3d-48a5-b2c9-1766e61b34ee.png)
   
   9. refresh table
   ```
   spark.sql("REFRESH TABLE `malawi`.`hudi_0_12_1_spark_test`")
   ```
   10. repeat step 6
   
   
![image](https://user-images.githubusercontent.com/62231347/204464598-e46e508f-6bd8-480d-a9fc-2141f83d455e.png)
   
   
   
   **Environment Description**
   
   * Hudi version : 0.12.1
   
   * Spark version : 3.1.3
   
   * Hive version : 3.1.1
   
   * Hadoop version : 3.1.0
   
   * Storage (HDFS/S3/GCS..) : HDFS
   
   * Running on Docker? (yes/no) : no
   
   


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