1
0

[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)
This commit is contained in:
Alexey Kudinkin
2022-02-24 13:23:13 -08:00
committed by GitHub
parent 521338b4d9
commit 85e8a5c4de
56 changed files with 1634 additions and 1010 deletions

View File

@@ -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<Object, Object> converterFn = null;
private transient Function1<Row, GenericRecord> 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();
}
/**

View File

@@ -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);
}

View File

@@ -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
}
}
}
}
}

View File

@@ -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))

View File

@@ -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 = {

View File

@@ -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
}

View File

@@ -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
}

View File

@@ -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.
*/