Copilot commented on code in PR #2846: URL: https://github.com/apache/sedona/pull/2846#discussion_r3122196312
########## spark/common/src/test/scala/org/apache/sedona/sql/geotiffMetadataTest.scala: ########## @@ -0,0 +1,222 @@ +/* + * 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.sedona.sql + +import org.apache.commons.io.FileUtils +import org.junit.Assert.assertEquals +import org.scalatest.BeforeAndAfterAll + +import java.io.File +import java.nio.file.Files + +class geotiffMetadataTest extends TestBaseScala with BeforeAndAfterAll { + + val rasterDir: String = resourceFolder + "raster/" + val singleFileLocation: String = resourceFolder + "raster/test1.tiff" + val tempDir: String = + Files.createTempDirectory("sedona_geotiffmetadata_test_").toFile.getAbsolutePath + + override def afterAll(): Unit = { + FileUtils.deleteDirectory(new File(tempDir)) + super.afterAll() + } + + describe("GeoTiffMetadata data source") { + + it("should read test1.tiff with exact metadata values") { + val df = sparkSession.read.format("geotiff.metadata").load(singleFileLocation) + assertEquals(1L, df.count()) + + val row = df.first() + assert(row.getAs[String]("path").endsWith("test1.tiff")) + assertEquals("GTiff", row.getAs[String]("driver")) + assertEquals(174803L, row.getAs[Long]("fileSize")) + assertEquals(512, row.getAs[Int]("width")) + assertEquals(517, row.getAs[Int]("height")) + assertEquals(1, row.getAs[Int]("numBands")) + assertEquals(3857, row.getAs[Int]("srid")) + assert(row.getAs[String]("crs").contains("EPSG")) + // test1.tiff has TileWidth/TileLength TIFF tags (internally tiled) + assertEquals(true, row.getAs[Boolean]("isTiled")) + } + + it("should return exact geoTransform for test1.tiff") { + val row = sparkSession.read + .format("geotiff.metadata") + .load(singleFileLocation) + .selectExpr( + "geoTransform.upperLeftX", + "geoTransform.upperLeftY", + "geoTransform.scaleX", + "geoTransform.scaleY", + "geoTransform.skewX", + "geoTransform.skewY") + .first() + assertEquals(-1.3095817809482181e7, row.getAs[Double]("upperLeftX"), 0.01) + assertEquals(4021262.7487925636, row.getAs[Double]("upperLeftY"), 0.01) + assertEquals(72.32861272132695, row.getAs[Double]("scaleX"), 1e-10) + assertEquals(-72.32861272132695, row.getAs[Double]("scaleY"), 1e-10) + assertEquals(0.0, row.getAs[Double]("skewX"), 1e-15) + assertEquals(0.0, row.getAs[Double]("skewY"), 1e-15) + } + + it("should return exact cornerCoordinates for test1.tiff") { + val row = sparkSession.read + .format("geotiff.metadata") + .load(singleFileLocation) + .selectExpr( + "cornerCoordinates.minX", + "cornerCoordinates.minY", + "cornerCoordinates.maxX", + "cornerCoordinates.maxY") + .first() + assertEquals(-1.3095817809482181e7, row.getAs[Double]("minX"), 0.01) + assertEquals(3983868.8560156375, row.getAs[Double]("minY"), 0.01) + assertEquals(-1.3058785559768861e7, row.getAs[Double]("maxX"), 0.01) + assertEquals(4021262.7487925636, row.getAs[Double]("maxY"), 0.01) + } + + it("should return exact band metadata for test1.tiff") { + val row = sparkSession.read + .format("geotiff.metadata") + .load(singleFileLocation) + .selectExpr("explode(bands) as b") + .selectExpr( + "b.band", + "b.dataType", + "b.colorInterpretation", + "b.noDataValue", + "b.blockWidth", + "b.blockHeight", + "b.description", + "b.unit") + .first() + assertEquals(1, row.getAs[Int]("band")) + assertEquals("UNSIGNED_8BITS", row.getAs[String]("dataType")) + assertEquals("Gray", row.getAs[String]("colorInterpretation")) + assert(row.isNullAt(row.fieldIndex("noDataValue"))) + assertEquals(256, row.getAs[Int]("blockWidth")) + assertEquals(256, row.getAs[Int]("blockHeight")) + assertEquals("GRAY_INDEX", row.getAs[String]("description")) + assert(row.isNullAt(row.fieldIndex("unit"))) + } + + it("should return empty overviews for non-COG test1.tiff") { + // test1.tiff has only 1 IFD (no internal overviews) + val row = sparkSession.read + .format("geotiff.metadata") + .load(singleFileLocation) + .selectExpr("size(overviews) as overviewCount") + .first() + assertEquals(0, row.getAs[Int]("overviewCount")) + } + + it("should cross-validate against raster data source") { + val metaRow = sparkSession.read.format("geotiff.metadata").load(singleFileLocation).first() + val rasterRow = sparkSession.read + .format("raster") + .option("retile", "false") + .load(singleFileLocation) + .selectExpr( + "RS_Width(rast) as width", + "RS_Height(rast) as height", + "RS_NumBands(rast) as numBands", + "RS_SRID(rast) as srid") + .first() + assertEquals(metaRow.getAs[Int]("width"), rasterRow.getAs[Int]("width")) + assertEquals(metaRow.getAs[Int]("height"), rasterRow.getAs[Int]("height")) + assertEquals(metaRow.getAs[Int]("numBands"), rasterRow.getAs[Int]("numBands")) + assertEquals(metaRow.getAs[Int]("srid"), rasterRow.getAs[Int]("srid")) + } + + it("should read multiple files via glob") { + val df = sparkSession.read.format("geotiff.metadata").load(rasterDir + "*.tiff") + // 7 .tiff files in the raster directory (excludes test3.tif) + assertEquals(7L, df.count()) + } + + it("should read files from directory with trailing slash") { + val df = sparkSession.read.format("geotiff.metadata").load(rasterDir) + // Recursive lookup finds all .tif/.tiff files including subdirectories + assertEquals(9L, df.count()) + } + Review Comment: The data source has special handling for directories *with* a trailing slash (recursive lookup + `pathGlobFilter`). There’s a test for the trailing-slash case, but no test for loading a directory path without the trailing slash, which can behave differently in `GeoTiffMetadataDataSource`. Add a test that loads `rasterDir.stripSuffix("/")` (or similar) to lock in the intended behavior. ########## spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/geotiffmetadata/GeoTiffMetadataPartitionReader.scala: ########## @@ -0,0 +1,405 @@ +/* + * 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 = {} + + private def readFileMetadata(partition: PartitionedFile): InternalRow = { + val path = new Path(new URI(partition.filePath.toString())) + 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 + val isTiled = GeoTiffMetadataPartitionReader.hasTiffTag(reader, 322) + val photometric = GeoTiffMetadataPartitionReader.extractPhotometricInterpretation(reader) + val tiffMetadata = GeoTiffMetadataPartitionReader.extractMetadata(reader) + val compression = GeoTiffMetadataPartitionReader.extractCompression(reader) + + raster = reader.read(null) + + lazy val width = RasterAccessors.getWidth(raster) + lazy val height = RasterAccessors.getHeight(raster) + lazy val numBands = RasterAccessors.numBands(raster) + lazy val srid = RasterAccessors.srid(raster) Review Comment: `raster = reader.read(null)` is executed for every file regardless of `readDataSchema`. That means queries that only need cheap fields (e.g., `path`, `fileSize`, `overviews`, `compression`) still build a `GridCoverage2D`, undermining the stated “no pixel data loaded” goal and limiting the benefit of schema-aware pruning. Consider avoiding `reader.read` unless at least one requested column actually requires a coverage (width/height/numBands/srid/crs/geoTransform/cornerCoordinates/bands). ########## 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.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) Review Comment: `createTable` only enables `recursiveFileLookup` + default `pathGlobFilter` when the *string* path ends with "/". A directory path without a trailing slash (common in Spark) will skip this branch, so the scan may include non-GeoTIFF files and fail when attempting to parse them. Consider detecting directories via Hadoop FS (e.g., `Path(...).getFileSystem(...).getFileStatus.isDirectory`) and applying the same filtering/recursion regardless of whether the input path ends with "/". ########## spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/geotiffmetadata/GeoTiffMetadataPartitionReader.scala: ########## @@ -0,0 +1,405 @@ +/* + * 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 = {} + + private def readFileMetadata(partition: PartitionedFile): InternalRow = { + val path = new Path(new URI(partition.filePath.toString())) + 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 + val isTiled = GeoTiffMetadataPartitionReader.hasTiffTag(reader, 322) + val photometric = GeoTiffMetadataPartitionReader.extractPhotometricInterpretation(reader) + val tiffMetadata = GeoTiffMetadataPartitionReader.extractMetadata(reader) + val compression = GeoTiffMetadataPartitionReader.extractCompression(reader) + Review Comment: TIFF IIO metadata extraction (`hasTiffTag`, `extractPhotometricInterpretation`, `extractMetadata`, `extractCompression`) runs unconditionally for every file before you even look at `readDataSchema`. This makes column pruning much less effective (e.g., selecting only `path,width,height` still pays the full metadata-tree traversal cost). Gate these calls on whether the corresponding columns (`isTiled`, `bands`, `metadata`, `compression`) are actually present in `readDataSchema`. -- 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]
