1
0

[HUDI-3130] Fixing Hive getSchema for RT tables addressing different partitions having different schemas (#4468)

* Fixing Hive getSchema for RT tables

* Addressing feedback

* temp diff

* fixing tests after spark datasource read support for metadata table is merged to master

* Adding multi-partition schema evolution tests to HoodieRealTimeRecordReader

Co-authored-by: Aditya Tiwari <aditya.tiwari@flipkart.com>
Co-authored-by: sivabalan <n.siva.b@gmail.com>
This commit is contained in:
Aditya Tiwari
2022-03-06 07:51:35 +05:30
committed by GitHub
parent 6a46130037
commit 051ad0b033
9 changed files with 174 additions and 56 deletions

View File

@@ -19,12 +19,13 @@
package org.apache.hudi
import org.apache.avro.Schema
import org.apache.avro.generic.{GenericRecord, GenericRecordBuilder}
import org.apache.avro.generic.{GenericRecord, GenericRecordBuilder, IndexedRecord}
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.avro.HoodieAvroUtils
import org.apache.hudi.common.config.HoodieMetadataConfig
import org.apache.hudi.common.engine.HoodieLocalEngineContext
import org.apache.hudi.common.fs.FSUtils
@@ -309,10 +310,15 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext,
}
}
private def mergeRowWithLog(curRow: InternalRow, curKey: String) = {
private def mergeRowWithLog(curRow: InternalRow, curKey: String) : org.apache.hudi.common.util.Option[IndexedRecord] = {
val historyAvroRecord = serializer.serialize(curRow).asInstanceOf[GenericRecord]
logRecords.get(curKey).getData
val mergedRec = logRecords.get(curKey).getData
.combineAndGetUpdateValue(historyAvroRecord, tableAvroSchema, payloadProps)
if (mergedRec.isPresent && mergedRec.get().getSchema != tableAvroSchema) {
org.apache.hudi.common.util.Option.of(HoodieAvroUtils.rewriteRecord(mergedRec.get().asInstanceOf[GenericRecord], tableAvroSchema).asInstanceOf[IndexedRecord])
} else {
mergedRec
}
}
}
}