[ 
https://issues.apache.org/jira/browse/HUDI-8818?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17922248#comment-17922248
 ] 

Shawn Chang commented on HUDI-8818:
-----------------------------------

I found this issue can also happen on Hudi 0.14 tables even if the table was 
later upgraded to table version 8 with a Hudi 1.0 writer. Should we reopen this 
Jira and make sure this doesn't happen with upgraded 0.14.0 table?

 

Reproduction steps: 
{code:java}
// use Hudi 0.14.0 to create the table 
spark-shell \--jars /usr/lib/hudi/hudi-spark-bundle.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"  

// scala script I used to create the table
import org.apache.hudi.DataSourceWriteOptions
import org.apache.spark.sql.SaveMode
val df1 = Seq( (100, "2015-01-01", "event_name_900", 
"2015-01-01T13:51:39.340396Z", "type1"), (101, "2015-01-01", "event_name_546", 
"2015-01-01T12:14:58.597216Z", "type2"), (102, "2015-01-01", "event_name_345", 
"2015-01-01T13:51:40.417052Z", "type3"), (103, "2015-01-01", "event_name_234", 
"2015-01-01T13:51:40.519832Z", "type4"), (104, "2015-01-01", "event_name_123", 
"2015-01-01T12:15:00.512679Z", "type1"), (105, "2015-01-01", "event_name_678", 
"2015-01-01T13:51:42.248818Z", "type2"), (106, "2015-01-01", "event_name_890", 
"2015-01-01T13:51:44.735360Z", "type3"), (107, "2015-01-01", "event_name_944", 
"2015-01-01T13:51:45.019544Z", "type4"), (108, "2015-01-01", "event_name_456", 
"2015-01-01T13:51:45.208007Z", "type1"), (109, "2015-01-01", "event_name_567", 
"2015-01-01T13:51:45.369689Z", "type2"), (110, "2015-01-01", "event_name_789", 
"2015-01-01T12:15:05.664947Z", "type3"), (111, "2015-01-01", "event_name_322", 
"2015-01-01T13:51:47.388239Z", "type4") ).toDF("event_id", "event_date", 
"event_name", "event_ts", "event_type")

val r = scala.util.Randomval 
num =  r.nextInt(99999)
var tableName = "yxchang_hudi_mor_simple_14_" + num
var tablePath = "<s3_bucket>" + tableName + "/"
df1.write.format("hudi") 
.option("hoodie.metadata.enable", "true") 
.option("hoodie.table.name", tableName)
.option("hoodie.datasource.write.operation", "insert") // use insert
.option("hoodie.datasource.write.table.type", "MERGE_ON_READ") 
.option("hoodie.datasource.write.recordkey.field", "event_id,event_date") 
.option("hoodie.datasource.write.partitionpath.field", "event_type")  
.option("hoodie.datasource.write.precombine.field", "event_ts") 
.option("hoodie.datasource.write.keygenerator.class", 
"org.apache.hudi.keygen.ComplexKeyGenerator") 
.option("hoodie.datasource.hive_sync.enable", "true") 
.option("hoodie.datasource.meta.sync.enable", "true") 
.option("hoodie.datasource.hive_sync.mode", "hms") 
.option("hoodie.datasource.hive_sync.database", "yxchang_nolf") 
.option("hoodie.datasource.hive_sync.table", tableName) 
.option("hoodie.datasource.hive_sync.partition_fields", "event_type") 
.option("hoodie.datasource.hive_sync.partition_extractor_class", 
"org.apache.hudi.hive.MultiPartKeysValueExtractor") 
.mode(SaveMode.Append) 
.save(tablePath)


// use Hudi 1.0 and DF to upgrade the table version to 8

val appendDf = Seq( (142, "2015-01-02", "event_name_922", 
"2015-01-01T13:51:39.340396Z", "type1"), (143, "2015-01-03", "event_name_533", 
"2015-01-01T12:14:58.597216Z", "type2"), (124, "2015-01-04", "event_name_344", 
"2015-01-01T13:51:40.417052Z", "type3"), (125, "2015-01-05", "event_name_266", 
"2015-01-01T13:51:40.519832Z", "type4"), (126, "2015-01-06", "event_name_177", 
"2015-01-01T12:15:00.512679Z", "type1"), (127, "2015-01-07", "event_name_688", 
"2015-01-01T13:51:42.248818Z", "type2"), (128, "2015-01-08", "event_name_891", 
"2015-01-01T13:51:44.735360Z", "type3"), (129, "2015-01-09", "event_name_945", 
"2015-01-01T13:51:45.019544Z", "type4"), (120, "2015-01-10", "event_name_450", 
"2015-01-01T13:51:45.208007Z", "type1"), (131, "2015-01-11", "event_name_562", 
"2015-01-01T13:51:45.369689Z", "type2"), (132, "2015-01-12", "event_name_786", 
"2015-01-01T12:15:05.664947Z", "type3"), (133, "2015-01-13", "event_name_328", 
"2015-01-01T13:51:47.388239Z", "type4") ).toDF("event_id", "event_date", 
"event_name", "event_ts", "event_type")
// Hudi 1.0
var tableName = "yxchang_hudi_mor_simple_14_31461"
var tablePath = "<s3_bucket>/yxchang_hudi_mor_simple_14_31461/"
appendDf.write.format("hudi").option("hoodie.metadata.enable", "true")
.option("hoodie.table.name", tableName)
.option("hoodie.datasource.write.operation", "insert") // use insert
.option("hoodie.datasource.write.table.type", "MERGE_ON_READ")
.option("hoodie.datasource.write.recordkey.field", "event_id,event_date")
.option("hoodie.datasource.write.partitionpath.field", "event_type") 
.option("hoodie.datasource.write.precombine.field", "event_ts")
.option("hoodie.datasource.write.keygenerator.class", 
"org.apache.hudi.keygen.ComplexKeyGenerator")
.option("hoodie.datasource.hive_sync.enable", "true")
.option("hoodie.datasource.meta.sync.enable", "true")
.option("hoodie.datasource.hive_sync.mode", "hms")
.option("hoodie.datasource.hive_sync.database", "yxchang_nolf")
.option("hoodie.datasource.hive_sync.table", tableName)
.option("hoodie.datasource.hive_sync.partition_fields", "event_type")
.option("hoodie.datasource.hive_sync.partition_extractor_class", 
"org.apache.hudi.hive.MultiPartKeysValueExtractor")
.mode(SaveMode.Append)
.save(tablePath)


// use spark-sql + hudi 1.0 to insert more data
spark-sql \--packages org.apache.hudi:hudi-spark3.5-bundle_2.12:1.0.0 \--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'

INSERT INTO yxchang_hudi_mor_simple_14_31461 VALUES  (133, '2015-01-13', 
'event_name_328', '2015-01-01T13:51:47.388239Z', 'type4');

{code}
Exception:
{code:java}
Config conflict(key     current value   existing value):hoodie.database.name:   
yxchang_nolforg.apache.hudi.exception.HoodieException: Config conflict(key      
current value   existing value):hoodie.database.name:   yxchang_nolf    at 
org.apache.hudi.HoodieWriterUtils$.validateTableConfig(HoodieWriterUtils.scala:256)
  at 
org.apache.hudi.HoodieSparkSqlWriterInternal.writeInternal(HoodieSparkSqlWriter.scala:245)
   at 
org.apache.hudi.HoodieSparkSqlWriterInternal.$anonfun$write$1(HoodieSparkSqlWriter.scala:190)
        at 
org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:108)
      at 
org.apache.spark.sql.execution.SQLExecution$.withTracker(SQLExecution.scala:384)
     at 
org.apache.spark.sql.execution.SQLExecution$.executeQuery$1(SQLExecution.scala:157)
  at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$10(SQLExecution.scala:220)
  at 
org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:108)
      at 
org.apache.spark.sql.execution.SQLExecution$.withTracker(SQLExecution.scala:384)
     at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$9(SQLExecution.scala:220)
   at 
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:405)
   at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:219)
   at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:901) at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:83)
       at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:74)
       at 
org.apache.spark.sql.adapter.BaseSpark3Adapter.sqlExecutionWithNewExecutionId(BaseSpark3Adapter.scala:105)
   at 
org.apache.hudi.HoodieSparkSqlWriterInternal.write(HoodieSparkSqlWriter.scala:212)
   at 
org.apache.hudi.HoodieSparkSqlWriter$.write(HoodieSparkSqlWriter.scala:127)  at 
org.apache.spark.sql.hudi.command.InsertIntoHoodieTableCommand$.run(InsertIntoHoodieTableCommand.scala:101)
  at 
org.apache.spark.sql.hudi.command.InsertIntoHoodieTableCommand.run(InsertIntoHoodieTableCommand.scala:62)
 {code}

> Hudi 1.0 cannot use SQL to write older versioned Hudi table
> -----------------------------------------------------------
>
>                 Key: HUDI-8818
>                 URL: https://issues.apache.org/jira/browse/HUDI-8818
>             Project: Apache Hudi
>          Issue Type: Sub-task
>            Reporter: Shawn Chang
>            Assignee: Davis Zhang
>            Priority: Blocker
>             Fix For: 1.0.1
>
>          Time Spent: 1.5h
>  Remaining Estimate: 0h
>
> When using Hudi 1.0 + Spark SQL to write a table created by Hudi 0.14 + Spark 
> 3.5.0 using Spark dataframe API, we noticed that the INSERT query would fail 
> to database config conflict
> {code:java}
> Config conflict(key    current value    existing value):
> hoodie.database.name:    yxchang_nolf
> org.apache.hudi.exception.HoodieException: Config conflict(key    current 
> value    existing value):
> hoodie.database.name:    yxchang_nolf
>     at 
> org.apache.hudi.HoodieWriterUtils$.validateTableConfig(HoodieWriterUtils.scala:256)
>     at 
> org.apache.hudi.HoodieSparkSqlWriterInternal.writeInternal(HoodieSparkSqlWriter.scala:245)
>     at 
> org.apache.hudi.HoodieSparkSqlWriterInternal.$anonfun$write$1(HoodieSparkSqlWriter.scala:190)
> {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to