Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 0 additions & 1 deletion hudi-utilities/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -251,7 +251,6 @@
<groupId>org.apache.kafka</groupId>
<artifactId>kafka_${scala.binary.version}</artifactId>
<version>${kafka.version}</version>
<scope>test</scope>
</dependency>

<!-- Httpcomponents -->
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -106,6 +106,16 @@ public static SchemaProvider createSchemaProvider(String schemaProviderClass, Ty
}
}

public static SchemaProvider createSchemaProvider(String schemaProviderClass,
TypedProperties cfg) throws IOException {
try {
return schemaProviderClass == null ? null :
(SchemaProvider) ReflectionUtils.loadClass(schemaProviderClass, cfg);
} catch (Throwable e) {
throw new HoodieException("Could not load schema provider class " + schemaProviderClass, e);
}
}

public static Option<Transformer> createTransformer(List<String> classNames) throws IOException {
try {
List<Transformer> transformers = new ArrayList<>();
Expand Down
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\"}]}");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ public class FilebasedSchemaProvider extends SchemaProvider {
* Configs supported.
*/
public static class Config {
private static final String SOURCE_SCHEMA_FILE_PROP = "hoodie.deltastreamer.schemaprovider.source.schema.file";
public static final String SOURCE_SCHEMA_FILE_PROP = "hoodie.deltastreamer.schemaprovider.source.schema.file";
private static final String TARGET_SCHEMA_FILE_PROP = "hoodie.deltastreamer.schemaprovider.target.schema.file";
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,10 @@ private static String fetchSchemaFromRegistry(String registryUrl) throws IOExcep
return node.get("schema").asText();
}

public SchemaRegistryProvider(TypedProperties properties) {
this(properties, null);
}

public SchemaRegistryProvider(TypedProperties props, JavaSparkContext jssc) {
super(props, jssc);
DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.SRC_SCHEMA_REGISTRY_URL_PROP));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.apache.hudi.utilities.schema.SchemaProvider;
import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen;
import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.CheckpointUtils;
import org.apache.hudi.utilities.sources.serde.HoodieAvroKafkaDecoder;

import io.confluent.kafka.serializers.KafkaAvroDeserializer;
import org.apache.avro.generic.GenericRecord;
Expand All @@ -45,11 +46,14 @@ public class AvroKafkaSource extends AvroSource {

private final KafkaOffsetGen offsetGen;

private static final String USE_CUSTOM_DESERIALIZER_PROP = "hoodie.deltastreamer.kafka.custom.avro.deserializer";

public AvroKafkaSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession,
SchemaProvider schemaProvider) {
super(props, sparkContext, sparkSession, schemaProvider);
boolean useCustomDeserializer = props.getBoolean(USE_CUSTOM_DESERIALIZER_PROP, false);
props.put("key.deserializer", StringDeserializer.class);
props.put("value.deserializer", KafkaAvroDeserializer.class);
props.put("value.deserializer", useCustomDeserializer ? HoodieAvroKafkaDecoder.class : KafkaAvroDeserializer.class);
offsetGen = new KafkaOffsetGen(props);
}

Expand Down
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());
}
}
}
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));
}
}