[HUDI-2114] Spark Query MOR Table Written By Flink Return Incorrect Timestamp Value (#3208)
This commit is contained in:
@@ -111,15 +111,9 @@ class MergeOnReadIncrementalRelation(val sqlContext: SQLContext,
|
||||
val lessThanFilter = LessThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.last.getTimestamp)
|
||||
filters :+isNotNullFilter :+ largerThanFilter :+ lessThanFilter
|
||||
}
|
||||
var requiredStructSchema = StructType(Seq())
|
||||
requiredColumns.foreach(col => {
|
||||
val field = tableStructSchema.find(_.name == col)
|
||||
if (field.isDefined) {
|
||||
requiredStructSchema = requiredStructSchema.add(field.get)
|
||||
}
|
||||
})
|
||||
val requiredAvroSchema = AvroConversionUtils
|
||||
.convertStructTypeToAvroSchema(requiredStructSchema, tableAvroSchema.getName, tableAvroSchema.getNamespace)
|
||||
val (requiredAvroSchema, requiredStructSchema) =
|
||||
MergeOnReadSnapshotRelation.getRequiredSchema(tableAvroSchema, requiredColumns)
|
||||
|
||||
val hoodieTableState = HoodieMergeOnReadTableState(
|
||||
tableStructSchema,
|
||||
requiredStructSchema,
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi
|
||||
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.hudi.common.model.HoodieBaseFile
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView
|
||||
@@ -94,16 +95,9 @@ class MergeOnReadSnapshotRelation(val sqlContext: SQLContext,
|
||||
override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = {
|
||||
log.debug(s" buildScan requiredColumns = ${requiredColumns.mkString(",")}")
|
||||
log.debug(s" buildScan filters = ${filters.mkString(",")}")
|
||||
var requiredStructSchema = StructType(Seq())
|
||||
requiredColumns.foreach(col => {
|
||||
val field = tableStructSchema.find(_.name == col)
|
||||
if (field.isDefined) {
|
||||
requiredStructSchema = requiredStructSchema.add(field.get)
|
||||
}
|
||||
})
|
||||
val requiredAvroSchema = AvroConversionUtils
|
||||
.convertStructTypeToAvroSchema(requiredStructSchema, tableAvroSchema.getName, tableAvroSchema.getNamespace)
|
||||
|
||||
val (requiredAvroSchema, requiredStructSchema) =
|
||||
MergeOnReadSnapshotRelation.getRequiredSchema(tableAvroSchema, requiredColumns)
|
||||
val fileIndex = buildFileIndex(filters)
|
||||
val hoodieTableState = HoodieMergeOnReadTableState(
|
||||
tableStructSchema,
|
||||
@@ -211,4 +205,15 @@ object MergeOnReadSnapshotRelation {
|
||||
// when create PartitionedFile.
|
||||
path.toUri.toString
|
||||
}
|
||||
|
||||
def getRequiredSchema(tableAvroSchema: Schema, requiredColumns: Array[String]): (Schema, StructType) = {
|
||||
// First get the required avro-schema, then convert the avro-schema to spark schema.
|
||||
val name2Fields = tableAvroSchema.getFields.asScala.map(f => f.name() -> f).toMap
|
||||
val requiredFields = requiredColumns.map(c => name2Fields(c))
|
||||
.map(f => new Schema.Field(f.name(), f.schema(), f.doc(), f.defaultVal(), f.order())).toList
|
||||
val requiredAvroSchema = Schema.createRecord(tableAvroSchema.getName, tableAvroSchema.getDoc,
|
||||
tableAvroSchema.getNamespace, tableAvroSchema.isError, requiredFields.asJava)
|
||||
val requiredStructSchema = AvroConversionUtils.convertAvroSchemaToStructType(requiredAvroSchema)
|
||||
(requiredAvroSchema, requiredStructSchema)
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user