Copilot commented on code in PR #2846:
URL: https://github.com/apache/sedona/pull/2846#discussion_r3125590969


##########
spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/geotiffmetadata/GeoTiffMetadataPartitionReader.scala:
##########
@@ -0,0 +1,454 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.sedona_sql.io.geotiffmetadata
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.sedona.common.raster.RasterAccessors
+import org.apache.sedona.common.raster.RasterBandAccessors
+import org.apache.sedona.common.raster.inputstream.HadoopImageInputStream
+import org.apache.sedona.common.utils.RasterUtils
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow
+import org.apache.spark.sql.catalyst.util.GenericArrayData
+import org.apache.spark.sql.connector.read.PartitionReader
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.unsafe.types.UTF8String
+import org.geotools.coverage.grid.GridCoverage2D
+import org.geotools.gce.geotiff.GeoTiffReader
+import org.geotools.referencing.crs.DefaultEngineeringCRS
+
+import java.net.URI
+import scala.collection.mutable
+import scala.util.Try
+
+class GeoTiffMetadataPartitionReader(
+    configuration: Configuration,
+    partitionedFiles: Array[PartitionedFile],
+    readDataSchema: StructType)
+    extends PartitionReader[InternalRow] {
+
+  private var currentFileIndex = 0
+  private var currentRow: InternalRow = _
+
+  override def next(): Boolean = {
+    if (currentFileIndex < partitionedFiles.length) {
+      currentRow = readFileMetadata(partitionedFiles(currentFileIndex))
+      currentFileIndex += 1
+      true
+    } else {
+      false
+    }
+  }
+
+  override def get(): InternalRow = currentRow
+
+  override def close(): Unit = {}
+
+  // Fields requiring reader.read() to build a GridCoverage2D
+  private val COVERAGE_FIELDS = Set(
+    "width",
+    "height",
+    "numBands",
+    "srid",
+    "crs",
+    "geoTransform",
+    "cornerCoordinates",
+    "bands")
+
+  private def readFileMetadata(partition: PartitionedFile): InternalRow = {
+    val requested = readDataSchema.fieldNames.toSet
+    val needCoverage = requested.exists(COVERAGE_FIELDS.contains)
+    val needIsTiled = requested.contains("isTiled")
+    val needBands = requested.contains("bands")
+    val needTiffMetadata = requested.contains("metadata")
+    val needCompression = requested.contains("compression")
+    val needReader =
+      needCoverage || needIsTiled || needBands || needTiffMetadata || 
needCompression ||
+        requested.contains("overviews")
+
+    val path = new Path(new URI(partition.filePath.toString()))
+
+    // Skip all I/O if only cheap fields (path, driver, fileSize) are requested
+    if (!needReader) {
+      return buildRow(path, partition, null, null, false, -1, Map.empty, null)
+    }
+
+    val imageStream = new HadoopImageInputStream(path, configuration)
+    var reader: GeoTiffReader = null
+    var raster: GridCoverage2D = null
+    try {
+      reader = new GeoTiffReader(
+        imageStream,
+        new org.geotools.util.factory.Hints(
+          org.geotools.util.factory.Hints.FORCE_LONGITUDE_FIRST_AXIS_ORDER,
+          java.lang.Boolean.TRUE))
+
+      // Extract TIFF IIO metadata BEFORE read() to avoid stream state issues.
+      // Only extract fields actually requested.
+      val isTiled =
+        if (needIsTiled) GeoTiffMetadataPartitionReader.hasTiffTag(reader, 
322) else false
+      val photometric =
+        if (needBands) 
GeoTiffMetadataPartitionReader.extractPhotometricInterpretation(reader)
+        else -1
+      val tiffMetadata =
+        if (needTiffMetadata) 
GeoTiffMetadataPartitionReader.extractMetadata(reader)
+        else Map.empty[String, String]
+      val compression =
+        if (needCompression) 
GeoTiffMetadataPartitionReader.extractCompression(reader) else null
+

Review Comment:
   The PR introduces `compression` and `metadata` extraction 
(`extractCompression` / `extractMetadata`), but the added test suite doesn’t 
assert either column’s behavior. To prevent regressions, add at least one 
assertion for `compression` (e.g., in the generated COG test) and a minimal 
assertion for `metadata` (e.g., that it’s non-null and/or contains a stable 
expected tag for `test1.tiff`).



##########
spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/geotiffmetadata/GeoTiffMetadataDataSource.scala:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.sedona_sql.io.geotiffmetadata
+
+import org.apache.hadoop.fs.Path
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.connector.catalog.TableProvider
+import org.apache.spark.sql.execution.datasources.FileFormat
+import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+import scala.collection.JavaConverters._
+import scala.util.Try
+
+/**
+ * A read-only Spark SQL data source that extracts GeoTIFF file metadata 
(dimensions, CRS, bands,
+ * overviews, compression, etc.) without loading raster pixel data into memory.
+ */
+class GeoTiffMetadataDataSource
+    extends FileDataSourceV2
+    with TableProvider
+    with DataSourceRegister {
+
+  override def shortName(): String = "geotiff.metadata"
+
+  private val loadTifPattern = "(.*)/([^/]*\\*[^/]*\\.(?i:tif|tiff))$".r
+
+  private def createTable(
+      options: CaseInsensitiveStringMap,
+      userSchema: Option[StructType] = None): Table = {
+    var paths = getPaths(options)
+    var optionsWithoutPaths = getOptionsWithoutPaths(options)
+    val tableName = getTableName(options, paths)
+
+    if (paths.size == 1) {
+      val head = paths.head
+      if (isDirectory(head)) {
+        // Directory (with or without trailing slash): recurse and filter to 
GeoTIFF files
+        val newOptions =
+          new java.util.HashMap[String, 
String](optionsWithoutPaths.asCaseSensitiveMap())
+        newOptions.put("recursiveFileLookup", "true")
+        if (!newOptions.containsKey("pathGlobFilter")) {
+          newOptions.put("pathGlobFilter", "*.{tif,tiff,TIF,TIFF}")
+        }
+        optionsWithoutPaths = new CaseInsensitiveStringMap(newOptions)
+      } else {
+        // Rewrite glob patterns like /path/to/some*glob*.tif into /path/to 
with
+        // pathGlobFilter="some*glob*.tif" to avoid listing .tif files as 
directories
+        head match {
+          case loadTifPattern(prefix, glob) =>
+            paths = Seq(prefix)
+            val newOptions =
+              new java.util.HashMap[String, 
String](optionsWithoutPaths.asCaseSensitiveMap())
+            newOptions.put("pathGlobFilter", glob)
+            optionsWithoutPaths = new CaseInsensitiveStringMap(newOptions)
+          case _ =>
+        }
+      }
+    }
+
+    new GeoTiffMetadataTable(
+      tableName,
+      sparkSession,
+      optionsWithoutPaths,
+      paths,
+      userSchema,
+      fallbackFileFormat)
+  }
+
+  override def getTable(options: CaseInsensitiveStringMap): Table = {
+    createTable(options)
+  }
+
+  override def getTable(options: CaseInsensitiveStringMap, schema: 
StructType): Table = {
+    createTable(options, Some(schema))
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType =
+    GeoTiffMetadataTable.SCHEMA
+
+  // Read-only data source — no V1 fallback needed
+  override def fallbackFileFormat: Class[_ <: FileFormat] = null
+
+  /**
+   * Check if a path points to a directory via Hadoop FileSystem, falling back 
to trailing-slash
+   * heuristic if FS access fails (e.g., path doesn't exist yet or glob 
patterns).
+   */
+  private def isDirectory(pathStr: String): Boolean = {
+    if (pathStr.endsWith("/")) return true
+    Try {
+      val hadoopConf = sparkSession.sessionState.newHadoopConf()
+      val path = new Path(pathStr)
+      val fs = path.getFileSystem(hadoopConf)
+      fs.getFileStatus(path).isDirectory
+    }.getOrElse(false)

Review Comment:
   `isDirectory` builds the Hadoop `Configuration` via 
`sparkSession.sessionState.newHadoopConf()` and ignores any per-read options 
(e.g., `fs.s3a.*`, custom FS impl settings) that are supplied as data source 
options. This can cause directory detection to incorrectly return `false`, 
which then skips setting `recursiveFileLookup`/`pathGlobFilter` and changes 
what files get discovered. Consider passing the effective options into 
`isDirectory` and using `newHadoopConfWithOptions(options.asScala.toMap)` (or 
equivalent) so directory detection uses the same FS configuration as the actual 
scan.



##########
spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/geotiffmetadata/GeoTiffMetadataPartitionReader.scala:
##########
@@ -0,0 +1,454 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.sedona_sql.io.geotiffmetadata
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.sedona.common.raster.RasterAccessors
+import org.apache.sedona.common.raster.RasterBandAccessors
+import org.apache.sedona.common.raster.inputstream.HadoopImageInputStream
+import org.apache.sedona.common.utils.RasterUtils
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow
+import org.apache.spark.sql.catalyst.util.GenericArrayData
+import org.apache.spark.sql.connector.read.PartitionReader
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.unsafe.types.UTF8String
+import org.geotools.coverage.grid.GridCoverage2D
+import org.geotools.gce.geotiff.GeoTiffReader
+import org.geotools.referencing.crs.DefaultEngineeringCRS
+
+import java.net.URI
+import scala.collection.mutable
+import scala.util.Try
+
+class GeoTiffMetadataPartitionReader(
+    configuration: Configuration,
+    partitionedFiles: Array[PartitionedFile],
+    readDataSchema: StructType)
+    extends PartitionReader[InternalRow] {
+
+  private var currentFileIndex = 0
+  private var currentRow: InternalRow = _
+
+  override def next(): Boolean = {
+    if (currentFileIndex < partitionedFiles.length) {
+      currentRow = readFileMetadata(partitionedFiles(currentFileIndex))
+      currentFileIndex += 1
+      true
+    } else {
+      false
+    }
+  }
+
+  override def get(): InternalRow = currentRow
+
+  override def close(): Unit = {}
+
+  // Fields requiring reader.read() to build a GridCoverage2D
+  private val COVERAGE_FIELDS = Set(
+    "width",
+    "height",
+    "numBands",
+    "srid",
+    "crs",
+    "geoTransform",
+    "cornerCoordinates",
+    "bands")
+
+  private def readFileMetadata(partition: PartitionedFile): InternalRow = {
+    val requested = readDataSchema.fieldNames.toSet
+    val needCoverage = requested.exists(COVERAGE_FIELDS.contains)
+    val needIsTiled = requested.contains("isTiled")
+    val needBands = requested.contains("bands")
+    val needTiffMetadata = requested.contains("metadata")
+    val needCompression = requested.contains("compression")
+    val needReader =
+      needCoverage || needIsTiled || needBands || needTiffMetadata || 
needCompression ||
+        requested.contains("overviews")
+
+    val path = new Path(new URI(partition.filePath.toString()))
+
+    // Skip all I/O if only cheap fields (path, driver, fileSize) are requested
+    if (!needReader) {
+      return buildRow(path, partition, null, null, false, -1, Map.empty, null)
+    }
+
+    val imageStream = new HadoopImageInputStream(path, configuration)
+    var reader: GeoTiffReader = null
+    var raster: GridCoverage2D = null
+    try {
+      reader = new GeoTiffReader(
+        imageStream,
+        new org.geotools.util.factory.Hints(
+          org.geotools.util.factory.Hints.FORCE_LONGITUDE_FIRST_AXIS_ORDER,
+          java.lang.Boolean.TRUE))
+
+      // Extract TIFF IIO metadata BEFORE read() to avoid stream state issues.
+      // Only extract fields actually requested.
+      val isTiled =
+        if (needIsTiled) GeoTiffMetadataPartitionReader.hasTiffTag(reader, 
322) else false
+      val photometric =
+        if (needBands) 
GeoTiffMetadataPartitionReader.extractPhotometricInterpretation(reader)
+        else -1
+      val tiffMetadata =
+        if (needTiffMetadata) 
GeoTiffMetadataPartitionReader.extractMetadata(reader)
+        else Map.empty[String, String]
+      val compression =
+        if (needCompression) 
GeoTiffMetadataPartitionReader.extractCompression(reader) else null
+
+      if (needCoverage) {
+        raster = reader.read(null)
+      }
+
+      buildRow(path, partition, reader, raster, isTiled, photometric, 
tiffMetadata, compression)
+    } finally {
+      if (raster != null) raster.dispose(true)
+      if (reader != null) reader.dispose()
+      imageStream.close()
+    }
+  }
+
+  private def buildRow(
+      path: Path,
+      partition: PartitionedFile,
+      reader: GeoTiffReader,
+      raster: GridCoverage2D,
+      isTiled: Boolean,
+      photometric: Int,
+      tiffMetadata: Map[String, String],
+      compression: String): InternalRow = {
+    lazy val width = RasterAccessors.getWidth(raster)
+    lazy val height = RasterAccessors.getHeight(raster)
+    lazy val numBands = RasterAccessors.numBands(raster)
+    lazy val srid = RasterAccessors.srid(raster)
+
+    lazy val crsStr = Try {
+      val crs = raster.getCoordinateReferenceSystem
+      if (crs == null || crs.isInstanceOf[DefaultEngineeringCRS]) null
+      else crs.toWKT
+    }.getOrElse(null)
+
+    lazy val affine = RasterUtils.getGDALAffineTransform(raster)
+    lazy val geoTransformRow = new GenericInternalRow(
+      Array[Any](
+        affine.getTranslateX,
+        affine.getTranslateY,
+        affine.getScaleX,
+        affine.getScaleY,
+        affine.getShearX,
+        affine.getShearY))
+
+    lazy val env = raster.getEnvelope2D
+    lazy val cornerCoordinatesRow = new GenericInternalRow(
+      Array[Any](env.getMinX, env.getMinY, env.getMaxX, env.getMaxY))
+
+    lazy val image = raster.getRenderedImage
+    lazy val tileWidth = image.getTileWidth
+    lazy val tileHeight = image.getTileHeight
+
+    lazy val bandsArray = GeoTiffMetadataPartitionReader
+      .buildBandsArray(raster, numBands, tileWidth, tileHeight, photometric)
+    lazy val overviewsArray = 
GeoTiffMetadataPartitionReader.buildOverviewsArray(reader)
+    lazy val metadataMap = 
GeoTiffMetadataPartitionReader.buildMetadataMap(tiffMetadata)
+
+    val fields = readDataSchema.fieldNames.map {
+      case "path" => UTF8String.fromString(path.toString)
+      case "driver" => UTF8String.fromString("GTiff")
+      case "fileSize" => partition.fileSize: Any
+      case "width" => width: Any
+      case "height" => height: Any
+      case "numBands" => numBands: Any
+      case "srid" => srid: Any
+      case "crs" =>
+        if (crsStr != null) UTF8String.fromString(crsStr) else null
+      case "geoTransform" => geoTransformRow
+      case "cornerCoordinates" => cornerCoordinatesRow
+      case "bands" => bandsArray
+      case "overviews" => overviewsArray
+      case "metadata" => metadataMap
+      case "isTiled" => isTiled: Any
+      case "compression" =>
+        if (compression != null) UTF8String.fromString(compression) else null
+      case other =>
+        throw new IllegalArgumentException(s"Unsupported field name: $other")
+    }
+
+    new GenericInternalRow(fields)
+  }
+}
+
+object GeoTiffMetadataPartitionReader {
+
+  // TIFF Photometric Interpretation values
+  private val PHOTOMETRIC_MIN_IS_WHITE = 0
+  private val PHOTOMETRIC_MIN_IS_BLACK = 1
+  private val PHOTOMETRIC_RGB = 2
+  private val PHOTOMETRIC_PALETTE = 3
+
+  def buildBandsArray(
+      raster: GridCoverage2D,
+      numBands: Int,
+      tileWidth: Int,
+      tileHeight: Int,
+      photometric: Int): GenericArrayData = {
+    val bands = (1 to numBands).map { i =>
+      val dataType = Try(RasterBandAccessors.getBandType(raster, 
i)).getOrElse(null)
+      val noDataValue = Try(RasterBandAccessors.getBandNoDataValue(raster, 
i)).getOrElse(null)
+      val description = Try {
+        val desc = raster.getSampleDimension(i - 1).getDescription
+        if (desc != null) desc.toString(java.util.Locale.ROOT) else null
+      }.getOrElse(null)
+      val unit = Try {
+        val units = raster.getSampleDimension(i - 1).getUnits
+        if (units != null) units.toString else null
+      }.getOrElse(null)
+      val colorInterp = resolveColorInterpretation(photometric, i, numBands)
+
+      new GenericInternalRow(
+        Array[Any](
+          i,
+          if (dataType != null) UTF8String.fromString(dataType) else null,
+          if (colorInterp != null) UTF8String.fromString(colorInterp) else 
null,
+          if (noDataValue != null) noDataValue.doubleValue() else null,
+          tileWidth,
+          tileHeight,
+          if (description != null) UTF8String.fromString(description) else 
null,
+          if (unit != null) UTF8String.fromString(unit) else null))
+    }.toArray
+    new GenericArrayData(bands)
+  }
+
+  private def resolveColorInterpretation(photometric: Int, band: Int, 
numBands: Int): String = {
+    photometric match {
+      case PHOTOMETRIC_MIN_IS_WHITE | PHOTOMETRIC_MIN_IS_BLACK =>
+        if (numBands == 1) "Gray"
+        else s"Gray$band"
+      case PHOTOMETRIC_RGB =>
+        band match {
+          case 1 => "Red"
+          case 2 => "Green"
+          case 3 => "Blue"
+          case 4 => "Alpha"
+          case _ => "Undefined"
+        }
+      case PHOTOMETRIC_PALETTE => "Palette"
+      case _ => "Undefined"
+    }
+  }
+
+  def buildOverviewsArray(reader: GeoTiffReader): GenericArrayData = {
+    try {
+      val layout = reader.getDatasetLayout
+      if (layout == null) return new GenericArrayData(Array.empty[InternalRow])
+
+      val numOverviews = layout.getNumInternalOverviews
+      if (numOverviews <= 0) return new 
GenericArrayData(Array.empty[InternalRow])
+
+      val resolutionLevels = reader.getResolutionLevels
+      if (resolutionLevels == null || resolutionLevels.length <= 1)
+        return new GenericArrayData(Array.empty[InternalRow])
+
+      // Get full dimensions from the reader without loading the coverage
+      val gridRange = reader.getOriginalGridRange
+      val fullWidth = gridRange.getSpan(0)
+      val fullHeight = gridRange.getSpan(1)
+
+      val count = Math.min(numOverviews, resolutionLevels.length - 1)
+      val fullResX = resolutionLevels(0)(0)
+      val fullResY = resolutionLevels(0)(1)
+      val overviews = (1 to count).map { level =>
+        val ovrResX = resolutionLevels(level)(0)
+        val ovrResY = resolutionLevels(level)(1)
+        new GenericInternalRow(
+          Array[Any](
+            level,
+            Math.round(fullWidth.toDouble * fullResX / ovrResX).toInt,
+            Math.round(fullHeight.toDouble * fullResY / ovrResY).toInt))
+      }.toArray
+      new GenericArrayData(overviews)
+    } catch {
+      case _: Exception => new GenericArrayData(Array.empty[InternalRow])
+    }
+  }
+
+  def buildMetadataMap(
+      metadata: Map[String, String]): 
org.apache.spark.sql.catalyst.util.MapData = {
+    if (metadata.isEmpty) return null

Review Comment:
   `buildMetadataMap` returns `null` when the metadata map is empty, while 
similar collection columns here (e.g., `overviews`) return an empty collection. 
This makes downstream queries treat “no metadata” as `null` rather than an 
empty map (e.g., `size(metadata)` becomes `null` instead of `0`) and forces 
additional `coalesce`/null-handling for common patterns. Consider returning an 
empty `MapData` for empty metadata to keep collection columns consistent.



-- 
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]

Reply via email to