This is an automated email from the ASF dual-hosted git repository.

jiayu pushed a commit to branch worktree-issue-2824
in repository https://gitbox.apache.org/repos/asf/sedona.git

commit 6539d1635b3b02e42a7504841d7f2de011ae8af2
Author: Jia Yu <[email protected]>
AuthorDate: Tue Apr 7 03:36:23 2026 -0700

    [GH-2824] Add schema-aware column pruning to sedonainfo extractors
    
    Pass requiredFields from Spark's readDataSchema to extractors so they
    can skip expensive work (bands, overviews, metadata, compression, CRS
    WKT) when those columns are not selected in the query.
---
 .../io/sedonainfo/GeoTiffMetadataExtractor.scala   | 35 ++++++++++++++--------
 .../io/sedonainfo/NetCdfMetadataExtractor.scala    |  3 +-
 .../sedonainfo/RasterFileMetadataExtractor.scala   | 10 ++++++-
 .../io/sedonainfo/SedonaInfoPartitionReader.scala  |  3 +-
 4 files changed, 36 insertions(+), 15 deletions(-)

diff --git 
a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/sedonainfo/GeoTiffMetadataExtractor.scala
 
b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/sedonainfo/GeoTiffMetadataExtractor.scala
index 2edf6e2b58..5eee085023 100644
--- 
a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/sedonainfo/GeoTiffMetadataExtractor.scala
+++ 
b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/sedonainfo/GeoTiffMetadataExtractor.scala
@@ -47,7 +47,8 @@ object GeoTiffMetadataExtractor extends 
RasterFileMetadataExtractor {
   override def extract(
       path: Path,
       fileSize: Long,
-      configuration: Configuration): RasterFileMetadata = {
+      configuration: Configuration,
+      requiredFields: Set[String] = Set.empty): RasterFileMetadata = {
     val imageStream = new HadoopImageInputStream(path, configuration)
     var reader: GeoTiffReader = null
     var raster: GridCoverage2D = null
@@ -58,11 +59,16 @@ object GeoTiffMetadataExtractor extends 
RasterFileMetadataExtractor {
           org.geotools.util.factory.Hints.FORCE_LONGITUDE_FIRST_AXIS_ORDER,
           java.lang.Boolean.TRUE))
 
+      val needAll = requiredFields.isEmpty
+      def need(field: String): Boolean = needAll || 
requiredFields.contains(field)
+
       // Extract TIFF IIO metadata BEFORE read() which may alter stream state
-      val isTiled = hasTiffTag(reader, TAG_TILE_WIDTH)
-      val photometric = extractPhotometricInterpretation(reader)
-      val tiffMetadata = extractMetadata(reader)
-      val compression = extractCompression(reader)
+      val isTiled = if (need("isTiled")) hasTiffTag(reader, TAG_TILE_WIDTH) 
else false
+      val photometric =
+        if (need("bands")) extractPhotometricInterpretation(reader) else -1
+      val tiffMetadata =
+        if (need("metadata")) extractMetadata(reader) else Map.empty[String, 
String]
+      val compression = if (need("compression")) extractCompression(reader) 
else null
 
       raster = reader.read(null)
 
@@ -71,11 +77,13 @@ object GeoTiffMetadataExtractor extends 
RasterFileMetadataExtractor {
       val numBands = RasterAccessors.numBands(raster)
       val srid = RasterAccessors.srid(raster)
 
-      val crsStr = Try {
-        val crs = raster.getCoordinateReferenceSystem
-        if (crs == null || crs.isInstanceOf[DefaultEngineeringCRS]) null
-        else crs.toWKT
-      }.getOrElse(null)
+      val crsStr = if (need("crs")) {
+        Try {
+          val crs = raster.getCoordinateReferenceSystem
+          if (crs == null || crs.isInstanceOf[DefaultEngineeringCRS]) null
+          else crs.toWKT
+        }.getOrElse(null)
+      } else null
 
       val affine = RasterUtils.getGDALAffineTransform(raster)
       val env = raster.getEnvelope2D
@@ -84,8 +92,11 @@ object GeoTiffMetadataExtractor extends 
RasterFileMetadataExtractor {
       val tileWidth = image.getTileWidth
       val tileHeight = image.getTileHeight
 
-      val bands = extractBands(raster, numBands, tileWidth, tileHeight, 
photometric)
-      val overviews = extractOverviews(reader, width, height)
+      val bands =
+        if (need("bands")) extractBands(raster, numBands, tileWidth, 
tileHeight, photometric)
+        else Seq.empty
+      val overviews =
+        if (need("overviews")) extractOverviews(reader, width, height) else 
Seq.empty
 
       RasterFileMetadata(
         path = path.toString,
diff --git 
a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/sedonainfo/NetCdfMetadataExtractor.scala
 
b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/sedonainfo/NetCdfMetadataExtractor.scala
index b65106f939..e27618b07b 100644
--- 
a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/sedonainfo/NetCdfMetadataExtractor.scala
+++ 
b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/sedonainfo/NetCdfMetadataExtractor.scala
@@ -41,7 +41,8 @@ object NetCdfMetadataExtractor extends 
RasterFileMetadataExtractor {
   override def extract(
       path: Path,
       fileSize: Long,
-      configuration: Configuration): RasterFileMetadata = {
+      configuration: Configuration,
+      requiredFields: Set[String] = Set.empty): RasterFileMetadata = {
     // Read file bytes via Hadoop FS, then open in memory
     val fs = path.getFileSystem(configuration)
     val status = fs.getFileStatus(path)
diff --git 
a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/sedonainfo/RasterFileMetadataExtractor.scala
 
b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/sedonainfo/RasterFileMetadataExtractor.scala
index 6319e35144..a9a0ae2cb9 100644
--- 
a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/sedonainfo/RasterFileMetadataExtractor.scala
+++ 
b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/sedonainfo/RasterFileMetadataExtractor.scala
@@ -33,8 +33,16 @@ trait RasterFileMetadataExtractor {
   /**
    * Extract metadata from the file at the given path. Implementations must 
not decode pixel data
    * — only headers/metadata should be read.
+   *
+   * @param requiredFields
+   *   Column names requested by Spark's column pruning. Extractors may skip 
expensive work for
+   *   fields not in this set. When empty, all fields are extracted.
    */
-  def extract(path: Path, fileSize: Long, configuration: Configuration): 
RasterFileMetadata
+  def extract(
+      path: Path,
+      fileSize: Long,
+      configuration: Configuration,
+      requiredFields: Set[String] = Set.empty): RasterFileMetadata
 
   /** Returns true if this extractor can handle the given file path (by 
extension). */
   def canHandle(path: Path): Boolean
diff --git 
a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/sedonainfo/SedonaInfoPartitionReader.scala
 
b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/sedonainfo/SedonaInfoPartitionReader.scala
index 68a59685ef..7588623a54 100644
--- 
a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/sedonainfo/SedonaInfoPartitionReader.scala
+++ 
b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/sedonainfo/SedonaInfoPartitionReader.scala
@@ -61,7 +61,8 @@ class SedonaInfoPartitionReader(
   private def readFileMetadata(partition: PartitionedFile): InternalRow = {
     val path = new Path(new URI(partition.filePath.toString()))
     val extractor = SedonaInfoPartitionReader.findExtractor(path)
-    val meta = extractor.extract(path, partition.fileSize, configuration)
+    val requiredFields = readDataSchema.fieldNames.toSet
+    val meta = extractor.extract(path, partition.fileSize, configuration, 
requiredFields)
     SedonaInfoPartitionReader.toInternalRow(meta, readDataSchema)
   }
 }

Reply via email to