[HUDI-1870] Add more Spark CI build tasks (#4022)
* [HUDI-1870] Add more Spark CI build tasks - build for spark3.0.x - build for spark-shade-unbundle-avro - fix build failures - delete unnecessary assertion for spark 3.0.x - use AvroConversionUtils#convertAvroSchemaToStructType instead of calling SchemaConverters#toSqlType directly to solve the compilation failures with spark-shade-unbundle-avro (#5) Co-authored-by: Yann <biyan900116@gmail.com>
This commit is contained in:
@@ -22,7 +22,9 @@ import java.util.Properties
|
||||
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.avro.generic.GenericRecord
|
||||
|
||||
import org.apache.hadoop.fs.{FileSystem, Path}
|
||||
|
||||
import org.apache.hudi.client.utils.SparkRowSerDe
|
||||
import org.apache.hudi.common.config.TypedProperties
|
||||
import org.apache.hudi.common.model.HoodieRecord
|
||||
@@ -30,9 +32,9 @@ import org.apache.hudi.common.table.HoodieTableMetaClient
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions
|
||||
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
|
||||
import org.apache.hudi.keygen.{BaseKeyGenerator, CustomAvroKeyGenerator, CustomKeyGenerator, KeyGenerator}
|
||||
|
||||
import org.apache.spark.SPARK_VERSION
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.avro.SchemaConverters
|
||||
import org.apache.spark.sql.catalyst.encoders.RowEncoder
|
||||
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, Literal}
|
||||
import org.apache.spark.sql.execution.datasources.{FileStatusCache, InMemoryFileIndex}
|
||||
@@ -137,13 +139,13 @@ object HoodieSparkUtils extends SparkAdapterSupport {
|
||||
def createRddInternal(df: DataFrame, writeSchema: Schema, latestTableSchema: Schema, structName: String, recordNamespace: String)
|
||||
: RDD[GenericRecord] = {
|
||||
// Use the write avro schema to derive the StructType which has the correct nullability information
|
||||
val writeDataType = SchemaConverters.toSqlType(writeSchema).dataType.asInstanceOf[StructType]
|
||||
val writeDataType = AvroConversionUtils.convertAvroSchemaToStructType(writeSchema)
|
||||
val encoder = RowEncoder.apply(writeDataType).resolveAndBind()
|
||||
val deserializer = sparkAdapter.createSparkRowSerDe(encoder)
|
||||
// if records were serialized with old schema, but an evolved schema was passed in with latestTableSchema, we need
|
||||
// latestTableSchema equivalent datatype to be passed in to AvroConversionHelper.createConverterToAvro()
|
||||
val reconciledDataType =
|
||||
if (latestTableSchema != null) SchemaConverters.toSqlType(latestTableSchema).dataType.asInstanceOf[StructType] else writeDataType
|
||||
if (latestTableSchema != null) AvroConversionUtils.convertAvroSchemaToStructType(latestTableSchema) else writeDataType
|
||||
// Note: deserializer.deserializeRow(row) is not capable of handling evolved schema. i.e. if Row was serialized in
|
||||
// old schema, but deserializer was created with an encoder with evolved schema, deserialization fails.
|
||||
// Hence we always need to deserialize in the same schema as serialized schema.
|
||||
|
||||
Reference in New Issue
Block a user