Skip to content
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

Pulsar IO - KafkaSource - allow to manage Avro Encoded messages #9448

Merged

Conversation

eolivelli
Copy link
Contributor

@eolivelli eolivelli commented Feb 3, 2021

Motivation

Currently KafkaSource allows only to deal with strings and byte arrays, it does not support records with Schema.
In Kafka we have the ability to encode messages using Avro and there is a Schema Registry (by Confluent®)

Modifications

Summary of changes:

  • allow current KafkaSource (KafkaBytesSource) to deal with io.confluent.kafka.serializers.KafkaAvroDeserializer and copy the raw bytes to the Pulsar topic, setting appropriately the Schema
  • this source support Schema Evolution end-to-end (i.e. add fields to the original schema in the Kafka world, and see the new fields in the Pulsar topic, without any reconfiguration or restart)
  • add Confluent® Schema Registry Client to the Kafka Connector NAR, the license is compatible with Apache 2 license and we can redistribute it
  • the configuration of the Schema Registry Client is done done in the consumerProperties property of the source (usually you add schema.registry.url)
  • add integration tests with Kafka and Schema Registry

Verifying this change

The patch introduces new integration tests.
The integration tests launch a Kafka Container and also a Confluent Schema Registry Container

Documentation

I will be happy to provide documentation once this patch is committed.

Copy link
Member

@sijie sijie left a comment

Choose a reason for hiding this comment

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

@eolivelli I think this approach makes unnecessary changes to the existing Pulsar schema library for achieving a very narrow scope. Pulsar and Kafka have very similar data models. Most of the time you should just transfer the bytes. The only thing you need to manage is to convert "Kafka SerDe" to "Pulsar Schema". You don't really need to write a special record to carry the schema information.

I wrote a schema-aware Kafka source at https://github.com/streamnative/pulsar-io-kafka, which we haven't contributed back yet.

  1. It only has one source connector that deals with byte[]. It works for all schemas. You don't need different source connectors.

  2. All what you need to do is to covert KafkaSerDe to Pulsar schema. See: https://github.com/streamnative/pulsar-io-kafka/blob/master/src/main/java/io/streamnative/connectors/kafka/KafkaSource.java#L338

  3. because Kafka encoded schema id as the first 4 bytes in Avro messages, hence you need to write a Schema wrapper. https://github.com/streamnative/pulsar-io-kafka/blob/master/src/main/java/io/streamnative/connectors/kafka/schema/KafkaAvroSchema.java

  4. Kafka also handles JSON slight differently. Hence we need to process that differently as well. https://github.com/streamnative/pulsar-io-kafka/blob/master/src/main/java/io/streamnative/connectors/kafka/schema/KafkaJsonSchema.java

  5. In a lot of cases, you just use Kafka's existing tools to covert Kafka schema into a standard AVRO schema to be stored as a Pulsar schema. That is the benefit of using an open-standard serialization framework rather than introducing your own type system. Reuse existing tools instead of reinventing a wheel.

Copy link
Contributor Author

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

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

@sijie
I think you are right in saying that we have to use open standards and existing tools, and to not reinvent the wheel.

Indeed this implementation of the KafkaSource is using exacly KafkaAvroDeserializer that is the tool that is available in the Kafka ecosystem to deal with Avro.

I prefer this solution because it is using only standard tools in the Kafka ecosystem.
I believe that in the long term this very simple code will be easily maintained here in the Pulsar codebase

I would like not to enter the details of how Kafka and Confluent serialize data (extract the schema id, extract the raw payload, connect to the schema registry....)

It is better to use the official library and use standard APIs.
in the future it will be easy to upgrade to new versions of the Kafka/Confluent client and support future changes/evolutions and enterprise features.

I believe this is a good enhancement to the Pulsar Sink framework:
we allowing Sources to push GenericRecords, and Pulsar will package the structure using the Schema provided by the Source using the Pulsar Schema API (Avro or whatever we will support in the future...).

If you prefer I can split the patch into two parts:

  • allow PulsarSource to deal with GenericRecord
  • enhance the standard KafkaSource

@sijie
Copy link
Member

sijie commented Feb 4, 2021

@eolivelli

Unfortunately, I disagree with you on this. This approach creates a very bad example for other people to follow.

First of all, it basically creates a "connector" class per schema type. This is a very bad practice. I would discourage a connector implementation going down this route. It is impossible to maintain.

Secondly, it solves a very narrow scoped problem by introducing a specific type of source connector in which the key is a string and the value is AVRO. All the key schema information is dropped. Key schema information is important to a lot of streaming use cases. You can't solve the problem with this approach. Following your approach, you will end up creating N*N connector classes (where N is the number of schema types).

I would encourage people not introducing a code change that is specialized for their own needs. The connector implementation should be beneficial to broader users.

I would like not to enter the details of how Kafka and Confluent serialize data

Unfortunately, the Kafka AVRO is a confluent open-source thing not a Kafka community thing.

We are always in the game of touching serialization details when converting a message from X format to Y format. You either do the conversation at a high level using abstraction or at a low level by realizing the serialization details.

The approach you proposed also has a bad performance because it will churn a lot of object allocations. Realizing the serialization details can save a lot of memory copy and serialization/deserialization.

we allowing Sources to push GenericRecords,

This is a good initiative. But it should be isolated from this KafkaSource change.

@eolivelli
Copy link
Contributor Author

@sijie there no problem in disagreeing,
let's find together the right way to provide features to the users in the best way for the project.

I am going to split the patch into two parts, this way we can make one step at a time.

First of all, it basically creates a "connector" class per schema type. This is a very bad practice. I would discourage a connector implementation going down this route. It is impossible to maintain.

We already have KafkaBytesSource and KafkaStringSource, so I am just adding a new flavour of the KafkaSource, in fact the implementation is just about adding a new subclass of KafkaAbstractSource.
I am following the current style.

In my plans I would like to work more on this KafkaSource and on the KafkaSink and try to make the structure better.

There is an open work that will allow to put more sinks on the same nar and provide a better user experience.
#3678

In the meanwhile users of the Kafka source can go with "--classname" (or they can select it from a Web UI for interactive Pulsar Management Consoles)

in which the key is a string and the value is AVRO

we can work on this issue as well (and that's on my backlog), I didn't want to introduce too many features.

I have users that are used to advanced data mapping mechanisms both for the key and for the value, so mapping the key is very important to me.
That said, currently the KafkaAbstractSource is working on a string key, that is a preexisting code

The approach you proposed also has a bad performance because it will churn a lot

of object allocations. Realizing the serialization details can save a lot of memory copy and serialization/deserialization.

I know about this fact, and I know how the StreamNative connector works.

Using the Java Model with GenericRecord adds that additional cost, but the benefit are:

  • to have simpler code, using code provided by the same vendors that are mantaining that serialization protocol
  • we can follow the evolutions just by upgrading the Confluent library
  • we are using pure Kafka/Pulsar APIs, totally integrated with the framework, this will allow us to leverage all future improvements

Copy link
Member

@lhotari lhotari left a comment

Choose a reason for hiding this comment

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

Good work @eolivelli . I put a few comments about some minor things.

@eolivelli eolivelli force-pushed the impl/kafka-schema-aware-with-tests branch from f06c417 to 8e456e3 Compare March 8, 2021 10:21
Copy link
Contributor Author

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

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

@sijie I have added Guava cache, removed AUTO_PRODUCE_BYTES, removed extractKey method and switched to ByteBuffer and addressed all of your comments.

Please take a look again

@eolivelli eolivelli requested a review from sijie March 8, 2021 10:42
Copy link
Member

@sijie sijie left a comment

Choose a reason for hiding this comment

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

@eolivelli overall looks good.

I see you removed the key handling from this PR. I think in general, we should structure the class hierarchy like the following. As we are introducing a new bytes connector, we need to make the key as pure bytes because that's the default behavior from Kafka.

KafkaAbstractSource -> KafkaAbstractStringKeySource -> KafkaStringSource
KafkaAbstractSource -> KafkaAbstractBytesKeySource -> KafkaBytesSource

<version>${kafka.confluent.avroserializer.version}</version>
</dependency>

<dependency>
Copy link
Member

Choose a reason for hiding this comment

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

Why do you need this dependency?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

good catch. I needed it initially.
now it is useless.

dropped

.properties(Collections.emptyMap())
.schema(definition.getBytes(StandardCharsets.UTF_8)
).build();
return new Schema<ByteBuffer>() {
Copy link
Member

Choose a reason for hiding this comment

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

I think you need to override the decode method and throw an UnsupportedException. Otherwise, if the decode method is used, it will cause StackOverflowException.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

@eolivelli
Copy link
Contributor Author

@sijie I have addressed your last comments.
I created a new issue regarding the management of the key.
#9848

this way we can move forward one step at a time

@eolivelli
Copy link
Contributor Author

@codelipenghui @sijie CI passed. probably we are good to go now :)

pulsar-io/kafka/pom.xml Outdated Show resolved Hide resolved
@eolivelli
Copy link
Contributor Author

/pulsarbot rerun-failure-checks

@eolivelli eolivelli requested a review from sijie March 12, 2021 11:08
@sijie sijie added this to the 2.8.0 milestone Mar 13, 2021
@sijie
Copy link
Member

sijie commented Mar 13, 2021

@freeznet @codelipenghui Can you review this PR?

@eolivelli
Copy link
Contributor Author

@codelipenghui can you please help merging this patch?

@codelipenghui codelipenghui merged commit d52a1b0 into apache:master Mar 15, 2021
@eolivelli eolivelli deleted the impl/kafka-schema-aware-with-tests branch March 15, 2021 07:46
@eolivelli
Copy link
Contributor Author

@thank you very much @codelipenghui and @sijie

fmiguelez pushed a commit to fmiguelez/pulsar that referenced this pull request Mar 16, 2021
…he#9448)

### Motivation
Currently KafkaSource allows only to deal with strings and byte arrays, it does not support records with Schema.
In Kafka we have the ability to encode messages using Avro and there is a Schema Registry (by Confluent®)

### Modifications

Summary of changes:
- allow current KafkaSource (`KafkaBytesSource`) to deal with `io.confluent.kafka.serializers.KafkaAvroDeserializer ` and copy the raw bytes to the Pulsar topic, setting appropriately the Schema
- this source support Schema Evolution end-to-end (i.e. add fields to the original schema in the Kafka world, and see the new fields in the Pulsar topic, without any reconfiguration or restart)
- add Confluent® Schema Registry Client to the Kafka Connector NAR, the license is compatible with Apache 2 license and we can redistribute it
- the configuration of the Schema Registry Client is done done in the consumerProperties property of the source (usually you add schema.registry.url)
- add integration tests with Kafka and Schema Registry

### Verifying this change

The patch introduces new integration tests.
The integration tests launch a Kafka Container and also a Confluent Schema Registry Container
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

5 participants