sathyaprakashg commented on pull request #2012:
URL: https://github.com/apache/hudi/pull/2012#issuecomment-686046387


   Below issue was causing build failure and i fixed it now by implementing a 
method `HoodieAvroUtils.addNamespaceToFixedField`. Not sure whether this is the 
best way to fix this issue. So, let me know if any sugestions.
   
   Below is the height field defintion in `triprec` schema
   ``` 
     {
         "name": "height",
         "type": {
           "type": "fixed",
           "name": "abc",
           "size": 5,
           "logicalType": "decimal",
           "precision": 10,
           "scale": 6
         }
       }
   ```
   When we use org.apache.spark.sql.avro.SchemaConverters.toAvroType method in 
`AvroConversionHelper` to convert Spark data type to avro type, spark-avro 
gives name as fixed and also adds namespace to it as per code 
[here](https://github.com/apache/spark/blob/master/external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala#L177)
   ```
     {
         "name": "height",
         "type": {
           "type": "fixed",
           "name": "fixed",
           "namespace": "hoodie.source.hoodie_source.height",
           "size": 5,
           "logicalType": "decimal",
           "precision": 10,
           "scale": 6
         }
       }
   ```
   First schema is used as reader schema and second one is used as writer 
schema in `HoodieAvroUtils.bytesToAvro`. This results in below error.
   ```
   ERROR org.apache.hudi.io.HoodieWriteHandle  - Error writing record 
HoodieRecord{key=HoodieKey { recordKey=c597b0c2-6f07-4c8e-b239-2752b60e0449 
partitionPath=default}, currentLocation='null', newLocation='null'}
   org.apache.avro.AvroTypeException: Found 
hoodie.source.hoodie_source.height.fixed, expecting abc
     at org.apache.avro.io.ResolvingDecoder.doAction(ResolvingDecoder.java:292)
     at org.apache.avro.io.parsing.Parser.advance(Parser.java:88)
     at 
org.apache.avro.io.ValidatingDecoder.checkFixed(ValidatingDecoder.java:135)
     at 
org.apache.avro.io.ValidatingDecoder.readFixed(ValidatingDecoder.java:146)
     at 
org.apache.avro.generic.GenericDatumReader.readFixed(GenericDatumReader.java:342)
     at 
org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:180)
     at 
org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:153)
     at 
org.apache.avro.generic.GenericDatumReader.readField(GenericDatumReader.java:232)
     at 
org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:222)
     at 
org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:175)
     at 
org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:153)
     at 
org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:145)
     at 
org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro(HoodieAvroUtils.java:142)
     at 
org.apache.hudi.common.model.OverwriteWithLatestAvroPayload.getInsertValue(OverwriteWithLatestAvroPayload.java:70)
     at 
org.apache.hudi.execution.LazyInsertIterable$HoodieInsertValueGenResult.<init>(LazyInsertIterable.java:92)
     at 
org.apache.hudi.execution.LazyInsertIterable.lambda$getTransformFunction$0(LazyInsertIterable.java:105)
     at 
org.apache.hudi.common.util.queue.BoundedInMemoryQueue.insertRecord(BoundedInMemoryQueue.java:170)
     at 
org.apache.hudi.common.util.queue.IteratorBasedQueueProducer.produce(IteratorBasedQueueProducer.java:46)
     at 
org.apache.hudi.common.util.queue.BoundedInMemoryExecutor.lambda$null$0(BoundedInMemoryExecutor.java:92)
     at java.util.concurrent.FutureTask.run(FutureTask.java:266)
     at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
     at java.util.concurrent.FutureTask.run(FutureTask.java:266)
     at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
     at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
     at java.lang.Thread.run(Thread.java:748)
   ```
   In order to fix this issue, approach i took is change the field name from 
`abc` to `fixed` in all the schema used in test and also add namespace to 
reader schema if it exists in writer schema for fixed field.
   
   If we just rename the field from `abc` to `fixed` but don't add namespace, 
then we get below error
   `org.apache.avro.AvroTypeException: Found 
hoodie.source.hoodie_source.height.fixed, expecting fixed`


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

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to