From 85e8a5c4de8fa59b769676149060eca13809cba6 Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Thu, 24 Feb 2022 13:23:13 -0800 Subject: [PATCH] [HUDI-1296] Support Metadata Table in Spark Datasource (#4789) * Bootstrapping initial support for Metadata Table in Spark Datasource - Consolidated Avro/Row conversion utilities to center around Spark's AvroDeserializer ; removed duplication - Bootstrapped HoodieBaseRelation - Updated HoodieMergeOnReadRDD to be able to handle Metadata Table - Modified MOR relations to be able to read different Base File formats (Parquet, HFile) --- .../hudi/keygen/ComplexAvroKeyGenerator.java | 12 +- .../hudi/keygen/BuiltinKeyGenerator.java | 20 +- .../hudi/keygen/ComplexKeyGenerator.java | 12 +- .../apache/hudi/AvroConversionHelper.scala | 380 ----------------- .../org/apache/hudi/AvroConversionUtils.scala | 99 ++++- .../org/apache/hudi/HoodieSparkUtils.scala | 85 ++-- .../avro/HoodieAvroDeserializerTrait.scala | 35 ++ .../sql/avro/HoodieAvroSerializerTrait.scala | 28 ++ .../apache/spark/sql/hudi/SparkAdapter.scala | 16 +- .../testutils/KeyGeneratorTestUtilities.java | 18 +- .../org/apache/hudi/avro/HoodieAvroUtils.java | 73 +++- .../hudi/avro/MercifulJsonConverter.java | 9 +- .../common/table/TableSchemaResolver.java | 3 +- .../hudi/metadata/BaseTableMetadata.java | 11 +- .../metadata/HoodieBackedTableMetadata.java | 12 +- .../hudi/metadata/HoodieMetadataPayload.java | 126 +++--- .../hudi/metadata/HoodieTableMetadata.java | 23 +- .../common/testutils/RawTripTestPayload.java | 17 + .../resources/timestamp-test-evolved.avsc | 40 +- .../utils/HoodieRealtimeInputFormatUtils.java | 7 +- .../org/apache/hudi/SparkRowWriteHelper.java | 29 +- .../org/apache/hudi/HoodieBaseRelation.scala | 117 ++++- .../apache/hudi/HoodieDataSourceHelper.scala | 24 +- .../apache/hudi/HoodieMergeOnReadRDD.scala | 223 +++++----- .../hudi/MergeOnReadIncrementalRelation.scala | 131 +++--- .../hudi/MergeOnReadSnapshotRelation.scala | 139 +++--- .../spark/sql/avro/HoodieAvroSerializer.scala | 13 +- .../catalyst/catalog/HoodieCatalogTable.scala | 4 +- .../spark/sql/hudi/ProvidesHoodieConfig.scala | 31 +- .../command/payload/ExpressionCodeGen.scala | 3 +- .../command/payload/ExpressionPayload.scala | 12 +- .../hudi/command/payload/SqlTypedRecord.scala | 13 +- .../command/MergeIntoHoodieTableCommand.scala | 8 +- .../hudi/keygen/TestComplexKeyGenerator.java | 3 +- .../hudi/keygen/TestCustomKeyGenerator.java | 5 +- .../keygen/TestGlobalDeleteKeyGenerator.java | 3 +- .../TestNonpartitionedKeyGenerator.java | 3 +- .../hudi/keygen/TestSimpleKeyGenerator.java | 3 +- .../TestTimestampBasedKeyGenerator.java | 126 +++--- ...stCreateKeyGeneratorByTypeWithFactory.java | 0 .../TestHoodieSparkKeyGeneratorFactory.java | 0 .../hudi/TestAvroConversionHelper.scala | 6 +- .../apache/hudi/TestDataSourceDefaults.scala | 6 +- .../hudi/TestHoodieSparkSqlWriter.scala | 55 ++- .../apache/hudi/TestHoodieSparkUtils.scala | 11 +- .../hudi/functional/TestMORDataSource.scala | 56 ++- ...TestMetadataTableWithSparkDataSource.scala | 98 +++++ hudi-spark-datasource/hudi-spark2/pom.xml | 9 + .../spark/sql/adapter/Spark2Adapter.scala | 10 +- .../sql/avro/PatchedAvroDeserializer.scala | 398 ++++++++++++++++++ .../avro/Spark2HoodieAvroDeserializer.scala | 33 ++ .../hudi-spark3-common/pom.xml | 9 + .../spark/sql/adapter/Spark3Adapter.scala | 15 +- .../avro/Spark3HoodieAvroDeserializer.scala} | 20 +- hudi-spark-datasource/hudi-spark3/pom.xml | 1 + pom.xml | 1 + 56 files changed, 1634 insertions(+), 1010 deletions(-) delete mode 100644 hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionHelper.scala create mode 100644 hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieAvroDeserializerTrait.scala create mode 100644 hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieAvroSerializerTrait.scala rename {hudi-client/hudi-spark-client => hudi-spark-datasource/hudi-spark}/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java (99%) rename {hudi-client/hudi-spark-client => hudi-spark-datasource/hudi-spark}/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java (99%) rename {hudi-client/hudi-spark-client => hudi-spark-datasource/hudi-spark}/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteKeyGenerator.java (99%) rename {hudi-client/hudi-spark-client => hudi-spark-datasource/hudi-spark}/src/test/java/org/apache/hudi/keygen/TestNonpartitionedKeyGenerator.java (99%) rename {hudi-client/hudi-spark-client => hudi-spark-datasource/hudi-spark}/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java (99%) rename {hudi-client/hudi-spark-client => hudi-spark-datasource/hudi-spark}/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java (80%) rename {hudi-client/hudi-spark-client => hudi-spark-datasource/hudi-spark}/src/test/java/org/apache/hudi/keygen/factory/TestCreateKeyGeneratorByTypeWithFactory.java (100%) rename {hudi-client/hudi-spark-client => hudi-spark-datasource/hudi-spark}/src/test/java/org/apache/hudi/keygen/factory/TestHoodieSparkKeyGeneratorFactory.java (100%) create mode 100644 hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataTableWithSparkDataSource.scala create mode 100644 hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/avro/PatchedAvroDeserializer.scala create mode 100644 hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/avro/Spark2HoodieAvroDeserializer.scala rename hudi-spark-datasource/{hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/HoodieAvroDeserializer.scala => hudi-spark3-common/src/main/scala/org/apache/spark/sql/avro/Spark3HoodieAvroDeserializer.scala} (67%) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java index fc87a83e3..dca0c2577 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java @@ -32,10 +32,14 @@ public class ComplexAvroKeyGenerator extends BaseKeyGenerator { public ComplexAvroKeyGenerator(TypedProperties props) { super(props); - this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()) - .split(",")).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList()); - this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key()) - .split(",")).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList()); + this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()).split(",")) + .map(String::trim) + .filter(s -> !s.isEmpty()) + .collect(Collectors.toList()); + this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key()).split(",")) + .map(String::trim) + .filter(s -> !s.isEmpty()) + .collect(Collectors.toList()); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java index 0b30f1989..fe03f60ee 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java @@ -18,28 +18,26 @@ package org.apache.hudi.keygen; +import org.apache.avro.generic.GenericRecord; import org.apache.hudi.ApiMaturityLevel; -import org.apache.hudi.AvroConversionHelper; +import org.apache.hudi.AvroConversionUtils; import org.apache.hudi.HoodieSparkUtils; import org.apache.hudi.PublicAPIMethod; import org.apache.hudi.client.utils.SparkRowSerDe; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieKeyException; - -import org.apache.avro.generic.GenericRecord; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.StructType; +import scala.Function1; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -import scala.Function1; - /** * Base class for the built-in key generators. Contains methods structured for * code reuse amongst them. @@ -48,7 +46,7 @@ public abstract class BuiltinKeyGenerator extends BaseKeyGenerator implements Sp private static final String STRUCT_NAME = "hoodieRowTopLevelField"; private static final String NAMESPACE = "hoodieRow"; - private transient Function1 converterFn = null; + private transient Function1 converterFn = null; private SparkRowSerDe sparkRowSerDe; protected StructType structType; @@ -69,10 +67,9 @@ public abstract class BuiltinKeyGenerator extends BaseKeyGenerator implements Sp @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) public String getRecordKey(Row row) { if (null == converterFn) { - converterFn = AvroConversionHelper.createConverterToAvro(row.schema(), STRUCT_NAME, NAMESPACE); + converterFn = AvroConversionUtils.createConverterToAvro(row.schema(), STRUCT_NAME, NAMESPACE); } - GenericRecord genericRecord = (GenericRecord) converterFn.apply(row); - return getKey(genericRecord).getRecordKey(); + return getKey(converterFn.apply(row)).getRecordKey(); } /** @@ -84,10 +81,9 @@ public abstract class BuiltinKeyGenerator extends BaseKeyGenerator implements Sp @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) public String getPartitionPath(Row row) { if (null == converterFn) { - converterFn = AvroConversionHelper.createConverterToAvro(row.schema(), STRUCT_NAME, NAMESPACE); + converterFn = AvroConversionUtils.createConverterToAvro(row.schema(), STRUCT_NAME, NAMESPACE); } - GenericRecord genericRecord = (GenericRecord) converterFn.apply(row); - return getKey(genericRecord).getPartitionPath(); + return getKey(converterFn.apply(row)).getPartitionPath(); } /** diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java index 8d02ce60f..2e2167f93 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java @@ -37,10 +37,14 @@ public class ComplexKeyGenerator extends BuiltinKeyGenerator { public ComplexKeyGenerator(TypedProperties props) { super(props); - this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()) - .split(",")).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList()); - this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key()) - .split(",")).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList()); + this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()).split(",")) + .map(String::trim) + .filter(s -> !s.isEmpty()) + .collect(Collectors.toList()); + this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key()).split(",")) + .map(String::trim) + .filter(s -> !s.isEmpty()) + .collect(Collectors.toList()); complexAvroKeyGenerator = new ComplexAvroKeyGenerator(props); } diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionHelper.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionHelper.scala deleted file mode 100644 index f968cbe1c..000000000 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionHelper.scala +++ /dev/null @@ -1,380 +0,0 @@ -/* - * 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 - -import java.nio.ByteBuffer -import java.sql.{Date, Timestamp} -import java.time.Instant - -import org.apache.avro.Conversions.DecimalConversion -import org.apache.avro.LogicalTypes.{TimestampMicros, TimestampMillis} -import org.apache.avro.Schema.Type._ -import org.apache.avro.generic.GenericData.{Fixed, Record} -import org.apache.avro.generic.{GenericData, GenericFixed, GenericRecord} -import org.apache.avro.{LogicalTypes, Schema} - -import org.apache.spark.sql.Row -import org.apache.spark.sql.avro.SchemaConverters -import org.apache.spark.sql.catalyst.expressions.GenericRow -import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.types._ - -import org.apache.hudi.AvroConversionUtils._ -import org.apache.hudi.exception.HoodieIncompatibleSchemaException - -import scala.collection.JavaConverters._ - -object AvroConversionHelper { - - private def createDecimal(decimal: java.math.BigDecimal, precision: Int, scale: Int): Decimal = { - if (precision <= Decimal.MAX_LONG_DIGITS) { - // Constructs a `Decimal` with an unscaled `Long` value if possible. - Decimal(decimal.unscaledValue().longValue(), precision, scale) - } else { - // Otherwise, resorts to an unscaled `BigInteger` instead. - Decimal(decimal, precision, scale) - } - } - - /** - * - * Returns a converter function to convert row in avro format to GenericRow of catalyst. - * - * @param sourceAvroSchema Source schema before conversion inferred from avro file by passed in - * by user. - * @param targetSqlType Target catalyst sql type after the conversion. - * @return returns a converter function to convert row in avro format to GenericRow of catalyst. - */ - def createConverterToRow(sourceAvroSchema: Schema, - targetSqlType: DataType): AnyRef => AnyRef = { - - def createConverter(avroSchema: Schema, sqlType: DataType, path: List[String]): AnyRef => AnyRef = { - val avroType = avroSchema.getType - (sqlType, avroType) match { - // Avro strings are in Utf8, so we have to call toString on them - case (StringType, STRING) | (StringType, ENUM) => - (item: AnyRef) => if (item == null) null else item.toString - // Byte arrays are reused by avro, so we have to make a copy of them. - case (IntegerType, INT) | (BooleanType, BOOLEAN) | (DoubleType, DOUBLE) | - (FloatType, FLOAT) | (LongType, LONG) => - identity - case (BinaryType, FIXED) => - (item: AnyRef) => - if (item == null) { - null - } else { - item.asInstanceOf[Fixed].bytes().clone() - } - case (BinaryType, BYTES) => - (item: AnyRef) => - if (item == null) { - null - } else { - val byteBuffer = item.asInstanceOf[ByteBuffer] - val bytes = new Array[Byte](byteBuffer.remaining) - byteBuffer.get(bytes) - bytes - } - case (d: DecimalType, FIXED) => - (item: AnyRef) => - if (item == null) { - null - } else { - val decimalConversion = new DecimalConversion - val bigDecimal = decimalConversion.fromFixed(item.asInstanceOf[GenericFixed], avroSchema, - LogicalTypes.decimal(d.precision, d.scale)) - createDecimal(bigDecimal, d.precision, d.scale) - } - case (d: DecimalType, BYTES) => - (item: AnyRef) => - if (item == null) { - null - } else { - val decimalConversion = new DecimalConversion - val bigDecimal = decimalConversion.fromBytes(item.asInstanceOf[ByteBuffer], avroSchema, - LogicalTypes.decimal(d.precision, d.scale)) - createDecimal(bigDecimal, d.precision, d.scale) - } - case (DateType, INT) => - (item: AnyRef) => - if (item == null) { - null - } else { - item match { - case integer: Integer => DateTimeUtils.toJavaDate(integer) - case _ => new Date(item.asInstanceOf[Long]) - } - } - case (TimestampType, LONG) => - (item: AnyRef) => - if (item == null) { - null - } else { - avroSchema.getLogicalType match { - case _: TimestampMillis => - new Timestamp(item.asInstanceOf[Long]) - case _: TimestampMicros => - new Timestamp(item.asInstanceOf[Long] / 1000) - case null => - new Timestamp(item.asInstanceOf[Long]) - case other => - throw new HoodieIncompatibleSchemaException( - s"Cannot convert Avro logical type $other to Catalyst Timestamp type.") - } - } - case (struct: StructType, RECORD) => - val length = struct.fields.length - val converters = new Array[AnyRef => AnyRef](length) - val avroFieldIndexes = new Array[Int](length) - var i = 0 - while (i < length) { - val sqlField = struct.fields(i) - val avroField = avroSchema.getField(sqlField.name) - if (avroField != null) { - val converter = createConverter(avroField.schema(), sqlField.dataType, - path :+ sqlField.name) - converters(i) = converter - avroFieldIndexes(i) = avroField.pos() - } else if (!sqlField.nullable) { - throw new HoodieIncompatibleSchemaException( - s"Cannot find non-nullable field ${sqlField.name} at path ${path.mkString(".")} " + - "in Avro schema\n" + - s"Source Avro schema: $sourceAvroSchema.\n" + - s"Target Catalyst type: $targetSqlType") - } - i += 1 - } - - (item: AnyRef) => { - if (item == null) { - null - } else { - val record = item.asInstanceOf[GenericRecord] - - val result = new Array[Any](length) - var i = 0 - while (i < converters.length) { - if (converters(i) != null) { - val converter = converters(i) - result(i) = converter(record.get(avroFieldIndexes(i))) - } - i += 1 - } - new GenericRow(result) - } - } - case (arrayType: ArrayType, ARRAY) => - val elementConverter = createConverter(avroSchema.getElementType, arrayType.elementType, - path) - val allowsNull = arrayType.containsNull - (item: AnyRef) => { - if (item == null) { - null - } else { - item.asInstanceOf[java.lang.Iterable[AnyRef]].asScala.map { element => - if (element == null && !allowsNull) { - throw new RuntimeException(s"Array value at path ${path.mkString(".")} is not " + - "allowed to be null") - } else { - elementConverter(element) - } - } - } - } - case (mapType: MapType, MAP) if mapType.keyType == StringType => - val valueConverter = createConverter(avroSchema.getValueType, mapType.valueType, path) - val allowsNull = mapType.valueContainsNull - (item: AnyRef) => { - if (item == null) { - null - } else { - item.asInstanceOf[java.util.Map[AnyRef, AnyRef]].asScala.map { x => - if (x._2 == null && !allowsNull) { - throw new RuntimeException(s"Map value at path ${path.mkString(".")} is not " + - "allowed to be null") - } else { - (x._1.toString, valueConverter(x._2)) - } - }.toMap - } - } - case (sqlType, UNION) => - if (avroSchema.getTypes.asScala.exists(_.getType == NULL)) { - val remainingUnionTypes = avroSchema.getTypes.asScala.filterNot(_.getType == NULL) - if (remainingUnionTypes.size == 1) { - createConverter(remainingUnionTypes.head, sqlType, path) - } else { - createConverter(Schema.createUnion(remainingUnionTypes.asJava), sqlType, path) - } - } else avroSchema.getTypes.asScala.map(_.getType) match { - case Seq(_) => createConverter(avroSchema.getTypes.get(0), sqlType, path) - case Seq(a, b) if Set(a, b) == Set(INT, LONG) && sqlType == LongType => - (item: AnyRef) => { - item match { - case null => null - case l: java.lang.Long => l - case i: java.lang.Integer => new java.lang.Long(i.longValue()) - } - } - case Seq(a, b) if Set(a, b) == Set(FLOAT, DOUBLE) && sqlType == DoubleType => - (item: AnyRef) => { - item match { - case null => null - case d: java.lang.Double => d - case f: java.lang.Float => new java.lang.Double(f.doubleValue()) - } - } - case other => - sqlType match { - case t: StructType if t.fields.length == avroSchema.getTypes.size => - val fieldConverters = t.fields.zip(avroSchema.getTypes.asScala).map { - case (field, schema) => - createConverter(schema, field.dataType, path :+ field.name) - } - - (item: AnyRef) => - if (item == null) { - null - } else { - val i = GenericData.get().resolveUnion(avroSchema, item) - val converted = new Array[Any](fieldConverters.length) - converted(i) = fieldConverters(i)(item) - new GenericRow(converted) - } - case _ => throw new HoodieIncompatibleSchemaException( - s"Cannot convert Avro schema to catalyst type because schema at path " + - s"${path.mkString(".")} is not compatible " + - s"(avroType = $other, sqlType = $sqlType). \n" + - s"Source Avro schema: $sourceAvroSchema.\n" + - s"Target Catalyst type: $targetSqlType") - } - } - case (left, right) => - throw new HoodieIncompatibleSchemaException( - s"Cannot convert Avro schema to catalyst type because schema at path " + - s"${path.mkString(".")} is not compatible (avroType = $left, sqlType = $right). \n" + - s"Source Avro schema: $sourceAvroSchema.\n" + - s"Target Catalyst type: $targetSqlType") - } - } - - createConverter(sourceAvroSchema, targetSqlType, List.empty[String]) - } - - def createConverterToAvro(dataType: DataType, - structName: String, - recordNamespace: String): Any => Any = { - dataType match { - case BinaryType => (item: Any) => - item match { - case null => null - case bytes: Array[Byte] => ByteBuffer.wrap(bytes) - } - case IntegerType | LongType | - FloatType | DoubleType | StringType | BooleanType => identity - case ByteType => (item: Any) => - if (item == null) null else item.asInstanceOf[Byte].intValue - case ShortType => (item: Any) => - if (item == null) null else item.asInstanceOf[Short].intValue - case dec: DecimalType => - val schema = SchemaConverters.toAvroType(dec, nullable = false, structName, recordNamespace) - (item: Any) => { - Option(item).map { _ => - val bigDecimalValue = item.asInstanceOf[java.math.BigDecimal] - val decimalConversions = new DecimalConversion() - decimalConversions.toFixed(bigDecimalValue, schema, LogicalTypes.decimal(dec.precision, dec.scale)) - }.orNull - } - case TimestampType => (item: Any) => - if (item == null) { - null - } else { - val timestamp = item match { - case i: Instant => Timestamp.from(i) - case t: Timestamp => t - } - // Convert time to microseconds since spark-avro by default converts TimestampType to - // Avro Logical TimestampMicros - timestamp.getTime * 1000 - } - case DateType => (item: Any) => - Option(item).map(_.asInstanceOf[Date].toLocalDate.toEpochDay.toInt).orNull - case ArrayType(elementType, _) => - val elementConverter = createConverterToAvro( - elementType, - structName, - recordNamespace) - (item: Any) => { - if (item == null) { - null - } else { - val sourceArray = item.asInstanceOf[Seq[Any]] - val sourceArraySize = sourceArray.size - val targetList = new java.util.ArrayList[Any](sourceArraySize) - var idx = 0 - while (idx < sourceArraySize) { - targetList.add(elementConverter(sourceArray(idx))) - idx += 1 - } - targetList - } - } - case MapType(StringType, valueType, _) => - val valueConverter = createConverterToAvro( - valueType, - structName, - recordNamespace) - (item: Any) => { - if (item == null) { - null - } else { - val javaMap = new java.util.HashMap[String, Any]() - item.asInstanceOf[Map[String, Any]].foreach { case (key, value) => - javaMap.put(key, valueConverter(value)) - } - javaMap - } - } - case structType: StructType => - val schema: Schema = convertStructTypeToAvroSchema(structType, structName, recordNamespace) - val childNameSpace = if (recordNamespace != "") s"$recordNamespace.$structName" else structName - val fieldConverters = structType.fields.map(field => - createConverterToAvro( - field.dataType, - field.name, - childNameSpace)) - (item: Any) => { - if (item == null) { - null - } else { - val record = new Record(schema) - val convertersIterator = fieldConverters.iterator - val fieldNamesIterator = dataType.asInstanceOf[StructType].fieldNames.iterator - val rowIterator = item.asInstanceOf[Row].toSeq.iterator - - while (convertersIterator.hasNext && rowIterator.hasNext) { - val converter = convertersIterator.next() - record.put(fieldNamesIterator.next(), converter(rowIterator.next())) - } - record - } - } - } - } -} diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala index d5a287233..a006eebf7 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala @@ -18,20 +18,105 @@ package org.apache.hudi -import org.apache.avro.Schema -import org.apache.avro.JsonProperties +import org.apache.avro.Schema.Type import org.apache.avro.generic.{GenericRecord, GenericRecordBuilder, IndexedRecord} +import org.apache.avro.{AvroRuntimeException, JsonProperties, Schema} +import org.apache.hudi.HoodieSparkUtils.sparkAdapter import org.apache.hudi.avro.HoodieAvroUtils import org.apache.spark.rdd.RDD import org.apache.spark.sql.avro.SchemaConverters -import org.apache.spark.sql.types.StructType +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.{Dataset, Row, SparkSession} -import scala.collection.JavaConverters._ import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ object AvroConversionUtils { + /** + * Check the nullability of the input Avro type and resolve it when it is nullable. The first + * return value is a [[Boolean]] indicating if the input Avro type is nullable. The second + * return value is either provided Avro type if it's not nullable, or its resolved non-nullable part + * in case it is + */ + def resolveAvroTypeNullability(avroType: Schema): (Boolean, Schema) = { + if (avroType.getType == Type.UNION) { + val fields = avroType.getTypes.asScala + val actualType = fields.filter(_.getType != Type.NULL) + if (fields.length != 2 || actualType.length != 1) { + throw new AvroRuntimeException( + s"Unsupported Avro UNION type $avroType: Only UNION of a null type and a non-null " + + "type is supported") + } + (true, actualType.head) + } else { + (false, avroType) + } + } + + /** + * Creates converter to transform Avro payload into Spark's Catalyst one + * + * @param rootAvroType Avro [[Schema]] to be transformed from + * @param rootCatalystType Catalyst [[StructType]] to be transformed into + * @return converter accepting Avro payload and transforming it into a Catalyst one (in the form of [[InternalRow]]) + */ + def createAvroToInternalRowConverter(rootAvroType: Schema, rootCatalystType: StructType): GenericRecord => Option[InternalRow] = + record => sparkAdapter.createAvroDeserializer(rootAvroType, rootCatalystType) + .deserialize(record) + .map(_.asInstanceOf[InternalRow]) + + /** + * Creates converter to transform Catalyst payload into Avro one + * + * @param rootCatalystType Catalyst [[StructType]] to be transformed from + * @param rootAvroType Avro [[Schema]] to be transformed into + * @param nullable whether Avro record is nullable + * @return converter accepting Catalyst payload (in the form of [[InternalRow]]) and transforming it into an Avro one + */ + def createInternalRowToAvroConverter(rootCatalystType: StructType, rootAvroType: Schema, nullable: Boolean): InternalRow => GenericRecord = { + row => sparkAdapter.createAvroSerializer(rootCatalystType, rootAvroType, nullable) + .serialize(row) + .asInstanceOf[GenericRecord] + } + + /** + * @deprecated please use [[AvroConversionUtils.createAvroToInternalRowConverter]] + */ + @Deprecated + def createConverterToRow(sourceAvroSchema: Schema, + targetSqlType: StructType): GenericRecord => Row = { + val encoder = RowEncoder.apply(targetSqlType).resolveAndBind() + val serde = sparkAdapter.createSparkRowSerDe(encoder) + val converter = AvroConversionUtils.createAvroToInternalRowConverter(sourceAvroSchema, targetSqlType) + + avro => converter.apply(avro).map(serde.deserializeRow).get + } + + /** + * @deprecated please use [[AvroConversionUtils.createInternalRowToAvroConverter]] + */ + @Deprecated + def createConverterToAvro(sourceSqlType: StructType, + structName: String, + recordNamespace: String): Row => GenericRecord = { + val encoder = RowEncoder.apply(sourceSqlType).resolveAndBind() + val serde = sparkAdapter.createSparkRowSerDe(encoder) + val avroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(sourceSqlType, structName, recordNamespace) + val (nullable, _) = resolveAvroTypeNullability(avroSchema) + + val converter = AvroConversionUtils.createInternalRowToAvroConverter(sourceSqlType, avroSchema, nullable) + + row => converter.apply(serde.serializeRow(row)) + } + + /** + * Creates [[org.apache.spark.sql.DataFrame]] from the provided [[RDD]] of [[GenericRecord]]s + * + * TODO convert directly from GenericRecord into InternalRow instead + */ def createDataFrame(rdd: RDD[GenericRecord], schemaStr: String, ss: SparkSession): Dataset[Row] = { if (rdd.isEmpty()) { ss.emptyDataFrame @@ -41,8 +126,8 @@ object AvroConversionUtils { else { val schema = new Schema.Parser().parse(schemaStr) val dataType = convertAvroSchemaToStructType(schema) - val convertor = AvroConversionHelper.createConverterToRow(schema, dataType) - records.map { x => convertor(x).asInstanceOf[Row] } + val converter = createConverterToRow(schema, dataType) + records.map { r => converter(r) } } }, convertAvroSchemaToStructType(new Schema.Parser().parse(schemaStr))) } @@ -57,7 +142,7 @@ object AvroConversionUtils { * @param recordNamespace Avro record namespace. * @return Avro schema corresponding to given struct type. */ - def convertStructTypeToAvroSchema(structType: StructType, + def convertStructTypeToAvroSchema(structType: DataType, structName: String, recordNamespace: String): Schema = { getAvroSchemaWithDefaults(SchemaConverters.toAvroType(structType, nullable = false, structName, recordNamespace)) diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala index 2db0cf226..b288289ac 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala @@ -18,13 +18,10 @@ package org.apache.hudi -import java.util.Properties - import org.apache.avro.Schema import org.apache.avro.generic.GenericRecord - import org.apache.hadoop.fs.{FileSystem, Path} - +import org.apache.hudi.avro.HoodieAvroUtils.rewriteRecord import org.apache.hudi.client.utils.SparkRowSerDe import org.apache.hudi.common.config.TypedProperties import org.apache.hudi.common.model.HoodieRecord @@ -32,7 +29,6 @@ import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.keygen.constant.KeyGeneratorOptions import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory import org.apache.hudi.keygen.{BaseKeyGenerator, CustomAvroKeyGenerator, CustomKeyGenerator, KeyGenerator} - import org.apache.spark.SPARK_VERSION import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.encoders.RowEncoder @@ -42,8 +38,8 @@ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{StringType, StructField, StructType} import org.apache.spark.sql.{DataFrame, SparkSession} +import java.util.Properties import scala.collection.JavaConverters._ -import scala.collection.JavaConverters.asScalaBufferConverter object HoodieSparkUtils extends SparkAdapterSupport { @@ -127,46 +123,49 @@ object HoodieSparkUtils extends SparkAdapterSupport { new InMemoryFileIndex(sparkSession, globbedPaths, Map(), Option.empty, fileStatusCache) } - def createRdd(df: DataFrame, structName: String, recordNamespace: String, reconcileToLatestSchema: Boolean, latestTableSchema: - org.apache.hudi.common.util.Option[Schema] = org.apache.hudi.common.util.Option.empty()): RDD[GenericRecord] = { - val dfWriteSchema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, recordNamespace) - var writeSchema : Schema = null; - var toReconcileSchema : Schema = null; - if (reconcileToLatestSchema && latestTableSchema.isPresent) { - // if reconcileToLatestSchema is set to true and latestSchema is present, then try to leverage latestTableSchema. - // this code path will handle situations where records are serialized in odl schema, but callers wish to convert - // to Rdd[GenericRecord] using different schema(could be evolved schema or could be latest table schema) - writeSchema = dfWriteSchema - toReconcileSchema = latestTableSchema.get() - } else { - // there are paths where callers wish to use latestTableSchema to convert to Rdd[GenericRecords] and not use - // row's schema. So use latestTableSchema if present. if not available, fallback to using row's schema. - writeSchema = if (latestTableSchema.isPresent) { latestTableSchema.get()} else { dfWriteSchema} - } - createRddInternal(df, writeSchema, toReconcileSchema, structName, recordNamespace) + /** + * @deprecated please use other overload [[createRdd]] + */ + def createRdd(df: DataFrame, structName: String, recordNamespace: String, reconcileToLatestSchema: Boolean, + latestTableSchema: org.apache.hudi.common.util.Option[Schema] = org.apache.hudi.common.util.Option.empty()): RDD[GenericRecord] = { + val latestTableSchemaConverted = if (latestTableSchema.isPresent && reconcileToLatestSchema) Some(latestTableSchema.get()) else None + createRdd(df, structName, recordNamespace, latestTableSchemaConverted) } - def createRddInternal(df: DataFrame, writeSchema: Schema, latestTableSchema: Schema, structName: String, recordNamespace: String) - : RDD[GenericRecord] = { - // Use the write avro schema to derive the StructType which has the correct nullability information - val writeDataType = AvroConversionUtils.convertAvroSchemaToStructType(writeSchema) - val encoder = RowEncoder.apply(writeDataType).resolveAndBind() - val deserializer = sparkAdapter.createSparkRowSerDe(encoder) - // if records were serialized with old schema, but an evolved schema was passed in with latestTableSchema, we need - // latestTableSchema equivalent datatype to be passed in to AvroConversionHelper.createConverterToAvro() - val reconciledDataType = - if (latestTableSchema != null) AvroConversionUtils.convertAvroSchemaToStructType(latestTableSchema) else writeDataType - // Note: deserializer.deserializeRow(row) is not capable of handling evolved schema. i.e. if Row was serialized in - // old schema, but deserializer was created with an encoder with evolved schema, deserialization fails. - // Hence we always need to deserialize in the same schema as serialized schema. - df.queryExecution.toRdd.map(row => deserializer.deserializeRow(row)) - .mapPartitions { records => - if (records.isEmpty) Iterator.empty - else { - val convertor = AvroConversionHelper.createConverterToAvro(reconciledDataType, structName, recordNamespace) - records.map { x => convertor(x).asInstanceOf[GenericRecord] } - } + def createRdd(df: DataFrame, structName: String, recordNamespace: String, readerAvroSchemaOpt: Option[Schema]): RDD[GenericRecord] = { + val writerSchema = df.schema + val writerAvroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(writerSchema, structName, recordNamespace) + val readerAvroSchema = readerAvroSchemaOpt.getOrElse(writerAvroSchema) + // We check whether passed in reader schema is identical to writer schema to avoid costly serde loop of + // making Spark deserialize its internal representation [[InternalRow]] into [[Row]] for subsequent conversion + // (and back) + val sameSchema = writerAvroSchema.equals(readerAvroSchema) + val (nullable, _) = AvroConversionUtils.resolveAvroTypeNullability(writerAvroSchema) + + // NOTE: We have to serialize Avro schema, and then subsequently parse it on the executor node, since Spark + // serializer is not able to digest it + val readerAvroSchemaStr = readerAvroSchema.toString + val writerAvroSchemaStr = writerAvroSchema.toString + // NOTE: We're accessing toRdd here directly to avoid [[InternalRow]] to [[Row]] conversion + df.queryExecution.toRdd.mapPartitions { rows => + if (rows.isEmpty) { + Iterator.empty + } else { + val transform: GenericRecord => GenericRecord = + if (sameSchema) identity + else { + val readerAvroSchema = new Schema.Parser().parse(readerAvroSchemaStr) + rewriteRecord(_, readerAvroSchema) + } + + // Since caller might request to get records in a different ("evolved") schema, we will be rewriting from + // existing Writer's schema into Reader's (avro) schema + val writerAvroSchema = new Schema.Parser().parse(writerAvroSchemaStr) + val convert = AvroConversionUtils.createInternalRowToAvroConverter(writerSchema, writerAvroSchema, nullable = nullable) + + rows.map { ir => transform(convert(ir)) } } + } } def getDeserializer(structType: StructType) : SparkRowSerDe = { diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieAvroDeserializerTrait.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieAvroDeserializerTrait.scala new file mode 100644 index 000000000..5c3035304 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieAvroDeserializerTrait.scala @@ -0,0 +1,35 @@ +/* + * 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.spark.sql.avro + +/** + * Deserializes Avro payload into Catalyst object + * + * NOTE: This is low-level component operating on Spark internal data-types (comprising [[InternalRow]]). + * If you're looking to convert Avro into "deserialized" [[Row]] (comprised of Java native types), + * please check [[AvroConversionUtils]] + */ +trait HoodieAvroDeserializerTrait { + final def deserialize(data: Any): Option[Any] = + doDeserialize(data) match { + case opt: Option[_] => opt // As of Spark 3.1, this will return data wrapped with Option, so we fetch the data + case row => Some(row) // For other Spark versions, return the data as is + } + + protected def doDeserialize(data: Any): Any +} diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieAvroSerializerTrait.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieAvroSerializerTrait.scala new file mode 100644 index 000000000..159d8da74 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieAvroSerializerTrait.scala @@ -0,0 +1,28 @@ +/* + * 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.spark.sql.avro + +/** + * Serializes Catalyst payload into Avro object + * + * NOTE: This is low-level component operating on Spark internal data-types (comprising [[InternalRow]]). + * If you're looking to convert "deserialized" [[Row]] into Avro, please check [[AvroConversionUtils]] + */ +trait HoodieAvroSerializerTrait { + def serialize(catalystData: Any): Any +} diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala index 20b4d3cc1..32ed2b16c 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala @@ -18,8 +18,9 @@ package org.apache.spark.sql.hudi -import org.apache.hudi.HoodieSparkUtils.sparkAdapter +import org.apache.avro.Schema import org.apache.hudi.client.utils.SparkRowSerDe +import org.apache.spark.sql.avro.{HoodieAvroDeserializerTrait, HoodieAvroSerializerTrait} import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder @@ -30,6 +31,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, SubqueryA import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier} import org.apache.spark.sql.execution.datasources.{FilePartition, LogicalRelation, PartitionedFile, SparkParsePartitionUtil} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.DataType import org.apache.spark.sql.{Row, SparkSession} import java.util.Locale @@ -40,6 +42,18 @@ import java.util.Locale */ trait SparkAdapter extends Serializable { + /** + * Creates instance of [[HoodieAvroSerializerTrait]] providing for ability to serialize + * Spark's [[InternalRow]] into Avro payloads + */ + def createAvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean): HoodieAvroSerializerTrait + + /** + * Creates instance of [[HoodieAvroDeserializerTrait]] providing for ability to deserialize + * Avro payloads into Spark's [[InternalRow]] + */ + def createAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType): HoodieAvroDeserializerTrait + /** * Create the SparkRowSerDe. */ diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/KeyGeneratorTestUtilities.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/KeyGeneratorTestUtilities.java index 37a58fb3e..c2256f40c 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/KeyGeneratorTestUtilities.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/KeyGeneratorTestUtilities.java @@ -18,12 +18,10 @@ package org.apache.hudi.testutils; -import org.apache.hudi.AvroConversionHelper; -import org.apache.hudi.AvroConversionUtils; - import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.AvroConversionUtils; import org.apache.spark.package$; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; @@ -33,16 +31,15 @@ import org.apache.spark.sql.catalyst.encoders.RowEncoder; import org.apache.spark.sql.catalyst.expressions.Attribute; import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema; import org.apache.spark.sql.types.StructType; +import scala.Function1; +import scala.collection.JavaConversions; +import scala.collection.JavaConverters; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.util.List; import java.util.stream.Collectors; -import scala.Function1; -import scala.collection.JavaConversions; -import scala.collection.JavaConverters; - public class KeyGeneratorTestUtilities { public static final String NESTED_COL_SCHEMA = "{\"type\":\"record\", \"name\":\"nested_col\",\"fields\": [" @@ -51,8 +48,7 @@ public class KeyGeneratorTestUtilities { + "{\"name\": \"timestamp\",\"type\": \"long\"},{\"name\": \"_row_key\", \"type\": \"string\"}," + "{\"name\": \"ts_ms\", \"type\": \"string\"}," + "{\"name\": \"pii_col\", \"type\": \"string\"}," - + "{\"name\": \"nested_col\",\"type\": " - + NESTED_COL_SCHEMA + "}" + + "{\"name\": \"nested_col\",\"type\": [\"null\", " + NESTED_COL_SCHEMA + "]}" + "]}"; public static final String TEST_STRUCTNAME = "test_struct_name"; @@ -86,8 +82,8 @@ public class KeyGeneratorTestUtilities { } public static Row getRow(GenericRecord record, Schema schema, StructType structType) { - Function1 converterFn = AvroConversionHelper.createConverterToRow(schema, structType); - Row row = (Row) converterFn.apply(record); + Function1 converterFn = AvroConversionUtils.createConverterToRow(schema, structType); + Row row = converterFn.apply(record); int fieldCount = structType.fieldNames().length; Object[] values = new Object[fieldCount]; for (int i = 0; i < fieldCount; i++) { diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java index ae0f5957b..209721e24 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java @@ -40,7 +40,6 @@ import org.apache.avro.io.EncoderFactory; import org.apache.avro.io.JsonDecoder; import org.apache.avro.io.JsonEncoder; import org.apache.avro.specific.SpecificRecordBase; - import org.apache.hudi.common.config.SerializableSchema; import org.apache.hudi.common.model.HoodieOperation; import org.apache.hudi.common.model.HoodieRecord; @@ -341,14 +340,24 @@ public class HoodieAvroUtils { /** * Given an Avro record with a given schema, rewrites it into the new schema while setting fields only from the new * schema. + * + * NOTE: This method is rewriting every record's field that is record itself recursively. It's + * caller's responsibility to make sure that no unnecessary re-writing occurs (by preemptively + * checking whether the record does require re-writing to adhere to the new schema) + * * NOTE: Here, the assumption is that you cannot go from an evolved schema (schema with (N) fields) - * to an older schema (schema with (N-1) fields). All fields present in the older record schema MUST be present in the - * new schema and the default/existing values are carried over. - * This particular method does the following things : - * a) Create a new empty GenericRecord with the new schema. - * b) For GenericRecord, copy over the data from the old schema to the new schema or set default values for all fields of this - * transformed schema - * c) For SpecificRecord, hoodie_metadata_fields have a special treatment. This is done because for code generated + * to an older schema (schema with (N-1) fields). All fields present in the older record schema MUST be present in the + * new schema and the default/existing values are carried over. + * + * This particular method does the following: + *
    + *
  1. Create a new empty GenericRecord with the new schema.
  2. + *
  3. For GenericRecord, copy over the data from the old schema to the new schema or set default values for all + * fields of this transformed schema
  4. + *
  5. For SpecificRecord, hoodie_metadata_fields have a special treatment (see below)
  6. + *
+ * + * For SpecificRecord we ignore Hudi Metadata fields, because for code generated * avro classes (HoodieMetadataRecord), the avro record is a SpecificBaseRecord type instead of a GenericRecord. * SpecificBaseRecord throws null pointer exception for record.get(name) if name is not present in the schema of the * record (which happens when converting a SpecificBaseRecord without hoodie_metadata_fields to a new record with it). @@ -360,16 +369,16 @@ public class HoodieAvroUtils { GenericRecord newRecord = new GenericData.Record(newSchema); boolean isSpecificRecord = oldRecord instanceof SpecificRecordBase; for (Schema.Field f : newSchema.getFields()) { - if (!isSpecificRecord) { - copyOldValueOrSetDefault(oldRecord, newRecord, f); - } else if (!isMetadataField(f.name())) { + if (!(isSpecificRecord && isMetadataField(f.name()))) { copyOldValueOrSetDefault(oldRecord, newRecord, f); } } + if (!GenericData.get().validate(newSchema, newRecord)) { throw new SchemaCompatibilityException( "Unable to validate the rewritten record " + oldRecord + " against schema " + newSchema); } + return newRecord; } @@ -383,18 +392,20 @@ public class HoodieAvroUtils { return records.stream().map(r -> rewriteRecord(r, newSchema)).collect(Collectors.toList()); } - private static void copyOldValueOrSetDefault(GenericRecord oldRecord, GenericRecord newRecord, Schema.Field f) { - // cache the result of oldRecord.get() to save CPU expensive hash lookup + private static void copyOldValueOrSetDefault(GenericRecord oldRecord, GenericRecord newRecord, Schema.Field field) { Schema oldSchema = oldRecord.getSchema(); - Object fieldValue = oldSchema.getField(f.name()) == null ? null : oldRecord.get(f.name()); - if (fieldValue == null) { - if (f.defaultVal() instanceof JsonProperties.Null) { - newRecord.put(f.name(), null); - } else { - newRecord.put(f.name(), f.defaultVal()); - } + Object fieldValue = oldSchema.getField(field.name()) == null ? null : oldRecord.get(field.name()); + + if (fieldValue != null) { + // In case field's value is a nested record, we have to rewrite it as well + Object newFieldValue = fieldValue instanceof GenericRecord + ? rewriteRecord((GenericRecord) fieldValue, resolveNullableSchema(field.schema())) + : fieldValue; + newRecord.put(field.name(), newFieldValue); + } else if (field.defaultVal() instanceof JsonProperties.Null) { + newRecord.put(field.name(), null); } else { - newRecord.put(f.name(), fieldValue); + newRecord.put(field.name(), field.defaultVal()); } } @@ -610,4 +621,24 @@ public class HoodieAvroUtils { SerializableSchema schema, boolean consistentLogicalTimestampEnabled) { return getRecordColumnValues(record, columns, schema.get(), consistentLogicalTimestampEnabled); } + + private static Schema resolveNullableSchema(Schema schema) { + if (schema.getType() != Schema.Type.UNION) { + return schema; + } + + List innerTypes = schema.getTypes(); + Schema nonNullType = + innerTypes.stream() + .filter(it -> it.getType() != Schema.Type.NULL) + .findFirst() + .orElse(null); + + if (innerTypes.size() != 2 || nonNullType == null) { + throw new AvroRuntimeException( + String.format("Unsupported Avro UNION type %s: Only UNION of a null type and a non-null type is supported", schema)); + } + + return nonNullType; + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/MercifulJsonConverter.java b/hudi-common/src/main/java/org/apache/hudi/avro/MercifulJsonConverter.java index d759a8deb..153351934 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/MercifulJsonConverter.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/MercifulJsonConverter.java @@ -18,15 +18,14 @@ package org.apache.hudi.avro; -import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.exception.HoodieIOException; - import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.avro.Schema; import org.apache.avro.Schema.Type; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieIOException; import java.io.IOException; import java.io.Serializable; @@ -293,7 +292,7 @@ public class MercifulJsonConverter { for (Object v : (List) value) { listRes.add(convertJsonToAvroField(v, name, elementSchema)); } - return Pair.of(true, listRes); + return Pair.of(true, new GenericData.Array<>(schema, listRes)); } }; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java index 3ca2122aa..a84a9482a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java @@ -143,7 +143,7 @@ public class TableSchemaResolver { * @throws Exception */ public Schema getTableAvroSchema() throws Exception { - return getTableAvroSchema(true); + return getTableAvroSchema(metaClient.getTableConfig().populateMetaFields()); } /** @@ -526,6 +526,7 @@ public class TableSchemaResolver { Schema tableAvroSchema = getTableAvroSchemaFromDataFile(); return tableAvroSchema.getField(HoodieRecord.OPERATION_METADATA_FIELD) != null; } catch (Exception e) { + LOG.info(String.format("Failed to read operation field from avro schema (%s)", e.getMessage())); return false; } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java index 459397bdc..3c648f38d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java @@ -19,8 +19,10 @@ package org.apache.hudi.metadata; -import org.apache.hudi.avro.model.HoodieMetadataColumnStats; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.model.HoodieMetadataBloomFilter; +import org.apache.hudi.avro.model.HoodieMetadataColumnStats; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.engine.HoodieEngineContext; @@ -38,9 +40,6 @@ import org.apache.hudi.common.util.hash.ColumnIndexID; import org.apache.hudi.common.util.hash.FileIndexID; import org.apache.hudi.common.util.hash.PartitionIndexID; import org.apache.hudi.exception.HoodieMetadataException; - -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -60,8 +59,8 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata { private static final Logger LOG = LogManager.getLogger(BaseTableMetadata.class); - static final long MAX_MEMORY_SIZE_IN_BYTES = 1024 * 1024 * 1024; - static final int BUFFER_SIZE = 10 * 1024 * 1024; + public static final long MAX_MEMORY_SIZE_IN_BYTES = 1024 * 1024 * 1024; + public static final int BUFFER_SIZE = 10 * 1024 * 1024; protected final transient HoodieEngineContext engineContext; protected final SerializableConfiguration hadoopConf; diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java index fb6ce7d5b..7b4dbd9a0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java @@ -290,7 +290,9 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata { final long baseFileOpenMs = baseFileReaderOpenTimePair.getValue(); // Open the log record scanner using the log files from the latest file slice - Pair logRecordScannerOpenTimePair = getLogRecordScanner(slice, partitionName); + List logFiles = slice.getLogFiles().collect(Collectors.toList()); + Pair logRecordScannerOpenTimePair = + getLogRecordScanner(logFiles, partitionName); HoodieMetadataMergedLogRecordReader logRecordScanner = logRecordScannerOpenTimePair.getKey(); final long logScannerOpenMs = logRecordScannerOpenTimePair.getValue(); @@ -343,9 +345,9 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata { return validInstantTimestamps; } - private Pair getLogRecordScanner(FileSlice slice, String partitionName) { + public Pair getLogRecordScanner(List logFiles, String partitionName) { HoodieTimer timer = new HoodieTimer().startTimer(); - List logFilePaths = slice.getLogFiles() + List sortedLogFilePaths = logFiles.stream() .sorted(HoodieLogFile.getLogFileComparator()) .map(o -> o.getPath().toString()) .collect(Collectors.toList()); @@ -363,7 +365,7 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata { HoodieMetadataMergedLogRecordReader logRecordScanner = HoodieMetadataMergedLogRecordReader.newBuilder() .withFileSystem(metadataMetaClient.getFs()) .withBasePath(metadataBasePath) - .withLogFilePaths(logFilePaths) + .withLogFilePaths(sortedLogFilePaths) .withReaderSchema(schema) .withLatestInstantTime(latestMetadataInstantTime) .withMaxMemorySizeInBytes(MAX_MEMORY_SIZE_IN_BYTES) @@ -378,7 +380,7 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata { Long logScannerOpenMs = timer.endTimer(); LOG.info(String.format("Opened %d metadata log files (dataset instant=%s, metadata instant=%s) in %d ms", - logFilePaths.size(), getLatestDataInstantTime(), latestMetadataInstantTime, logScannerOpenMs)); + sortedLogFilePaths.size(), getLatestDataInstantTime(), latestMetadataInstantTime, logScannerOpenMs)); return Pair.of(logRecordScanner, logScannerOpenMs); } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java index 3730cccee..221b52e77 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java @@ -18,6 +18,13 @@ package org.apache.hudi.metadata; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.model.HoodieMetadataBloomFilter; import org.apache.hudi.avro.model.HoodieMetadataColumnStats; import org.apache.hudi.avro.model.HoodieMetadataFileInfo; @@ -37,14 +44,6 @@ import org.apache.hudi.common.util.hash.PartitionIndexID; import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.io.storage.HoodieHFileReader; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; - import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; @@ -56,6 +55,9 @@ import java.util.Properties; import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.TypeUtils.unsafeCast; +import static org.apache.hudi.common.util.ValidationUtils.checkArgument; +import static org.apache.hudi.common.util.ValidationUtils.checkState; import static org.apache.hudi.metadata.HoodieTableMetadata.RECORDKEY_PARTITION_LIST; /** @@ -95,8 +97,8 @@ public class HoodieMetadataPayload implements HoodieRecordPayload record) { - if (record.isPresent()) { + public HoodieMetadataPayload(Option recordOpt) { + if (recordOpt.isPresent()) { + GenericRecord record = recordOpt.get(); // This can be simplified using SpecificData.deepcopy once this bug is fixed // https://issues.apache.org/jira/browse/AVRO-1811 - key = record.get().get(KEY_FIELD_NAME).toString(); - type = (int) record.get().get(SCHEMA_FIELD_NAME_TYPE); - if (record.get().get(SCHEMA_FIELD_NAME_METADATA) != null) { - filesystemMetadata = (Map) record.get().get("filesystemMetadata"); + // + // NOTE: {@code HoodieMetadataRecord} has to always carry both "key" nad "type" fields + // for it to be handled appropriately, therefore these fields have to be reflected + // in any (read-)projected schema + key = record.get(KEY_FIELD_NAME).toString(); + type = (int) record.get(SCHEMA_FIELD_NAME_TYPE); + + Map metadata = getNestedFieldValue(record, SCHEMA_FIELD_NAME_METADATA); + if (metadata != null) { + filesystemMetadata = metadata; filesystemMetadata.keySet().forEach(k -> { GenericRecord v = filesystemMetadata.get(k); - filesystemMetadata.put(k.toString(), new HoodieMetadataFileInfo((Long) v.get("size"), (Boolean) v.get("isDeleted"))); + filesystemMetadata.put(k, new HoodieMetadataFileInfo((Long) v.get("size"), (Boolean) v.get("isDeleted"))); }); } if (type == METADATA_TYPE_BLOOM_FILTER) { - final GenericRecord metadataRecord = (GenericRecord) record.get().get(SCHEMA_FIELD_ID_BLOOM_FILTER); - if (metadataRecord == null) { - throw new HoodieMetadataException("Valid " + SCHEMA_FIELD_ID_BLOOM_FILTER + " record expected for type: " + METADATA_TYPE_BLOOM_FILTER); + GenericRecord bloomFilterRecord = getNestedFieldValue(record, SCHEMA_FIELD_ID_BLOOM_FILTER); + // NOTE: Only legitimate reason for {@code BloomFilterMetadata} to not be present is when + // it's not been read from the storage (ie it's not been a part of projected schema). + // Otherwise, it has to be present or the record would be considered invalid + if (bloomFilterRecord == null) { + checkArgument(record.getSchema().getField(SCHEMA_FIELD_ID_BLOOM_FILTER) == null, + String.format("Valid %s record expected for type: %s", SCHEMA_FIELD_ID_BLOOM_FILTER, METADATA_TYPE_COLUMN_STATS)); + } else { + bloomFilterMetadata = new HoodieMetadataBloomFilter( + (String) bloomFilterRecord.get(BLOOM_FILTER_FIELD_TYPE), + (String) bloomFilterRecord.get(BLOOM_FILTER_FIELD_TIMESTAMP), + (ByteBuffer) bloomFilterRecord.get(BLOOM_FILTER_FIELD_BLOOM_FILTER), + (Boolean) bloomFilterRecord.get(BLOOM_FILTER_FIELD_IS_DELETED) + ); } - bloomFilterMetadata = new HoodieMetadataBloomFilter( - (String) metadataRecord.get(BLOOM_FILTER_FIELD_TYPE), - (String) metadataRecord.get(BLOOM_FILTER_FIELD_TIMESTAMP), - (ByteBuffer) metadataRecord.get(BLOOM_FILTER_FIELD_BLOOM_FILTER), - (Boolean) metadataRecord.get(BLOOM_FILTER_FIELD_IS_DELETED) - ); } if (type == METADATA_TYPE_COLUMN_STATS) { - GenericRecord v = (GenericRecord) record.get().get(SCHEMA_FIELD_ID_COLUMN_STATS); - if (v == null) { - throw new HoodieMetadataException("Valid " + SCHEMA_FIELD_ID_COLUMN_STATS + " record expected for type: " + METADATA_TYPE_COLUMN_STATS); + GenericRecord columnStatsRecord = getNestedFieldValue(record, SCHEMA_FIELD_ID_COLUMN_STATS); + // NOTE: Only legitimate reason for {@code ColumnStatsMetadata} to not be present is when + // it's not been read from the storage (ie it's not been a part of projected schema). + // Otherwise, it has to be present or the record would be considered invalid + if (columnStatsRecord == null) { + checkArgument(record.getSchema().getField(SCHEMA_FIELD_ID_COLUMN_STATS) == null, + String.format("Valid %s record expected for type: %s", SCHEMA_FIELD_ID_COLUMN_STATS, METADATA_TYPE_COLUMN_STATS)); + } else { + columnStatMetadata = HoodieMetadataColumnStats.newBuilder() + .setFileName((String) columnStatsRecord.get(COLUMN_STATS_FIELD_RESOURCE_NAME)) + .setMinValue((String) columnStatsRecord.get(COLUMN_STATS_FIELD_MIN_VALUE)) + .setMaxValue((String) columnStatsRecord.get(COLUMN_STATS_FIELD_MAX_VALUE)) + .setValueCount((Long) columnStatsRecord.get(COLUMN_STATS_FIELD_VALUE_COUNT)) + .setNullCount((Long) columnStatsRecord.get(COLUMN_STATS_FIELD_NULL_COUNT)) + .setTotalSize((Long) columnStatsRecord.get(COLUMN_STATS_FIELD_TOTAL_SIZE)) + .setTotalUncompressedSize((Long) columnStatsRecord.get(COLUMN_STATS_FIELD_TOTAL_UNCOMPRESSED_SIZE)) + .setIsDeleted((Boolean) columnStatsRecord.get(COLUMN_STATS_FIELD_IS_DELETED)) + .build(); } - columnStatMetadata = HoodieMetadataColumnStats.newBuilder() - .setFileName((String) v.get(COLUMN_STATS_FIELD_RESOURCE_NAME)) - .setMinValue((String) v.get(COLUMN_STATS_FIELD_MIN_VALUE)) - .setMaxValue((String) v.get(COLUMN_STATS_FIELD_MAX_VALUE)) - .setValueCount((Long) v.get(COLUMN_STATS_FIELD_VALUE_COUNT)) - .setNullCount((Long) v.get(COLUMN_STATS_FIELD_NULL_COUNT)) - .setTotalSize((Long) v.get(COLUMN_STATS_FIELD_TOTAL_SIZE)) - .setTotalUncompressedSize((Long) v.get(COLUMN_STATS_FIELD_TOTAL_UNCOMPRESSED_SIZE)) - .setIsDeleted((Boolean) v.get(COLUMN_STATS_FIELD_IS_DELETED)) - .build(); } } } @@ -175,12 +194,12 @@ public class HoodieMetadataPayload implements HoodieRecordPayload(key, metadataPayload); } @@ -273,9 +292,9 @@ public class HoodieMetadataPayload implements HoodieRecordPayload { HoodieKey key = new HoodieKey(getColumnStatsIndexKey(partitionName, columnRangeMetadata), MetadataPartitionType.COLUMN_STATS.getPartitionPath()); - HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(), METADATA_TYPE_COLUMN_STATS, + HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(), HoodieMetadataColumnStats.newBuilder() .setFileName(new Path(columnRangeMetadata.getFilePath()).getName()) .setMinValue(columnRangeMetadata.getMinValue() == null ? null : @@ -473,7 +492,7 @@ public class HoodieMetadataPayload implements HoodieRecordPayload T getNestedFieldValue(GenericRecord record, String fieldName) { + // NOTE: This routine is more lightweight than {@code HoodieAvroUtils.getNestedFieldVal} + if (record.getSchema().getField(fieldName) == null) { + return null; + } + + return unsafeCast(record.get(fieldName)); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java index 6a5df050a..52fdbd993 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java @@ -18,14 +18,14 @@ package org.apache.hudi.metadata; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.model.HoodieMetadataColumnStats; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.Option; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieMetadataException; @@ -35,6 +35,8 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.Map; +import static org.apache.hudi.common.util.ValidationUtils.checkArgument; + /** * Interface that supports querying various pieces of metadata about a hudi table. */ @@ -58,12 +60,19 @@ public interface HoodieTableMetadata extends Serializable, AutoCloseable { static final String METADATA_TABLE_REL_PATH = HoodieTableMetaClient.METAFOLDER_NAME + Path.SEPARATOR + "metadata"; /** - * Return the base path of the Metadata Table. - * - * @param tableBasePath The base path of the dataset + * Return the base-path of the Metadata Table for the given Dataset identified by base-path */ - static String getMetadataTableBasePath(String tableBasePath) { - return tableBasePath + Path.SEPARATOR + METADATA_TABLE_REL_PATH; + static String getMetadataTableBasePath(String dataTableBasePath) { + return dataTableBasePath + Path.SEPARATOR + METADATA_TABLE_REL_PATH; + } + + /** + * Returns the base path of the Dataset provided the base-path of the Metadata Table of this + * Dataset + */ + static String getDataTableBasePathFromMetadataTable(String metadataTableBasePath) { + checkArgument(isMetadataTable(metadataTableBasePath)); + return metadataTableBasePath.substring(0, metadataTableBasePath.lastIndexOf(METADATA_TABLE_REL_PATH) - 1); } /** diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/RawTripTestPayload.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/RawTripTestPayload.java index 8bd10823d..c052b63ab 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/RawTripTestPayload.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/RawTripTestPayload.java @@ -80,6 +80,23 @@ public class RawTripTestPayload implements HoodieRecordPayload recordsToStrings(List records) { return records.stream().map(RawTripTestPayload::recordToString).filter(Option::isPresent).map(Option::get) .collect(Collectors.toList()); diff --git a/hudi-common/src/test/resources/timestamp-test-evolved.avsc b/hudi-common/src/test/resources/timestamp-test-evolved.avsc index beb36329e..7a52ca6f2 100644 --- a/hudi-common/src/test/resources/timestamp-test-evolved.avsc +++ b/hudi-common/src/test/resources/timestamp-test-evolved.avsc @@ -20,7 +20,43 @@ "type": "record", "name": "User", "fields": [ - {"name": "field1", "type": ["null", "string"], "default": null}, - {"name": "createTime", "type": ["null", "long"], "default": null} + { + "name": "field1", + "type": [ + "null", + "string" + ], + "default": null + }, + { + "name": "createTime", + "type": [ + "null", + "long" + ], + "default": null + }, + { + "name": "createTimeString", + "type": [ + "null", + "string" + ], + "default": null + }, + { + "name": "createTimeDecimal", + "type": [ + "null", + { + "name": "decimalFixed", + "type": "fixed", + "logicalType": "decimal", + "precision": 20, + "scale": 4, + "size": 10 + } + ] + } ] } \ No newline at end of file diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java index 396782d96..d2501ee8d 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java @@ -68,13 +68,13 @@ public class HoodieRealtimeInputFormatUtils extends HoodieInputFormatUtils { } // Return parquet file with a list of log files in the same file group. - public static List, List>> groupLogsByBaseFile(Configuration conf, List partitionPaths) { + public static List, List>> groupLogsByBaseFile(Configuration conf, List partitionPaths) { Set partitionSet = new HashSet<>(partitionPaths); // TODO(vc): Should we handle also non-hoodie splits here? Map partitionsToMetaClient = getTableMetaClientByPartitionPath(conf, partitionSet); // Get all the base file and it's log files pairs in required partition paths. - List, List>> baseAndLogsList = new ArrayList<>(); + List, List>> baseAndLogsList = new ArrayList<>(); partitionSet.forEach(partitionPath -> { // for each partition path obtain the data & log file groupings, then map back to inputsplits HoodieTableMetaClient metaClient = partitionsToMetaClient.get(partitionPath); @@ -91,8 +91,7 @@ public class HoodieRealtimeInputFormatUtils extends HoodieInputFormatUtils { .orElse(Stream.empty()); latestFileSlices.forEach(fileSlice -> { - List logFilePaths = fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()) - .map(logFile -> logFile.getPath().toString()).collect(Collectors.toList()); + List logFilePaths = fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList()); baseAndLogsList.add(Pair.of(fileSlice.getBaseFile(), logFilePaths)); }); } catch (Exception e) { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/SparkRowWriteHelper.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/SparkRowWriteHelper.java index 6f5dd3713..ea9c9b2c0 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/SparkRowWriteHelper.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/SparkRowWriteHelper.java @@ -19,7 +19,6 @@ package org.apache.hudi; import org.apache.hudi.common.model.HoodieRecord; - import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.api.java.function.ReduceFunction; import org.apache.spark.sql.Dataset; @@ -30,14 +29,13 @@ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder; import org.apache.spark.sql.catalyst.encoders.RowEncoder; import org.apache.spark.sql.catalyst.expressions.Attribute; import org.apache.spark.sql.types.StructType; - -import java.util.List; -import java.util.stream.Collectors; - import scala.Tuple2; import scala.collection.JavaConversions; import scala.collection.JavaConverters; +import java.util.List; +import java.util.stream.Collectors; + /** * Helper class to assist in deduplicating Rows for BulkInsert with Rows. */ @@ -55,20 +53,13 @@ public class SparkRowWriteHelper { } public Dataset deduplicateRows(Dataset inputDf, String preCombineField, boolean isGlobalIndex) { - ExpressionEncoder encoder = getEncoder(inputDf.schema()); - - return inputDf.groupByKey( - (MapFunction) value -> - isGlobalIndex ? (value.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD)) : - (value.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD) + "+" + value.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD)), Encoders.STRING()) - .reduceGroups((ReduceFunction) (v1, v2) -> { - if (((Comparable) v1.getAs(preCombineField)).compareTo(((Comparable) v2.getAs(preCombineField))) >= 0) { - return v1; - } else { - return v2; - } - } - ).map((MapFunction, Row>) value -> value._2, encoder); + return inputDf.groupByKey((MapFunction) value -> + isGlobalIndex + ? (value.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD)) + : (value.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD) + "+" + value.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD)), Encoders.STRING()) + .reduceGroups((ReduceFunction) (v1, v2) -> + ((Comparable) v1.getAs(preCombineField)).compareTo(v2.getAs(preCombineField)) >= 0 ? v1 : v2) + .map((MapFunction, Row>) value -> value._2, getEncoder(inputDf.schema())); } private ExpressionEncoder getEncoder(StructType schema) { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala index 74aa8a3d9..1e2946dd2 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala @@ -18,17 +18,30 @@ package org.apache.hudi import org.apache.avro.Schema - +import org.apache.avro.generic.GenericRecord +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hbase.io.hfile.CacheConfig +import org.apache.hudi.common.config.SerializableConfiguration +import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.model.HoodieFileFormat import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} - +import org.apache.hudi.common.util.StringUtils +import org.apache.hudi.io.storage.HoodieHFileReader +import org.apache.hudi.metadata.HoodieTableMetadata import org.apache.spark.internal.Logging -import org.apache.spark.sql.{SQLContext, SparkSession} import org.apache.spark.sql.avro.SchemaConverters -import org.apache.spark.sql.sources.{BaseRelation, PrunedFilteredScan} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.sources.{BaseRelation, Filter, PrunedFilteredScan} import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{SQLContext, SparkSession} +import scala.collection.JavaConverters._ import scala.util.Try +case class HoodieTableSchema(structTypeSchema: StructType, avroSchemaStr: String) + /** * Hoodie BaseRelation which extends [[PrunedFilteredScan]]. */ @@ -41,15 +54,105 @@ abstract class HoodieBaseRelation( protected val sparkSession: SparkSession = sqlContext.sparkSession - protected val tableAvroSchema: Schema = { + protected lazy val tableAvroSchema: Schema = { val schemaUtil = new TableSchemaResolver(metaClient) - Try (schemaUtil.getTableAvroSchema).getOrElse(SchemaConverters.toAvroType(userSchema.get)) + Try(schemaUtil.getTableAvroSchema).getOrElse( + // If there is no commit in the table, we can't get the schema + // t/h [[TableSchemaResolver]], fallback to the provided [[userSchema]] instead. + userSchema match { + case Some(s) => SchemaConverters.toAvroType(s) + case _ => throw new IllegalArgumentException("User-provided schema is required in case the table is empty") + } + ) } protected val tableStructSchema: StructType = AvroConversionUtils.convertAvroSchemaToStructType(tableAvroSchema) protected val partitionColumns: Array[String] = metaClient.getTableConfig.getPartitionFields.orElse(Array.empty) - override def schema: StructType = userSchema.getOrElse(tableStructSchema) + protected def getPrecombineFieldProperty: Option[String] = + Option(metaClient.getTableConfig.getPreCombineField) + .orElse(optParams.get(DataSourceWriteOptions.PRECOMBINE_FIELD.key)) match { + // NOTE: This is required to compensate for cases when empty string is used to stub + // property value to avoid it being set with the default value + // TODO(HUDI-3456) cleanup + case Some(f) if !StringUtils.isNullOrEmpty(f) => Some(f) + case _ => None + } + override def schema: StructType = tableStructSchema +} + +object HoodieBaseRelation { + + def isMetadataTable(metaClient: HoodieTableMetaClient) = + HoodieTableMetadata.isMetadataTable(metaClient.getBasePath) + + /** + * Returns file-reader routine accepting [[PartitionedFile]] and returning an [[Iterator]] + * over [[InternalRow]] + */ + def createBaseFileReader(spark: SparkSession, + partitionSchema: StructType, + tableSchema: HoodieTableSchema, + requiredSchema: HoodieTableSchema, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = { + val hfileReader = createHFileReader( + spark = spark, + tableSchema = tableSchema, + requiredSchema = requiredSchema, + filters = filters, + options = options, + hadoopConf = hadoopConf + ) + val parquetReader = HoodieDataSourceHelper.buildHoodieParquetReader( + sparkSession = spark, + dataSchema = tableSchema.structTypeSchema, + partitionSchema = partitionSchema, + requiredSchema = requiredSchema.structTypeSchema, + filters = filters, + options = options, + hadoopConf = hadoopConf + ) + + partitionedFile => { + val extension = FSUtils.getFileExtension(partitionedFile.filePath) + if (HoodieFileFormat.PARQUET.getFileExtension.equals(extension)) { + parquetReader.apply(partitionedFile) + } else if (HoodieFileFormat.HFILE.getFileExtension.equals(extension)) { + hfileReader.apply(partitionedFile) + } else { + throw new UnsupportedOperationException(s"Base file format not supported by Spark DataSource ($partitionedFile)") + } + } + } + + private def createHFileReader(spark: SparkSession, + tableSchema: HoodieTableSchema, + requiredSchema: HoodieTableSchema, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = { + val hadoopConfBroadcast = + spark.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) + + partitionedFile => { + val hadoopConf = hadoopConfBroadcast.value.get() + val reader = new HoodieHFileReader[GenericRecord](hadoopConf, new Path(partitionedFile.filePath), + new CacheConfig(hadoopConf)) + + val requiredRowSchema = requiredSchema.structTypeSchema + // NOTE: Schema has to be parsed at this point, since Avro's [[Schema]] aren't serializable + // to be passed from driver to executor + val requiredAvroSchema = new Schema.Parser().parse(requiredSchema.avroSchemaStr) + val avroToRowConverter = AvroConversionUtils.createAvroToInternalRowConverter(requiredAvroSchema, requiredRowSchema) + + reader.getRecordIterator(requiredAvroSchema).asScala + .map(record => { + avroToRowConverter.apply(record.asInstanceOf[GenericRecord]).get + }) + } + } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDataSourceHelper.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDataSourceHelper.scala index a04f54621..fb12549f6 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDataSourceHelper.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDataSourceHelper.scala @@ -20,18 +20,16 @@ package org.apache.hudi import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileStatus - import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Expression, PredicateHelper, SpecificInternalRow, SubqueryExpression, UnsafeProjection} +import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat -import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile} import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer object HoodieDataSourceHelper extends PredicateHelper { @@ -77,14 +75,13 @@ object HoodieDataSourceHelper extends PredicateHelper { * Wrapper `buildReaderWithPartitionValues` of [[ParquetFileFormat]] * to deal with [[ColumnarBatch]] when enable parquet vectorized reader if necessary. */ - def buildHoodieParquetReader( - sparkSession: SparkSession, - dataSchema: StructType, - partitionSchema: StructType, - requiredSchema: StructType, - filters: Seq[Filter], - options: Map[String, String], - hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = { + def buildHoodieParquetReader(sparkSession: SparkSession, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = { val readParquetFile: PartitionedFile => Iterator[Any] = new ParquetFileFormat().buildReaderWithPartitionValues( sparkSession = sparkSession, @@ -98,11 +95,10 @@ object HoodieDataSourceHelper extends PredicateHelper { file: PartitionedFile => { val iter = readParquetFile(file) - val rows = iter.flatMap(_ match { + iter.flatMap { case r: InternalRow => Seq(r) case b: ColumnarBatch => b.rowIterator().asScala - }) - rows + } } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala index 8903ee6a4..96fe47e02 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala @@ -23,14 +23,19 @@ import org.apache.avro.generic.{GenericRecord, GenericRecordBuilder} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hudi.HoodieDataSourceHelper._ +import org.apache.hudi.HoodieMergeOnReadRDD.resolveAvroSchemaNullability +import org.apache.hudi.MergeOnReadSnapshotRelation.getFilePath +import org.apache.hudi.common.config.HoodieMetadataConfig +import org.apache.hudi.common.engine.HoodieLocalEngineContext import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.fs.FSUtils.getRelativePartitionPath import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner import org.apache.hudi.config.HoodiePayloadConfig import org.apache.hudi.exception.HoodieException import org.apache.hudi.hadoop.config.HoodieRealtimeConfig -import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS +import org.apache.hudi.metadata.HoodieTableMetadata.getDataTableBasePathFromMetadataTable +import org.apache.hudi.metadata.{HoodieBackedTableMetadata, HoodieTableMetadata} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.avro.{HoodieAvroDeserializer, HoodieAvroSerializer} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeProjection import org.apache.spark.sql.execution.datasources.PartitionedFile @@ -48,51 +53,38 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, @transient config: Configuration, fullSchemaFileReader: PartitionedFile => Iterator[InternalRow], requiredSchemaFileReader: PartitionedFile => Iterator[InternalRow], - tableState: HoodieMergeOnReadTableState) + tableState: HoodieMergeOnReadTableState, + tableSchema: HoodieTableSchema, + requiredSchema: HoodieTableSchema) extends RDD[InternalRow](sc, Nil) { private val confBroadcast = sc.broadcast(new SerializableWritable(config)) - private val preCombineField = tableState.preCombineField - private val recordKeyFieldOpt = tableState.recordKeyFieldOpt - private val payloadProps = if (preCombineField.isDefined) { - HoodiePayloadConfig.newBuilder - .withPayloadOrderingField(preCombineField.get) - .build.getProps - } else { - new Properties() - } - - private val requiredSchema = tableState.requiredStructSchema - - private val requiredFieldPosition = HoodieSparkUtils.collectFieldIndexes(requiredSchema, - tableState.tableStructSchema - ) + private val recordKeyField = tableState.recordKeyField + private val payloadProps = tableState.preCombineFieldOpt + .map(preCombineField => + HoodiePayloadConfig.newBuilder + .withPayloadOrderingField(preCombineField) + .build + .getProps + ) + .getOrElse(new Properties()) override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { val mergeOnReadPartition = split.asInstanceOf[HoodieMergeOnReadPartition] val iter = mergeOnReadPartition.split match { - case dataFileOnlySplit if dataFileOnlySplit.logPaths.isEmpty => - val rows = requiredSchemaFileReader(dataFileOnlySplit.dataFile.get) - extractRequiredSchema(rows, requiredSchema, requiredFieldPosition) + case dataFileOnlySplit if dataFileOnlySplit.logFiles.isEmpty => + requiredSchemaFileReader(dataFileOnlySplit.dataFile.get) case logFileOnlySplit if logFileOnlySplit.dataFile.isEmpty => logFileIterator(logFileOnlySplit, getConfig) - case skipMergeSplit if skipMergeSplit.mergeType - .equals(DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL) => - skipMergeFileIterator( - skipMergeSplit, - requiredSchemaFileReader(skipMergeSplit.dataFile.get), - getConfig - ) - case payloadCombineSplit if payloadCombineSplit.mergeType - .equals(DataSourceReadOptions.REALTIME_PAYLOAD_COMBINE_OPT_VAL) => - payloadCombineFileIterator( - payloadCombineSplit, - fullSchemaFileReader(payloadCombineSplit.dataFile.get), - getConfig - ) + case skipMergeSplit if skipMergeSplit.mergeType.equals(DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL) => + skipMergeFileIterator(skipMergeSplit, requiredSchemaFileReader(skipMergeSplit.dataFile.get), getConfig) + case payloadCombineSplit + if payloadCombineSplit.mergeType.equals(DataSourceReadOptions.REALTIME_PAYLOAD_COMBINE_OPT_VAL) => + payloadCombineFileIterator(payloadCombineSplit, fullSchemaFileReader(payloadCombineSplit.dataFile.get), + getConfig) case _ => throw new HoodieException(s"Unable to select an Iterator to read the Hoodie MOR File Split for " + s"file path: ${mergeOnReadPartition.split.dataFile.get.filePath}" + - s"log paths: ${mergeOnReadPartition.split.logPaths.toString}" + + s"log paths: ${mergeOnReadPartition.split.logFiles.toString}" + s"hoodie table path: ${mergeOnReadPartition.split.tablePath}" + s"spark partition Index: ${mergeOnReadPartition.index}" + s"merge type: ${mergeOnReadPartition.split.mergeType}") @@ -121,12 +113,15 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, private def logFileIterator(split: HoodieMergeOnReadFileSplit, config: Configuration): Iterator[InternalRow] = - new Iterator[InternalRow] with Closeable { - private val tableAvroSchema = new Schema.Parser().parse(tableState.tableAvroSchema) - private val requiredAvroSchema = new Schema.Parser().parse(tableState.requiredAvroSchema) + new Iterator[InternalRow] with Closeable with SparkAdapterSupport { + private val tableAvroSchema = new Schema.Parser().parse(tableSchema.avroSchemaStr) + private val requiredAvroSchema = new Schema.Parser().parse(requiredSchema.avroSchemaStr) + private val requiredFieldPosition = + requiredSchema.structTypeSchema + .map(f => tableAvroSchema.getField(f.name).pos()).toList private val recordBuilder = new GenericRecordBuilder(requiredAvroSchema) - private val deserializer = HoodieAvroDeserializer(requiredAvroSchema, tableState.requiredStructSchema) - private val unsafeProjection = UnsafeProjection.create(tableState.requiredStructSchema) + private val deserializer = sparkAdapter.createAvroDeserializer(requiredAvroSchema, requiredSchema.structTypeSchema) + private val unsafeProjection = UnsafeProjection.create(requiredSchema.structTypeSchema) private var logScanner = HoodieMergeOnReadRDD.scanLog(split, tableAvroSchema, config) private val logRecords = logScanner.getRecords private val logRecordsKeyIterator = logRecords.keySet().iterator().asScala @@ -141,9 +136,10 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, // delete record found, skipping this.hasNext } else { - val requiredAvroRecord = AvroConversionUtils - .buildAvroRecordBySchema(curAvroRecord.get(), requiredAvroSchema, requiredFieldPosition, recordBuilder) - recordToLoad = unsafeProjection(deserializer.deserializeData(requiredAvroRecord).asInstanceOf[InternalRow]) + val requiredAvroRecord = AvroConversionUtils.buildAvroRecordBySchema(curAvroRecord.get(), requiredAvroSchema, + requiredFieldPosition, recordBuilder) + val rowOpt = deserializer.deserialize(requiredAvroRecord) + recordToLoad = unsafeProjection(rowOpt.get.asInstanceOf[InternalRow]) true } } else { @@ -169,12 +165,15 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, private def skipMergeFileIterator(split: HoodieMergeOnReadFileSplit, baseFileIterator: Iterator[InternalRow], config: Configuration): Iterator[InternalRow] = - new Iterator[InternalRow] with Closeable { - private val tableAvroSchema = new Schema.Parser().parse(tableState.tableAvroSchema) - private val requiredAvroSchema = new Schema.Parser().parse(tableState.requiredAvroSchema) + new Iterator[InternalRow] with Closeable with SparkAdapterSupport { + private val tableAvroSchema = new Schema.Parser().parse(tableSchema.avroSchemaStr) + private val requiredAvroSchema = new Schema.Parser().parse(requiredSchema.avroSchemaStr) + private val requiredFieldPosition = + requiredSchema.structTypeSchema + .map(f => tableAvroSchema.getField(f.name).pos()).toList private val recordBuilder = new GenericRecordBuilder(requiredAvroSchema) - private val deserializer = HoodieAvroDeserializer(requiredAvroSchema, tableState.requiredStructSchema) - private val unsafeProjection = UnsafeProjection.create(tableState.requiredStructSchema) + private val deserializer = sparkAdapter.createAvroDeserializer(requiredAvroSchema, requiredSchema.structTypeSchema) + private val unsafeProjection = UnsafeProjection.create(requiredSchema.structTypeSchema) private var logScanner = HoodieMergeOnReadRDD.scanLog(split, tableAvroSchema, config) private val logRecords = logScanner.getRecords private val logRecordsKeyIterator = logRecords.keySet().iterator().asScala @@ -185,7 +184,7 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, override def hasNext: Boolean = { if (baseFileIterator.hasNext) { val curRow = baseFileIterator.next() - recordToLoad = unsafeProjection(createInternalRowWithSchema(curRow, requiredSchema, requiredFieldPosition)) + recordToLoad = unsafeProjection(curRow) true } else { if (logRecordsKeyIterator.hasNext) { @@ -195,9 +194,10 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, // delete record found, skipping this.hasNext } else { - val requiredAvroRecord = AvroConversionUtils - .buildAvroRecordBySchema(curAvroRecord.get(), requiredAvroSchema, requiredFieldPosition, recordBuilder) - recordToLoad = unsafeProjection(deserializer.deserializeData(requiredAvroRecord).asInstanceOf[InternalRow]) + val requiredAvroRecord = AvroConversionUtils.buildAvroRecordBySchema(curAvroRecord.get(), requiredAvroSchema, + requiredFieldPosition, recordBuilder) + val rowOpt = deserializer.deserialize(requiredAvroRecord) + recordToLoad = unsafeProjection(rowOpt.get.asInstanceOf[InternalRow]) true } } else { @@ -224,18 +224,22 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, private def payloadCombineFileIterator(split: HoodieMergeOnReadFileSplit, baseFileIterator: Iterator[InternalRow], config: Configuration): Iterator[InternalRow] = - new Iterator[InternalRow] with Closeable { - private val tableAvroSchema = new Schema.Parser().parse(tableState.tableAvroSchema) - private val requiredAvroSchema = new Schema.Parser().parse(tableState.requiredAvroSchema) - private val serializer = HoodieAvroSerializer(tableState.tableStructSchema, tableAvroSchema, false) - private val requiredDeserializer = HoodieAvroDeserializer(requiredAvroSchema, tableState.requiredStructSchema) + new Iterator[InternalRow] with Closeable with SparkAdapterSupport { + private val tableAvroSchema = new Schema.Parser().parse(tableSchema.avroSchemaStr) + private val requiredAvroSchema = new Schema.Parser().parse(requiredSchema.avroSchemaStr) + private val requiredFieldPosition = + requiredSchema.structTypeSchema + .map(f => tableAvroSchema.getField(f.name).pos()).toList + private val serializer = sparkAdapter.createAvroSerializer(tableSchema.structTypeSchema, tableAvroSchema, + resolveAvroSchemaNullability(tableAvroSchema)) + private val requiredDeserializer = sparkAdapter.createAvroDeserializer(requiredAvroSchema, requiredSchema.structTypeSchema) private val recordBuilder = new GenericRecordBuilder(requiredAvroSchema) - private val unsafeProjection = UnsafeProjection.create(tableState.requiredStructSchema) + private val unsafeProjection = UnsafeProjection.create(requiredSchema.structTypeSchema) private var logScanner = HoodieMergeOnReadRDD.scanLog(split, tableAvroSchema, config) private val logRecords = logScanner.getRecords private val logRecordsKeyIterator = logRecords.keySet().iterator().asScala private val keyToSkip = mutable.Set.empty[String] - private val recordKeyPosition = if (recordKeyFieldOpt.isEmpty) HOODIE_RECORD_KEY_COL_POS else tableState.tableStructSchema.fieldIndex(recordKeyFieldOpt.get) + private val recordKeyPosition = tableSchema.structTypeSchema.fieldIndex(recordKeyField) private var recordToLoad: InternalRow = _ @@ -253,20 +257,15 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, this.hasNext } else { // load merged record as InternalRow with required schema - val requiredAvroRecord = AvroConversionUtils - .buildAvroRecordBySchema( - mergedAvroRecord.get(), - requiredAvroSchema, - requiredFieldPosition, - recordBuilder - ) - recordToLoad = unsafeProjection(requiredDeserializer - .deserializeData(requiredAvroRecord).asInstanceOf[InternalRow]) + val requiredAvroRecord = AvroConversionUtils.buildAvroRecordBySchema(mergedAvroRecord.get(), requiredAvroSchema, + requiredFieldPosition, recordBuilder) + val rowOpt = requiredDeserializer.deserialize(requiredAvroRecord) + recordToLoad = unsafeProjection(rowOpt.get.asInstanceOf[InternalRow]) true } } else { // No merge needed, load current row with required schema - recordToLoad = unsafeProjection(createInternalRowWithSchema(curRow, requiredSchema, requiredFieldPosition)) + recordToLoad = unsafeProjection(createInternalRowWithSchema(curRow, requiredSchema.structTypeSchema, requiredFieldPosition)) true } } else { @@ -287,8 +286,8 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, requiredFieldPosition, recordBuilder ) - recordToLoad = unsafeProjection(requiredDeserializer - .deserializeData(requiredAvroRecord).asInstanceOf[InternalRow]) + val rowOpt = requiredDeserializer.deserialize(requiredAvroRecord) + recordToLoad = unsafeProjection(rowOpt.get.asInstanceOf[InternalRow]) true } } @@ -312,8 +311,8 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, private def mergeRowWithLog(curRow: InternalRow, curKey: String) = { val historyAvroRecord = serializer.serialize(curRow).asInstanceOf[GenericRecord] - logRecords.get(curKey).getData.combineAndGetUpdateValue( - historyAvroRecord, tableAvroSchema, payloadProps) + logRecords.get(curKey).getData + .combineAndGetUpdateValue(historyAvroRecord, tableAvroSchema, payloadProps) } } } @@ -323,32 +322,60 @@ private object HoodieMergeOnReadRDD { def scanLog(split: HoodieMergeOnReadFileSplit, logSchema: Schema, config: Configuration): HoodieMergedLogRecordScanner = { val fs = FSUtils.getFs(split.tablePath, config) - val partitionPath: String = if (split.logPaths.isEmpty || split.logPaths.get.asJava.isEmpty) { - null + val logFiles = split.logFiles.get + + if (HoodieTableMetadata.isMetadataTable(split.tablePath)) { + val metadataConfig = HoodieMetadataConfig.newBuilder().enable(true).build() + val dataTableBasePath = getDataTableBasePathFromMetadataTable(split.tablePath) + val metadataTable = new HoodieBackedTableMetadata( + new HoodieLocalEngineContext(config), metadataConfig, + dataTableBasePath, + config.get(HoodieRealtimeConfig.SPILLABLE_MAP_BASE_PATH_PROP, HoodieRealtimeConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH)) + + // NOTE: In case of Metadata Table partition path equates to partition name (since there's just one level + // of indirection among MT partitions) + val relativePartitionPath = getRelativePartitionPath(new Path(split.tablePath), getPartitionPath(split)) + metadataTable.getLogRecordScanner(logFiles.asJava, relativePartitionPath).getLeft } else { - new Path(split.logPaths.get.asJava.get(0)).getParent.getName + val logRecordScannerBuilder = HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(fs) + .withBasePath(split.tablePath) + .withLogFilePaths(split.logFiles.get.map(logFile => getFilePath(logFile.getPath)).asJava) + .withReaderSchema(logSchema) + .withLatestInstantTime(split.latestCommit) + .withReadBlocksLazily( + Try(config.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, + HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED).toBoolean) + .getOrElse(false)) + .withReverseReader(false) + .withBufferSize( + config.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP, + HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE)) + .withMaxMemorySizeInBytes(split.maxCompactionMemoryInBytes) + .withSpillableMapBasePath( + config.get(HoodieRealtimeConfig.SPILLABLE_MAP_BASE_PATH_PROP, + HoodieRealtimeConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH)) + + if (logFiles.nonEmpty) { + logRecordScannerBuilder.withPartition(getRelativePartitionPath(new Path(split.tablePath), logFiles.head.getPath.getParent)) + } + + logRecordScannerBuilder.build() } - val logRecordScannerBuilder = HoodieMergedLogRecordScanner.newBuilder() - .withFileSystem(fs) - .withBasePath(split.tablePath) - .withLogFilePaths(split.logPaths.get.asJava) - .withReaderSchema(logSchema) - .withLatestInstantTime(split.latestCommit) - .withReadBlocksLazily( - Try(config.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, - HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED).toBoolean) - .getOrElse(false)) - .withReverseReader(false) - .withBufferSize( - config.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP, - HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE)) - .withMaxMemorySizeInBytes(split.maxCompactionMemoryInBytes) - .withSpillableMapBasePath( - config.get(HoodieRealtimeConfig.SPILLABLE_MAP_BASE_PATH_PROP, - HoodieRealtimeConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH)) - if (partitionPath != null) { - logRecordScannerBuilder.withPartition(partitionPath) + } + + private def getPartitionPath(split: HoodieMergeOnReadFileSplit): Path = { + // Determine partition path as an immediate parent folder of either + // - The base file + // - Some log file + split.dataFile.map(baseFile => new Path(baseFile.filePath)) + .getOrElse(split.logFiles.get.head.getPath) + .getParent + } + + private def resolveAvroSchemaNullability(schema: Schema) = { + AvroConversionUtils.resolveAvroTypeNullability(schema) match { + case (nullable, _) => nullable } - logRecordScannerBuilder.build() } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala index cda7b81f0..b9d18c68d 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala @@ -17,8 +17,10 @@ package org.apache.hudi +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{GlobPattern, Path} import org.apache.hadoop.mapred.JobConf +import org.apache.hudi.HoodieBaseRelation.createBaseFileReader import org.apache.hudi.common.model.HoodieRecord import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.table.view.HoodieTableFileSystemView @@ -35,17 +37,17 @@ import org.apache.spark.sql.{Row, SQLContext} import scala.collection.JavaConversions._ /** - * Experimental. - * Relation, that implements the Hoodie incremental view for Merge On Read table. - * - */ + * Experimental. + * Relation, that implements the Hoodie incremental view for Merge On Read table. + * + */ class MergeOnReadIncrementalRelation(sqlContext: SQLContext, val optParams: Map[String, String], val userSchema: Option[StructType], val metaClient: HoodieTableMetaClient) extends HoodieBaseRelation(sqlContext, metaClient, optParams, userSchema) { - private val conf = sqlContext.sparkContext.hadoopConfiguration + private val conf = new Configuration(sqlContext.sparkContext.hadoopConfiguration) private val jobConf = new JobConf(conf) private val commitTimeline = metaClient.getCommitsAndCompactionTimeline.filterCompletedInstants() @@ -75,84 +77,89 @@ class MergeOnReadIncrementalRelation(sqlContext: SQLContext, private val fileIndex = if (commitsToReturn.isEmpty) List() else buildFileIndex() - private val preCombineField = { - val preCombineFieldFromTableConfig = metaClient.getTableConfig.getPreCombineField - if (preCombineFieldFromTableConfig != null) { - Some(preCombineFieldFromTableConfig) - } else { - // get preCombineFiled from the options if this is a old table which have not store - // the field to hoodie.properties - optParams.get(DataSourceReadOptions.READ_PRE_COMBINE_FIELD.key) - } + private val preCombineFieldOpt = getPrecombineFieldProperty + + // Record filters making sure that only records w/in the requested bounds are being fetched as part of the + // scan collected by this relation + private lazy val incrementalSpanRecordsFilters: Seq[Filter] = { + val isNotNullFilter = IsNotNull(HoodieRecord.COMMIT_TIME_METADATA_FIELD) + val largerThanFilter = GreaterThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.head.getTimestamp) + val lessThanFilter = LessThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.last.getTimestamp) + Seq(isNotNullFilter, largerThanFilter, lessThanFilter) + } + + private lazy val mandatoryColumns = { + // NOTE: This columns are required for Incremental flow to be able to handle the rows properly, even in + // cases when no columns are requested to be fetched (for ex, when using {@code count()} API) + Seq(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD) ++ + preCombineFieldOpt.map(Seq(_)).getOrElse(Seq()) } override def needConversion: Boolean = false - override def unhandledFilters(filters: Array[Filter]): Array[Filter] = { - if (fileIndex.isEmpty) { - filters - } else { - val isNotNullFilter = IsNotNull(HoodieRecord.COMMIT_TIME_METADATA_FIELD) - val largerThanFilter = GreaterThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.head.getTimestamp) - val lessThanFilter = LessThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.last.getTimestamp) - filters :+ isNotNullFilter :+ largerThanFilter :+ lessThanFilter - } - } - override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = { if (fileIndex.isEmpty) { sqlContext.sparkContext.emptyRDD[Row] } else { logDebug(s"buildScan requiredColumns = ${requiredColumns.mkString(",")}") logDebug(s"buildScan filters = ${filters.mkString(",")}") + // config to ensure the push down filter for parquet will be applied. sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.filterPushdown", "true") sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.recordLevelFilter.enabled", "true") sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.enableVectorizedReader", "false") - val pushDownFilter = { - val isNotNullFilter = IsNotNull(HoodieRecord.COMMIT_TIME_METADATA_FIELD) - val largerThanFilter = GreaterThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.head.getTimestamp) - val lessThanFilter = LessThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.last.getTimestamp) - filters :+ isNotNullFilter :+ largerThanFilter :+ lessThanFilter - } + + val fetchedColumns: Array[String] = appendMandatoryColumns(requiredColumns) + val (requiredAvroSchema, requiredStructSchema) = - HoodieSparkUtils.getRequiredSchema(tableAvroSchema, requiredColumns) + HoodieSparkUtils.getRequiredSchema(tableAvroSchema, fetchedColumns) - val hoodieTableState = HoodieMergeOnReadTableState( - tableStructSchema, - requiredStructSchema, - tableAvroSchema.toString, - requiredAvroSchema.toString, - fileIndex, - preCombineField, - Option.empty - ) - val fullSchemaParquetReader = HoodieDataSourceHelper.buildHoodieParquetReader( - sparkSession = sqlContext.sparkSession, - dataSchema = tableStructSchema, - partitionSchema = StructType(Nil), - requiredSchema = tableStructSchema, - filters = pushDownFilter, + val partitionSchema = StructType(Nil) + val tableSchema = HoodieTableSchema(tableStructSchema, tableAvroSchema.toString) + val requiredSchema = HoodieTableSchema(requiredStructSchema, requiredAvroSchema.toString) + + val fullSchemaParquetReader = createBaseFileReader( + spark = sqlContext.sparkSession, + partitionSchema = partitionSchema, + tableSchema = tableSchema, + requiredSchema = tableSchema, + // This file-reader is used to read base file records, subsequently merging them with the records + // stored in delta-log files. As such, we have to read _all_ records from the base file, while avoiding + // applying any user-defined filtering _before_ we complete combining them w/ delta-log records (to make sure that + // we combine them correctly) + // + // The only filtering applicable here is the filtering to make sure we're only fetching records that + // fall into incremental span of the timeline being queried + filters = incrementalSpanRecordsFilters, options = optParams, - hadoopConf = sqlContext.sparkSession.sessionState.newHadoopConf() + // NOTE: We have to fork the Hadoop Config here as Spark will be modifying it + // to configure Parquet reader appropriately + hadoopConf = new Configuration(conf) ) - - val requiredSchemaParquetReader = HoodieDataSourceHelper.buildHoodieParquetReader( - sparkSession = sqlContext.sparkSession, - dataSchema = tableStructSchema, - partitionSchema = StructType(Nil), - requiredSchema = tableStructSchema, - filters = pushDownFilter, + val requiredSchemaParquetReader = createBaseFileReader( + spark = sqlContext.sparkSession, + partitionSchema = partitionSchema, + tableSchema = tableSchema, + requiredSchema = requiredSchema, + filters = filters ++ incrementalSpanRecordsFilters, options = optParams, - hadoopConf = sqlContext.sparkSession.sessionState.newHadoopConf() + // NOTE: We have to fork the Hadoop Config here as Spark will be modifying it + // to configure Parquet reader appropriately + hadoopConf = new Configuration(conf) ) + val hoodieTableState = HoodieMergeOnReadTableState(fileIndex, HoodieRecord.RECORD_KEY_METADATA_FIELD, preCombineFieldOpt) + + // TODO implement incremental span record filtering w/in RDD to make sure returned iterator is appropriately + // filtered, since file-reader might not be capable to perform filtering val rdd = new HoodieMergeOnReadRDD( sqlContext.sparkContext, jobConf, fullSchemaParquetReader, requiredSchemaParquetReader, - hoodieTableState + hoodieTableState, + tableSchema, + requiredSchema ) rdd.asInstanceOf[RDD[Row]] } @@ -206,10 +213,9 @@ class MergeOnReadIncrementalRelation(sqlContext: SQLContext, } val logPath = if (f.getLatestFileSlice.isPresent) { - //If log path doesn't exist, we still include an empty path to avoid using + // If log path doesn't exist, we still include an empty path to avoid using // the default parquet reader to ensure the push down filter will be applied. - Option(f.getLatestFileSlice.get().getLogFiles.iterator().toList - .map(logfile => logfile.getPath.toString)) + Option(f.getLatestFileSlice.get().getLogFiles.iterator().toList) } else { Option.empty @@ -219,4 +225,9 @@ class MergeOnReadIncrementalRelation(sqlContext: SQLContext, latestCommit, metaClient.getBasePath, maxCompactionMemoryInBytes, mergeType) }) } + + private def appendMandatoryColumns(requestedColumns: Array[String]): Array[String] = { + val missing = mandatoryColumns.filter(col => !requestedColumns.contains(col)) + requestedColumns ++ missing + } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala index 2e7dcf8d7..7c1a3540c 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala @@ -18,43 +18,37 @@ package org.apache.hudi -import org.apache.hudi.common.model.HoodieLogFile -import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapred.JobConf +import org.apache.hudi.HoodieBaseRelation.{createBaseFileReader, isMetadataTable} +import org.apache.hudi.common.model.{HoodieLogFile, HoodieRecord} +import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.table.view.HoodieTableFileSystemView import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes - -import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapred.JobConf - -import org.apache.spark.internal.Logging +import org.apache.hudi.metadata.HoodieMetadataPayload 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.expressions.Expression import org.apache.spark.sql.execution.datasources.{FileStatusCache, PartitionedFile} -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.hudi.HoodieSqlCommonUtils -import org.apache.spark.sql.{Row, SQLContext} -import org.apache.spark.sql.sources.{BaseRelation, Filter, PrunedFilteredScan} +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{Row, SQLContext} import scala.collection.JavaConverters._ case class HoodieMergeOnReadFileSplit(dataFile: Option[PartitionedFile], - logPaths: Option[List[String]], + logFiles: Option[List[HoodieLogFile]], latestCommit: String, tablePath: String, maxCompactionMemoryInBytes: Long, mergeType: String) -case class HoodieMergeOnReadTableState(tableStructSchema: StructType, - requiredStructSchema: StructType, - tableAvroSchema: String, - requiredAvroSchema: String, - hoodieRealtimeFileSplits: List[HoodieMergeOnReadFileSplit], - preCombineField: Option[String], - recordKeyFieldOpt: Option[String]) +case class HoodieMergeOnReadTableState(hoodieRealtimeFileSplits: List[HoodieMergeOnReadFileSplit], + recordKeyField: String, + preCombineFieldOpt: Option[String]) class MergeOnReadSnapshotRelation(sqlContext: SQLContext, optParams: Map[String, String], @@ -63,7 +57,7 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext, val metaClient: HoodieTableMetaClient) extends HoodieBaseRelation(sqlContext, metaClient, optParams, userSchema) { - private val conf = sqlContext.sparkContext.hadoopConfiguration + private val conf = new Configuration(sqlContext.sparkContext.hadoopConfiguration) private val jobConf = new JobConf(conf) private val mergeType = optParams.getOrElse( @@ -72,19 +66,21 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext, private val maxCompactionMemoryInBytes = getMaxCompactionMemoryInBytes(jobConf) - private val preCombineField = { - val preCombineFieldFromTableConfig = metaClient.getTableConfig.getPreCombineField - if (preCombineFieldFromTableConfig != null) { - Some(preCombineFieldFromTableConfig) - } else { - // get preCombineFiled from the options if this is a old table which have not store - // the field to hoodie.properties - optParams.get(DataSourceReadOptions.READ_PRE_COMBINE_FIELD.key) - } + // If meta fields are enabled, always prefer key from the meta field as opposed to user-specified one + // NOTE: This is historical behavior which is preserved as is + private val recordKeyField = { + if (metaClient.getTableConfig.populateMetaFields()) HoodieRecord.RECORD_KEY_METADATA_FIELD + else metaClient.getTableConfig.getRecordKeyFieldProp } - private var recordKeyFieldOpt = Option.empty[String] - if (!metaClient.getTableConfig.populateMetaFields()) { - recordKeyFieldOpt = Option(metaClient.getTableConfig.getRecordKeyFieldProp) + + private val preCombineFieldOpt = getPrecombineFieldProperty + + private lazy val mandatoryColumns = { + if (isMetadataTable(metaClient)) { + Seq(HoodieMetadataPayload.KEY_FIELD_NAME, HoodieMetadataPayload.SCHEMA_FIELD_NAME_TYPE) + } else { + Seq(recordKeyField) ++ preCombineFieldOpt.map(Seq(_)).getOrElse(Seq()) + } } override def needConversion: Boolean = false @@ -96,45 +92,56 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext, log.debug(s" buildScan requiredColumns = ${requiredColumns.mkString(",")}") log.debug(s" buildScan filters = ${filters.mkString(",")}") + // NOTE: In case list of requested columns doesn't contain the Primary Key one, we + // have to add it explicitly so that + // - Merging could be performed correctly + // - In case 0 columns are to be fetched (for ex, when doing {@code count()} on Spark's [[Dataset]], + // Spark still fetches all the rows to execute the query correctly + // + // It's okay to return columns that have not been requested by the caller, as those nevertheless will be + // filtered out upstream + val fetchedColumns: Array[String] = appendMandatoryColumns(requiredColumns) + val (requiredAvroSchema, requiredStructSchema) = - HoodieSparkUtils.getRequiredSchema(tableAvroSchema, requiredColumns) + HoodieSparkUtils.getRequiredSchema(tableAvroSchema, fetchedColumns) val fileIndex = buildFileIndex(filters) - val hoodieTableState = HoodieMergeOnReadTableState( - tableStructSchema, - requiredStructSchema, - tableAvroSchema.toString, - requiredAvroSchema.toString, - fileIndex, - preCombineField, - recordKeyFieldOpt - ) - val fullSchemaParquetReader = HoodieDataSourceHelper.buildHoodieParquetReader( - sparkSession = sqlContext.sparkSession, - dataSchema = tableStructSchema, - partitionSchema = StructType(Nil), - requiredSchema = tableStructSchema, + + val partitionSchema = StructType(Nil) + val tableSchema = HoodieTableSchema(tableStructSchema, tableAvroSchema.toString) + val requiredSchema = HoodieTableSchema(requiredStructSchema, requiredAvroSchema.toString) + + val fullSchemaParquetReader = createBaseFileReader( + spark = sqlContext.sparkSession, + partitionSchema = partitionSchema, + tableSchema = tableSchema, + requiredSchema = tableSchema, + // This file-reader is used to read base file records, subsequently merging them with the records + // stored in delta-log files. As such, we have to read _all_ records from the base file, while avoiding + // applying any filtering _before_ we complete combining them w/ delta-log records (to make sure that + // we combine them correctly) filters = Seq.empty, options = optParams, - hadoopConf = sqlContext.sparkSession.sessionState.newHadoopConf() + // NOTE: We have to fork the Hadoop Config here as Spark will be modifying it + // to configure Parquet reader appropriately + hadoopConf = new Configuration(conf) ) - - val requiredSchemaParquetReader = HoodieDataSourceHelper.buildHoodieParquetReader( - sparkSession = sqlContext.sparkSession, - dataSchema = tableStructSchema, - partitionSchema = StructType(Nil), - requiredSchema = tableStructSchema, + val requiredSchemaParquetReader = createBaseFileReader( + spark = sqlContext.sparkSession, + partitionSchema = partitionSchema, + tableSchema = tableSchema, + requiredSchema = requiredSchema, filters = filters, options = optParams, - hadoopConf = sqlContext.sparkSession.sessionState.newHadoopConf() + // NOTE: We have to fork the Hadoop Config here as Spark will be modifying it + // to configure Parquet reader appropriately + hadoopConf = new Configuration(conf) ) - val rdd = new HoodieMergeOnReadRDD( - sqlContext.sparkContext, - jobConf, - fullSchemaParquetReader, - requiredSchemaParquetReader, - hoodieTableState - ) + val tableState = HoodieMergeOnReadTableState(fileIndex, recordKeyField, preCombineFieldOpt) + + val rdd = new HoodieMergeOnReadRDD(sqlContext.sparkContext, jobConf, fullSchemaParquetReader, + requiredSchemaParquetReader, tableState, tableSchema, requiredSchema) + rdd.asInstanceOf[RDD[Row]] } @@ -214,8 +221,7 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext, Option.empty } - val logPaths = fileSlice.getLogFiles.sorted(HoodieLogFile.getLogFileComparator).iterator().asScala - .map(logFile => MergeOnReadSnapshotRelation.getFilePath(logFile.getPath)).toList + val logPaths = fileSlice.getLogFiles.sorted(HoodieLogFile.getLogFileComparator).iterator().asScala.toList val logPathsOptional = if (logPaths.isEmpty) Option.empty else Option(logPaths) HoodieMergeOnReadFileSplit(partitionedFile, logPathsOptional, queryInstant, metaClient.getBasePath, @@ -225,6 +231,11 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext, } } } + + private def appendMandatoryColumns(requestedColumns: Array[String]): Array[String] = { + val missing = mandatoryColumns.filter(col => !requestedColumns.contains(col)) + requestedColumns ++ missing + } } object MergeOnReadSnapshotRelation { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/HoodieAvroSerializer.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/HoodieAvroSerializer.scala index b464c2dc5..050efbd3d 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/HoodieAvroSerializer.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/HoodieAvroSerializer.scala @@ -20,9 +20,10 @@ package org.apache.spark.sql.avro import org.apache.avro.Schema import org.apache.spark.sql.types.DataType -/** - * As AvroSerializer cannot be access out of the spark.sql.avro package since spark 3.1, we define - * this class to be accessed by other class. - */ -case class HoodieAvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean) - extends AvroSerializer(rootCatalystType, rootAvroType, nullable) +class HoodieAvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean) + extends HoodieAvroSerializerTrait { + + val avroSerializer = new AvroSerializer(rootCatalystType, rootAvroType, nullable) + + override def serialize(catalystData: Any): Any = avroSerializer.serialize(catalystData) +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala index f14ccbe60..98823d142 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala @@ -25,17 +25,15 @@ import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} import org.apache.hudi.common.util.ValidationUtils import org.apache.hudi.keygen.ComplexKeyGenerator import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory - import org.apache.spark.internal.Logging -import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.avro.SchemaConverters import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.hudi.HoodieOptionConfig import org.apache.spark.sql.hudi.HoodieSqlCommonUtils._ import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.sql.{AnalysisException, SparkSession} import java.util.{Locale, Properties} - import scala.collection.JavaConverters._ import scala.collection.mutable diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala index a124575e0..d6745b679 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala @@ -42,17 +42,21 @@ trait ProvidesHoodieConfig extends Logging { val tableConfig = hoodieCatalogTable.tableConfig val tableId = hoodieCatalogTable.table.identifier + // NOTE: Here we fallback to "" to make sure that null value is not overridden with + // default value ("ts") + // TODO(HUDI-3456) clean up val preCombineField = Option(tableConfig.getPreCombineField).getOrElse("") + require(hoodieCatalogTable.primaryKeys.nonEmpty, s"There are no primary key in table ${hoodieCatalogTable.table.identifier}, cannot execute update operator") val enableHive = isEnableHive(sparkSession) withSparkConf(sparkSession, catalogProperties) { - Map( + Map.apply( "path" -> hoodieCatalogTable.tableLocation, RECORDKEY_FIELD.key -> hoodieCatalogTable.primaryKeys.mkString(","), - PRECOMBINE_FIELD.key -> preCombineField, TBL_NAME.key -> hoodieCatalogTable.tableName, + PRECOMBINE_FIELD.key -> preCombineField, HIVE_STYLE_PARTITIONING.key -> tableConfig.getHiveStylePartitioningEnable, URL_ENCODE_PARTITIONING.key -> tableConfig.getUrlEncodePartitioning, KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName, @@ -70,6 +74,7 @@ trait ProvidesHoodieConfig extends Logging { HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> "200", SqlKeyGenerator.PARTITION_SCHEMA -> hoodieCatalogTable.partitionSchema.toDDL ) + .filter { case(_, v) => v != null } } } @@ -98,8 +103,12 @@ trait ProvidesHoodieConfig extends Logging { val options = hoodieCatalogTable.catalogProperties ++ tableConfig.getProps.asScala.toMap ++ extraOptions val parameters = withSparkConf(sparkSession, options)() - val preCombineColumn = hoodieCatalogTable.preCombineKey.getOrElse("") - val partitionFields = hoodieCatalogTable.partitionFields.mkString(",") + val partitionFieldsStr = hoodieCatalogTable.partitionFields.mkString(",") + + // NOTE: Here we fallback to "" to make sure that null value is not overridden with + // default value ("ts") + // TODO(HUDI-3456) clean up + val preCombineField = hoodieCatalogTable.preCombineKey.getOrElse("") val hiveStylePartitioningEnable = Option(tableConfig.getHiveStylePartitioningEnable).getOrElse("true") val urlEncodePartitioning = Option(tableConfig.getUrlEncodePartitioning).getOrElse("false") @@ -115,7 +124,7 @@ trait ProvidesHoodieConfig extends Logging { DataSourceWriteOptions.SQL_INSERT_MODE.defaultValue())) val isNonStrictMode = insertMode == InsertMode.NON_STRICT val isPartitionedTable = hoodieCatalogTable.partitionFields.nonEmpty - val hasPrecombineColumn = preCombineColumn.nonEmpty + val hasPrecombineColumn = hoodieCatalogTable.preCombineKey.nonEmpty val operation = (enableBulkInsert, isOverwrite, dropDuplicate, isNonStrictMode, isPartitionedTable) match { case (true, _, _, false, _) => @@ -147,37 +156,41 @@ trait ProvidesHoodieConfig extends Logging { } else { classOf[OverwriteWithLatestAvroPayload].getCanonicalName } - logInfo(s"insert statement use write operation type: $operation, payloadClass: $payloadClassName") + + logInfo(s"Insert statement use write operation type: $operation, payloadClass: $payloadClassName") val enableHive = isEnableHive(sparkSession) + withSparkConf(sparkSession, options) { Map( "path" -> path, TABLE_TYPE.key -> tableType, TBL_NAME.key -> hoodieCatalogTable.tableName, - PRECOMBINE_FIELD.key -> preCombineColumn, OPERATION.key -> operation, HIVE_STYLE_PARTITIONING.key -> hiveStylePartitioningEnable, URL_ENCODE_PARTITIONING.key -> urlEncodePartitioning, KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName, SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME -> keyGeneratorClassName, RECORDKEY_FIELD.key -> hoodieCatalogTable.primaryKeys.mkString(","), - PARTITIONPATH_FIELD.key -> partitionFields, + PRECOMBINE_FIELD.key -> preCombineField, + PARTITIONPATH_FIELD.key -> partitionFieldsStr, PAYLOAD_CLASS_NAME.key -> payloadClassName, ENABLE_ROW_WRITER.key -> enableBulkInsert.toString, HoodieWriteConfig.COMBINE_BEFORE_INSERT.key -> String.valueOf(hasPrecombineColumn), + HIVE_PARTITION_FIELDS.key -> partitionFieldsStr, META_SYNC_ENABLED.key -> enableHive.toString, HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(), HIVE_USE_JDBC.key -> "false", HIVE_DATABASE.key -> hoodieCatalogTable.table.identifier.database.getOrElse("default"), HIVE_TABLE.key -> hoodieCatalogTable.table.identifier.table, HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true", - HIVE_PARTITION_FIELDS.key -> partitionFields, HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName, HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> "200", HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> "200", SqlKeyGenerator.PARTITION_SCHEMA -> hoodieCatalogTable.partitionSchema.toDDL ) + .filter { case (_, v) => v != null } } } + } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionCodeGen.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionCodeGen.scala index 509746bae..947291d10 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionCodeGen.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionCodeGen.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.hudi.command.payload -import java.util.UUID import org.apache.avro.generic.{GenericRecord, IndexedRecord} import org.apache.hudi.sql.IExpressionEvaluator import org.apache.spark.executor.InputMetrics @@ -37,6 +36,8 @@ import org.apache.spark.{TaskContext, TaskKilledException} import org.codehaus.commons.compiler.CompileException import org.codehaus.janino.{ClassBodyEvaluator, InternalCompilerException} +import java.util.UUID + /** * Do CodeGen for expression based on IndexedRecord. * The mainly difference with the spark's CodeGen for expression is that diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala index 0800d1712..e59a60932 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala @@ -17,14 +17,9 @@ package org.apache.spark.sql.hudi.command.payload -import java.util.{Base64, Properties} -import java.util.concurrent.Callable - 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 @@ -34,13 +29,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.avro.{AvroSerializer, 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 java.util.concurrent.Callable +import java.util.{Base64, Properties} import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer @@ -315,7 +311,7 @@ object ExpressionPayload { val conditionEvaluator = ExpressionCodeGen.doCodeGen(Seq(condition), conditionSerializer) val assignSqlType = AvroConversionUtils.convertAvroSchemaToStructType(writeSchema) - val assignSerializer = new HoodieAvroSerializer(assignSqlType, writeSchema, false) + val assignSerializer = new AvroSerializer(assignSqlType, writeSchema, false) val assignmentEvaluator = ExpressionCodeGen.doCodeGen(assignments, assignSerializer) conditionEvaluator -> assignmentEvaluator } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/SqlTypedRecord.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/SqlTypedRecord.scala index 749761443..29025877b 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/SqlTypedRecord.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/SqlTypedRecord.scala @@ -17,22 +17,19 @@ package org.apache.spark.sql.hudi.command.payload -import org.apache.avro.generic.IndexedRecord import org.apache.avro.Schema - -import org.apache.hudi.AvroConversionUtils - -import org.apache.spark.sql.avro.HoodieAvroDeserializer +import org.apache.avro.generic.IndexedRecord +import org.apache.hudi.{AvroConversionUtils, SparkAdapterSupport} import org.apache.spark.sql.catalyst.InternalRow /** * A sql typed record which will convert the avro field to sql typed value. */ -class SqlTypedRecord(val record: IndexedRecord) extends IndexedRecord { +class SqlTypedRecord(val record: IndexedRecord) extends IndexedRecord with SparkAdapterSupport { private lazy val sqlType = AvroConversionUtils.convertAvroSchemaToStructType(getSchema) - private lazy val avroDeserializer = HoodieAvroDeserializer(record.getSchema, sqlType) - private lazy val sqlRow = avroDeserializer.deserializeData(record).asInstanceOf[InternalRow] + private lazy val avroDeserializer = sparkAdapter.createAvroDeserializer(record.getSchema, sqlType) + private lazy val sqlRow = avroDeserializer.deserialize(record).get.asInstanceOf[InternalRow] override def put(i: Int, v: Any): Unit = { record.put(i, v) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala index 29386dc43..1d9aedd2a 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala @@ -443,13 +443,18 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Hoodie val partitionColumns = tableConfig.getPartitionFieldProp.split(",").map(_.toLowerCase) val partitionSchema = StructType(tableSchema.filter(f => partitionColumns.contains(f.name))) + // NOTE: Here we fallback to "" to make sure that null value is not overridden with + // default value ("ts") + // TODO(HUDI-3456) clean up + val preCombineField = hoodieCatalogTable.preCombineKey.getOrElse("") + // Enable the hive sync by default if spark have enable the hive metastore. val enableHive = isEnableHive(sparkSession) withSparkConf(sparkSession, hoodieCatalogTable.catalogProperties) { Map( "path" -> path, RECORDKEY_FIELD.key -> tableConfig.getRecordKeyFieldProp, - PRECOMBINE_FIELD.key -> hoodieCatalogTable.preCombineKey.getOrElse(""), + PRECOMBINE_FIELD.key -> preCombineField, TBL_NAME.key -> hoodieCatalogTable.tableName, PARTITIONPATH_FIELD.key -> tableConfig.getPartitionFieldProp, PAYLOAD_CLASS_NAME.key -> classOf[ExpressionPayload].getCanonicalName, @@ -470,6 +475,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Hoodie HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> "200", SqlKeyGenerator.PARTITION_SCHEMA -> partitionSchema.toDDL ) + .filter { case (_, v) => v != null } } } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java similarity index 99% rename from hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java rename to hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java index 87deef2a5..735277d95 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java @@ -18,13 +18,12 @@ package org.apache.hudi.keygen; +import org.apache.avro.generic.GenericRecord; import org.apache.hudi.AvroConversionUtils; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.exception.HoodieKeyException; - -import org.apache.avro.generic.GenericRecord; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.testutils.KeyGeneratorTestUtilities; import org.apache.spark.sql.Row; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java similarity index 99% rename from hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java rename to hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java index 4b590d937..26a2b439a 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java @@ -18,10 +18,9 @@ package org.apache.hudi.keygen; -import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.config.TypedProperties; - import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.keygen.constant.KeyGeneratorType; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteKeyGenerator.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteKeyGenerator.java similarity index 99% rename from hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteKeyGenerator.java rename to hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteKeyGenerator.java index aa9568b7a..a0d90e028 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteKeyGenerator.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteKeyGenerator.java @@ -18,11 +18,10 @@ package org.apache.hudi.keygen; +import org.apache.avro.generic.GenericRecord; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.exception.HoodieKeyException; - -import org.apache.avro.generic.GenericRecord; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.testutils.KeyGeneratorTestUtilities; import org.apache.spark.sql.Row; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestNonpartitionedKeyGenerator.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestNonpartitionedKeyGenerator.java similarity index 99% rename from hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestNonpartitionedKeyGenerator.java rename to hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestNonpartitionedKeyGenerator.java index 0760de112..297b07779 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestNonpartitionedKeyGenerator.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestNonpartitionedKeyGenerator.java @@ -18,12 +18,11 @@ package org.apache.hudi.keygen; +import org.apache.avro.generic.GenericRecord; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.exception.HoodieKeyException; - -import org.apache.avro.generic.GenericRecord; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.testutils.KeyGeneratorTestUtilities; import org.apache.spark.sql.Row; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java similarity index 99% rename from hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java rename to hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java index 0fc90c83a..7dea9e414 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java @@ -18,13 +18,12 @@ package org.apache.hudi.keygen; +import org.apache.avro.generic.GenericRecord; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.exception.HoodieKeyException; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.testutils.KeyGeneratorTestUtilities; - -import org.apache.avro.generic.GenericRecord; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; import org.junit.jupiter.api.Assertions; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java similarity index 80% rename from hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java rename to hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java index d6d0879cf..1fc4b9f1e 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java @@ -18,18 +18,18 @@ package org.apache.hudi.keygen; -import org.apache.hudi.AvroConversionHelper; +import org.apache.avro.Conversions; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericFixed; +import org.apache.avro.generic.GenericRecord; import org.apache.hudi.AvroConversionUtils; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.testutils.SchemaTestUtil; - -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; import org.apache.hudi.exception.HoodieKeyGeneratorException; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.testutils.KeyGeneratorTestUtilities; - import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema; @@ -37,12 +37,12 @@ import org.apache.spark.sql.types.StructType; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import scala.Function1; +import scala.Tuple2; import java.io.IOException; import java.math.BigDecimal; -import scala.Function1; - import static org.junit.jupiter.api.Assertions.assertEquals; public class TestTimestampBasedKeyGenerator { @@ -69,7 +69,21 @@ public class TestTimestampBasedKeyGenerator { properties.setProperty(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), "false"); } - private TypedProperties getBaseKeyConfig(String timestampType, String dateFormat, String timezone, String scalarType) { + private Row genericRecordToRow(GenericRecord baseRecord) { + Function1 convertor = AvroConversionUtils.createConverterToRow(baseRecord.getSchema(), structType); + Row row = convertor.apply(baseRecord); + int fieldCount = structType.fieldNames().length; + Object[] values = new Object[fieldCount]; + for (int i = 0; i < fieldCount; i++) { + values[i] = row.get(i); + } + return new GenericRowWithSchema(values, structType); + } + + private TypedProperties getBaseKeyConfig(String partitionPathField, String timestampType, String dateFormat, String timezone, String scalarType) { + TypedProperties properties = new TypedProperties(this.properties); + + properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), partitionPathField); properties.setProperty(KeyGeneratorOptions.Config.TIMESTAMP_TYPE_FIELD_PROP, timestampType); properties.setProperty(KeyGeneratorOptions.Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, dateFormat); properties.setProperty(KeyGeneratorOptions.Config.TIMESTAMP_TIMEZONE_FORMAT_PROP, timezone); @@ -81,18 +95,17 @@ public class TestTimestampBasedKeyGenerator { return properties; } - private Row genericRecordToRow(GenericRecord baseRecord) { - Function1 convertor = AvroConversionHelper.createConverterToRow(baseRecord.getSchema(), structType); - Row row = (Row) convertor.apply(baseRecord); - int fieldCount = structType.fieldNames().length; - Object[] values = new Object[fieldCount]; - for (int i = 0; i < fieldCount; i++) { - values[i] = row.get(i); - } - return new GenericRowWithSchema(values, structType); - } + private TypedProperties getBaseKeyConfig(String partitionPathField, + String timestampType, + String inputFormatList, + String inputFormatDelimiterRegex, + String inputTimezone, + String outputFormat, + String outputTimezone) { + TypedProperties properties = new TypedProperties(this.properties); + + properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), partitionPathField); - private TypedProperties getBaseKeyConfig(String timestampType, String inputFormatList, String inputFormatDelimiterRegex, String inputTimezone, String outputFormat, String outputTimezone) { if (timestampType != null) { properties.setProperty(KeyGeneratorOptions.Config.TIMESTAMP_TYPE_FIELD_PROP, timestampType); } @@ -118,7 +131,7 @@ public class TestTimestampBasedKeyGenerator { public void testTimestampBasedKeyGenerator() throws IOException { // timezone is GMT+8:00 baseRecord.put("createTime", 1578283932000L); - properties = getBaseKeyConfig("EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT+8:00", null); + properties = getBaseKeyConfig("createTime", "EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT+8:00", null); TimestampBasedKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties); HoodieKey hk1 = keyGen.getKey(baseRecord); assertEquals("2020-01-06 12", hk1.getPartitionPath()); @@ -128,68 +141,60 @@ public class TestTimestampBasedKeyGenerator { assertEquals("2020-01-06 12", keyGen.getPartitionPath(internalRow, baseRow.schema())); // timezone is GMT+8:00, createTime is BigDecimal - baseRecord.put("createTime", new BigDecimal(1578283932000.00001)); - properties = getBaseKeyConfig("EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT+8:00", null); + BigDecimal decimal = new BigDecimal("1578283932000.0001"); + Conversions.DecimalConversion conversion = new Conversions.DecimalConversion(); + Tuple2 resolvedNullableSchema = AvroConversionUtils.resolveAvroTypeNullability(schema.getField("createTimeDecimal").schema()); + GenericFixed avroDecimal = conversion.toFixed(decimal, resolvedNullableSchema._2, LogicalTypes.decimal(20, 4)); + baseRecord.put("createTimeDecimal", avroDecimal); + properties = getBaseKeyConfig("createTimeDecimal", "EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT+8:00", null); keyGen = new TimestampBasedKeyGenerator(properties); HoodieKey bigDecimalKey = keyGen.getKey(baseRecord); assertEquals("2020-01-06 12", bigDecimalKey.getPartitionPath()); - - // test w/ Row baseRow = genericRecordToRow(baseRecord); assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow)); // timezone is GMT - properties = getBaseKeyConfig("EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT", null); + properties = getBaseKeyConfig("createTime", "EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT", null); keyGen = new TimestampBasedKeyGenerator(properties); HoodieKey hk2 = keyGen.getKey(baseRecord); assertEquals("2020-01-06 04", hk2.getPartitionPath()); - - // test w/ Row assertEquals("2020-01-06 04", keyGen.getPartitionPath(baseRow)); // timestamp is DATE_STRING, timezone is GMT+8:00 - baseRecord.put("createTime", "2020-01-06 12:12:12"); - properties = getBaseKeyConfig("DATE_STRING", "yyyy-MM-dd hh", "GMT+8:00", null); + baseRecord.put("createTimeString", "2020-01-06 12:12:12"); + properties = getBaseKeyConfig("createTimeString", "DATE_STRING", "yyyy-MM-dd hh", "GMT+8:00", null); properties.setProperty("hoodie.deltastreamer.keygen.timebased.input.dateformat", "yyyy-MM-dd hh:mm:ss"); keyGen = new TimestampBasedKeyGenerator(properties); HoodieKey hk3 = keyGen.getKey(baseRecord); assertEquals("2020-01-06 12", hk3.getPartitionPath()); - - // test w/ Row baseRow = genericRecordToRow(baseRecord); assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow)); // timezone is GMT - properties = getBaseKeyConfig("DATE_STRING", "yyyy-MM-dd hh", "GMT", null); + properties = getBaseKeyConfig("createTimeString", "DATE_STRING", "yyyy-MM-dd hh", "GMT", null); keyGen = new TimestampBasedKeyGenerator(properties); HoodieKey hk4 = keyGen.getKey(baseRecord); assertEquals("2020-01-06 12", hk4.getPartitionPath()); - - // test w/ Row assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow)); // timezone is GMT+8:00, createTime is null baseRecord.put("createTime", null); - properties = getBaseKeyConfig("EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT+8:00", null); + properties = getBaseKeyConfig("createTime", "EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT+8:00", null); keyGen = new TimestampBasedKeyGenerator(properties); HoodieKey hk5 = keyGen.getKey(baseRecord); assertEquals("1970-01-01 08", hk5.getPartitionPath()); - - // test w/ Row baseRow = genericRecordToRow(baseRecord); assertEquals("1970-01-01 08", keyGen.getPartitionPath(baseRow)); internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow); assertEquals("1970-01-01 08", keyGen.getPartitionPath(internalRow, baseRow.schema())); // timestamp is DATE_STRING, timezone is GMT, createTime is null - baseRecord.put("createTime", null); - properties = getBaseKeyConfig("DATE_STRING", "yyyy-MM-dd hh:mm:ss", "GMT", null); + baseRecord.put("createTimeString", null); + properties = getBaseKeyConfig("createTime", "DATE_STRING", "yyyy-MM-dd hh:mm:ss", "GMT", null); properties.setProperty("hoodie.deltastreamer.keygen.timebased.input.dateformat", "yyyy-MM-dd hh:mm:ss"); keyGen = new TimestampBasedKeyGenerator(properties); HoodieKey hk6 = keyGen.getKey(baseRecord); assertEquals("1970-01-01 12:00:00", hk6.getPartitionPath()); - - // test w/ Row baseRow = genericRecordToRow(baseRecord); assertEquals("1970-01-01 12:00:00", keyGen.getPartitionPath(baseRow)); internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow); @@ -202,7 +207,7 @@ public class TestTimestampBasedKeyGenerator { baseRecord.put("createTime", 20000L); // timezone is GMT - properties = getBaseKeyConfig("SCALAR", "yyyy-MM-dd hh", "GMT", "days"); + properties = getBaseKeyConfig("createTime", "SCALAR", "yyyy-MM-dd hh", "GMT", "days"); TimestampBasedKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties); HoodieKey hk1 = keyGen.getKey(baseRecord); assertEquals(hk1.getPartitionPath(), "2024-10-04 12"); @@ -215,7 +220,7 @@ public class TestTimestampBasedKeyGenerator { // timezone is GMT, createTime is null baseRecord.put("createTime", null); - properties = getBaseKeyConfig("SCALAR", "yyyy-MM-dd hh", "GMT", "days"); + properties = getBaseKeyConfig("createTime", "SCALAR", "yyyy-MM-dd hh", "GMT", "days"); keyGen = new TimestampBasedKeyGenerator(properties); HoodieKey hk2 = keyGen.getKey(baseRecord); assertEquals("1970-01-02 12", hk2.getPartitionPath()); @@ -227,8 +232,8 @@ public class TestTimestampBasedKeyGenerator { assertEquals("1970-01-02 12", keyGen.getPartitionPath(internalRow, baseRow.schema())); // timezone is GMT. number of days store integer in mysql - baseRecord.put("createTime", 18736); - properties = getBaseKeyConfig("SCALAR", "yyyy-MM-dd", "GMT", "DAYS"); + baseRecord.put("createTime", 18736L); + properties = getBaseKeyConfig("createTime", "SCALAR", "yyyy-MM-dd", "GMT", "DAYS"); keyGen = new TimestampBasedKeyGenerator(properties); HoodieKey scalarSecondsKey = keyGen.getKey(baseRecord); assertEquals("2021-04-19", scalarSecondsKey.getPartitionPath()); @@ -245,7 +250,7 @@ public class TestTimestampBasedKeyGenerator { baseRecord = SchemaTestUtil.generateAvroRecordFromJson(schema, 1, "001", "f1"); baseRecord.put("createTime", 1638513806000000L); - properties = getBaseKeyConfig("SCALAR", "yyyy/MM/dd", "GMT", "MICROSECONDS"); + properties = getBaseKeyConfig("createTime", "SCALAR", "yyyy/MM/dd", "GMT", "MICROSECONDS"); properties.setProperty(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), "true"); TimestampBasedKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties); HoodieKey hk1 = keyGen.getKey(baseRecord); @@ -259,7 +264,7 @@ public class TestTimestampBasedKeyGenerator { // timezone is GMT, createTime is null baseRecord.put("createTime", null); - properties = getBaseKeyConfig("SCALAR", "yyyy/MM/dd", "GMT", "MICROSECONDS"); + properties = getBaseKeyConfig("createTime", "SCALAR", "yyyy/MM/dd", "GMT", "MICROSECONDS"); properties.setProperty(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), "true"); keyGen = new TimestampBasedKeyGenerator(properties); HoodieKey hk2 = keyGen.getKey(baseRecord); @@ -274,8 +279,9 @@ public class TestTimestampBasedKeyGenerator { @Test public void test_ExpectsMatch_SingleInputFormat_ISO8601WithMsZ_OutputTimezoneAsUTC() throws IOException { - baseRecord.put("createTime", "2020-04-01T13:01:33.428Z"); + baseRecord.put("createTimeString", "2020-04-01T13:01:33.428Z"); properties = this.getBaseKeyConfig( + "createTimeString", "DATE_STRING", "yyyy-MM-dd'T'HH:mm:ss.SSSZ", "", @@ -292,8 +298,9 @@ public class TestTimestampBasedKeyGenerator { @Test public void test_ExpectsMatch_SingleInputFormats_ISO8601WithMsZ_OutputTimezoneAsInputDateTimeZone() throws IOException { - baseRecord.put("createTime", "2020-04-01T13:01:33.428Z"); + baseRecord.put("createTimeString", "2020-04-01T13:01:33.428Z"); properties = this.getBaseKeyConfig( + "createTimeString", "DATE_STRING", "yyyy-MM-dd'T'HH:mm:ss.SSSZ", "", @@ -310,8 +317,9 @@ public class TestTimestampBasedKeyGenerator { @Test public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsZ_OutputTimezoneAsUTC() throws IOException { - baseRecord.put("createTime", "2020-04-01T13:01:33.428Z"); + baseRecord.put("createTimeString", "2020-04-01T13:01:33.428Z"); properties = this.getBaseKeyConfig( + "createTimeString", "DATE_STRING", "yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ", "", @@ -328,8 +336,9 @@ public class TestTimestampBasedKeyGenerator { @Test public void test_ExpectsMatch_MultipleInputFormats_ISO8601NoMsZ_OutputTimezoneAsUTC() throws IOException { - baseRecord.put("createTime", "2020-04-01T13:01:33Z"); + baseRecord.put("createTimeString", "2020-04-01T13:01:33Z"); properties = this.getBaseKeyConfig( + "createTimeString", "DATE_STRING", "yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ", "", @@ -346,8 +355,9 @@ public class TestTimestampBasedKeyGenerator { @Test public void test_ExpectsMatch_MultipleInputFormats_ISO8601NoMsWithOffset_OutputTimezoneAsUTC() throws IOException { - baseRecord.put("createTime", "2020-04-01T13:01:33-05:00"); + baseRecord.put("createTimeString", "2020-04-01T13:01:33-05:00"); properties = this.getBaseKeyConfig( + "createTimeString", "DATE_STRING", "yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ", "", @@ -364,8 +374,9 @@ public class TestTimestampBasedKeyGenerator { @Test public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsWithOffset_OutputTimezoneAsUTC() throws IOException { - baseRecord.put("createTime", "2020-04-01T13:01:33.123-05:00"); + baseRecord.put("createTimeString", "2020-04-01T13:01:33.123-05:00"); properties = this.getBaseKeyConfig( + "createTimeString", "DATE_STRING", "yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ", "", @@ -382,8 +393,9 @@ public class TestTimestampBasedKeyGenerator { @Test public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsZ_OutputTimezoneAsEST() throws IOException { - baseRecord.put("createTime", "2020-04-01T13:01:33.123Z"); + baseRecord.put("createTimeString", "2020-04-01T13:01:33.123Z"); properties = this.getBaseKeyConfig( + "createTimeString", "DATE_STRING", "yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ", "", @@ -400,8 +412,9 @@ public class TestTimestampBasedKeyGenerator { @Test public void test_Throws_MultipleInputFormats_InputDateNotMatchingFormats() throws IOException { - baseRecord.put("createTime", "2020-04-01 13:01:33.123-05:00"); + baseRecord.put("createTimeString", "2020-04-01 13:01:33.123-05:00"); properties = this.getBaseKeyConfig( + "createTimeString", "DATE_STRING", "yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ", "", @@ -417,8 +430,9 @@ public class TestTimestampBasedKeyGenerator { @Test public void test_ExpectsMatch_MultipleInputFormats_ShortDate_OutputCustomDate() throws IOException { - baseRecord.put("createTime", "20200401"); + baseRecord.put("createTimeString", "20200401"); properties = this.getBaseKeyConfig( + "createTimeString", "DATE_STRING", "yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ,yyyyMMdd", "", diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/factory/TestCreateKeyGeneratorByTypeWithFactory.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/factory/TestCreateKeyGeneratorByTypeWithFactory.java similarity index 100% rename from hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/factory/TestCreateKeyGeneratorByTypeWithFactory.java rename to hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/factory/TestCreateKeyGeneratorByTypeWithFactory.java diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/factory/TestHoodieSparkKeyGeneratorFactory.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/factory/TestHoodieSparkKeyGeneratorFactory.java similarity index 100% rename from hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/factory/TestHoodieSparkKeyGeneratorFactory.java rename to hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/factory/TestHoodieSparkKeyGeneratorFactory.java diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroConversionHelper.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroConversionHelper.scala index e29944529..686d09ccf 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroConversionHelper.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroConversionHelper.scala @@ -18,13 +18,13 @@ package org.apache.hudi -import java.time.LocalDate - import org.apache.avro.Schema import org.apache.avro.generic.GenericData import org.apache.spark.sql.catalyst.expressions.GenericRow import org.scalatest.{FunSuite, Matchers} +import java.time.LocalDate + class TestAvroConversionHelper extends FunSuite with Matchers { val dateSchema = s""" @@ -42,7 +42,7 @@ class TestAvroConversionHelper extends FunSuite with Matchers { test("Logical type: date") { val schema = new Schema.Parser().parse(dateSchema) - val convertor = AvroConversionHelper.createConverterToRow(schema, AvroConversionUtils.convertAvroSchemaToStructType(schema)) + val convertor = AvroConversionUtils.createConverterToRow(schema, AvroConversionUtils.convertAvroSchemaToStructType(schema)) val dateOutputData = dateInputData.map(x => { val record = new GenericData.Record(schema) {{ put("date", x) }} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala index d3be8c9b3..7fc7d318d 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala @@ -243,7 +243,7 @@ class TestDataSourceDefaults { val partitionPathProp: String = props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD.key) val STRUCT_NAME: String = "hoodieRowTopLevelField" val NAMESPACE: String = "hoodieRow" - var converterFn: Function1[Any, Any] = _ + var converterFn: Function1[Row, GenericRecord] = _ override def getKey(record: GenericRecord): HoodieKey = { new HoodieKey(HoodieAvroUtils.getNestedFieldValAsString(record, recordKeyProp, true, false), @@ -251,13 +251,13 @@ class TestDataSourceDefaults { } override def getRecordKey(row: Row): String = { - if (null == converterFn) converterFn = AvroConversionHelper.createConverterToAvro(row.schema, STRUCT_NAME, NAMESPACE) + if (null == converterFn) converterFn = AvroConversionUtils.createConverterToAvro(row.schema, STRUCT_NAME, NAMESPACE) val genericRecord = converterFn.apply(row).asInstanceOf[GenericRecord] getKey(genericRecord).getRecordKey } override def getPartitionPath(row: Row): String = { - if (null == converterFn) converterFn = AvroConversionHelper.createConverterToAvro(row.schema, STRUCT_NAME, NAMESPACE) + if (null == converterFn) converterFn = AvroConversionUtils.createConverterToAvro(row.schema, STRUCT_NAME, NAMESPACE) val genericRecord = converterFn.apply(row).asInstanceOf[GenericRecord] getKey(genericRecord).getPartitionPath } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala index beabfbe9b..b5186fb1a 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala @@ -32,13 +32,13 @@ import org.apache.hudi.functional.TestBootstrap import org.apache.hudi.hive.HiveSyncConfig import org.apache.hudi.keygen.{ComplexKeyGenerator, NonpartitionedKeyGenerator, SimpleKeyGenerator} import org.apache.hudi.testutils.DataSourceTestUtils -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql._ import org.apache.spark.sql.functions.{expr, lit} import org.apache.spark.sql.hudi.HoodieSparkSessionExtension import org.apache.spark.sql.hudi.command.SqlKeyGenerator import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} +import org.apache.spark.{SparkConf, SparkContext} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue, fail} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.params.ParameterizedTest @@ -827,33 +827,32 @@ class TestHoodieSparkSqlWriter { /** * Test case for non partition table with metatable support. */ - @Test - def testNonPartitionTableWithMetatableSupport(): Unit = { - List(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL).foreach { tableType => - val options = Map(DataSourceWriteOptions.TABLE_TYPE.key -> tableType, - DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "col3", - DataSourceWriteOptions.RECORDKEY_FIELD.key -> "keyid", - DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "", - DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.NonpartitionedKeyGenerator", - HoodieWriteConfig.TBL_NAME.key -> "hoodie_test", - "hoodie.insert.shuffle.parallelism" -> "1", - "hoodie.metadata.enable" -> "true") - val df = spark.range(0, 10).toDF("keyid") - .withColumn("col3", expr("keyid")) - .withColumn("age", expr("keyid + 1000")) - df.write.format("hudi") - .options(options.updated(DataSourceWriteOptions.OPERATION.key, "insert")) - .mode(SaveMode.Overwrite).save(tempBasePath) - // upsert same record again - val df_update = spark.range(0, 10).toDF("keyid") - .withColumn("col3", expr("keyid")) - .withColumn("age", expr("keyid + 2000")) - df_update.write.format("hudi") - .options(options.updated(DataSourceWriteOptions.OPERATION.key, "upsert")) - .mode(SaveMode.Append).save(tempBasePath) - assert(spark.read.format("hudi").load(tempBasePath).count() == 10) - assert(spark.read.format("hudi").load(tempBasePath).where("age >= 2000").count() == 10) - } + @ParameterizedTest + @EnumSource(value = classOf[HoodieTableType]) + def testNonPartitionTableWithMetatableSupport(tableType: HoodieTableType): Unit = { + val options = Map(DataSourceWriteOptions.TABLE_TYPE.key -> tableType.name, + DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "col3", + DataSourceWriteOptions.RECORDKEY_FIELD.key -> "keyid", + DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "", + DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.NonpartitionedKeyGenerator", + HoodieWriteConfig.TBL_NAME.key -> "hoodie_test", + "hoodie.insert.shuffle.parallelism" -> "1", + "hoodie.metadata.enable" -> "true") + val df = spark.range(0, 10).toDF("keyid") + .withColumn("col3", expr("keyid")) + .withColumn("age", expr("keyid + 1000")) + df.write.format("hudi") + .options(options.updated(DataSourceWriteOptions.OPERATION.key, "insert")) + .mode(SaveMode.Overwrite).save(tempBasePath) + // upsert same record again + val df_update = spark.range(0, 10).toDF("keyid") + .withColumn("col3", expr("keyid")) + .withColumn("age", expr("keyid + 2000")) + df_update.write.format("hudi") + .options(options.updated(DataSourceWriteOptions.OPERATION.key, "upsert")) + .mode(SaveMode.Append).save(tempBasePath) + assert(spark.read.format("hudi").load(tempBasePath).count() == 10) + assert(spark.read.format("hudi").load(tempBasePath).where("age >= 2000").count() == 10) } /** diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala index acab7dae0..9f00b5dcd 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala @@ -20,22 +20,18 @@ package org.apache.hudi import org.apache.avro.Schema import org.apache.avro.generic.GenericRecord - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path - +import org.apache.hudi.exception.SchemaCompatibilityException import org.apache.hudi.testutils.DataSourceTestUtils - import org.apache.spark.sql.types.{StructType, TimestampType} import org.apache.spark.sql.{Row, SparkSession} - import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test import org.junit.jupiter.api.io.TempDir import java.io.File import java.nio.file.Paths - import scala.collection.JavaConverters class TestHoodieSparkUtils { @@ -232,8 +228,9 @@ class TestHoodieSparkUtils { fail("createRdd should fail, because records don't have a column which is not nullable in the passed in schema") } catch { case e: Exception => - e.getCause.asInstanceOf[NullPointerException] - assertTrue(e.getMessage.contains("null of string in field new_nested_col of")) + val cause = e.getCause + assertTrue(cause.isInstanceOf[SchemaCompatibilityException]) + assertTrue(e.getMessage.contains("Unable to validate the rewritten record {\"innerKey\": \"innerKey1_2\", \"innerValue\": 2} against schema")) } spark.stop() } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala index 2ef59872a..ed6ef87b8 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala @@ -17,11 +17,11 @@ package org.apache.hudi.functional +import org.apache.avro.generic.GenericRecord import org.apache.hadoop.fs.Path - import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.common.config.HoodieMetadataConfig -import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieTableType} +import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieRecord, HoodieRecordPayload, HoodieTableType} import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.testutils.HoodieTestDataGenerator import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings @@ -30,18 +30,18 @@ import org.apache.hudi.index.HoodieIndex.IndexType import org.apache.hudi.keygen.NonpartitionedKeyGenerator import org.apache.hudi.keygen.constant.KeyGeneratorOptions.Config import org.apache.hudi.testutils.{DataSourceTestUtils, HoodieClientTestBase} -import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieSparkUtils} +import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieSparkUtils} import org.apache.log4j.LogManager - +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.BooleanType - import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.CsvSource +import java.util import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ @@ -349,11 +349,15 @@ class TestMORDataSource extends HoodieClientTestBase { // First Operation: // Producing parquet files to three default partitions. // SNAPSHOT view on MOR table with parquet files only. + + // Overriding the partition-path field + val opts = commonOpts + (DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition_path") + val hoodieRecords1 = dataGen.generateInserts("001", 100) - val records1 = recordsToStrings(hoodieRecords1).toList - val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) + + val inputDF1 = toDataset(hoodieRecords1) inputDF1.write.format("org.apache.hudi") - .options(commonOpts) + .options(opts) .option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) @@ -376,11 +380,10 @@ class TestMORDataSource extends HoodieClientTestBase { // Second Operation: // Upsert 50 update records // Snopshot view should read 100 records - val records2 = recordsToStrings(dataGen.generateUniqueUpdates("002", 50)) - .toList - val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2)) + val records2 = dataGen.generateUniqueUpdates("002", 50) + val inputDF2 = toDataset(records2) inputDF2.write.format("org.apache.hudi") - .options(commonOpts) + .options(opts) .mode(SaveMode.Append) .save(basePath) val hudiSnapshotDF2 = spark.read.format("org.apache.hudi") @@ -424,17 +427,31 @@ class TestMORDataSource extends HoodieClientTestBase { verifyShow(hudiIncDF2) verifyShow(hudiIncDF1Skipmerge) - val record3 = recordsToStrings(dataGen.generateUpdatesWithTS("003", hoodieRecords1, -1)) - spark.read.json(spark.sparkContext.parallelize(record3, 2)) - .write.format("org.apache.hudi").options(commonOpts) + val record3 = dataGen.generateUpdatesWithTS("003", hoodieRecords1, -1) + val inputDF3 = toDataset(record3) + inputDF3.write.format("org.apache.hudi").options(opts) .mode(SaveMode.Append).save(basePath) + val hudiSnapshotDF3 = spark.read.format("org.apache.hudi") .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .load(basePath + "/*/*/*/*") + + verifyShow(hudiSnapshotDF3); + assertEquals(100, hudiSnapshotDF3.count()) assertEquals(0, hudiSnapshotDF3.filter("rider = 'rider-003'").count()) } + private def toDataset(records: util.List[HoodieRecord[_]]) = { + val avroRecords = records.map(_.getData + .asInstanceOf[HoodieRecordPayload[_]] + .getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA) + .get + .asInstanceOf[GenericRecord]) + val rdd: RDD[GenericRecord] = spark.sparkContext.parallelize(avroRecords, 2) + AvroConversionUtils.createDataFrame(rdd, HoodieTestDataGenerator.AVRO_SCHEMA.toString, spark) + } + @Test def testVectorizedReader() { spark.conf.set("spark.sql.parquet.enableVectorizedReader", true) @@ -553,15 +570,10 @@ class TestMORDataSource extends HoodieClientTestBase { .orderBy(desc("_hoodie_commit_time")) .head() assertEquals(sampleRow.getDouble(0), sampleRow.get(0)) - assertEquals(sampleRow.getLong(1), sampleRow.get(1)) + assertEquals(sampleRow.getDate(1), sampleRow.get(1)) assertEquals(sampleRow.getString(2), sampleRow.get(2)) assertEquals(sampleRow.getSeq(3), sampleRow.get(3)) - if (HoodieSparkUtils.gteqSpark3_2) { - // Since Spark3.2, the `nation` column is parsed as String, not Struct. - assertEquals(sampleRow.getString(4), sampleRow.get(4)) - } else { - assertEquals(sampleRow.getStruct(4), sampleRow.get(4)) - } + assertEquals(sampleRow.getAs[Array[Byte]](4), sampleRow.get(4)) } def verifyShow(df: DataFrame): Unit = { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataTableWithSparkDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataTableWithSparkDataSource.scala new file mode 100644 index 000000000..918202e97 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataTableWithSparkDataSource.scala @@ -0,0 +1,98 @@ +/* + * 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.config.HoodieMetadataConfig +import org.apache.hudi.common.testutils.HoodieTestDataGenerator +import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings +import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.testutils.SparkClientFunctionalTestHarness +import org.apache.spark.sql.SaveMode +import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.{Tag, Test} + +import scala.collection.JavaConverters._ + +@Tag("functional") +class TestMetadataTableWithSparkDataSource extends SparkClientFunctionalTestHarness { + + val hudi = "org.apache.hudi" + var commonOpts = Map( + "hoodie.insert.shuffle.parallelism" -> "4", + "hoodie.upsert.shuffle.parallelism" -> "4", + "hoodie.bulkinsert.shuffle.parallelism" -> "2", + "hoodie.delete.shuffle.parallelism" -> "1", + DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key", + DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition", + DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp", + HoodieWriteConfig.TBL_NAME.key -> "hoodie_test" + ) + + @Test + def testReadability(): Unit = { + val dataGen = new HoodieTestDataGenerator() + + val opts: Map[String, String] = commonOpts ++ Map( + HoodieMetadataConfig.ENABLE.key -> "true", + HoodieMetadataConfig.COMPACT_NUM_DELTA_COMMITS.key -> "1" + ) + + // Insert records + val newRecords = dataGen.generateInserts("001", 100) + val newRecordsDF = parseRecords(recordsToStrings(newRecords).asScala) + + newRecordsDF.write.format(hudi) + .options(opts) + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .mode(SaveMode.Append) + .save(basePath) + + // Update records + val updatedRecords = dataGen.generateUpdates("002", newRecords) + val updatedRecordsDF = parseRecords(recordsToStrings(updatedRecords).asScala) + + updatedRecordsDF.write.format(hudi) + .options(opts) + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) + .mode(SaveMode.Append) + .save(basePath) + + val metadataDF = spark.read.format(hudi).load(s"$basePath/.hoodie/metadata") + + // Smoke test + metadataDF.show() + + // Query w/ 0 requested columns should be working fine + assertEquals(4, metadataDF.count()) + + val expectedKeys = Seq("2015/03/16", "2015/03/17", "2016/03/15", "__all_partitions__") + val keys = metadataDF.select("key") + .collect() + .map(_.getString(0)) + .toSeq + .sorted + + assertEquals(expectedKeys, keys) + } + + private def parseRecords(records: Seq[String]) = { + spark.read.json(spark.sparkContext.parallelize(records, 2)) + } +} diff --git a/hudi-spark-datasource/hudi-spark2/pom.xml b/hudi-spark-datasource/hudi-spark2/pom.xml index 228360354..3fb6cf3dd 100644 --- a/hudi-spark-datasource/hudi-spark2/pom.xml +++ b/hudi-spark-datasource/hudi-spark2/pom.xml @@ -199,6 +199,15 @@ org.apache.spark spark-sql_${scala.binary.version} ${spark2.version} + provided + true + + + + org.apache.spark + spark-avro_${scala.binary.version} + ${spark2.version} + provided true diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala index 94d1c80fc..5dfa7d957 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala @@ -17,9 +17,10 @@ package org.apache.spark.sql.adapter +import org.apache.avro.Schema import org.apache.hudi.Spark2RowSerDe import org.apache.hudi.client.utils.SparkRowSerDe - +import org.apache.spark.sql.avro.{HoodieAvroDeserializerTrait, HoodieAvroSerializerTrait, Spark2HoodieAvroDeserializer, HoodieAvroSerializer} import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.{Expression, Like} @@ -31,6 +32,7 @@ import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFil import org.apache.spark.sql.hudi.SparkAdapter import org.apache.spark.sql.hudi.parser.HoodieSpark2ExtendedSqlParser import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.DataType import org.apache.spark.sql.{Row, SparkSession} import scala.collection.mutable.ArrayBuffer @@ -40,6 +42,12 @@ import scala.collection.mutable.ArrayBuffer */ class Spark2Adapter extends SparkAdapter { + def createAvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean): HoodieAvroSerializerTrait = + new HoodieAvroSerializer(rootCatalystType, rootAvroType, nullable) + + def createAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType): HoodieAvroDeserializerTrait = + new Spark2HoodieAvroDeserializer(rootAvroType, rootCatalystType) + override def createSparkRowSerDe(encoder: ExpressionEncoder[Row]): SparkRowSerDe = { new Spark2RowSerDe(encoder) } diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/avro/PatchedAvroDeserializer.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/avro/PatchedAvroDeserializer.scala new file mode 100644 index 000000000..8d9948c58 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/avro/PatchedAvroDeserializer.scala @@ -0,0 +1,398 @@ +/* + * 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.spark.sql.avro + +import org.apache.avro.Conversions.DecimalConversion +import org.apache.avro.LogicalTypes.{TimestampMicros, TimestampMillis} +import org.apache.avro.Schema.Type._ +import org.apache.avro.generic._ +import org.apache.avro.util.Utf8 +import org.apache.avro.{LogicalTypes, Schema, SchemaBuilder} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{SpecificInternalRow, UnsafeArrayData} +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, DateTimeUtils, GenericArrayData} +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +import java.math.BigDecimal +import java.nio.ByteBuffer +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +/** + * A deserializer to deserialize data in avro format to data in catalyst format. + * + * NOTE: This is a version of {@code AvroDeserializer} impl from Spark 2.4.4 w/ the fix for SPARK-30267 + * applied on top of it + */ +class PatchedAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) { + private lazy val decimalConversions = new DecimalConversion() + + private val converter: Any => Any = rootCatalystType match { + // A shortcut for empty schema. + case st: StructType if st.isEmpty => + (data: Any) => InternalRow.empty + + case st: StructType => + val resultRow = new SpecificInternalRow(st.map(_.dataType)) + val fieldUpdater = new RowUpdater(resultRow) + val writer = getRecordWriter(rootAvroType, st, Nil) + (data: Any) => { + val record = data.asInstanceOf[GenericRecord] + writer(fieldUpdater, record) + resultRow + } + + case _ => + val tmpRow = new SpecificInternalRow(Seq(rootCatalystType)) + val fieldUpdater = new RowUpdater(tmpRow) + val writer = newWriter(rootAvroType, rootCatalystType, Nil) + (data: Any) => { + writer(fieldUpdater, 0, data) + tmpRow.get(0, rootCatalystType) + } + } + + def deserialize(data: Any): Any = converter(data) + + /** + * Creates a writer to write avro values to Catalyst values at the given ordinal with the given + * updater. + */ + private def newWriter( + avroType: Schema, + catalystType: DataType, + path: List[String]): (CatalystDataUpdater, Int, Any) => Unit = + (avroType.getType, catalystType) match { + case (NULL, NullType) => (updater, ordinal, _) => + updater.setNullAt(ordinal) + + // TODO: we can avoid boxing if future version of avro provide primitive accessors. + case (BOOLEAN, BooleanType) => (updater, ordinal, value) => + updater.setBoolean(ordinal, value.asInstanceOf[Boolean]) + + case (INT, IntegerType) => (updater, ordinal, value) => + updater.setInt(ordinal, value.asInstanceOf[Int]) + + case (INT, DateType) => (updater, ordinal, value) => + updater.setInt(ordinal, value.asInstanceOf[Int]) + + case (LONG, LongType) => (updater, ordinal, value) => + updater.setLong(ordinal, value.asInstanceOf[Long]) + + case (LONG, TimestampType) => avroType.getLogicalType match { + case _: TimestampMillis => (updater, ordinal, value) => + updater.setLong(ordinal, value.asInstanceOf[Long] * 1000) + case _: TimestampMicros => (updater, ordinal, value) => + updater.setLong(ordinal, value.asInstanceOf[Long]) + case null => (updater, ordinal, value) => + // For backward compatibility, if the Avro type is Long and it is not logical type, + // the value is processed as timestamp type with millisecond precision. + updater.setLong(ordinal, value.asInstanceOf[Long] * 1000) + case other => throw new IncompatibleSchemaException( + s"Cannot convert Avro logical type ${other} to Catalyst Timestamp type.") + } + + // Before we upgrade Avro to 1.8 for logical type support, spark-avro converts Long to Date. + // For backward compatibility, we still keep this conversion. + case (LONG, DateType) => (updater, ordinal, value) => + updater.setInt(ordinal, (value.asInstanceOf[Long] / DateTimeUtils.MILLIS_PER_DAY).toInt) + + case (FLOAT, FloatType) => (updater, ordinal, value) => + updater.setFloat(ordinal, value.asInstanceOf[Float]) + + case (DOUBLE, DoubleType) => (updater, ordinal, value) => + updater.setDouble(ordinal, value.asInstanceOf[Double]) + + case (STRING, StringType) => (updater, ordinal, value) => + val str = value match { + case s: String => UTF8String.fromString(s) + case s: Utf8 => + val bytes = new Array[Byte](s.getByteLength) + System.arraycopy(s.getBytes, 0, bytes, 0, s.getByteLength) + UTF8String.fromBytes(bytes) + } + updater.set(ordinal, str) + + case (ENUM, StringType) => (updater, ordinal, value) => + updater.set(ordinal, UTF8String.fromString(value.toString)) + + case (FIXED, BinaryType) => (updater, ordinal, value) => + updater.set(ordinal, value.asInstanceOf[GenericFixed].bytes().clone()) + + case (BYTES, BinaryType) => (updater, ordinal, value) => + val bytes = value match { + case b: ByteBuffer => + val bytes = new Array[Byte](b.remaining) + b.get(bytes) + bytes + case b: Array[Byte] => b + case other => throw new RuntimeException(s"$other is not a valid avro binary.") + } + updater.set(ordinal, bytes) + + case (FIXED, d: DecimalType) => (updater, ordinal, value) => + val bigDecimal = decimalConversions.fromFixed(value.asInstanceOf[GenericFixed], avroType, + LogicalTypes.decimal(d.precision, d.scale)) + val decimal = createDecimal(bigDecimal, d.precision, d.scale) + updater.setDecimal(ordinal, decimal) + + case (BYTES, d: DecimalType) => (updater, ordinal, value) => + val bigDecimal = decimalConversions.fromBytes(value.asInstanceOf[ByteBuffer], avroType, + LogicalTypes.decimal(d.precision, d.scale)) + val decimal = createDecimal(bigDecimal, d.precision, d.scale) + updater.setDecimal(ordinal, decimal) + + case (RECORD, st: StructType) => + val writeRecord = getRecordWriter(avroType, st, path) + (updater, ordinal, value) => + val row = new SpecificInternalRow(st) + writeRecord(new RowUpdater(row), value.asInstanceOf[GenericRecord]) + updater.set(ordinal, row) + + case (ARRAY, ArrayType(elementType, containsNull)) => + val elementWriter = newWriter(avroType.getElementType, elementType, path) + val elementPath = path :+ "element" + (updater, ordinal, value) => + val collection = value.asInstanceOf[java.util.Collection[Any]] + val len = collection.size() + val result = createArrayData(elementType, len) + val elementUpdater = new ArrayDataUpdater(result) + + var i = 0 + val iter = collection.iterator() + while (iter.hasNext) { + val element = iter.next() + if (element == null) { + if (!containsNull) { + throw new RuntimeException( + s"Array value at path '${elementPath.mkString(".")}' is not allowed to be null") + } else { + elementUpdater.setNullAt(i) + } + } else { + elementWriter(elementUpdater, i, element) + } + i += 1 + } + + updater.set(ordinal, result) + + case (MAP, MapType(keyType, valueType, valueContainsNull)) if keyType == StringType => + val keyWriter = newWriter(SchemaBuilder.builder().stringType(), StringType, path) + val valueWriter = newWriter(avroType.getValueType, valueType, path) + (updater, ordinal, value) => + val map = value.asInstanceOf[java.util.Map[AnyRef, AnyRef]] + val keyArray = createArrayData(keyType, map.size()) + val keyUpdater = new ArrayDataUpdater(keyArray) + val valueArray = createArrayData(valueType, map.size()) + val valueUpdater = new ArrayDataUpdater(valueArray) + val iter = map.entrySet().iterator() + var i = 0 + while (iter.hasNext) { + val entry = iter.next() + assert(entry.getKey != null) + keyWriter(keyUpdater, i, entry.getKey) + if (entry.getValue == null) { + if (!valueContainsNull) { + throw new RuntimeException(s"Map value at path ${path.mkString(".")} is not " + + "allowed to be null") + } else { + valueUpdater.setNullAt(i) + } + } else { + valueWriter(valueUpdater, i, entry.getValue) + } + i += 1 + } + + updater.set(ordinal, new ArrayBasedMapData(keyArray, valueArray)) + + case (UNION, _) => + val allTypes = avroType.getTypes.asScala + val nonNullTypes = allTypes.filter(_.getType != NULL) + val nonNullAvroType = Schema.createUnion(nonNullTypes.asJava) + if (nonNullTypes.nonEmpty) { + if (nonNullTypes.length == 1) { + newWriter(nonNullTypes.head, catalystType, path) + } else { + nonNullTypes.map(_.getType) match { + case Seq(a, b) if Set(a, b) == Set(INT, LONG) && catalystType == LongType => + (updater, ordinal, value) => value match { + case null => updater.setNullAt(ordinal) + case l: java.lang.Long => updater.setLong(ordinal, l) + case i: java.lang.Integer => updater.setLong(ordinal, i.longValue()) + } + + case Seq(a, b) if Set(a, b) == Set(FLOAT, DOUBLE) && catalystType == DoubleType => + (updater, ordinal, value) => value match { + case null => updater.setNullAt(ordinal) + case d: java.lang.Double => updater.setDouble(ordinal, d) + case f: java.lang.Float => updater.setDouble(ordinal, f.doubleValue()) + } + + case _ => + catalystType match { + case st: StructType if st.length == nonNullTypes.size => + val fieldWriters = nonNullTypes.zip(st.fields).map { + case (schema, field) => newWriter(schema, field.dataType, path :+ field.name) + }.toArray + (updater, ordinal, value) => { + val row = new SpecificInternalRow(st) + val fieldUpdater = new RowUpdater(row) + val i = GenericData.get().resolveUnion(nonNullAvroType, value) + fieldWriters(i)(fieldUpdater, i, value) + updater.set(ordinal, row) + } + + case _ => + throw new IncompatibleSchemaException( + s"Cannot convert Avro to catalyst because schema at path " + + s"${path.mkString(".")} is not compatible " + + s"(avroType = $avroType, sqlType = $catalystType).\n" + + s"Source Avro schema: $rootAvroType.\n" + + s"Target Catalyst type: $rootCatalystType") + } + } + } + } else { + (updater, ordinal, value) => updater.setNullAt(ordinal) + } + + case _ => + throw new IncompatibleSchemaException( + s"Cannot convert Avro to catalyst because schema at path ${path.mkString(".")} " + + s"is not compatible (avroType = $avroType, sqlType = $catalystType).\n" + + s"Source Avro schema: $rootAvroType.\n" + + s"Target Catalyst type: $rootCatalystType") + } + + // TODO: move the following method in Decimal object on creating Decimal from BigDecimal? + private def createDecimal(decimal: BigDecimal, precision: Int, scale: Int): Decimal = { + if (precision <= Decimal.MAX_LONG_DIGITS) { + // Constructs a `Decimal` with an unscaled `Long` value if possible. + Decimal(decimal.unscaledValue().longValue(), precision, scale) + } else { + // Otherwise, resorts to an unscaled `BigInteger` instead. + Decimal(decimal, precision, scale) + } + } + + private def getRecordWriter( + avroType: Schema, + sqlType: StructType, + path: List[String]): (CatalystDataUpdater, GenericRecord) => Unit = { + val validFieldIndexes = ArrayBuffer.empty[Int] + val fieldWriters = ArrayBuffer.empty[(CatalystDataUpdater, Any) => Unit] + + val length = sqlType.length + var i = 0 + while (i < length) { + val sqlField = sqlType.fields(i) + val avroField = avroType.getField(sqlField.name) + if (avroField != null) { + validFieldIndexes += avroField.pos() + + val baseWriter = newWriter(avroField.schema(), sqlField.dataType, path :+ sqlField.name) + val ordinal = i + val fieldWriter = (fieldUpdater: CatalystDataUpdater, value: Any) => { + if (value == null) { + fieldUpdater.setNullAt(ordinal) + } else { + baseWriter(fieldUpdater, ordinal, value) + } + } + fieldWriters += fieldWriter + } else if (!sqlField.nullable) { + throw new IncompatibleSchemaException( + s""" + |Cannot find non-nullable field ${path.mkString(".")}.${sqlField.name} in Avro schema. + |Source Avro schema: $rootAvroType. + |Target Catalyst type: $rootCatalystType. + """.stripMargin) + } + i += 1 + } + + (fieldUpdater, record) => { + var i = 0 + while (i < validFieldIndexes.length) { + fieldWriters(i)(fieldUpdater, record.get(validFieldIndexes(i))) + i += 1 + } + } + } + + private def createArrayData(elementType: DataType, length: Int): ArrayData = elementType match { + case BooleanType => UnsafeArrayData.fromPrimitiveArray(new Array[Boolean](length)) + case ByteType => UnsafeArrayData.fromPrimitiveArray(new Array[Byte](length)) + case ShortType => UnsafeArrayData.fromPrimitiveArray(new Array[Short](length)) + case IntegerType => UnsafeArrayData.fromPrimitiveArray(new Array[Int](length)) + case LongType => UnsafeArrayData.fromPrimitiveArray(new Array[Long](length)) + case FloatType => UnsafeArrayData.fromPrimitiveArray(new Array[Float](length)) + case DoubleType => UnsafeArrayData.fromPrimitiveArray(new Array[Double](length)) + case _ => new GenericArrayData(new Array[Any](length)) + } + + /** + * A base interface for updating values inside catalyst data structure like `InternalRow` and + * `ArrayData`. + */ + sealed trait CatalystDataUpdater { + def set(ordinal: Int, value: Any): Unit + + def setNullAt(ordinal: Int): Unit = set(ordinal, null) + def setBoolean(ordinal: Int, value: Boolean): Unit = set(ordinal, value) + def setByte(ordinal: Int, value: Byte): Unit = set(ordinal, value) + def setShort(ordinal: Int, value: Short): Unit = set(ordinal, value) + def setInt(ordinal: Int, value: Int): Unit = set(ordinal, value) + def setLong(ordinal: Int, value: Long): Unit = set(ordinal, value) + def setDouble(ordinal: Int, value: Double): Unit = set(ordinal, value) + def setFloat(ordinal: Int, value: Float): Unit = set(ordinal, value) + def setDecimal(ordinal: Int, value: Decimal): Unit = set(ordinal, value) + } + + final class RowUpdater(row: InternalRow) extends CatalystDataUpdater { + override def set(ordinal: Int, value: Any): Unit = row.update(ordinal, value) + + override def setNullAt(ordinal: Int): Unit = row.setNullAt(ordinal) + override def setBoolean(ordinal: Int, value: Boolean): Unit = row.setBoolean(ordinal, value) + override def setByte(ordinal: Int, value: Byte): Unit = row.setByte(ordinal, value) + override def setShort(ordinal: Int, value: Short): Unit = row.setShort(ordinal, value) + override def setInt(ordinal: Int, value: Int): Unit = row.setInt(ordinal, value) + override def setLong(ordinal: Int, value: Long): Unit = row.setLong(ordinal, value) + override def setDouble(ordinal: Int, value: Double): Unit = row.setDouble(ordinal, value) + override def setFloat(ordinal: Int, value: Float): Unit = row.setFloat(ordinal, value) + override def setDecimal(ordinal: Int, value: Decimal): Unit = + row.setDecimal(ordinal, value, value.precision) + } + + final class ArrayDataUpdater(array: ArrayData) extends CatalystDataUpdater { + override def set(ordinal: Int, value: Any): Unit = array.update(ordinal, value) + + override def setNullAt(ordinal: Int): Unit = array.setNullAt(ordinal) + override def setBoolean(ordinal: Int, value: Boolean): Unit = array.setBoolean(ordinal, value) + override def setByte(ordinal: Int, value: Byte): Unit = array.setByte(ordinal, value) + override def setShort(ordinal: Int, value: Short): Unit = array.setShort(ordinal, value) + override def setInt(ordinal: Int, value: Int): Unit = array.setInt(ordinal, value) + override def setLong(ordinal: Int, value: Long): Unit = array.setLong(ordinal, value) + override def setDouble(ordinal: Int, value: Double): Unit = array.setDouble(ordinal, value) + override def setFloat(ordinal: Int, value: Float): Unit = array.setFloat(ordinal, value) + override def setDecimal(ordinal: Int, value: Decimal): Unit = array.update(ordinal, value) + } +} diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/avro/Spark2HoodieAvroDeserializer.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/avro/Spark2HoodieAvroDeserializer.scala new file mode 100644 index 000000000..ac2c82f70 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/avro/Spark2HoodieAvroDeserializer.scala @@ -0,0 +1,33 @@ +/* + * 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.spark.sql.avro + +import org.apache.avro.Schema +import org.apache.spark.sql.types.DataType + +/** + * This is Spark 2 implementation for the [[HoodieAvroDeserializerTrait]] leveraging [[PatchedAvroDeserializer]], + * which is just copied over version of [[AvroDeserializer]] from Spark 2.4.4 w/ SPARK-30267 being back-ported to it + */ +class Spark2HoodieAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) + extends HoodieAvroDeserializerTrait { + + private val avroDeserializer = new PatchedAvroDeserializer(rootAvroType, rootCatalystType) + + def doDeserialize(data: Any): Any = avroDeserializer.deserialize(data) +} diff --git a/hudi-spark-datasource/hudi-spark3-common/pom.xml b/hudi-spark-datasource/hudi-spark3-common/pom.xml index 8fd46c7b3..30e7bda2e 100644 --- a/hudi-spark-datasource/hudi-spark3-common/pom.xml +++ b/hudi-spark-datasource/hudi-spark3-common/pom.xml @@ -168,6 +168,15 @@ org.apache.spark spark-sql_2.12 ${spark3.version} + provided + true + + + + org.apache.spark + spark-avro_2.12 + ${spark3.version} + provided true diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/Spark3Adapter.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/Spark3Adapter.scala index a1c41e80a..8f073bb1c 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/Spark3Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/Spark3Adapter.scala @@ -17,10 +17,11 @@ package org.apache.spark.sql.adapter +import org.apache.avro.Schema import org.apache.hudi.Spark3RowSerDe import org.apache.hudi.client.utils.SparkRowSerDe import org.apache.hudi.spark3.internal.ReflectUtil -import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.avro.{HoodieAvroDeserializerTrait, HoodieAvroSerializerTrait, Spark3HoodieAvroDeserializer, HoodieAvroSerializer} import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.{Expression, Like} @@ -30,18 +31,24 @@ import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoStatement, Join, J import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.connector.catalog.Table +import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.execution.datasources.{FilePartition, LogicalRelation, PartitionedFile, Spark3ParsePartitionUtil, SparkParsePartitionUtil} import org.apache.spark.sql.hudi.SparkAdapter import org.apache.spark.sql.internal.SQLConf - -import scala.collection.JavaConverters.mapAsScalaMapConverter +import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.{Row, SparkSession} /** * The adapter for spark3. */ class Spark3Adapter extends SparkAdapter { + def createAvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean): HoodieAvroSerializerTrait = + new HoodieAvroSerializer(rootCatalystType, rootAvroType, nullable) + + def createAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType): HoodieAvroDeserializerTrait = + new Spark3HoodieAvroDeserializer(rootAvroType, rootCatalystType) + override def createSparkRowSerDe(encoder: ExpressionEncoder[Row]): SparkRowSerDe = { new Spark3RowSerDe(encoder) } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/HoodieAvroDeserializer.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/avro/Spark3HoodieAvroDeserializer.scala similarity index 67% rename from hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/HoodieAvroDeserializer.scala rename to hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/avro/Spark3HoodieAvroDeserializer.scala index 1678dc05d..fa03f5d84 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/HoodieAvroDeserializer.scala +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/avro/Spark3HoodieAvroDeserializer.scala @@ -18,20 +18,15 @@ package org.apache.spark.sql.avro import org.apache.avro.Schema - import org.apache.hudi.HoodieSparkUtils - import org.apache.spark.sql.types.DataType -/** - * This is to be compatible with the type returned by Spark 3.1 - * and other spark versions for AvroDeserializer - */ -case class HoodieAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) { +class Spark3HoodieAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) + extends HoodieAvroDeserializerTrait { + // SPARK-34404: As of Spark3.2, there is no AvroDeserializer's constructor with Schema and DataType arguments. + // So use the reflection to get AvroDeserializer instance. private val avroDeserializer = if (HoodieSparkUtils.isSpark3_2) { - // SPARK-34404: As of Spark3.2, there is no AvroDeserializer's constructor with Schema and DataType arguments. - // So use the reflection to get AvroDeserializer instance. val constructor = classOf[AvroDeserializer].getConstructor(classOf[Schema], classOf[DataType], classOf[String]) constructor.newInstance(rootAvroType, rootCatalystType, "EXCEPTION") } else { @@ -39,10 +34,5 @@ case class HoodieAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataTy constructor.newInstance(rootAvroType, rootCatalystType) } - def deserializeData(data: Any): Any = { - avroDeserializer.deserialize(data) match { - case Some(r) => r // As of spark 3.1, this will return data wrapped with Option, so we fetch the data. - case o => o // for other spark version, return the data directly. - } - } + def doDeserialize(data: Any): Any = avroDeserializer.deserialize(data) } diff --git a/hudi-spark-datasource/hudi-spark3/pom.xml b/hudi-spark-datasource/hudi-spark3/pom.xml index d8dba8384..722a1b410 100644 --- a/hudi-spark-datasource/hudi-spark3/pom.xml +++ b/hudi-spark-datasource/hudi-spark3/pom.xml @@ -158,6 +158,7 @@ org.apache.spark spark-sql_2.12 ${spark3.version} + provided true diff --git a/pom.xml b/pom.xml index a0f7c2c6e..1b28ae1bb 100644 --- a/pom.xml +++ b/pom.xml @@ -349,6 +349,7 @@ 3 @{argLine} + false ${surefire-log4j.file}