Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SEDONA-699] Fix issue with not closing parquet files. #1749

Merged
merged 3 commits into from
Jan 17, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@ 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 @@ -67,11 +69,11 @@ object GeoParquetMetadataPartitionReaderFactory {
partitionedFile: PartitionedFile,
readDataSchema: StructType): Iterator[InternalRow] = {
val filePath = partitionedFile.filePath
val metadata = ParquetFileReader
.open(HadoopInputFile.fromPath(new Path(filePath), configuration))
.getFooter
.getFileMetaData
.getKeyValueMetaData

val footer = ParquetFileReader
.readFooter(configuration, new Path(filePath), ParquetMetadataConverter.NO_FILTER)

val metadata = footer.getFileMetaData.getKeyValueMetaData
val row = GeoParquetMetaData.parseKeyValueMetaData(metadata) match {
case Some(geo) =>
val geoColumnsMap = geo.columns.map { case (columnName, columnMetadata) =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.spark.sql.execution.datasources.v2.geoparquet.metadata

import org.apache.hadoop.conf.Configuration
import org.apache.parquet.ParquetReadOptions
import org.apache.parquet.hadoop.ParquetFileReader
import org.apache.parquet.hadoop.util.HadoopInputFile
import org.apache.spark.broadcast.Broadcast
Expand Down Expand Up @@ -66,12 +67,14 @@ object GeoParquetMetadataPartitionReaderFactory {
configuration: Configuration,
partitionedFile: PartitionedFile,
readDataSchema: StructType): Iterator[InternalRow] = {
val inputFile = HadoopInputFile.fromPath(partitionedFile.toPath, configuration)
val inputStream = inputFile.newStream()

val footer = ParquetFileReader
.readFooter(inputFile, ParquetReadOptions.builder().build(), inputStream)

val filePath = partitionedFile.toPath.toString
val metadata = ParquetFileReader
.open(HadoopInputFile.fromPath(partitionedFile.toPath, configuration))
.getFooter
.getFileMetaData
.getKeyValueMetaData
val metadata = footer.getFileMetaData.getKeyValueMetaData
val row = GeoParquetMetaData.parseKeyValueMetaData(metadata) match {
case Some(geo) =>
val geoColumnsMap = geo.columns.map { case (columnName, columnMetadata) =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.spark.sql.execution.datasources.v2.geoparquet.metadata

import org.apache.hadoop.conf.Configuration
import org.apache.parquet.ParquetReadOptions
import org.apache.parquet.hadoop.ParquetFileReader
import org.apache.parquet.hadoop.util.HadoopInputFile
import org.apache.spark.broadcast.Broadcast
Expand Down Expand Up @@ -66,12 +67,15 @@ object GeoParquetMetadataPartitionReaderFactory {
configuration: Configuration,
partitionedFile: PartitionedFile,
readDataSchema: StructType): Iterator[InternalRow] = {

val inputFile = HadoopInputFile.fromPath(partitionedFile.toPath, configuration)
val inputStream = inputFile.newStream()

val footer = ParquetFileReader
.readFooter(inputFile, ParquetReadOptions.builder().build(), inputStream)

val filePath = partitionedFile.toPath.toString
val metadata = ParquetFileReader
.open(HadoopInputFile.fromPath(partitionedFile.toPath, configuration))
.getFooter
.getFileMetaData
.getKeyValueMetaData
val metadata = footer.getFileMetaData.getKeyValueMetaData
val row = GeoParquetMetaData.parseKeyValueMetaData(metadata) match {
case Some(geo) =>
val geoColumnsMap = geo.columns.map { case (columnName, columnMetadata) =>
Expand Down
Loading