-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-1648] Added custom kafka meta fields and custom kafka avro decoder. #2598
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
6 commits
Select commit
Hold shift + click to select a range
1068595
Added custom kafka fields and custom kafka avro decoder.
2d6112b
Reduced KafkaAvroSchemaDeserializer to just call a super class method…
2c86577
Added unit tests, minor code improvements.
1860266
Addresed style issues.
c706d1c
Merge remote-tracking branch 'upstream/master' into custom-deserializer
9bc13c2
Apparently reflection utils can't create a class if one of the parame…
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
89 changes: 89 additions & 0 deletions
89
...-utilities/src/main/java/org/apache/hudi/utilities/deser/KafkaAvroSchemaDeserializer.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,89 @@ | ||
| /* | ||
| * 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.deser; | ||
|
|
||
| import io.confluent.kafka.serializers.AbstractKafkaAvroDeserializer; | ||
| import io.confluent.kafka.serializers.KafkaAvroDeserializer; | ||
| import org.apache.avro.Schema; | ||
| import org.apache.hudi.common.config.TypedProperties; | ||
| import org.apache.hudi.common.util.ReflectionUtils; | ||
| import org.apache.hudi.exception.HoodieException; | ||
| import org.apache.hudi.utilities.schema.SchemaProvider; | ||
| import org.apache.kafka.common.errors.SerializationException; | ||
|
|
||
| import java.util.Map; | ||
| import java.util.Map.Entry; | ||
| import java.util.Objects; | ||
|
|
||
| /** | ||
| * Extending {@link KafkaAvroSchemaDeserializer} as we need to be able to inject reader schema during deserialization. | ||
| */ | ||
| public class KafkaAvroSchemaDeserializer extends KafkaAvroDeserializer { | ||
|
|
||
| private static final String SCHEMA_PROVIDER_CLASS_PROP = "hoodie.deltastreamer.schemaprovider.class"; | ||
| private Schema sourceSchema; | ||
|
|
||
| public KafkaAvroSchemaDeserializer() {} | ||
|
|
||
| @Override | ||
| public void configure(Map<String, ?> configs, boolean isKey) { | ||
| super.configure(configs, isKey); | ||
| try { | ||
| TypedProperties props = getConvertToTypedProperties(configs); | ||
| String className = props.getString(SCHEMA_PROVIDER_CLASS_PROP); | ||
| SchemaProvider schemaProvider = (SchemaProvider) ReflectionUtils.loadClass(className, props); | ||
| sourceSchema = Objects.requireNonNull(schemaProvider).getSourceSchema(); | ||
| } catch (Throwable e) { | ||
| throw new HoodieException(e); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Pretty much copy-paste from the {@link AbstractKafkaAvroDeserializer} except line 87: | ||
| * DatumReader reader = new GenericDatumReader(schema, sourceSchema); | ||
| * <p> | ||
| * We need to inject reader schema during deserialization or later stages of the pipeline break. | ||
| * | ||
| * @param includeSchemaAndVersion | ||
| * @param topic | ||
| * @param isKey | ||
| * @param payload | ||
| * @param readerSchema | ||
| * @return | ||
| * @throws SerializationException | ||
| */ | ||
| @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 TypedProperties getConvertToTypedProperties(Map<String, ?> configs) { | ||
| TypedProperties typedProperties = new TypedProperties(); | ||
| for (Entry<String, ?> entry : configs.entrySet()) { | ||
| typedProperties.put(entry.getKey(), entry.getValue()); | ||
| } | ||
| return typedProperties; | ||
| } | ||
| } |
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
119 changes: 119 additions & 0 deletions
119
...ities/src/main/java/org/apache/hudi/utilities/sources/helpers/AvroKafkaSourceHelpers.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,119 @@ | ||
| /* | ||
| * 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.helpers; | ||
|
|
||
| import org.apache.avro.Schema; | ||
| import org.apache.avro.generic.GenericRecord; | ||
| import org.apache.kafka.clients.consumer.ConsumerRecord; | ||
|
|
||
| import java.util.ArrayList; | ||
| import java.util.Arrays; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
|
|
||
| public class AvroKafkaSourceHelpers { | ||
|
|
||
| public static final String INJECT_KAFKA_META_FIELDS = "hoodie.deltastreamer.source.inject_kafka_fields"; | ||
|
|
||
| public static final String KAFKA_PARTITION_META_FIELD = "_hudi_kafka_partition"; | ||
| public static final String KAFKA_OFFSET_META_FIELD = "_hudi_kafka_offset"; | ||
| public static final String KAFKA_TOPIC_META_FIELD = "_hudi_kafka_topic"; | ||
| public static final String KAFKA_KEY_META_FIELD = "_hudi_kafka_key"; | ||
| public static final String KAFKA_META_FIELDS_PATTERN = "<KAFKA_FIELDS>"; | ||
|
|
||
| public static final String ALL_KAFKA_META_FIELDS = String.join( | ||
| ",", | ||
| AvroKafkaSourceHelpers.KAFKA_PARTITION_META_FIELD, | ||
| AvroKafkaSourceHelpers.KAFKA_OFFSET_META_FIELD, | ||
| AvroKafkaSourceHelpers.KAFKA_TOPIC_META_FIELD, | ||
| AvroKafkaSourceHelpers.KAFKA_KEY_META_FIELD); | ||
|
|
||
| public static String transform(String sql) { | ||
| return sql.replaceAll(KAFKA_META_FIELDS_PATTERN, ALL_KAFKA_META_FIELDS); | ||
| } | ||
|
|
||
| public static GenericRecord addKafkaFields(ConsumerRecord<Object, Object> obj) { | ||
| GenericRecord record = (GenericRecord) obj.value(); | ||
| record.put(AvroKafkaSourceHelpers.KAFKA_OFFSET_META_FIELD, obj.offset()); | ||
| record.put(AvroKafkaSourceHelpers.KAFKA_PARTITION_META_FIELD, obj.partition()); | ||
| record.put(AvroKafkaSourceHelpers.KAFKA_TOPIC_META_FIELD, obj.topic()); | ||
| record.put(AvroKafkaSourceHelpers.KAFKA_KEY_META_FIELD, obj.key()); | ||
| return record; | ||
| } | ||
|
|
||
| private static boolean isKafkaMetadataField(String fieldName) { | ||
| return AvroKafkaSourceHelpers.KAFKA_PARTITION_META_FIELD.equals(fieldName) | ||
| || AvroKafkaSourceHelpers.KAFKA_OFFSET_META_FIELD.equals(fieldName) | ||
| || AvroKafkaSourceHelpers.KAFKA_TOPIC_META_FIELD.equals(fieldName) | ||
| || AvroKafkaSourceHelpers.KAFKA_KEY_META_FIELD.equals(fieldName); | ||
| } | ||
|
|
||
| /** | ||
| * Adds the Kafka metadata fields to the given schema, so later on the appropriate data can be injected. | ||
| */ | ||
| public static Schema addKafkaMetadataFields(Schema schema) { | ||
|
|
||
| final List<Schema.Field> parentFields = new ArrayList<>(); | ||
| final List<Schema.Field> schemaFields = schema.getFields(); | ||
|
|
||
| for (Schema.Field field : schemaFields) { | ||
| if (!isKafkaMetadataField(field.name())) { | ||
| Schema.Field newField = new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultVal()); | ||
| for (Map.Entry<String, Object> prop : field.getObjectProps().entrySet()) { | ||
| newField.addProp(prop.getKey(), prop.getValue()); | ||
| } | ||
| parentFields.add(newField); | ||
| } | ||
| } | ||
|
|
||
| final Schema.Field partitionField = | ||
| new Schema.Field( | ||
| AvroKafkaSourceHelpers.KAFKA_PARTITION_META_FIELD, | ||
| Schema.createUnion(Arrays.asList(Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.INT))), | ||
| "", | ||
| Schema.NULL_VALUE); | ||
| final Schema.Field offsetField = | ||
| new Schema.Field( | ||
| AvroKafkaSourceHelpers.KAFKA_OFFSET_META_FIELD, | ||
| Schema.createUnion(Arrays.asList(Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.LONG))), | ||
| "", | ||
| Schema.NULL_VALUE); | ||
| final Schema.Field topicField = | ||
| new Schema.Field( | ||
| AvroKafkaSourceHelpers.KAFKA_TOPIC_META_FIELD, | ||
| Schema.createUnion(Arrays.asList(Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.STRING))), | ||
| "", | ||
| Schema.NULL_VALUE); | ||
| final Schema.Field keyField = | ||
| new Schema.Field( | ||
| AvroKafkaSourceHelpers.KAFKA_KEY_META_FIELD, | ||
| Schema.createUnion(Arrays.asList(Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.STRING))), | ||
| "", | ||
| Schema.NULL_VALUE); | ||
|
|
||
| parentFields.add(partitionField); | ||
| parentFields.add(offsetField); | ||
| parentFields.add(topicField); | ||
| parentFields.add(keyField); | ||
|
|
||
| final Schema mergedSchema = Schema.createRecord(schema.getName(), schema.getDoc(), schema.getNamespace(), false); | ||
| mergedSchema.setFields(parentFields); | ||
| return mergedSchema; | ||
| } | ||
| } |
Oops, something went wrong.
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.