[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:
@@ -18,6 +18,7 @@
|
||||
package org.apache.hudi
|
||||
|
||||
import org.apache.hadoop.fs.Path
|
||||
|
||||
import org.apache.hudi.DataSourceReadOptions._
|
||||
import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord}
|
||||
import org.apache.hudi.DataSourceWriteOptions.{BOOTSTRAP_OPERATION_OPT_VAL, OPERATION}
|
||||
@@ -26,8 +27,9 @@ import org.apache.hudi.common.model.HoodieTableType.{COPY_ON_WRITE, MERGE_ON_REA
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.hudi.hadoop.HoodieROTablePathFilter
|
||||
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.sql.avro.SchemaConverters
|
||||
|
||||
import org.apache.spark.sql.execution.datasources.{DataSource, FileStatusCache, HadoopFsRelation}
|
||||
import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat
|
||||
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
|
||||
@@ -217,8 +219,7 @@ class DefaultSource extends RelationProvider
|
||||
// the table schema evolution.
|
||||
val tableSchemaResolver = new TableSchemaResolver(metaClient)
|
||||
try {
|
||||
Some(SchemaConverters.toSqlType(tableSchemaResolver.getTableAvroSchema)
|
||||
.dataType.asInstanceOf[StructType])
|
||||
Some(AvroConversionUtils.convertAvroSchemaToStructType(tableSchemaResolver.getTableAvroSchema))
|
||||
} catch {
|
||||
case _: Throwable =>
|
||||
None // If there is no commit in the table, we can not get the schema
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
package org.apache.hudi
|
||||
|
||||
import org.apache.hadoop.fs.{FileStatus, Path}
|
||||
|
||||
import org.apache.hudi.DataSourceReadOptions.{QUERY_TYPE, QUERY_TYPE_SNAPSHOT_OPT_VAL}
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig
|
||||
@@ -26,10 +27,10 @@ import org.apache.hudi.common.model.FileSlice
|
||||
import org.apache.hudi.common.model.HoodieTableType.MERGE_ON_READ
|
||||
import org.apache.hudi.common.table.view.{FileSystemViewStorageConfig, HoodieTableFileSystemView}
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.sql.{Column, SparkSession}
|
||||
import org.apache.spark.sql.avro.SchemaConverters
|
||||
import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, InterpretedPredicate}
|
||||
import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils}
|
||||
import org.apache.spark.sql.catalyst.{InternalRow, expressions}
|
||||
@@ -38,6 +39,7 @@ import org.apache.spark.sql.hudi.{DataSkippingUtils, HoodieSqlUtils}
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.unsafe.types.UTF8String
|
||||
|
||||
import java.util.Properties
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
@@ -96,8 +98,7 @@ case class HoodieFileIndex(
|
||||
*/
|
||||
lazy val schema: StructType = schemaSpec.getOrElse({
|
||||
val schemaUtil = new TableSchemaResolver(metaClient)
|
||||
SchemaConverters.toSqlType(schemaUtil.getTableAvroSchema)
|
||||
.dataType.asInstanceOf[StructType]
|
||||
AvroConversionUtils.convertAvroSchemaToStructType(schemaUtil.getTableAvroSchema)
|
||||
})
|
||||
|
||||
/**
|
||||
|
||||
@@ -23,7 +23,8 @@ import java.util.{Date, Locale, Properties}
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hudi.SparkAdapterSupport
|
||||
|
||||
import org.apache.hudi.{AvroConversionUtils, SparkAdapterSupport}
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext
|
||||
import org.apache.hudi.common.config.DFSPropertiesConfiguration
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig
|
||||
@@ -31,9 +32,8 @@ import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.model.HoodieRecord
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
|
||||
|
||||
import org.apache.spark.SPARK_VERSION
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.sql.avro.SchemaConverters
|
||||
import org.apache.spark.sql.{Column, DataFrame, SparkSession}
|
||||
import org.apache.spark.sql.catalyst.TableIdentifier
|
||||
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
|
||||
@@ -46,6 +46,7 @@ import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.sql.types.{DataType, NullType, StringType, StructField, StructType}
|
||||
|
||||
import java.text.SimpleDateFormat
|
||||
|
||||
import scala.collection.immutable.Map
|
||||
|
||||
object HoodieSqlUtils extends SparkAdapterSupport {
|
||||
@@ -83,8 +84,7 @@ object HoodieSqlUtils extends SparkAdapterSupport {
|
||||
catch {
|
||||
case _: Throwable => None
|
||||
}
|
||||
avroSchema.map(SchemaConverters.toSqlType(_).dataType
|
||||
.asInstanceOf[StructType]).map(removeMetaFields)
|
||||
avroSchema.map(AvroConversionUtils.convertAvroSchemaToStructType).map(removeMetaFields)
|
||||
}
|
||||
|
||||
def getAllPartitionPaths(spark: SparkSession, table: CatalogTable): Seq[String] = {
|
||||
|
||||
@@ -19,10 +19,13 @@ package org.apache.spark.sql.hudi.command.payload
|
||||
|
||||
import java.util.{Base64, Properties}
|
||||
import java.util.concurrent.Callable
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
import com.google.common.cache.CacheBuilder
|
||||
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.avro.generic.{GenericData, GenericRecord, IndexedRecord}
|
||||
|
||||
import org.apache.hudi.AvroConversionUtils
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.avro.HoodieAvroUtils
|
||||
import org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro
|
||||
@@ -31,12 +34,14 @@ import org.apache.hudi.common.util.{ValidationUtils, Option => HOption}
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.io.HoodieWriteHandle
|
||||
import org.apache.hudi.sql.IExpressionEvaluator
|
||||
|
||||
import org.apache.spark.sql.avro.{AvroSerializer, HoodieAvroSerializer, SchemaConverters}
|
||||
import org.apache.spark.sql.catalyst.expressions.Expression
|
||||
import org.apache.spark.sql.hudi.SerDeUtils
|
||||
import org.apache.spark.sql.hudi.command.payload.ExpressionPayload.getEvaluator
|
||||
import org.apache.spark.sql.types.{StructField, StructType}
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.collection.mutable.ArrayBuffer
|
||||
|
||||
/**
|
||||
@@ -309,7 +314,7 @@ object ExpressionPayload {
|
||||
SchemaConverters.toAvroType(conditionType), false)
|
||||
val conditionEvaluator = ExpressionCodeGen.doCodeGen(Seq(condition), conditionSerializer)
|
||||
|
||||
val assignSqlType = SchemaConverters.toSqlType(writeSchema).dataType.asInstanceOf[StructType]
|
||||
val assignSqlType = AvroConversionUtils.convertAvroSchemaToStructType(writeSchema)
|
||||
val assignSerializer = new HoodieAvroSerializer(assignSqlType, writeSchema, false)
|
||||
val assignmentEvaluator = ExpressionCodeGen.doCodeGen(assignments, assignSerializer)
|
||||
conditionEvaluator -> assignmentEvaluator
|
||||
|
||||
@@ -19,16 +19,18 @@ package org.apache.spark.sql.hudi.command.payload
|
||||
|
||||
import org.apache.avro.generic.IndexedRecord
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.spark.sql.avro.{HooodieAvroDeserializer, SchemaConverters}
|
||||
|
||||
import org.apache.hudi.AvroConversionUtils
|
||||
|
||||
import org.apache.spark.sql.avro.HooodieAvroDeserializer
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.types._
|
||||
|
||||
/**
|
||||
* A sql typed record which will convert the avro field to sql typed value.
|
||||
*/
|
||||
class SqlTypedRecord(val record: IndexedRecord) extends IndexedRecord {
|
||||
|
||||
private lazy val sqlType = SchemaConverters.toSqlType(getSchema).dataType.asInstanceOf[StructType]
|
||||
private lazy val sqlType = AvroConversionUtils.convertAvroSchemaToStructType(getSchema)
|
||||
private lazy val avroDeserializer = HooodieAvroDeserializer(record.getSchema, sqlType)
|
||||
private lazy val sqlRow = avroDeserializer.deserializeData(record).asInstanceOf[InternalRow]
|
||||
|
||||
|
||||
@@ -22,16 +22,17 @@ import java.nio.charset.StandardCharsets
|
||||
import java.util.Date
|
||||
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hudi.{DataSourceReadOptions, IncrementalRelation, MergeOnReadIncrementalRelation, SparkAdapterSupport}
|
||||
|
||||
import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, IncrementalRelation, MergeOnReadIncrementalRelation, SparkAdapterSupport}
|
||||
import org.apache.hudi.common.model.HoodieTableType
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.util.{FileIOUtils, TablePathUtils}
|
||||
|
||||
import org.apache.spark.sql.hudi.streaming.HoodieStreamSource.VERSION
|
||||
import org.apache.spark.sql.hudi.streaming.HoodieSourceOffset.INIT_OFFSET
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.avro.SchemaConverters
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.catalyst.encoders.RowEncoder
|
||||
import org.apache.spark.sql.execution.streaming.{HDFSMetadataLog, Offset, Source}
|
||||
@@ -118,8 +119,7 @@ class HoodieStreamSource(
|
||||
override def schema: StructType = {
|
||||
schemaOption.getOrElse {
|
||||
val schemaUtil = new TableSchemaResolver(metaClient)
|
||||
SchemaConverters.toSqlType(schemaUtil.getTableAvroSchema)
|
||||
.dataType.asInstanceOf[StructType]
|
||||
AvroConversionUtils.convertAvroSchemaToStructType(schemaUtil.getTableAvroSchema)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user