Hi Ji, 
Moving this discussion to https://github.com/apache/hudi/issues/2063 which you 
have opened. I have added a possible workaround in the comments. Please try it 
out and respond in the issue. 
Thanks,Balaji.V

    On Monday, September 7, 2020, 10:11:13 AM PDT, Jl Liu (cadl) 
<ctrlaltdelete...@gmail.com> wrote:  
 
 Thanks~ 

I got another question about schema evolution. I don’t found document on 
homepage and wiki. If I change type from INT to LONG, will Audi overwrite total 
parquet files of the partition? 

I disable schema compatibility check and write LONG type data to existed INT 
type hudi table successfully, but got “Parquet column cannot be converted in 
file xxx.parquet. Column: [xxx], Expected: int, Found: INT64” error on read. It 
seems like some parquet files with different schema stored in the same 
directory, I can’t read them together.



> 2020年9月8日 上午12:30,Sivabalan <n.siv...@gmail.com> 写道:
> 
> Actually, I guess it is a bug in hudi. reader and writer schema arguments
> are called wrongly. (reader is sent for writer and writer is sent for
> reader). Will file a bug. Then, as you expect, INT should be evolvable to
> LONG, where as vice versa is incompatible.
> 
> 
> On Mon, Sep 7, 2020 at 12:17 PM Sivabalan <n.siv...@gmail.com> wrote:
> 
>> Hudi relies on avro's Schema compatability check. Looks like as per avro
>> SchemaCompatability, INT can't be evolved to a LONG, but LONG to INT is
>> allowed.
>> 
>> Check line no 339 here
>> <https://github.com/apache/avro/blob/master/lang/java/avro/src/main/java/org/apache/avro/SchemaCompatibility.java>
>> .
>> Also, check their test case here
>> <https://github.com/apache/avro/blob/master/lang/java/avro/src/test/java/org/apache/avro/TestSchemaCompatibilityTypeMismatch.java>
>>  at
>> line 44.
>> 
>> 
>> 
>> On Mon, Sep 7, 2020 at 12:02 PM Prashant Wason <pwa...@uber.com.invalid>
>> wrote:
>> 
>>> Yes, the schema change looks fine. That would mean its an issue with the
>>> schema compatibility checker. The are explicit checks for such cases so
>>> can't say where the issue lies.
>>> 
>>> I am out on a vacation this week. I will look into this as soon as I am
>>> back.
>>> 
>>> Thanks
>>> Prashant
>>> 
>>> On Sun, Sep 6, 2020, 11:18 AM Vinoth Chandar <vin...@apache.org> wrote:
>>> 
>>>> That does sound like a backwards compatible change.
>>>> @prashant , any ideas here? (since you have the best context on the
>>> schema
>>>> validation checks)
>>>> 
>>>> On Thu, Sep 3, 2020 at 8:12 PM cadl <ctrlaltdelete...@gmail.com> wrote:
>>>> 
>>>>> Hi All,
>>>>> 
>>>>> I want to change the type of one column in my COW table, from int to
>>>> long.
>>>>> When I set “hoodie.avro.schema.validate = true” and upsert new data
>>> with
>>>>> long type, I got a “Failed upsert schema compatibility check” error.
>>>> Dose
>>>>> it break backwards compatibility? If I disable
>>>> hoodie.avro.schema.validate,
>>>>> I can upsert and read normally.
>>>>> 
>>>>> 
>>>>> code demo:
>>> https://gist.github.com/cadl/be433079747aeea88c9c1f45321cc2eb
>>>>> 
>>>>> stacktrace:
>>>>> 
>>>>> 
>>>>> org.apache.hudi.exception.HoodieUpsertException: Failed upsert schema
>>>>> compatibility check.
>>>>>  at
>>>>> 
>>>> 
>>> org.apache.hudi.table.HoodieTable.validateUpsertSchema(HoodieTable.java:572)
>>>>>  at
>>>>> 
>>>> 
>>> org.apache.hudi.client.HoodieWriteClient.upsert(HoodieWriteClient.java:190)
>>>>>  at
>>>>> 
>>>> 
>>> org.apache.hudi.DataSourceUtils.doWriteOperation(DataSourceUtils.java:260)
>>>>>  at
>>>>> 
>>>> 
>>> org.apache.hudi.HoodieSparkSqlWriter$.write(HoodieSparkSqlWriter.scala:169)
>>>>>  at
>>>> org.apache.hudi.DefaultSource.createRelation(DefaultSource.scala:125)
>>>>>  at
>>>>> 
>>>> 
>>> org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand.run(SaveIntoDataSourceCommand.scala:45)
>>>>>  at
>>>>> 
>>>> 
>>> org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70)
>>>>>  at
>>>>> 
>>>> 
>>> org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68)
>>>>>  at
>>>>> 
>>>> 
>>> org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:86)
>>>>>  at
>>>>> 
>>>> 
>>> org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
>>>>>  at
>>>>> 
>>>> 
>>> org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
>>>>>  at
>>>>> 
>>>> 
>>> org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
>>>>>  at
>>>>> 
>>>> 
>>> org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
>>>>>  at
>>>>> 
>>>> 
>>> org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
>>>>>  at
>>>> org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
>>>>>  at
>>>>> 
>>>> 
>>> org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:80)
>>>>>  at
>>>>> 
>>>> 
>>> org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:80)
>>>>>  at
>>>>> 
>>>> 
>>> org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:676)
>>>>>  at
>>>>> 
>>>> 
>>> org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:676)
>>>>>  at
>>>>> 
>>>> 
>>> org.apache.spark.sql.execution.SQLExecution$$anonfun$withNewExecutionId$1.apply(SQLExecution.scala:78)
>>>>>  at
>>>>> 
>>>> 
>>> org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:125)
>>>>>  at
>>>>> 
>>>> 
>>> org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:73)
>>>>>  at
>>>>> 
>>>> 
>>> org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:676)
>>>>>  at
>>>>> 
>>>> 
>>> org.apache.spark.sql.DataFrameWriter.saveToV1Source(DataFrameWriter.scala:285)
>>>>>  at
>>> org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:271)
>>>>>  at
>>> org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:229)
>>>>>  ... 69 elided
>>>>> Caused by: org.apache.hudi.exception.HoodieException: Failed schema
>>>>> compatibility check for writerSchema
>>>>> 
>>>> 
>>> :{"type":"record","name":"foo_record","namespace":"hoodie.foo","fields":[{"name":"_hoodie_commit_time","type":["null","string"],"doc":"","default":null},{"name":"_hoodie_commit_seqno","type":["null","string"],"doc":"","default":null},{"name":"_hoodie_record_key","type":["null","string"],"doc":"","default":null},{"name":"_hoodie_partition_path","type":["null","string"],"doc":"","default":null},{"name":"_hoodie_file_name","type":["null","string"],"doc":"","default":null},{"name":"a","type":"long"},{"name":"b","type":"string"},{"name":"__row_key","type":"int"},{"name":"__row_version","type":"int"}]},
>>>>> table schema
>>>>> 
>>>> 
>>> :{"type":"record","name":"foo_record","namespace":"hoodie.foo","fields":[{"name":"_hoodie_commit_time","type":["null","string"],"doc":"","default":null},{"name":"_hoodie_commit_seqno","type":["null","string"],"doc":"","default":null},{"name":"_hoodie_record_key","type":["null","string"],"doc":"","default":null},{"name":"_hoodie_partition_path","type":["null","string"],"doc":"","default":null},{"name":"_hoodie_file_name","type":["null","string"],"doc":"","default":null},{"name":"a","type":"int"},{"name":"b","type":"string"},{"name":"__row_key","type":"int"},{"name":"__row_version","type":"int"}]},
>>>>> base path :file:///jfs/cadl/hudi_data/schema/foo
>>>>>  at
>>>> org.apache.hudi.table.HoodieTable.validateSchema(HoodieTable.java:564)
>>>>>  at
>>>>> 
>>>> 
>>> org.apache.hudi.table.HoodieTable.validateUpsertSchema(HoodieTable.java:570)
>>>>>  ... 94 more.
>>>> 
>>> 
>> 
>> 
>> --
>> Regards,
>> -Sivabalan
>> 
> 
> 
> -- 
> Regards,
> -Sivabalan
  

Reply via email to