Hi,

I think deleting field is supported with Avro both backward and forward as long 
as the field is optional  and provide default value.

A simple exemple of Avro optional field defined using a union type and a 
default value:
{ "name": "foo", "type": ["null", "string"], "default": "null" }
Readers will use default value when field is not present.

I believe problem here is Parquet which does not support field deletion. 
One option is to set Parquet field value to null. Parquet will use RLE encoding 
for efficient encoding of all null values in "deleted" field.

Regards,
Benoit

> On 6 Feb 2020, at 17:57, Nishith <[email protected]> wrote:
> 

Pratakysh,

Deleting fields isn’t Avro schema backwards compatible. Hudi relies on Avro 
schema evolution rules which helps to prevent breaking of existing queries on 
such tables - say someone was querying that field that is now deleted.
You can read more here -> https://avro.apache.org/docs/1.8.2/spec.html
That being said, I’m also looking at how we can support schema evolution 
slightly differently - somethings could be more in our control and not break 
reader queries - but that’s not in the near future.

Thanks

Sent from my iPhone

> On Feb 5, 2020, at 11:22 PM, Pratyaksh Sharma <[email protected]> wrote:
> 
> Hi Vinoth,
> 
> We do not have any standard documentation for the said approach as it was
> self thought through. Just logging a conversation from #general channel for
> record purpose -
> 
> "Hello people, I'm doing a POC to use HUDI in our data pipeline, but I got
> an error and I didnt find any solution for this... I wrote some parquet
> files with HUDI using INSERT_OPERATION_OPT_VAL, MOR_STORAGE_TYPE_OPT_VAL
> and sync with hive and worked perfectly. But after that, I try to wrote
> another file in the same table (with some schema changes, just delete and
> add some columns) and got this error Caused by:
> org.apache.parquet.io.InvalidRecordException:
> Parquet/Avro schema mismatch: Avro field 'field' not found. Anyone know
> what to do?"
> 
>>> On Sun, Jan 5, 2020 at 2:00 AM Vinoth Chandar <[email protected]> wrote:
>> 
>> In my experience, you need to follow some rules on evolving and keep the
>> data backwards compatible. Or the only other option is to rewrite the
>> entire dataset :), which is very expensive.
>> 
>> If you have some pointers to learn more about any approach you are
>> suggesting, happy to read up.
>> 
>> On Wed, Jan 1, 2020 at 10:26 PM Pratyaksh Sharma <[email protected]>
>> wrote:
>> 
>>> Hi Vinoth,
>>> 
>>> As you explained above and as per what is mentioned in this FAQ (
>>> 
>>> 
>> https://cwiki.apache.org/confluence/display/HUDI/FAQ#FAQ-What'sHudi'sschemaevolutionstory
>>> ),
>>> Hudi is able to maintain schema evolution only if the schema is
>> *backwards
>>> compatible*. What about the case when it is backwards incompatible? This
>>> might be the case when for some reason you are unable to enforce things
>>> like not deleting fields or not change the order. Ideally we should be
>> full
>>> proof and be able to support schema evolution in every case possible. In
>>> such a case, creating a Uber schema can be useful. WDYT?
>>> 
>>> On Wed, Jan 1, 2020 at 12:49 AM Vinoth Chandar <[email protected]>
>> wrote:
>>> 
>>>> Hi Syed,
>>>> 
>>>> Typically, I have been the Confluent/avro schema registry used as a the
>>>> source of truth and Hive schema is just a translation. Thats how the
>>>> hudi-hive sync also works..
>>>> Have you considered making fields optional in the avro schema so that
>>> even
>>>> if the source data does not have few of them, there will be nulls..
>>>> In general, the two places I have dealt with this, all made it works
>>> using
>>>> the schema evolution rules avro supports.. and enforcing things like
>> not
>>>> deleting fields, not changing order etc.
>>>> 
>>>> Hope that atleast helps a bit
>>>> 
>>>> thanks
>>>> vinoth
>>>> 
>>>> On Sun, Dec 29, 2019 at 11:55 PM Syed Abdul Kather <[email protected]
>>> 
>>>> wrote:
>>>> 
>>>>> Hi Team,
>>>>> 
>>>>> We have pull data from Kafka generated by Debezium. The schema
>>> maintained
>>>>> in the schema registry by confluent framework during the population
>> of
>>>>> data.
>>>>> 
>>>>> *Problem Statement Here: *
>>>>> 
>>>>> All the addition/deletion of columns is maintained in schema
>> registry.
>>>>> During running the Hudi pipeline, We have custom schema registry
>> that
>>>>> pulls the latest schema from the schema registry as well as from hive
>>>>> metastore and we create a uber schema (so that missing the columns
>> from
>>>> the
>>>>> schema registry will be pulled from hive metastore) But is there any
>>>> better
>>>>> approach to solve this problem?.
>>>>> 
>>>>> 
>>>>> 
>>>>> 
>>>>>           Thanks and Regards,
>>>>>       S SYED ABDUL KATHER
>>>>> 
>>>> 
>>> 
>> 

Reply via email to