[HUDI-1129] Improving schema evolution support in hudi (#2927)
* Adding support to ingest records with old schema after table's schema is evolved * Rebasing against latest master - Trimming test file to be < 800 lines - Renaming config names * Addressing feedback Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
committed by
GitHub
parent
73d898322b
commit
1196736185
@@ -357,7 +357,7 @@ object AvroConversionHelper {
|
||||
val fieldNamesIterator = dataType.asInstanceOf[StructType].fieldNames.iterator
|
||||
val rowIterator = item.asInstanceOf[Row].toSeq.iterator
|
||||
|
||||
while (convertersIterator.hasNext) {
|
||||
while (convertersIterator.hasNext && rowIterator.hasNext) {
|
||||
val converter = convertersIterator.next()
|
||||
record.put(fieldNamesIterator.next(), converter(rowIterator.next()))
|
||||
}
|
||||
|
||||
@@ -92,22 +92,43 @@ object HoodieSparkUtils extends SparkAdapterSupport {
|
||||
new InMemoryFileIndex(sparkSession, globbedPaths, Map(), Option.empty, fileStatusCache)
|
||||
}
|
||||
|
||||
def createRdd(df: DataFrame, structName: String, recordNamespace: String): RDD[GenericRecord] = {
|
||||
val avroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, recordNamespace)
|
||||
createRdd(df, avroSchema, structName, recordNamespace)
|
||||
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)
|
||||
}
|
||||
|
||||
def createRdd(df: DataFrame, avroSchema: Schema, structName: String, recordNamespace: String)
|
||||
def createRddInternal(df: DataFrame, writeSchema: Schema, latestTableSchema: Schema, structName: String, recordNamespace: String)
|
||||
: RDD[GenericRecord] = {
|
||||
// Use the Avro schema to derive the StructType which has the correct nullability information
|
||||
val dataType = SchemaConverters.toSqlType(avroSchema).dataType.asInstanceOf[StructType]
|
||||
val encoder = RowEncoder.apply(dataType).resolveAndBind()
|
||||
// Use the write avro schema to derive the StructType which has the correct nullability information
|
||||
val writeDataType = SchemaConverters.toSqlType(writeSchema).dataType.asInstanceOf[StructType]
|
||||
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) SchemaConverters.toSqlType(latestTableSchema).dataType.asInstanceOf[StructType] 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(dataType, structName, recordNamespace)
|
||||
val convertor = AvroConversionHelper.createConverterToAvro(reconciledDataType, structName, recordNamespace)
|
||||
records.map { x => convertor(x).asInstanceOf[GenericRecord] }
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user