sbernauer opened a new pull request #3111:
URL: https://github.com/apache/hudi/pull/3111


   Hi Hudi Team!
   
   ## What is the purpose of the pull request
   We are using Deltastreamer to ingest events from kafka into an S3 store.
   We had some incident this week after some of our producers switched to a new 
schema version, but some remained on older schem versions.
   
   We stumbled over this PR https://github.com/apache/hudi/pull/2619. With the 
following settings we can enbale the Custom avro kafka deserializer
   ```
   
hoodie.deltastreamer.source.kafka.value.deserializer.class=org.apache.hudi.utilities.deser.KafkaAvroSchemaDeserializer
   
hoodie.deltastreamer.schemaprovider.class=org.apache.hudi.utilities.schema.FilebasedSchemaProvider
   ```
   
   Doing so we noticed that KafkaAvroSchemaDeserializer tries instanciating the 
configured SchemaProvider via reflection. It calls the constructor in 
https://github.com/apache/hudi/blob/cdb9b48170ef98634babd8954392efb1c1b90fcf/hudi-utilities/src/main/java/org/apache/hudi/utilities/deser/KafkaAvroSchemaDeserializer.java#L55
 resulting in
   ```
   Caused by: java.lang.NoSuchMethodException: 
org.apache.hudi.utilities.schema.SchemaProviderWithPostProcessor.<init>(org.apache.hudi.common.config.TypedProperties)
   ```
   There are multiple problems here
   * The Class has not the needed constructor
   * As we are using FilebasedSchemaProvider it calls the wrong class wrapping 
the FilebasedSchemaProvider
   * The FilebasedSchemaProvider needs the JavaSparkContext to work correctly. 
I think we dont have access to the JavaSparkContext in 
KafkaAvroSchemaDeserializer
   
   ## Brief change log
   
   This PR modifies the KafkaAvroSchemaDeserializer, so that it does not rely 
on Reflection to call the SchemaProvider.
   Instead it uses the normal progam flow to ask the SchemProvider for the 
sourceSchema.
   It then passes the sourceSchema as Property to the 
KafkaAvroSchemaDeserializer so that it can be used for deserialization.
   
   Anyway: I wonder if it is a good idea to make the usage of 
KafkaAvroSchemaDeserializer the default. IMHO it woud make sense.
   
   ## Verify this pull request
   You have to enable the feature with
   ```
   
hoodie.deltastreamer.source.kafka.value.deserializer.class=org.apache.hudi.utilities.deser.KafkaAvroSchemaDeserializer
   ```
   
   The tests are WIP
   
   ## Committer checklist
   
    - [ ] Has a corresponding JIRA in PR title & commit
    
    - [ ] Commit message is descriptive of the change
    
    - [ ] CI is green
   
    - [ ] Necessary doc changes done or have another open PR
          
    - [ ] For large changes, please consider breaking it into sub-tasks under 
an umbrella JIRA.


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