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


##########
docs/tutorial/files/geotiffmetadata-sedona-spark.md:
##########
@@ -0,0 +1,188 @@
+<!--
+ 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.
+ -->
+
+# GeoTiffMetadata - GeoTIFF File Metadata
+
+GeoTiffMetadata is a Spark data source that reads GeoTIFF file metadata 
without decoding pixel data, similar to 
[gdalinfo](https://gdal.org/en/stable/programs/gdalinfo.html). It returns one 
row per file with metadata including dimensions, coordinate system, band 
information, tiling, overviews, and compression.
+
+This is useful for:
+
+* Cataloging and inventorying large collections of raster files
+* Detecting Cloud Optimized GeoTIFFs (COGs) by checking tiling and overview 
status
+* Inspecting file properties before loading full raster data
+* Building spatial indexes over raster file collections
+
+![Schema Overview](../../image/geotiff_metadata/schema_overview.svg 
"geotiff.metadata output schema")
+
+## COG detection
+
+Cloud Optimized GeoTIFFs (COGs) are GeoTIFF files with internal tiling and 
overviews optimized for cloud access. The `geotiff.metadata` data source 
reports these properties directly:
+
+![COG Structure](../../image/geotiff_metadata/cog_structure.svg "How COG 
properties map to geotiff.metadata fields")
+
+```python
+df = sedona.read.format("geotiff.metadata").load("/path/to/rasters/")
+cogs = df.filter("isTiled AND size(overviews) > 0")
+cogs.select("path", "compression", "overviews").show(truncate=False)
+```
+
+## Read GeoTIFF metadata
+
+=== "Scala"
+
+    ```scala
+    val df = sedona.read.format("geotiff.metadata").load("/path/to/rasters/")
+    df.show()
+    ```
+
+=== "Java"
+
+    ```java
+    Dataset<Row> df = 
sedona.read().format("geotiff.metadata").load("/path/to/rasters/");
+    df.show();
+    ```
+
+=== "Python"
+
+    ```python
+    df = sedona.read.format("geotiff.metadata").load("/path/to/rasters/")
+    df.show()
+    ```
+
+You can also use glob patterns:
+
+```python
+df = sedona.read.format("geotiff.metadata").load("/path/to/rasters/*.tif")
+```
+
+Or load a single file:
+
+```python
+df = sedona.read.format("geotiff.metadata").load("/path/to/image.tiff")
+```
+
+## Output schema
+
+Each row represents one GeoTIFF file with the following columns:
+
+| Column | Type | Description |
+|--------|------|-------------|
+| `path` | String | File path |
+| `driver` | String | Format driver (`"GTiff"`) |
+| `fileSize` | Long | File size in bytes |
+| `width` | Int | Image width in pixels |
+| `height` | Int | Image height in pixels |
+| `numBands` | Int | Number of bands |
+| `srid` | Int | EPSG code (0 if unknown) |
+| `crs` | String | Coordinate Reference System as WKT |
+| `geoTransform` | Struct | Affine transform parameters |
+| `cornerCoordinates` | Struct | Bounding box |
+| `bands` | Array[Struct] | Per-band metadata |
+| `overviews` | Array[Struct] | Overview (pyramid) levels |
+| `metadata` | Map[String, String] | File-wide TIFF metadata tags |
+| `isTiled` | Boolean | Whether the file uses internal tiling |
+| `compression` | String | Compression type (e.g., `"LZW"`, `"Deflate"`) |
+

Review Comment:
   The output schema table is using `||` at the start of each row (e.g., `|| 
Column | Type | Description |`), which is not valid Markdown table syntax and 
won’t render as a table in MkDocs. Convert this to a standard Markdown table 
using single `|` row delimiters (matching the style used in other tutorial 
pages).



##########
spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/geotiffmetadata/GeoTiffMetadataTable.scala:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.FileStatus
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.connector.catalog.SupportsRead
+import org.apache.spark.sql.connector.catalog.TableCapability
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.connector.write.LogicalWriteInfo
+import org.apache.spark.sql.connector.write.WriteBuilder
+import org.apache.spark.sql.execution.datasources.FileFormat
+import org.apache.spark.sql.execution.datasources.v2.FileTable
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+import java.util.{Set => JSet}
+
+case class GeoTiffMetadataTable(
+    name: String,
+    sparkSession: SparkSession,
+    options: CaseInsensitiveStringMap,
+    paths: Seq[String],
+    userSpecifiedSchema: Option[StructType],
+    fallbackFileFormat: Class[_ <: FileFormat])
+    extends FileTable(sparkSession, options, paths, userSpecifiedSchema)
+    with SupportsRead {
+
+  override def inferSchema(files: Seq[FileStatus]): Option[StructType] =
+    Some(userSpecifiedSchema.getOrElse(GeoTiffMetadataTable.SCHEMA))
+
+  override def formatName: String = "GeoTiffMetadata"
+
+  override def capabilities(): JSet[TableCapability] =
+    java.util.EnumSet.of(TableCapability.BATCH_READ)
+
+  override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder 
= {
+    GeoTiffMetadataScanBuilder(sparkSession, fileIndex, schema, dataSchema, 
options)
+  }
+
+  def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder =

Review Comment:
   `newWriteBuilder` is intended to override the table write path, but it’s 
missing the `override` modifier. In Scala this will fail compilation (or 
silently define a new method if the signature ever diverges), and it’s 
inconsistent with other FileTable implementations in this repo. Add `override` 
to the method definition (and keep throwing `UnsupportedOperationException` for 
read-only behavior).



##########
spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/geotiffmetadata/GeoTiffMetadataDataSource.scala:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.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.sedona_sql.io.raster.RasterFileFormat
+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._
+
+/**
+ * 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) {
+      if (paths.head.endsWith("/")) {
+        // Trailing-slash directories: 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
+        paths.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
+
+  override def fallbackFileFormat: Class[_ <: FileFormat] = 
classOf[RasterFileFormat]
+}

Review Comment:
   This data source is described as read-only and 
`GeoTiffMetadataTable.newWriteBuilder` throws, but `fallbackFileFormat` is set 
to `RasterFileFormat`. Other read-only metadata sources in this repo (e.g., 
`GeoParquetMetadataDataSource`) set `fallbackFileFormat` to `null` to avoid 
triggering V1 file source fallback behavior. Consider returning `null` here as 
well to prevent Spark from attempting any V1 fallback paths for this format.



##########
spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/geotiffmetadata/GeoTiffMetadataScanBuilder.scala:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.connector.read.Batch
+import org.apache.spark.sql.connector.read.InputPartition
+import org.apache.spark.sql.connector.read.PartitionReaderFactory
+import org.apache.spark.sql.connector.read.Scan
+import org.apache.spark.sql.connector.read.SupportsPushDownLimit
+import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex
+import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder
+import org.apache.spark.sql.execution.datasources.FilePartition
+import org.apache.spark.sql.execution.datasources.v2.FileScan
+import org.apache.spark.sql.sedona_sql.io.raster.RasterInputPartition
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+import org.apache.spark.util.SerializableConfiguration
+
+import scala.collection.JavaConverters._
+
+case class GeoTiffMetadataScanBuilder(
+    sparkSession: SparkSession,
+    fileIndex: PartitioningAwareFileIndex,
+    schema: StructType,
+    dataSchema: StructType,
+    options: CaseInsensitiveStringMap)
+    extends FileScanBuilder(sparkSession, fileIndex, dataSchema)
+    with SupportsPushDownLimit {
+
+  private var pushedLimit: Option[Int] = None
+
+  override def build(): Scan = {
+    GeoTiffMetadataScan(
+      sparkSession,
+      fileIndex,
+      dataSchema,
+      readDataSchema(),
+      readPartitionSchema(),
+      options,
+      pushedDataFilters,
+      partitionFilters,
+      dataFilters,
+      pushedLimit)
+  }
+
+  override def pushLimit(limit: Int): Boolean = {
+    pushedLimit = Some(limit)
+    true
+  }
+
+  override def isPartiallyPushed: Boolean = false
+}
+
+case class GeoTiffMetadataScan(
+    sparkSession: SparkSession,
+    fileIndex: PartitioningAwareFileIndex,
+    dataSchema: StructType,
+    readDataSchema: StructType,
+    readPartitionSchema: StructType,
+    options: CaseInsensitiveStringMap,
+    pushedFilters: Array[Filter],
+    partitionFilters: Seq[Expression] = Seq.empty,
+    dataFilters: Seq[Expression] = Seq.empty,
+    pushedLimit: Option[Int] = None)
+    extends FileScan
+    with Batch {
+
+  override def isSplitable(path: org.apache.hadoop.fs.Path): Boolean = false
+
+  private lazy val inputPartitions = {
+    var partitions = super.planInputPartitions()
+
+    // Limit the number of files to read
+    pushedLimit.foreach { limit =>
+      var remaining = limit
+      partitions = partitions.iterator
+        .takeWhile(_ => remaining > 0)
+        .map { partition =>
+          val filePartition = partition.asInstanceOf[FilePartition]
+          val files = filePartition.files
+          if (files.length <= remaining) {
+            remaining -= files.length
+            filePartition
+          } else {
+            val selectedFiles = files.take(remaining)
+            remaining = 0
+            FilePartition(filePartition.index, selectedFiles)
+          }

Review Comment:
   The LIMIT pushdown partition trimming uses `asInstanceOf[FilePartition]` 
when iterating partitions. If `super.planInputPartitions()` ever returns a 
non-`FilePartition` implementation, this will throw a `ClassCastException` 
before your later pattern match can produce the intended 
`IllegalArgumentException`. Use a pattern match (or safe cast with explicit 
error) inside the LIMIT trimming logic too.



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