You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2020/05/05 03:37:02 UTC

[GitHub] [incubator-hudi] afilipchik commented on a change in pull request #1562: [HUDI-837]: implemented custom deserializer for AvroKafkaSource

afilipchik commented on a change in pull request #1562:
URL: https://github.com/apache/incubator-hudi/pull/1562#discussion_r419843893



##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/serde/HoodieAvroKafkaDeserializer.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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 org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.schema.SchemaProvider;
+
+import io.confluent.kafka.serializers.AbstractKafkaAvroDeserializer;
+import io.confluent.kafka.serializers.KafkaAvroDeserializerConfig;
+import kafka.serializer.Decoder;
+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 HoodieAvroKafkaDeserializer extends AbstractKafkaAvroDeserializer implements Decoder<Object> {

Review comment:
       probably should be called Confluent... as it assumes that schema registry is from confluent.

##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/serde/HoodieAvroKafkaDeserializer.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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 org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.schema.SchemaProvider;
+
+import io.confluent.kafka.serializers.AbstractKafkaAvroDeserializer;
+import io.confluent.kafka.serializers.KafkaAvroDeserializerConfig;
+import kafka.serializer.Decoder;
+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 HoodieAvroKafkaDeserializer extends AbstractKafkaAvroDeserializer implements Decoder<Object> {
+
+  private final Schema sourceSchema;
+  private static final String SCHEMA_PROVIDER_CLASS_PROP = "hoodie.deltastreamer.schemaprovider.class";
+
+  public HoodieAvroKafkaDeserializer(VerifiableProperties properties) {
+    this.configure(new KafkaAvroDeserializerConfig(properties.props()));
+    TypedProperties typedProperties = new TypedProperties();
+    copyProperties(typedProperties, properties.props());
+    try {
+      SchemaProvider schemaProvider = UtilHelpers.createSchemaProvider(

Review comment:
       Oh, right. We actually have internal implementation which is very similar to this one, except reader schema thing.

##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/serde/HoodieAvroKafkaDeserializer.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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 org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.schema.SchemaProvider;
+
+import io.confluent.kafka.serializers.AbstractKafkaAvroDeserializer;
+import io.confluent.kafka.serializers.KafkaAvroDeserializerConfig;
+import kafka.serializer.Decoder;
+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 HoodieAvroKafkaDeserializer extends AbstractKafkaAvroDeserializer implements Decoder<Object> {
+
+  private final Schema sourceSchema;
+  private static final String SCHEMA_PROVIDER_CLASS_PROP = "hoodie.deltastreamer.schemaprovider.class";
+
+  public HoodieAvroKafkaDeserializer(VerifiableProperties 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);
+      sourceSchema = Objects.requireNonNull(schemaProvider).getSourceSchema();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public Object fromBytes(byte[] bytes) {
+    return deserialize(bytes);
+  }
+
+  @Override
+  protected Object deserialize(boolean includeSchemaAndVersion, String topic, Boolean isKey, byte[] payload, Schema readerSchema) throws SerializationException {
+    if (readerSchema != null) {
+      return super.deserialize(includeSchemaAndVersion, topic, isKey, payload, readerSchema);
+    }

Review comment:
       What does this check do? Should we always use sourceSchema over reader schema if it is not null as if I specified source schema I expect it to be used? 
   
   What situation will have reader schema specified and preferred?

##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroKafkaSource.java
##########
@@ -45,11 +46,14 @@
 
   private final KafkaOffsetGen offsetGen;
 
+  private final String useCustomDeserializerProp = "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(useCustomDeserializerProp, false);
     props.put("key.deserializer", StringDeserializer.class);
-    props.put("value.deserializer", KafkaAvroDeserializer.class);
+    props.put("value.deserializer", useCustomDeserializer ? HoodieAvroKafkaDeserializer.class : KafkaAvroDeserializer.class);

Review comment:
       is it fair to assume confluent is default for avro?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org