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 @@ -287,7 +287,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 @@ -40,23 +40,28 @@ 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";
// public static final String SOURCE_SCHEMA_PROP = "hoodie.deltastreamer.schemaprovider.source.schema";
}

private final FileSystem fs;

private final Schema sourceSchema;

private Schema targetSchema;
private TypedProperties props;

public FilebasedSchemaProvider(TypedProperties props, JavaSparkContext jssc) {
super(props, jssc);
this.props = props;
DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.SOURCE_SCHEMA_FILE_PROP));
String sourceFile = props.getString(Config.SOURCE_SCHEMA_FILE_PROP);
this.fs = FSUtils.getFs(sourceFile, jssc.hadoopConfiguration(), true);
try {
this.sourceSchema = new Schema.Parser().parse(this.fs.open(new Path(sourceFile)));
// props.setProperty(Config.SOURCE_SCHEMA_PROP, sourceSchema.toString());
if (props.containsKey(Config.TARGET_SCHEMA_FILE_PROP)) {
this.targetSchema =
new Schema.Parser().parse(fs.open(new Path(props.getString(Config.TARGET_SCHEMA_FILE_PROP))));
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,130 @@
/*
* 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.serde;

import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.utilities.UtilHelpers;
import org.apache.hudi.utilities.schema.SchemaProvider;
import org.apache.hudi.utilities.serde.config.HoodieKafkaAvroDeserializationConfig;

import kafka.utils.VerifiableProperties;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.io.DatumReader;
import org.apache.avro.io.DecoderFactory;
import org.apache.avro.specific.SpecificDatumReader;
import org.apache.kafka.common.errors.SerializationException;

import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Map;
import java.util.Objects;
import java.util.Properties;

import static org.apache.hudi.utilities.serde.config.HoodieKafkaAvroDeserializationConfig.SCHEMA_PROVIDER_CLASS_PROP;

public class AbstractHoodieKafkaAvroDeserializer {

private final DecoderFactory decoderFactory = DecoderFactory.get();
private boolean useSpecificAvroReader = false;
private Schema sourceSchema;
private Schema targetSchema;

public AbstractHoodieKafkaAvroDeserializer(VerifiableProperties properties) {
// this.sourceSchema = new Schema.Parser().parse(properties.props().getProperty(FilebasedSchemaProvider.Config.SOURCE_SCHEMA_PROP));
TypedProperties typedProperties = new TypedProperties();
copyProperties(typedProperties, properties.props());
try {
SchemaProvider schemaProvider = UtilHelpers.createSchemaProvider(
typedProperties.getString(SCHEMA_PROVIDER_CLASS_PROP), typedProperties, null);
this.sourceSchema = Objects.requireNonNull(schemaProvider).getSourceSchema();
this.targetSchema = Objects.requireNonNull(schemaProvider).getTargetSchema();
} catch (IOException e) {
throw new RuntimeException(e);
}
}

private void copyProperties(TypedProperties typedProperties, Properties properties) {
for (Map.Entry<Object, Object> entry : properties.entrySet()) {
typedProperties.put(entry.getKey(), entry.getValue());
}
}

protected void configure(HoodieKafkaAvroDeserializationConfig config) {
useSpecificAvroReader = config
.getBoolean(HoodieKafkaAvroDeserializationConfig.SPECIFIC_AVRO_READER_CONFIG);
}

protected Object deserialize(byte[] payload) throws SerializationException {
return deserialize(null, null, payload, targetSchema);
}

/**
* Just like single-parameter version but accepts an Avro schema to use for reading.
*
* @param payload serialized data
* @param readerSchema schema to use for Avro read (optional, enables Avro projection)
* @return the deserialized object
*/
protected Object deserialize(byte[] payload, Schema readerSchema) throws SerializationException {
return deserialize(null, null, payload, readerSchema);
}

protected Object deserialize(String topic, Boolean isKey, byte[] payload, Schema readerSchema) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Original AbstractKafkaAvroDeserializer decoder has a check

if (payload == null) {
      return null;
}

Is removal of this check safe?

Copy link
Contributor

Choose a reason for hiding this comment

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

Do you know where readerSchema come from?

Copy link
Contributor

Choose a reason for hiding this comment

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

It should be coming from the configured schema provider.

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 assumes the message starts with schema version (code looks like from Confluent deserializer). It doesn't belong to AbstractHoodieKafkaAvroDeserializer.java

int length = buffer.limit() - 1 - 4;
Object result;
if (sourceSchema.getType().equals(Schema.Type.BYTES)) {
byte[] bytes = new byte[length];
buffer.get(bytes, 0, length);
result = bytes;
} else {
int start = buffer.position() + buffer.arrayOffset();
DatumReader reader = this.getDatumReader(sourceSchema, readerSchema);
Object object = reader.read(null, this.decoderFactory.binaryDecoder(buffer.array(), start, length, null));
if (sourceSchema.getType().equals(Schema.Type.STRING)) {
object = object.toString();
}
result = object;
}
return result;
} catch (IOException | RuntimeException e) {
throw new SerializationException("Error deserializing payload: ", e);
}
}

private ByteBuffer getByteBuffer(byte[] payload) {
ByteBuffer buffer = ByteBuffer.wrap(payload);
if (buffer.get() != 0) {
throw new SerializationException("Unknown magic byte!");
} else {
return buffer;
}
}

private DatumReader getDatumReader(Schema writerSchema, Schema readerSchema) {
if (this.useSpecificAvroReader) {
return new SpecificDatumReader(writerSchema, readerSchema);
} else {
return new GenericDatumReader(writerSchema, readerSchema);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,44 @@
/*
* 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.serde;

import org.apache.hudi.utilities.serde.config.HoodieKafkaAvroDeserializationConfig;

import kafka.serializer.Decoder;
import kafka.utils.VerifiableProperties;
import org.apache.avro.Schema;
import org.apache.kafka.common.errors.SerializationException;

public class HoodieKafkaAvroDecoder extends AbstractHoodieKafkaAvroDeserializer implements Decoder<Object> {

public HoodieKafkaAvroDecoder(VerifiableProperties properties) {
super(properties);
configure(new HoodieKafkaAvroDeserializationConfig(properties.props()));
}

@Override
public Object fromBytes(byte[] bytes) {
return deserialize(bytes);
}

@Override
protected Object deserialize(String topic, Boolean isKey, byte[] payload, Schema readerSchema) throws SerializationException {
return super.deserialize(topic, isKey, payload, readerSchema);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
/*
* 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.serde.config;

import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.config.ConfigDef;

import java.util.Map;

public class HoodieKafkaAvroDeserializationConfig extends AbstractConfig {

public static final String SPECIFIC_AVRO_READER_CONFIG = "specific.avro.reader";
public static final boolean SPECIFIC_AVRO_READER_DEFAULT = false;
public static final String SPECIFIC_AVRO_READER_DOC = "If true, tries to look up the SpecificRecord class ";

public static final String SCHEMA_PROVIDER_CLASS_PROP = "hoodie.deltastreamer.schemaprovider.class";

private static ConfigDef config;

static {
config = new ConfigDef()
.define(SPECIFIC_AVRO_READER_CONFIG, ConfigDef.Type.BOOLEAN, SPECIFIC_AVRO_READER_DEFAULT,
ConfigDef.Importance.LOW, SPECIFIC_AVRO_READER_DOC);
}

public HoodieKafkaAvroDeserializationConfig(Map<?,?> props) {
super(config, props);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,13 @@

package org.apache.hudi.utilities.sources;

import org.apache.hudi.DataSourceUtils;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics;
import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
import org.apache.hudi.utilities.schema.SchemaProvider;
import org.apache.hudi.utilities.serde.HoodieKafkaAvroDecoder;
import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen;
import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.CheckpointUtils;

Expand All @@ -37,6 +40,8 @@
import org.apache.spark.streaming.kafka010.LocationStrategies;
import org.apache.spark.streaming.kafka010.OffsetRange;

import java.util.Collections;

/**
* Reads avro serialized Kafka data, based on the confluent schema-registry.
*/
Expand All @@ -45,15 +50,21 @@ public class AvroKafkaSource extends AvroSource {
private static final Logger LOG = LogManager.getLogger(AvroKafkaSource.class);

private final KafkaOffsetGen offsetGen;

private final HoodieDeltaStreamerMetrics metrics;
private final String useSchemaRegistry = "hoodie.deltastreamer.source.avro.serde.useSchemaRegistry";

public AvroKafkaSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession,
SchemaProvider schemaProvider, HoodieDeltaStreamerMetrics metrics) {
super(props, sparkContext, sparkSession, schemaProvider);
this.metrics = metrics;
boolean useSchemaRegistryForDeserialization = props.getBoolean(useSchemaRegistry, true);
props.put("key.deserializer", StringDeserializer.class);
props.put("value.deserializer", KafkaAvroDeserializer.class);
if (useSchemaRegistryForDeserialization) {
props.put("value.deserializer", KafkaAvroDeserializer.class);
} else {
DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(FilebasedSchemaProvider.Config.SOURCE_SCHEMA_FILE_PROP));
props.put("value.deserializer", HoodieKafkaAvroDecoder.class);
Copy link
Contributor

Choose a reason for hiding this comment

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

Would be nice to have an ability to configure which decoder class is to use for value.deserializer to be able to handle internal data decodings specifics.

}
offsetGen = new KafkaOffsetGen(props);
}

Expand Down
Loading