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 2021/01/04 16:11:37 UTC

[GitHub] [hudi] vburenin commented on a change in pull request #2380: [Hudi 73] Adding support for vanilla AvroKafkaSource

vburenin commented on a change in pull request #2380:
URL: https://github.com/apache/hudi/pull/2380#discussion_r551397436



##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/serde/AbstractHoodieKafkaAvroDeserializer.java
##########
@@ -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) {

Review comment:
       Original AbstractKafkaAvroDeserializer decoder has a check 
   ```    
   if (payload == null) {
         return null;
   }
   ```
   Is removal of this check safe?

##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroKafkaSource.java
##########
@@ -45,15 +50,21 @@
   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);

Review comment:
       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.

##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/serde/AbstractHoodieKafkaAvroDeserializer.java
##########
@@ -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) {

Review comment:
       Do you know where readerSchema come from?




----------------------------------------------------------------
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