Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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._
Expand Down Expand Up @@ -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)
}

/**
Expand All @@ -154,41 +154,48 @@ 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
val containsNullSchema = innerFields.foldLeft(false)((nullFieldEncountered, schema) => nullFieldEncountered | schema.getType == Schema.Type.NULL)
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
Expand Down
27 changes: 27 additions & 0 deletions hudi-common/src/test/resources/simple-test-doced.avsc
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
{
"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string","doc":"name_comment"},
{"name": "favorite_number", "type": "int","doc":"favorite_number_comment"},
{"name": "favorite_color", "type": "string"}
]
}
Original file line number Diff line number Diff line change
Expand Up @@ -316,6 +316,8 @@ public static HiveSyncConfig buildHiveSyncConfig(TypedProperties props, String b
if (props.containsKey(HiveExternalCatalog.CREATED_SPARK_VERSION())) {
hiveSyncConfig.sparkVersion = props.getString(HiveExternalCatalog.CREATED_SPARK_VERSION());
}
hiveSyncConfig.syncComment = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_SYNC_COMMENT().key(),
DataSourceWriteOptions.HIVE_SYNC_COMMENT().defaultValue()));
return hiveSyncConfig;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -555,6 +555,11 @@ object DataSourceWriteOptions {
.withDocumentation("Whether sync hive metastore bucket specification when using bucket index." +
"The specification is 'CLUSTERED BY (trace_id) SORTED BY (trace_id ASC) INTO 65536 BUCKETS'")

val HIVE_SYNC_COMMENT: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.sync_comment")
.defaultValue("false")
.withDocumentation("Whether to sync the table column comments while syncing the table.")

// Async Compaction - Enabled by default for MOR
val ASYNC_COMPACT_ENABLE: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.compaction.async.enable")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -600,6 +600,7 @@ object HoodieSparkSqlWriter {
hiveSyncConfig.serdeProperties = hoodieConfig.getString(HIVE_TABLE_SERDE_PROPERTIES)
hiveSyncConfig.tableProperties = hoodieConfig.getString(HIVE_TABLE_PROPERTIES)
hiveSyncConfig.sparkVersion = SPARK_VERSION
hiveSyncConfig.syncComment = hoodieConfig.getStringOrDefault(HIVE_SYNC_COMMENT).toBoolean
hiveSyncConfig
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -161,4 +161,220 @@ class TestAvroConversionUtils extends FunSuite with Matchers {

assert(avroSchema.equals(expectedAvroSchema))
}

test("test convertStructTypeToAvroSchema with Nested StructField comment") {
val mapType = DataTypes.createMapType(StringType, new StructType().add("mapKey", "string", false, "mapKeyComment").add("mapVal", "integer", true))
val arrayType = ArrayType(new StructType().add("arrayKey", "string", false).add("arrayVal", "integer", true, "arrayValComment"))
val innerStruct = new StructType().add("innerKey","string",false, "innerKeyComment").add("value", "long", true, "valueComment")

val struct = new StructType().add("key", "string", false).add("version", "string", true, "versionComment")
.add("data1",innerStruct,false).add("data2",innerStruct,true)
.add("nullableMap", mapType, true).add("map",mapType,false)
.add("nullableArray", arrayType, true).add("array",arrayType,false)

val avroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(struct, "SchemaName", "SchemaNS")

val expectedSchemaStr = s"""
{
"type": "record",
"name": "SchemaName",
"namespace": "SchemaNS",
"fields": [
{
"name": "key",
"type": "string"
},
{
"name": "version",
"type": [
"null",
"string"
],
"doc": "versionComment",
"default": null
},
{
"name": "data1",
"type": {
"type": "record",
"name": "data1",
"namespace": "SchemaNS.SchemaName",
"fields": [
{
"name": "innerKey",
"type": "string",
"doc": "innerKeyComment"
},
{
"name": "value",
"type": [
"null",
"long"
],
"doc": "valueComment",
"default": null
}
]
}
},
{
"name": "data2",
"type": [
"null",
{
"type": "record",
"name": "data2",
"namespace": "SchemaNS.SchemaName",
"fields": [
{
"name": "innerKey",
"type": "string",
"doc": "innerKeyComment"
},
{
"name": "value",
"type": [
"null",
"long"
],
"doc": "valueComment",
"default": null
}
]
}
],
"default": null
},
{
"name": "nullableMap",
"type": [
"null",
{
"type": "map",
"values": [
{
"type": "record",
"name": "nullableMap",
"namespace": "SchemaNS.SchemaName",
"fields": [
{
"name": "mapKey",
"type": "string",
"doc": "mapKeyComment"
},
{
"name": "mapVal",
"type": [
"null",
"int"
],
"default": null
}
]
},
"null"
]
}
],
"default": null
},
{
"name": "map",
"type": {
"type": "map",
"values": [
{
"type": "record",
"name": "map",
"namespace": "SchemaNS.SchemaName",
"fields": [
{
"name": "mapKey",
"type": "string",
"doc": "mapKeyComment"
},
{
"name": "mapVal",
"type": [
"null",
"int"
],
"default": null
}
]
},
"null"
]
}
},
{
"name": "nullableArray",
"type": [
"null",
{
"type": "array",
"items": [
{
"type": "record",
"name": "nullableArray",
"namespace": "SchemaNS.SchemaName",
"fields": [
{
"name": "arrayKey",
"type": "string"
},
{
"name": "arrayVal",
"type": [
"null",
"int"
],
"doc": "arrayValComment",
"default": null
}
]
},
"null"
]
}
],
"default": null
},
{
"name": "array",
"type": {
"type": "array",
"items": [
{
"type": "record",
"name": "array",
"namespace": "SchemaNS.SchemaName",
"fields": [
{
"name": "arrayKey",
"type": "string"
},
{
"name": "arrayVal",
"type": [
"null",
"int"
],
"doc": "arrayValComment",
"default": null
}
]
},
"null"
]
}
}
]
}}
"""

val expectedAvroSchema = new Schema.Parser().parse(expectedSchemaStr)

assert(avroSchema.equals(expectedAvroSchema))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -132,6 +132,9 @@ public class HiveSyncConfig implements Serializable {
@Parameter(names = {"--spark-version"}, description = "The spark version", required = false)
public String sparkVersion;

@Parameter(names = {"--sync-comment"}, description = "synchronize table comments to hive")
public boolean syncComment = false;

// enhance the similar function in child class
public static HiveSyncConfig copy(HiveSyncConfig cfg) {
HiveSyncConfig newConfig = new HiveSyncConfig();
Expand Down Expand Up @@ -159,6 +162,7 @@ public static HiveSyncConfig copy(HiveSyncConfig cfg) {
newConfig.withOperationField = cfg.withOperationField;
newConfig.isConditionalSync = cfg.isConditionalSync;
newConfig.sparkVersion = cfg.sparkVersion;
newConfig.syncComment = cfg.syncComment;
return newConfig;
}

Expand Down Expand Up @@ -193,6 +197,7 @@ public String toString() {
+ ", sparkSchemaLengthThreshold=" + sparkSchemaLengthThreshold
+ ", withOperationField=" + withOperationField
+ ", isConditionalSync=" + isConditionalSync
+ ", syncComment=" + syncComment
+ '}';
}

Expand Down
Loading