Skip to content

Conversation

@pratyakshsharma
Copy link
Contributor

Tips

What is the purpose of the pull request

Supports kafka installations without schema-registry by allowing FileBasedSchemaProvider to be integrated to AvroKafkaSource.

Brief change log

  • Implemented HoodieKafkaAvroDecoder.
  • Introduced a property for configuring AvroKafkaSource with or without schema-registry setup.

Verify this pull request

(Please pick either of the following options)

This pull request is a trivial rework / code cleanup without any test coverage.

(or)

This pull request is already covered by existing tests, such as (please describe tests).

(or)

This change added tests and can be verified as follows:

(example:)

  • Added integration tests for end-to-end.
  • Added HoodieClientWriteTest to verify the change.
  • Manually verified the change by running a job locally.

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.

@codecov-io
Copy link

codecov-io commented Apr 26, 2020

Codecov Report

Merging #1565 into master will decrease coverage by 0.03%.
The diff coverage is 60.00%.

Impacted file tree graph

@@             Coverage Diff              @@
##             master    #1565      +/-   ##
============================================
- Coverage     71.82%   71.79%   -0.04%     
- Complexity      294      303       +9     
============================================
  Files           383      386       +3     
  Lines         16549    16598      +49     
  Branches       1663     1667       +4     
============================================
+ Hits          11887    11916      +29     
- Misses         3930     3946      +16     
- Partials        732      736       +4     
Impacted Files Coverage Δ Complexity Δ
...apache/hudi/utilities/sources/AvroKafkaSource.java 0.00% <0.00%> (ø) 0.00 <0.00> (ø)
...ies/serde/AbstractHoodieKafkaAvroDeserializer.java 60.60% <60.60%> (ø) 6.00 <6.00> (?)
...e/hudi/utilities/serde/HoodieKafkaAvroDecoder.java 75.00% <75.00%> (ø) 1.00 <1.00> (?)
...hudi/utilities/schema/FilebasedSchemaProvider.java 83.33% <100.00%> (+2.08%) 5.00 <0.00> (ø)
...e/config/HoodieKafkaAvroDeserializationConfig.java 100.00% <100.00%> (ø) 2.00 <2.00> (?)
...che/hudi/common/util/BufferedRandomAccessFile.java 54.38% <0.00%> (-0.88%) 0.00% <0.00%> (ø%)

Continue to review full report at Codecov.

Legend - Click here to learn more
Δ = absolute <relative> (impact), ø = not affected, ? = missing data
Powered by Codecov. Last update 19ca0b5...a71f602. Read the comment docs.

@vinothchandar vinothchandar self-assigned this Apr 27, 2020
protected Object deserialize(String topic, Boolean isKey, byte[] payload, Schema readerSchema) {
try {
ByteBuffer buffer = this.getByteBuffer(payload);
int id = buffer.getInt();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this looks specific to confluent schema registry. Is it a good idea to assume all the messages will start with magic int?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Contributor Author

@pratyakshsharma pratyakshsharma Apr 28, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this looks specific to confluent schema registry. Is it a good idea to assume all the messages will start with magic int?

Ok, the actual purpose of this PR was to support kafka installations without schema-registry. I messed it up at this point. Thank you for pointing this out. Will handle this accordingly.

any reason it it not inspired by the latest version?

No specific reason. The implementation is inspired from the kafka-avro-serializer version we are using currently. :)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@vinothchandar @afilipchik I explored a bit around this. Deserialization process has to happen exactly opposite to how the data was serialized. Also it is highly discouraged to not use schema-registry when working with avro and kafka [1][2][3]
That said, I guess if some one wants to use vanilla AvroKafkaSource, he will have to serialize the data without actual schema-registry setup. Hence as a matter of practice and a good alternative can be mandating use of MockSchemaRegistryClient.java class for the same and hence the code written above specific to schema-registry should be fine.

I am suggesting use of MockSchemaRegistryClient.java class since that has been used by Confluent for writing their test cases and this will bring a standard practice to follow for using HoodieKafkaAvroDecoder.java class introduced with this PR.

If we agree to this, I will mention the same as a comment in the code above.

[1] confluentinc/confluent-kafka-dotnet#530
[2] https://medium.com/@igorvlahek1/no-need-for-schema-registry-in-your-spring-kafka-tests-a5b81468a0e1
[3] https://stackoverflow.com/questions/45635726/kafkaavroserializer-for-serializing-avro-without-schema-registry-url

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if we are reusing code, we need to be also mindful of updates needed for NOTICE and LICENSE>

}

protected Object deserialize(byte[] payload) throws SerializationException {
return deserialize(null, null, payload, sourceSchema);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it looks like sourceSchema is used both as writer and reader. Why? Avro is painful when writer schema is lost as some evolutions become impossible.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Did not get this point. What do you mean by writer schema getting lost?

Copy link
Contributor Author

@pratyakshsharma pratyakshsharma Apr 28, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

So basically what I want to understand here is - what specific advantages you get by having separate writer and reader schemas apart from being able to handle field renames using aliases? I tried to go through avro library, but still am not convinced about this. If you could point me to some useful documentation or blog regarding this, that would be great.

Copy link
Contributor

@afilipchik afilipchik May 5, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

let me find it. We had to patch kafka -> avro deser in hudi 0.5 as it was breaking in some situations.
In general based on spec at https://avro.apache.org/docs/current/spec.html:

Data Serialization and Deserialization

Binary encoded Avro data does not include type information or field names. The benefit is that the serialized data is small, but as a result a schema must always be used in order to read Avro data correctly. The best way to ensure that the schema is structurally identical to the one used to write the data is to use the exact same schema.

Therefore, files or systems that store Avro data should always include the writer's schema for that data. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data. In general, it is advisable that any reader of Avro data should use a schema that is the same (as defined more fully in Parsing Canonical Form for Schemas) as the schema that was used to write the data in order to deserialize it correctly. Deserializing data into a newer schema is accomplished by specifying an additional schema, the results of which are described in Schema Resolution.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@afilipchik got a chance to explore this?

result = bytes;
} else {
int start = buffer.position() + buffer.arrayOffset();
DatumReader reader = this.getDatumReader(sourceSchema, readerSchema);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what the difference here bt sourceSchema and readerSchema ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

They are both same. :)

@vinothchandar
Copy link
Member

Overall, this PR is nice in the sense that it let's us read data from Kafka using AVRO, with a fixed schema..
but then, it cannot handle evolutions that well (this is an expected tradeoff , right).. I feel we should understand little better how people are sending vanilla avro data into kafka today, beofre we can resolve the open items here..

@vinothchandar vinothchandar removed their assignment May 14, 2020
@pratyakshsharma
Copy link
Contributor Author

So handling schema evolutions without schema-registry is going to be really tricky. I tried googling around this stuff, and found the below 2 links. These might be useful in what we want to achieve -

  1. https://stackoverflow.com/questions/37290303/producing-and-consuming-avro-messages-from-kafka-without-confluent-components
  2. https://github.com/farmdawgnation/registryless-avro-converter

Particularly the second repository aims at serializing and deserializing avro data without schema-registry using Confluent and Avro libraries. At a high level, it looks like they are also not handling schema evolution in their code. I would need some time to go through it in depth though.
Also if you see the description of jira (https://issues.apache.org/jira/browse/HUDI-73), it mentions integration of AvroKafkaSource with FilebasedSchemaProvider (which is what is done in this PR :) ). If we really want to integrate it with FilebasedSchemaProvider, then I do not think it is feasible to handle schema evolution, since as a user, one cannot keep on changing schema files on every evolution. Thoughts? @vinothchandar

@vinothchandar vinothchandar added the area:schema Schema evolution and data types label Oct 9, 2020
@afilipchik
Copy link
Contributor

Hey, any way to push it over the line? We are upgrading to 0.6 and in the current state evolutions on top of Confluent Schema registry is borked

@vinothchandar
Copy link
Member

@pratyakshsharma any comments? do you have cycles in the next couple weeks to get this over finish line.

We can target this for 0.7.0 as well .

@pratyakshsharma
Copy link
Contributor Author

Hi, I am currently facing issues with my laptop. Replacement is on its way. I will pick this up as soon as I get the new laptop, most probably in another 2 days. :)

Copy link
Contributor

@nsivabalan nsivabalan left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@afilipchik @pratyakshsharma : can you folks summarize whats pending wrt this patch. I can take this up and try to get this in upcoming release. Would appreciate if you can go over what needs to be done to get this to closure.

Here is my understanding so far.
For those who don't want to use schema registry, this patch adds support to use file based schema for deser kafka avro data.
AbstractHoodieKafkaAvroDeserializer is the class that has the core logic to deser. and HoodieKafkaAvroDecoder is the decoder impl which leverages AbstractHoodieKafkaAvroDeserializer.

One downside of this approach is regards to schema evolution, since user has to keep updating the schema as and when required. but based on Pratyaksh's last comment, looks like we can't get past it.

Do I need to research more on these lines? as to how to support schema evolution.

@vinothchandar vinothchandar added the priority:critical Production degraded; pipelines stalled label Feb 11, 2021
@vinothchandar vinothchandar removed the priority:critical Production degraded; pipelines stalled label Mar 14, 2021
@n3nash
Copy link
Contributor

n3nash commented Mar 30, 2021

@nsivabalan Should we close this PR in favor of the recent Kafka Avro Source that was landed ?

@pratyakshsharma
Copy link
Contributor Author

@nsivabalan Should we close this PR in favor of the recent Kafka Avro Source that was landed ?

@n3nash can you please point me to the specific PR you are referring to here?

@n3nash
Copy link
Contributor

n3nash commented Mar 31, 2021

@pratyakshsharma I'm referring to this commit -> 900de34

@pratyakshsharma
Copy link
Contributor Author

@pratyakshsharma I'm referring to this commit -> 900de34

@n3nash I went through the commit you are referring to. It basically takes care of passing in custom reader schema for deserialization. It does not take care of kafka setup without schema-registry. Still there is some work that needs to be done in this regard. I will sync up with @nsivabalan for the same.

@pratyakshsharma
Copy link
Contributor Author

@afilipchik @pratyakshsharma : can you folks summarize whats pending wrt this patch. I can take this up and try to get this in upcoming release. Would appreciate if you can go over what needs to be done to get this to closure.

@nsivabalan Basically we need to figure out/ decide on the below items -

  1. Is it okay to mandate use of MockSchemaRegistryClient while serializing data before sending it to kafka for users?
  2. Referring to this comment - [HUDI-73]: implemented vanilla AvroKafkaSource #1565 (comment), I guess the actual scope of this PR was supporting vanilla kafka setup w/o supporting schema evolution. If indeed, we want to move forward with supporting schema evolutions along with vanilla setup, we need to explore around how to achieve the same.

Any thoughts are welcome. :)

@n3nash
Copy link
Contributor

n3nash commented Apr 6, 2021

@pratyakshsharma Thanks for providing that context. @nsivabalan Once you have shared your thoughts, we can redo the scope of this PR and either rebase and fix this or open a new one.

@vinothchandar
Copy link
Member

Closing due to inactivity

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

area:schema Schema evolution and data types

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants