jonvex commented on code in PR #11770:
URL: https://github.com/apache/hudi/pull/11770#discussion_r1715655134
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala:
##########
@@ -251,4 +259,20 @@ class
HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState,
override def close(): Unit = closeableFileGroupRecordIterator.close()
}
}
+
+ private def readBaseFile(file: PartitionedFile, parquetFileReader:
SparkParquetReader, requestedSchema: StructType,
+ remainingPartitionSchema: StructType,
fixedPartitionIndexes: Set[Int], requiredSchema: StructType,
+ partitionSchema: StructType, outputSchema:
StructType, filters: Seq[Filter],
+ storageConf: StorageConfiguration[Configuration]):
Iterator[InternalRow] = {
+ if (requestedSchema.equals(requiredSchema)) {
+ parquetFileReader.read(file, requiredSchema, partitionSchema,
internalSchemaOpt, filters, storageConf)
+ } else {
+ val partitionValues =
InternalRow.fromSeq(file.partitionValues.toSeq(partitionSchema).zipWithIndex.filter(p
=> fixedPartitionIndexes.contains(p._2)).map(p => p._1))
+ val modifiedFile = PartitionedFile(partitionValues, file.filePath,
file.start, file.length, file.locations, file.modificationTime, file.fileSize,
file.otherConstantMetadataColumnValues)
+ val iter = parquetFileReader.read(modifiedFile, requestedSchema,
remainingPartitionSchema, internalSchemaOpt, filters, storageConf)
+ val unsafeProjection =
generateUnsafeProjection(StructType(requestedSchema.fields ++
remainingPartitionSchema.fields), outputSchema)
+ iter.map(row => unsafeProjection(row))
Review Comment:
need to do a projection here. The case in mind is that partition schema is
"a,b,c" mandatoryFields is "a,c",
then we will read (dataSchema + a + c) and append b. So the final schema
will be (data schema + a + c +b)
but expected output is (data schema + a + b +c)
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]