Skip to content

Commit

Permalink
Fix the geoparquet footer reader.
Browse files Browse the repository at this point in the history
  • Loading branch information
Imbruced committed Jan 16, 2025
1 parent 2f96f87 commit 289c9e7
Showing 1 changed file with 4 additions and 4 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,9 @@
package org.apache.spark.sql.execution.datasources.v2.geoparquet.metadata

import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.parquet.ParquetReadOptions
import org.apache.parquet.format.converter.ParquetMetadataConverter
import org.apache.parquet.hadoop.ParquetFileReader
import org.apache.parquet.hadoop.util.HadoopInputFile
import org.apache.spark.broadcast.Broadcast
Expand Down Expand Up @@ -66,13 +68,11 @@ object GeoParquetMetadataPartitionReaderFactory {
configuration: Configuration,
partitionedFile: PartitionedFile,
readDataSchema: StructType): Iterator[InternalRow] = {
val inputFile = HadoopInputFile.fromPath(partitionedFile.toPath, configuration)
val inputStream = inputFile.newStream()
val filePath = partitionedFile.filePath

val footer = ParquetFileReader
.readFooter(inputFile, ParquetReadOptions.builder().build(), inputStream)
.readFooter(configuration, new Path(filePath), ParquetMetadataConverter.NO_FILTER)

val filePath = partitionedFile.toPath.toString
val metadata = footer.getFileMetaData.getKeyValueMetaData
val row = GeoParquetMetaData.parseKeyValueMetaData(metadata) match {
case Some(geo) =>
Expand Down

0 comments on commit 289c9e7

Please sign in to comment.