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 2022/02/24 07:38:26 UTC

[GitHub] [hudi] vinothchandar commented on a change in pull request #4789: [HUDI-1296] Support Metadata Table in Spark Datasource

vinothchandar commented on a change in pull request #4789:
URL: https://github.com/apache/hudi/pull/4789#discussion_r813615713



##########
File path: hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieAvroDeserializerTrait.scala
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.spark.sql.avro
+
+/**
+ * Deserializes Avro payload into Catalyst object
+ *
+ * NOTE: This is low-level component operating on Spark internal data-types (comprising [[InternalRow]]).
+ *       If you're looking to convert Avro into "deserialized" [[Row]] (comprised of Java native types),
+ *       please check [[AvroConversionUtils]]
+ */
+trait HoodieAvroDeserializerTrait {
+  final def deserialize(data: Any): Option[Any] =
+    doDeserialize(data) match {
+      case opt: Option[_] => opt    // As of Spark 3.1, this will return data wrapped with Option, so we fetch the data

Review comment:
       if there is code specific to spark versions, can you move them into the adapters? how does this work with spark 2.x? 

##########
File path: hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionHelper.scala
##########
@@ -1,380 +0,0 @@
-/*
- * 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
-
-import java.nio.ByteBuffer
-import java.sql.{Date, Timestamp}
-import java.time.Instant
-
-import org.apache.avro.Conversions.DecimalConversion
-import org.apache.avro.LogicalTypes.{TimestampMicros, TimestampMillis}
-import org.apache.avro.Schema.Type._
-import org.apache.avro.generic.GenericData.{Fixed, Record}
-import org.apache.avro.generic.{GenericData, GenericFixed, GenericRecord}
-import org.apache.avro.{LogicalTypes, Schema}
-
-import org.apache.spark.sql.Row
-import org.apache.spark.sql.avro.SchemaConverters
-import org.apache.spark.sql.catalyst.expressions.GenericRow
-import org.apache.spark.sql.catalyst.util.DateTimeUtils
-import org.apache.spark.sql.types._
-
-import org.apache.hudi.AvroConversionUtils._
-import org.apache.hudi.exception.HoodieIncompatibleSchemaException
-
-import scala.collection.JavaConverters._
-
-object AvroConversionHelper {
-
-  private def createDecimal(decimal: java.math.BigDecimal, precision: Int, scale: Int): Decimal = {
-    if (precision <= Decimal.MAX_LONG_DIGITS) {
-      // Constructs a `Decimal` with an unscaled `Long` value if possible.
-      Decimal(decimal.unscaledValue().longValue(), precision, scale)
-    } else {
-      // Otherwise, resorts to an unscaled `BigInteger` instead.
-      Decimal(decimal, precision, scale)
-    }
-  }
-
-  /**
-    *
-    * Returns a converter function to convert row in avro format to GenericRow of catalyst.
-    *
-    * @param sourceAvroSchema Source schema before conversion inferred from avro file by passed in
-    *                         by user.
-    * @param targetSqlType    Target catalyst sql type after the conversion.
-    * @return returns a converter function to convert row in avro format to GenericRow of catalyst.
-    */
-  def createConverterToRow(sourceAvroSchema: Schema,

Review comment:
       what happens to all this code? consolidated? if so, are we sure there are no subtle differences from the different conversion implementations?

##########
File path: hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala
##########
@@ -18,20 +18,105 @@
 
 package org.apache.hudi
 
-import org.apache.avro.Schema
-import org.apache.avro.JsonProperties
+import org.apache.avro.Schema.Type
 import org.apache.avro.generic.{GenericRecord, GenericRecordBuilder, IndexedRecord}
+import org.apache.avro.{AvroRuntimeException, JsonProperties, Schema}
+import org.apache.hudi.HoodieSparkUtils.sparkAdapter
 import org.apache.hudi.avro.HoodieAvroUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.avro.SchemaConverters
-import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.encoders.RowEncoder
+import org.apache.spark.sql.types.{DataType, StructType}
 import org.apache.spark.sql.{Dataset, Row, SparkSession}
 
-import scala.collection.JavaConverters._
 import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
 
 object AvroConversionUtils {
 
+  /**
+   * Check the nullability of the input Avro type and resolve it when it is nullable. The first
+   * return value is a [[Boolean]] indicating if the input Avro type is nullable. The second
+   * return value is either provided Avro type if it's not nullable, or its resolved non-nullable part
+   * in case it is
+   */
+  def resolveAvroTypeNullability(avroType: Schema): (Boolean, Schema) = {

Review comment:
       My usual comment. :) where are these methods from? what has changed

##########
File path: hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala
##########
@@ -41,8 +107,8 @@ object AvroConversionUtils {
         else {
           val schema = new Schema.Parser().parse(schemaStr)
           val dataType = convertAvroSchemaToStructType(schema)
-          val convertor = AvroConversionHelper.createConverterToRow(schema, dataType)
-          records.map { x => convertor(x).asInstanceOf[Row] }
+          val converter = createConverterToRow(schema, dataType)

Review comment:
       >we even have out own HoodieInternalRow extension
   This is a different code path altogether. 
   
   but I think its okay to depend on InternalRow. Spark's undocumented APIs are a pain regardless

##########
File path: hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala
##########
@@ -41,8 +107,8 @@ object AvroConversionUtils {
         else {
           val schema = new Schema.Parser().parse(schemaStr)
           val dataType = convertAvroSchemaToStructType(schema)
-          val convertor = AvroConversionHelper.createConverterToRow(schema, dataType)
-          records.map { x => convertor(x).asInstanceOf[Row] }
+          val converter = createConverterToRow(schema, dataType)

Review comment:
       To be clear, we already have a row writer path, which is exactly on par with any other framework's parquet writing. On the upsert path, where this code is used, the bottleneck is indexing and we already have significant speedups due to the rich set of indexing schemes.That said, using internalrow is ok




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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