-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-837]: implemented custom deserializer for AvroKafkaSource #1562
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
Closed
Closed
Changes from all commits
Commits
Show all changes
5 commits
Select commit
Hold shift + click to select a range
4447a56
[HUDI-837]: implemented custom deserializer for AvroKafkaSource
pratyakshsharma 911d35d
[HUDI-837]: code review comments addressed
pratyakshsharma 9f80200
[HUDI-837]: added test cases
pratyakshsharma 684e5da
[HUDI-837]: small changes in DummySchemaProvider
pratyakshsharma e798873
[HUDI-837]: code review comments addressed
pratyakshsharma File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
37 changes: 37 additions & 0 deletions
37
hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/DummySchemaProvider.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,37 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.hudi.utilities.schema; | ||
|
|
||
| import org.apache.avro.Schema; | ||
| import org.apache.hudi.common.config.TypedProperties; | ||
|
|
||
| /** | ||
| * Schema provider to be used in HoodieAvroKafkaDeserializer test cases. | ||
| */ | ||
| public class DummySchemaProvider extends SchemaProvider { | ||
|
|
||
| public DummySchemaProvider(TypedProperties properties) { | ||
| super(properties, null); | ||
| } | ||
|
|
||
| public Schema getSourceSchema() { | ||
| return new Schema.Parser().parse("{\"namespace\":\"example.avro\",\"type\":\"record\",\"name\":\"User\",\"fields\":" | ||
| + "[{\"name\":\"name\",\"type\":\"string\"},{\"name\":\"favorite_number\",\"type\":\"int\"},{\"name\":\"favorite_color\",\"type\":\"string\"}]}"); | ||
| } | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
83 changes: 83 additions & 0 deletions
83
...ilities/src/main/java/org/apache/hudi/utilities/sources/serde/HoodieAvroKafkaDecoder.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,83 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.hudi.utilities.sources.serde; | ||
|
|
||
| import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; | ||
| import io.confluent.kafka.serializers.KafkaAvroDecoder; | ||
| import org.apache.hudi.common.config.TypedProperties; | ||
| import org.apache.hudi.utilities.UtilHelpers; | ||
| import org.apache.hudi.utilities.schema.SchemaProvider; | ||
|
|
||
| import io.confluent.kafka.serializers.KafkaAvroDeserializerConfig; | ||
| import kafka.utils.VerifiableProperties; | ||
| import org.apache.avro.Schema; | ||
| import org.apache.kafka.common.errors.SerializationException; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.Map; | ||
| import java.util.Objects; | ||
| import java.util.Properties; | ||
|
|
||
| /** | ||
| * This is a custom implementation of kafka.serializer.Decoder<T> which aims at deserializing all the incoming messages | ||
| * with same schema (which is latest). | ||
| */ | ||
| public class HoodieAvroKafkaDecoder extends KafkaAvroDecoder { | ||
|
|
||
| private final Schema sourceSchema; | ||
| public static final String SCHEMA_PROVIDER_CLASS_PROP = "hoodie.deltastreamer.schemaprovider.class"; | ||
|
|
||
| public HoodieAvroKafkaDecoder(SchemaRegistryClient client, VerifiableProperties properties) { | ||
| super(client); | ||
| TypedProperties typedProperties = new TypedProperties(); | ||
| copyProperties(typedProperties, properties.props()); | ||
| try { | ||
| SchemaProvider schemaProvider = UtilHelpers.createSchemaProvider( | ||
| typedProperties.getString(SCHEMA_PROVIDER_CLASS_PROP), typedProperties); | ||
| this.sourceSchema = Objects.requireNonNull(schemaProvider).getSourceSchema(); | ||
| } catch (IOException e) { | ||
| throw new RuntimeException(e); | ||
| } | ||
| } | ||
|
|
||
| public HoodieAvroKafkaDecoder(VerifiableProperties properties) { | ||
| super(properties); | ||
| this.configure(new KafkaAvroDeserializerConfig(properties.props())); | ||
| TypedProperties typedProperties = new TypedProperties(); | ||
| copyProperties(typedProperties, properties.props()); | ||
| try { | ||
| SchemaProvider schemaProvider = UtilHelpers.createSchemaProvider( | ||
| typedProperties.getString(SCHEMA_PROVIDER_CLASS_PROP), typedProperties); | ||
| this.sourceSchema = Objects.requireNonNull(schemaProvider).getSourceSchema(); | ||
| } catch (IOException e) { | ||
| throw new RuntimeException(e); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| protected Object deserialize(boolean includeSchemaAndVersion, String topic, Boolean isKey, byte[] payload, Schema readerSchema) throws SerializationException { | ||
| return super.deserialize(includeSchemaAndVersion, topic, isKey, payload, sourceSchema); | ||
| } | ||
|
|
||
| private void copyProperties(TypedProperties typedProperties, Properties properties) { | ||
| for (Map.Entry<Object, Object> entry : properties.entrySet()) { | ||
| typedProperties.put(entry.getKey(), entry.getValue()); | ||
| } | ||
| } | ||
| } |
74 changes: 74 additions & 0 deletions
74
...ies/src/test/java/org/apache/hudi/utilities/sources/serde/TestHoodieAvroKafkaDecoder.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,74 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.hudi.utilities.sources.serde; | ||
|
|
||
| import io.confluent.kafka.serializers.KafkaAvroDeserializerConfig; | ||
| import org.apache.hudi.common.config.TypedProperties; | ||
|
|
||
| import io.confluent.kafka.schemaregistry.client.MockSchemaRegistryClient; | ||
| import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; | ||
| import io.confluent.kafka.serializers.KafkaAvroDeserializer; | ||
| import io.confluent.kafka.serializers.KafkaAvroSerializer; | ||
| import kafka.utils.VerifiableProperties; | ||
| import org.apache.avro.Schema; | ||
| import org.apache.avro.generic.GenericData; | ||
| import org.apache.avro.generic.GenericRecord; | ||
| import org.apache.hudi.utilities.schema.DummySchemaProvider; | ||
| import org.junit.jupiter.api.Assertions; | ||
| import org.junit.jupiter.api.BeforeAll; | ||
| import org.junit.jupiter.api.Test; | ||
|
|
||
| public class TestHoodieAvroKafkaDecoder { | ||
|
|
||
| private static KafkaAvroSerializer avroSerializer; | ||
| private static SchemaRegistryClient client; | ||
| private static KafkaAvroDeserializer avroDeserializer; | ||
| private static String topic; | ||
|
|
||
| @BeforeAll | ||
| public static void init() { | ||
| client = new MockSchemaRegistryClient(); | ||
| avroDeserializer = new KafkaAvroDeserializer(client); | ||
| avroSerializer = new KafkaAvroSerializer(client); | ||
| topic = "test_topic"; | ||
| } | ||
|
|
||
| private GenericRecord createAvroRecord(Schema schema) { | ||
| GenericRecord avroRecord = new GenericData.Record(schema); | ||
| avroRecord.put("name", "testUser"); | ||
| avroRecord.put("favorite_number", 13); | ||
| avroRecord.put("favorite_color", "dummy_color"); | ||
| return avroRecord; | ||
| } | ||
|
|
||
| @Test | ||
| public void testKafkaDeserializer() { | ||
| TypedProperties typedProperties = new TypedProperties(); | ||
| typedProperties.setProperty(KafkaAvroDeserializerConfig.SCHEMA_REGISTRY_URL_CONFIG, "http://localhost:8081"); | ||
| typedProperties.setProperty(HoodieAvroKafkaDecoder.SCHEMA_PROVIDER_CLASS_PROP, DummySchemaProvider.class.getName()); | ||
| VerifiableProperties verifiableProperties = new VerifiableProperties(typedProperties); | ||
| DummySchemaProvider schemaProvider = new DummySchemaProvider(typedProperties); | ||
| byte[] bytes; | ||
| GenericRecord record = createAvroRecord(schemaProvider.getSourceSchema()); | ||
| bytes = avroSerializer.serialize(topic, record); | ||
| HoodieAvroKafkaDecoder kafkaAvroDecoder = new HoodieAvroKafkaDecoder(client, verifiableProperties); | ||
| Assertions.assertEquals(record, avroDeserializer.deserialize(topic, bytes)); | ||
| Assertions.assertEquals(record, kafkaAvroDecoder.fromBytes(bytes)); | ||
| } | ||
| } |
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.