New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SUPPORT] Support for Schema evolution. Facing an error #1845
Comments
Thanks Sebastian for the detailed information. From looking at the code related to your originally seen exception (not the test), it has to do with plain avro schema compatibility rules. Would you mind providing both versions of schema. From the test code in PR, there are 4 different fields (various combination of nullable vs default types). I would like to see which field specifically is contributing to the issue. |
Thanks for your fast reply! The PR adds a new Test and improves 2 existing tests. The mentioned 4 new cols in TestHoodieAvroUtils increase the number of tested cases, the tests are still working. Regarding the both versions of my schema, i can only provide a diff but it should be sufficient. The new field has the same type (union of null and string, default of null) as reproduced in the test here https://github.com/apache/hudi/pull/1844/files#diff-07dd5ed6077721a382c35b2700da0883R130. diff Event.json_schema Event-aged.json_schema
|
@sbernauer : Are you appending this field to the end of the schema ? Otherwise looks ok. Although honestly, I have not seen the usage of "avro.java.string": "String" before. @prashantwason @n3nash : Any ideas what is wrong here ? |
@bvaradar, yes i am appending the field to end of the schema (as reproduced in the test). The definition of the event is outside my scope, we just consume this events ;) |
@sbernauer I briefly looked at the test-case you added and I see what you are trying to reproduce. The issue seems to be as follows :
|
Thanks @n3nash Also, I see that you are using spark and Hadoop 3.x. Please make sure you are using the right version of avro that we support - 1.8.2 at runtime. That could also be an issue. |
Hi @bvaradar, I've created a test using pure avro DataFileWriter to reproduce my DeltaStreamer-test and it kind of works.
The second test 1.) Write record of B using schema B I found another post https://stackoverflow.com/questions/34733604/avro-schema-doesnt-honor-backward-compatibilty, it provides a testcase for schema evolution using a BinaryDecoder and reproduces the EOF exception. My setup is using avro 1.8.2. |
The javadoc for DatumWriter says: "Stores in a file a sequence of data conforming to a schema. The schema is stored in the file with the data. Each datum in a file is of the same schema." Your test case has records with different schema being written to the same file. Please try using the GenericDatumReader construtor which specifies both the reader/writer schema. |
Yes, to be correct you have to specify both reader and writer schema to GenericDatumReader. Hudi currenty only passes one schema here
I think that's a problem I would have used GenericDatumReader with reader and writer schema, but first I wanted to append new evoluted events to an existing file. Looking at "Each datum in a file is of the same schema" this seems expected behavior to be impossible. "Each datum in a file is of the same schema" - How is the strategy of Hudi when there are Upserts for an evoluted event? The new events can't simply be written to an existing file. |
Log blocks are read using both reader/write schema. |
The issue is probably related to using the wrong schema for Upsert. Let me expand your original steps to reproduce and you can correct me: Steps to reproduce:
If in step 4 you are using SCHEMA_V1 then it wont be able to read in records ingested at step 3 as that is a newer schema. |
At this step I used SCHEMA_V2 |
Is there anything we can do further to resolve this issue? |
@sbernauer : Do you mean using SCHEMA_v1 at step 4. Otherwise, both reader and writer schema is same. right ? There are couple of issues here :
|
For issue (1) : Opened https://issues.apache.org/jira/browse/HUDI-1128 |
Thanks @bvaradar for opening the tickets! |
I could have been more precise. At Step 4 we consume events without the new optional field (we could call them events produced with SCHEMA_V1). The Deltastreamer is passed the SCHEMA_v2. |
cc @nsivabalan as well. This is related to the schema evolution PR. |
@n3nash The concrete thing to do here, is to just take and repro this scenario with master branch and see if its still an issue. |
Hi @vinothchandar, @nsivabalan and me are working on Slack trying to get our schema evolution usecase running. Currently only Inserts, no Upserts for CoW. And more complicated avro schema evolution than adding a field at the end of the schema. I'm trying out his patch in #2927 |
@nsivabalan Can you please reply above ? |
yes, I am in sync w/ @sbernauer via slack. He confirmed that the PR we have put up works for him (older records able to be upserted to hudi after schema evolved w/ hudi table). He is doing more testing for now. |
@sbernauer : closing this issue out as we have the fixes landed already. Feel free to open up a new issue if you have more requirements. thanks! |
Hi Hudi team,
In https://cwiki.apache.org/confluence/display/HUDI/FAQ#FAQ-What'sHudi'sschemaevolutionstory you describe, that "As long as the schema passed to Hudi [...] is backwards compatible [...] Hudi will seamlessly handle read/write of old and new data and also keep the Hive schema up-to date."
We need and try to use this mechanism but are failing.
Steps to reproduce:
I've created a test to reproduce this behaviour here: #1844. I hope the test is helpful.
The test cause the following exception, but i think they are quite related.
I think the mailing list entry from avro could help a lot: http://apache-avro.679487.n3.nabble.com/AVRO-schema-evolution-adding-optional-column-with-default-fails-deserialization-td4043025.html
The corresponding place in code is here:
hudi/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java
Line 107 in bf1d36f
Thanks a lot in advance!
Cheers,
Sebastian
Expected behavior
As stated in FAQ schema evolution should not crash the DeltaStreamer
Environment Description
Hudi version : current master branch
Spark version : 3.0.0
Hive version : 3.1.2
Hadoop version : 3.2.1
Storage (HDFS/S3/GCS..) : S3
Running on Docker? (yes/no) : yes, running on kubernetes
The text was updated successfully, but these errors were encountered: