Skip to content
Open
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 @@ -170,7 +170,7 @@ object AvroConversionUtils {
val loader: java.util.function.Function[Schema, StructType] = key => {
try {
HoodieSparkAvroSchemaConverters.toSqlType(key) match {
case (dataType, _) => dataType.asInstanceOf[StructType]
case (dataType, _, _) => dataType.asInstanceOf[StructType]
}
} catch {
case e: Exception => throw new HoodieSchemaException("Failed to convert avro schema to struct type: " + avroSchema, e)
Expand All @@ -185,7 +185,7 @@ object AvroConversionUtils {
def convertAvroSchemaToDataType(avroSchema: Schema): DataType = {
try {
HoodieSparkAvroSchemaConverters.toSqlType(avroSchema) match {
case (dataType, _) => dataType
case (dataType, _, _) => dataType
}
} catch {
case e: Exception => throw new HoodieSchemaException("Failed to convert avro schema to DataType: " + avroSchema, e)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ import org.apache.spark.sql.types.DataType
*/
trait HoodieAvroSchemaConverters {

def toSqlType(avroSchema: Schema): (DataType, Boolean)
def toSqlType(avroSchema: Schema): (DataType, Boolean, Option[String])

def toAvroType(catalystType: DataType, nullable: Boolean, recordName: String, nameSpace: String = ""): Schema

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,9 +29,9 @@ import org.apache.spark.sql.types.DataType
*/
object HoodieSparkAvroSchemaConverters extends HoodieAvroSchemaConverters {

override def toSqlType(avroSchema: Schema): (DataType, Boolean) =
override def toSqlType(avroSchema: Schema): (DataType, Boolean, Option[String]) =
SchemaConverters.toSqlType(avroSchema) match {
case SchemaType(dataType, nullable) => (dataType, nullable)
case SchemaType(dataType, nullable, doc) => (dataType, nullable, doc)
}

override def toAvroType(catalystType: DataType, nullable: Boolean, recordName: String, nameSpace: String): Schema =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ private[sql] object SchemaConverters {
*
* @since 2.4.0
*/
case class SchemaType(dataType: DataType, nullable: Boolean)
case class SchemaType(dataType: DataType, nullable: Boolean, doc: Option[String])

/**
* Converts an Avro schema to a corresponding Spark SQL schema.
Expand All @@ -62,33 +62,33 @@ private[sql] object SchemaConverters {
private val unionFieldMemberPrefix = "member"

private def toSqlTypeHelper(avroSchema: Schema, existingRecordNames: Set[String]): SchemaType = {
avroSchema.getType match {
case INT => avroSchema.getLogicalType match {
case _: Date => SchemaType(DateType, nullable = false)
case _ => SchemaType(IntegerType, nullable = false)
(avroSchema.getType, Option(avroSchema.getDoc)) match {
case (INT, doc) => avroSchema.getLogicalType match {
case _: Date => SchemaType(DateType, nullable = false, doc)
case _ => SchemaType(IntegerType, nullable = false, doc)
}
case STRING => SchemaType(StringType, nullable = false)
case BOOLEAN => SchemaType(BooleanType, nullable = false)
case BYTES | FIXED => avroSchema.getLogicalType match {
case (STRING, doc) => SchemaType(StringType, nullable = false, doc)
case (BOOLEAN, doc) => SchemaType(BooleanType, nullable = false, doc)
case (BYTES | FIXED, doc) => avroSchema.getLogicalType match {
// For FIXED type, if the precision requires more bytes than fixed size, the logical
// type will be null, which is handled by Avro library.
case d: Decimal => SchemaType(DecimalType(d.getPrecision, d.getScale), nullable = false)
case _ => SchemaType(BinaryType, nullable = false)
case d: Decimal => SchemaType(DecimalType(d.getPrecision, d.getScale), nullable = false, doc)
case _ => SchemaType(BinaryType, nullable = false, doc)
}

case DOUBLE => SchemaType(DoubleType, nullable = false)
case FLOAT => SchemaType(FloatType, nullable = false)
case LONG => avroSchema.getLogicalType match {
case _: TimestampMillis | _: TimestampMicros => SchemaType(TimestampType, nullable = false)
case _: LocalTimestampMillis | _: LocalTimestampMicros => SchemaType(TimestampNTZType, nullable = false)
case _ => SchemaType(LongType, nullable = false)
case (DOUBLE, doc) => SchemaType(DoubleType, nullable = false, doc)
case (FLOAT, doc) => SchemaType(FloatType, nullable = false, doc)
case (LONG, doc) => avroSchema.getLogicalType match {
case _: TimestampMillis | _: TimestampMicros => SchemaType(TimestampType, nullable = false, doc)
case _: LocalTimestampMillis | _: LocalTimestampMicros => SchemaType(TimestampNTZType, nullable = false, doc)
case _ => SchemaType(LongType, nullable = false, doc)
}

case ENUM => SchemaType(StringType, nullable = false)
case (ENUM, doc) => SchemaType(StringType, nullable = false, doc)

case NULL => SchemaType(NullType, nullable = true)
case (NULL, doc) => SchemaType(NullType, nullable = true, doc)

case RECORD =>
case (RECORD, doc) =>
if (existingRecordNames.contains(avroSchema.getFullName)) {
throw new IncompatibleSchemaException(
s"""
Expand All @@ -107,21 +107,21 @@ private[sql] object SchemaConverters {
StructField(f.name, schemaType.dataType, schemaType.nullable, metadata)
}

SchemaType(StructType(fields.toSeq), nullable = false)
SchemaType(StructType(fields.toSeq), nullable = false, doc)

case ARRAY =>
case (ARRAY, doc) =>
val schemaType = toSqlTypeHelper(avroSchema.getElementType, existingRecordNames)
SchemaType(
ArrayType(schemaType.dataType, containsNull = schemaType.nullable),
nullable = false)
nullable = false, doc)

case MAP =>
case (MAP, doc) =>
val schemaType = toSqlTypeHelper(avroSchema.getValueType, existingRecordNames)
SchemaType(
MapType(StringType, schemaType.dataType, valueContainsNull = schemaType.nullable),
nullable = false)
nullable = false, doc)

case UNION =>
case (UNION, doc) =>
if (avroSchema.getTypes.asScala.exists(_.getType == NULL)) {
// In case of a union with null, eliminate it and make a recursive call
val remainingUnionTypes = avroSchema.getTypes.asScala.filterNot(_.getType == NULL)
Expand All @@ -135,20 +135,21 @@ private[sql] object SchemaConverters {
case Seq(t1) =>
toSqlTypeHelper(avroSchema.getTypes.get(0), existingRecordNames)
case Seq(t1, t2) if Set(t1, t2) == Set(INT, LONG) =>
SchemaType(LongType, nullable = false)
SchemaType(LongType, nullable = false, doc)
case Seq(t1, t2) if Set(t1, t2) == Set(FLOAT, DOUBLE) =>
SchemaType(DoubleType, nullable = false)
SchemaType(DoubleType, nullable = false, doc)
case _ =>
// Convert complex unions to struct types where field names are member0, member1, etc.
// This is consistent with the behavior when converting between Avro and Parquet.
val fields = avroSchema.getTypes.asScala.zipWithIndex.map {
case (s, i) =>
val schemaType = toSqlTypeHelper(s, existingRecordNames)
// All fields are nullable because only one of them is set at a time
StructField(s"$unionFieldMemberPrefix$i", schemaType.dataType, nullable = true)
val metadata = if(schemaType.doc.isDefined) new MetadataBuilder().putString("comment", schemaType.doc.get).build() else Metadata.empty
StructField(s"$unionFieldMemberPrefix$i", schemaType.dataType, nullable = true, metadata)
}

SchemaType(StructType(fields.toSeq), nullable = false)
SchemaType(StructType(fields.toSeq), nullable = false, doc)
}

case other => throw new IncompatibleSchemaException(s"Unsupported type $other")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -192,14 +192,17 @@ public void testCreateAndGetHoodieTable(HoodieTableType tableType) throws Except
+ "{\"name\":\"uuid\",\"type\":\"integer\",\"nullable\":false,\"metadata\":{}},"
+ "{\"name\":\"name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},"
+ "{\"name\":\"age\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},"
+ "{\"name\":\"infos\",\"type\":{\"type\":\"array\", \"elementType\":\"string\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}},"
+ "{\"name\":\"infos\",\"type\":{\"type\":\"array\",\"elementType\":\"string\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}},"
+ "{\"name\":\"ts_3\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},"
+ "{\"name\":\"ts_6\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},"
+ "{\"name\":\"par1\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}";
assertEquals(expectedAvroSchemaStr, avroSchemaStr);
// Use robust JSON comparison instead of brittle string comparison
ObjectMapper mapper = new ObjectMapper();
JsonNode expectedNode = mapper.readTree(expectedAvroSchemaStr);
JsonNode actualNode = mapper.readTree(avroSchemaStr);
assertEquals(expectedNode, actualNode, "Schema JSON structure doesn't match");

// validate array field nullable
ObjectMapper mapper = new ObjectMapper();
JsonNode arrayFieldTypeNode = mapper.readTree(avroSchemaStr).get("fields").get(8).get("type");
assertThat(arrayFieldTypeNode.get("type").asText(), is("array"));
assertThat(arrayFieldTypeNode.get("containsNull").asBoolean(), is(true));
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,84 @@
/*
* 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.
*/

package org.apache.hudi.functional

import org.apache.hudi.DataSourceWriteOptions
import org.apache.hudi.common.model.{HoodieTableType}
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.testutils.HoodieClientTestUtils.getSparkConfForTest

import org.apache.spark.SparkContext
import org.apache.spark.sql._
import org.apache.spark.sql.hudi.HoodieSparkSessionExtension
import org.apache.spark.sql.types.StructType
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.BeforeEach
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.EnumSource


class TestColumnComments {
var spark : SparkSession = _
var sqlContext: SQLContext = _
var sc : SparkContext = _

def initSparkContext(): Unit = {
val sparkConf = getSparkConfForTest(getClass.getSimpleName)
spark = SparkSession.builder()
.withExtensions(new HoodieSparkSessionExtension)
.config(sparkConf)
.getOrCreate()
sc = spark.sparkContext
sc.setLogLevel("ERROR")
sqlContext = spark.sqlContext
}

@BeforeEach
def setUp() {
initSparkContext()
}

@ParameterizedTest
@EnumSource(value = classOf[HoodieTableType], names = Array("COPY_ON_WRITE", "MERGE_ON_READ"))
def testColumnCommentWithSparkDatasource(tableType: HoodieTableType): Unit = {
val basePath = java.nio.file.Files.createTempDirectory("hoodie_comments_path").toAbsolutePath.toString
val opts = Map(
HoodieWriteConfig.TBL_NAME.key -> "hoodie_comments",
DataSourceWriteOptions.TABLE_TYPE.key -> tableType.toString,
DataSourceWriteOptions.OPERATION.key -> "bulk_insert",
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition"
)
val inputDF = spark.sql("select '0' as _row_key, '1' as content, '2' as partition, '3' as ts")
val struct = new StructType()
.add("_row_key", "string", true, "dummy comment")
.add("content", "string", true)
.add("partition", "string", true)
.add("ts", "string", true)
spark.createDataFrame(inputDF.rdd, struct)
.write.format("hudi")
.options(opts)
.mode(SaveMode.Overwrite)
.save(basePath)
spark.read.format("hudi").load(basePath).registerTempTable("test_tbl")

// now confirm the comment is present at read time
assertEquals(1, spark.sql("desc extended test_tbl")
.filter("col_name = '_row_key' and comment = 'dummy comment'").count)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ class TestAvroSerDe extends SparkAdapterSupport {
}

val schema = HoodieSchema.fromAvroSchema(HoodieMetadataColumnStats.SCHEMA$)
val SchemaType(catalystSchema, _) = SchemaConverters.toSqlType(schema.getAvroSchema)
val SchemaType(catalystSchema, _, _) = SchemaConverters.toSqlType(schema.getAvroSchema)

val deserializer = sparkAdapter.createAvroDeserializer(schema, catalystSchema)
val serializer = sparkAdapter.createAvroSerializer(catalystSchema, schema, nullable = false)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,22 +22,40 @@ import org.apache.hudi.avro.model.HoodieMetadataColumnStats

import org.apache.avro.JsonProperties
import org.apache.spark.sql.avro.SchemaConverters.SchemaType
import org.apache.spark.sql.types._
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.Test

class TestSchemaConverters {

/**
* Helper method to strip metadata from DataType structures for comparison.
* This allows us to compare the core schema structure while ignoring documentation/comments.
*/
private def stripMetadata(dataType: DataType): DataType = dataType match {
case StructType(fields) => StructType(fields.map(f =>
StructField(f.name, stripMetadata(f.dataType), f.nullable, Metadata.empty)))
case ArrayType(elementType, containsNull) => ArrayType(stripMetadata(elementType), containsNull)
case MapType(keyType, valueType, valueContainsNull) =>
MapType(stripMetadata(keyType), stripMetadata(valueType), valueContainsNull)
case other => other
}

@Test
def testAvroUnionConversion(): Unit = {
val originalAvroSchema = HoodieMetadataColumnStats.SCHEMA$

val SchemaType(convertedStructType, _) = SchemaConverters.toSqlType(originalAvroSchema)
val SchemaType(convertedStructType, _, _) = SchemaConverters.toSqlType(originalAvroSchema)
val convertedAvroSchema = SchemaConverters.toAvroType(convertedStructType)

// NOTE: Here we're validating that converting Avro -> Catalyst and Catalyst -> Avro are inverse
// transformations, but since it's not an easy endeavor to match Avro schemas, we match
// derived Catalyst schemas instead
assertEquals(convertedStructType, SchemaConverters.toSqlType(convertedAvroSchema).dataType)
// transformations for the core data structure. We strip metadata (comments/docs) since
// the toAvroType method doesn't preserve documentation from StructField metadata, making
// perfect round-trip conversion with docs challenging for complex union schemas.
val firstConversion = stripMetadata(convertedStructType)
val secondConversion = stripMetadata(SchemaConverters.toSqlType(convertedAvroSchema).dataType)

assertEquals(firstConversion, secondConversion)
// validate that the doc string and default null value are set
originalAvroSchema.getFields.forEach { field =>
val convertedField = convertedAvroSchema.getField(field.name())
Expand Down
6 changes: 6 additions & 0 deletions hudi-sync/hudi-adb-sync/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,12 @@
<artifactId>hudi-sync-common</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hudi</groupId>
<artifactId>hudi-io</artifactId>
<version>${project.version}</version>
<classifier>shaded</classifier>
</dependency>
<dependency>
<groupId>org.apache.hudi</groupId>
<artifactId>hudi-hive-sync</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.schema.HoodieSchema;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.common.util.ConfigUtils;
import org.apache.hudi.common.util.HadoopConfigUtils;
import org.apache.hudi.common.util.Option;
Expand All @@ -33,6 +34,7 @@
import org.apache.hudi.sync.common.util.SparkDataSourceTableUtils;

import com.beust.jcommander.JCommander;
import org.apache.avro.Schema;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat;
import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe;
Expand Down Expand Up @@ -219,13 +221,20 @@ private void syncSchema(String tableName, boolean tableExists, boolean useRealTi
Map<String, String> tableProperties = ConfigUtils.toMap(config.getString(ADB_SYNC_TABLE_PROPERTIES));
Map<String, String> serdeProperties = ConfigUtils.toMap(config.getString(ADB_SYNC_SERDE_PROPERTIES));
if (config.getBoolean(ADB_SYNC_SYNC_AS_SPARK_DATA_SOURCE_TABLE)) {
Map<String, String> sparkTableProperties = SparkDataSourceTableUtils.getSparkTableProperties(config.getSplitStrings(META_SYNC_PARTITION_FIELDS),
config.getString(META_SYNC_SPARK_VERSION), config.getInt(ADB_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD), schema);
Map<String, String> sparkSerdeProperties = SparkDataSourceTableUtils.getSparkSerdeProperties(readAsOptimized, config.getString(META_SYNC_BASE_PATH));
tableProperties.putAll(sparkTableProperties);
serdeProperties.putAll(sparkSerdeProperties);
LOG.info("Sync as spark datasource table, tableName:{}, tableExists:{}, tableProperties:{}, sederProperties:{}",
tableName, tableExists, tableProperties, serdeProperties);
try {
// Always include metadata fields for ADB sync
Schema avroSchema = new TableSchemaResolver(syncClient.getMetaClient()).getTableAvroSchema(true);
HoodieSchema hoodieSchema = HoodieSchema.fromAvroSchema(avroSchema);
Map<String, String> sparkTableProperties = SparkDataSourceTableUtils.getSparkTableProperties(config.getSplitStrings(META_SYNC_PARTITION_FIELDS),
config.getString(META_SYNC_SPARK_VERSION), config.getInt(ADB_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD), hoodieSchema);
Map<String, String> sparkSerdeProperties = SparkDataSourceTableUtils.getSparkSerdeProperties(readAsOptimized, config.getString(META_SYNC_BASE_PATH));
tableProperties.putAll(sparkTableProperties);
serdeProperties.putAll(sparkSerdeProperties);
LOG.info("Sync as spark datasource table, tableName:{}, tableExists:{}, tableProperties:{}, sederProperties:{}",
tableName, tableExists, tableProperties, serdeProperties);
} catch (Exception e) {
throw new HoodieAdbSyncException("Failed to get Avro schema for ADB sync", e);
}
}

// Check and sync schema
Expand Down
Loading
Loading