1
0

[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:
Raymond Xu
2021-11-22 02:16:45 -08:00
committed by GitHub
parent 8281cbf762
commit 02f7ca2b05
10 changed files with 48 additions and 32 deletions

View File

@@ -21,18 +21,19 @@ package org.apache.hudi.integ.testsuite.utils
import org.apache.avro.Schema
import org.apache.avro.generic.GenericRecord
import org.apache.hudi.HoodieSparkUtils
import org.apache.hudi.{AvroConversionUtils, HoodieSparkUtils}
import org.apache.hudi.common.model.HoodieRecord
import org.apache.hudi.common.util.Option
import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config
import org.apache.hudi.integ.testsuite.generator.GenericRecordFullPayloadGenerator
import org.apache.hudi.integ.testsuite.utils.SparkSqlUtils.getFieldNamesAndTypes
import org.apache.hudi.utilities.schema.RowBasedSchemaProvider
import org.apache.spark.api.java.JavaRDD
import org.apache.spark.sql.avro.SchemaConverters
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.{AnalysisException, SparkSession}
import org.apache.spark.sql.SparkSession
import org.apache.spark.storage.StorageLevel
import org.slf4j.Logger
import scala.math.BigDecimal.RoundingMode.RoundingMode
@@ -139,7 +140,7 @@ object SparkSqlUtils {
*/
def getFieldNamesAndTypes(avroSchemaString: String): Array[(String, String)] = {
val schema = new Schema.Parser().parse(avroSchemaString)
val structType = SchemaConverters.toSqlType(schema).dataType.asInstanceOf[StructType]
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
structType.fields.map(field => (field.name, field.dataType.simpleString))
}