1
0

[HUDI-3018] Adding validation to dataframe scheme to ensure reserved field does not have diff data type (#4852)

This commit is contained in:
Sivabalan Narayanan
2022-02-27 11:59:23 -05:00
committed by GitHub
parent 2f99e8458a
commit d5444ff7ff
5 changed files with 33 additions and 13 deletions

View File

@@ -19,11 +19,10 @@ package org.apache.hudi
import org.apache.avro.Schema
import org.apache.avro.generic.GenericRecord
import org.apache.avro.reflect.AvroSchema
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hadoop.hive.conf.HiveConf
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.HoodieWriterUtils._
import org.apache.hudi.avro.HoodieAvroUtils
@@ -45,9 +44,7 @@ import org.apache.hudi.keygen.{TimestampBasedAvroKeyGenerator, TimestampBasedKey
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
import org.apache.hudi.sync.common.AbstractSyncTool
import org.apache.hudi.table.BulkInsertPartitioner
import org.apache.log4j.LogManager
import org.apache.spark.SPARK_VERSION
import org.apache.spark.api.java.JavaSparkContext
import org.apache.spark.sql.hive.HiveExternalCatalog
@@ -58,7 +55,6 @@ import org.apache.spark.sql._
import org.apache.spark.SparkContext
import java.util.Properties
import scala.collection.JavaConversions._
import scala.collection.mutable
import scala.collection.mutable.ListBuffer
@@ -242,6 +238,7 @@ object HoodieSparkSqlWriter {
if (reconcileSchema) {
schema = getLatestTableSchema(fs, basePath, sparkContext, schema)
}
validateSchemaForHoodieIsDeleted(schema)
sparkContext.getConf.registerAvroSchemas(schema)
log.info(s"Registered avro schema : ${schema.toString(true)}")
@@ -432,6 +429,14 @@ object HoodieSparkSqlWriter {
}
}
def validateSchemaForHoodieIsDeleted(schema: Schema): Unit = {
if (schema.getField(HoodieRecord.HOODIE_IS_DELETED) != null &&
AvroConversionUtils.resolveAvroTypeNullability(schema.getField(HoodieRecord.HOODIE_IS_DELETED).schema())._2.getType != Schema.Type.BOOLEAN) {
throw new HoodieException(HoodieRecord.HOODIE_IS_DELETED + " has to be BOOLEAN type. Passed in dataframe's schema has type "
+ schema.getField(HoodieRecord.HOODIE_IS_DELETED).schema().getType)
}
}
def bulkInsertAsRow(sqlContext: SQLContext,
parameters: Map[String, String],
df: DataFrame,
@@ -454,6 +459,7 @@ object HoodieSparkSqlWriter {
if (dropPartitionColumns) {
schema = generateSchemaWithoutPartitionColumns(partitionColumns, schema)
}
validateSchemaForHoodieIsDeleted(schema)
sparkContext.getConf.registerAvroSchemas(schema)
log.info(s"Registered avro schema : ${schema.toString(true)}")
if (parameters(INSERT_DROP_DUPS.key).toBoolean) {