1
0

[HUDI-3383] Sync column comments while syncing a hive table (#4960)

Desc: Add a hive sync config(hoodie.datasource.hive_sync.sync_comment). This config defaults to false.
While syncing data source to hudi, add column comments to source avro schema, and the sync_comment is true, syncing column comments to the hive table.
This commit is contained in:
MrSleeping123
2022-03-10 09:44:39 +08:00
committed by GitHub
parent 548000b0d6
commit 8859b48b2a
13 changed files with 467 additions and 14 deletions

View File

@@ -26,7 +26,7 @@ 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.types.{DataType, StructType}
import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructType}
import org.apache.spark.sql.{Dataset, Row, SparkSession}
import scala.collection.JavaConversions._
@@ -144,7 +144,7 @@ object AvroConversionUtils {
def convertStructTypeToAvroSchema(structType: DataType,
structName: String,
recordNamespace: String): Schema = {
getAvroSchemaWithDefaults(SchemaConverters.toAvroType(structType, nullable = false, structName, recordNamespace))
getAvroSchemaWithDefaults(SchemaConverters.toAvroType(structType, nullable = false, structName, recordNamespace), structType)
}
/**
@@ -154,13 +154,20 @@ object AvroConversionUtils {
* @param schema input avro schema
* @return Avro schema with null default set to nullable fields
*/
def getAvroSchemaWithDefaults(schema: Schema): Schema = {
def getAvroSchemaWithDefaults(schema: Schema, dataType: DataType): Schema = {
schema.getType match {
case Schema.Type.RECORD => {
val structType = dataType.asInstanceOf[StructType]
val structFields = structType.fields
val modifiedFields = schema.getFields.map(field => {
val newSchema = getAvroSchemaWithDefaults(field.schema())
val i: Int = structType.fieldIndex(field.name())
val comment: String = if (structFields(i).metadata.contains("comment")) {
structFields(i).metadata.getString("comment")
} else {
field.doc()
}
val newSchema = getAvroSchemaWithDefaults(field.schema(), structFields(i).dataType)
field.schema().getType match {
case Schema.Type.UNION => {
val innerFields = newSchema.getTypes
@@ -168,27 +175,27 @@ object AvroConversionUtils {
if(containsNullSchema) {
// Need to re shuffle the fields in list because to set null as default, null schema must be head in union schema
val restructuredNewSchema = Schema.createUnion(List(Schema.create(Schema.Type.NULL)) ++ innerFields.filter(innerSchema => !(innerSchema.getType == Schema.Type.NULL)))
new Schema.Field(field.name(), restructuredNewSchema, field.doc(), JsonProperties.NULL_VALUE)
new Schema.Field(field.name(), restructuredNewSchema, comment, JsonProperties.NULL_VALUE)
} else {
new Schema.Field(field.name(), newSchema, field.doc(), field.defaultVal())
new Schema.Field(field.name(), newSchema, comment, field.defaultVal())
}
}
case _ => new Schema.Field(field.name(), newSchema, field.doc(), field.defaultVal())
case _ => new Schema.Field(field.name(), newSchema, comment, field.defaultVal())
}
}).toList
Schema.createRecord(schema.getName, schema.getDoc, schema.getNamespace, schema.isError, modifiedFields)
}
case Schema.Type.UNION => {
Schema.createUnion(schema.getTypes.map(innerSchema => getAvroSchemaWithDefaults(innerSchema)))
Schema.createUnion(schema.getTypes.map(innerSchema => getAvroSchemaWithDefaults(innerSchema, dataType)))
}
case Schema.Type.MAP => {
Schema.createMap(getAvroSchemaWithDefaults(schema.getValueType))
Schema.createMap(getAvroSchemaWithDefaults(schema.getValueType, dataType.asInstanceOf[MapType].valueType))
}
case Schema.Type.ARRAY => {
Schema.createArray(getAvroSchemaWithDefaults(schema.getElementType))
Schema.createArray(getAvroSchemaWithDefaults(schema.getElementType, dataType.asInstanceOf[ArrayType].elementType))
}
case _ => schema