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


##########
spark/common/src/test/scala/org/apache/sedona/sql/rasterIOTest.scala:
##########
@@ -239,5 +248,282 @@ class rasterIOTest extends TestBaseScala with 
BeforeAndAfter with GivenWhenThen
     }
   }
 
-  override def afterAll(): Unit = FileUtils.deleteDirectory(new File(tempDir))
+  describe("Raster read test") {
+    it("should read geotiff using raster source with explicit tiling") {
+      val rasterDf = sparkSession.read
+        .format("raster")
+        .options(Map("retile" -> "true", "tileWidth" -> "64"))
+        .load(rasterdatalocation)
+      assert(rasterDf.count() > 100)
+      rasterDf.collect().foreach { row =>

Review Comment:
   This test triggers two full scans of the raster source (`count()` then 
`collect()`), which increases CI time and doubles IO for no added coverage. 
Consider collecting once (or using `take`) and asserting on the collected row 
count instead of calling `count()` separately.
   ```suggestion
         val rasterRows = rasterDf.collect()
         assert(rasterRows.length > 100)
         rasterRows.foreach { row =>
   ```



##########
spark/common/src/test/scala/org/apache/sedona/sql/rasterIOTest.scala:
##########
@@ -239,5 +248,282 @@ class rasterIOTest extends TestBaseScala with 
BeforeAndAfter with GivenWhenThen
     }
   }
 
-  override def afterAll(): Unit = FileUtils.deleteDirectory(new File(tempDir))
+  describe("Raster read test") {
+    it("should read geotiff using raster source with explicit tiling") {
+      val rasterDf = sparkSession.read
+        .format("raster")
+        .options(Map("retile" -> "true", "tileWidth" -> "64"))
+        .load(rasterdatalocation)
+      assert(rasterDf.count() > 100)
+      rasterDf.collect().foreach { row =>
+        val raster = row.getAs[Object](0).asInstanceOf[GridCoverage2D]
+        assert(raster.getGridGeometry.getGridRange2D.width <= 64)
+        assert(raster.getGridGeometry.getGridRange2D.height <= 64)
+        val x = row.getInt(1)
+        val y = row.getInt(2)
+        assert(x >= 0 && y >= 0)
+        raster.dispose(true)
+      }
+
+      // Test projection push-down
+      rasterDf.selectExpr("y", "rast as r").collect().foreach { row =>
+        val raster = row.getAs[Object](1).asInstanceOf[GridCoverage2D]
+        assert(raster.getGridGeometry.getGridRange2D.width <= 64)
+        assert(raster.getGridGeometry.getGridRange2D.height <= 64)
+        val y = row.getInt(0)
+        assert(y >= 0)
+        raster.dispose(true)
+      }
+    }
+
+    it("should tile geotiff using raster source with padding enabled") {
+      val rasterDf = sparkSession.read
+        .format("raster")
+        .options(Map("retile" -> "true", "tileWidth" -> "64", "padWithNoData" 
-> "true"))
+        .load(rasterdatalocation)
+      assert(rasterDf.count() > 100)
+      rasterDf.collect().foreach { row =>

Review Comment:
   This test also performs `count()` followed by `collect()`, causing two 
actions / scans. Consider a single action (e.g., `val rows = collect()` and 
assert on `rows.length`) to keep the test runtime and IO down.



##########
spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/raster/RasterPartitionReader.scala:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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.raster
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.sedona.common.raster.RasterConstructors
+import org.apache.sedona.common.raster.inputstream.HadoopImageInputStream
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter
+import org.apache.spark.sql.connector.read.PartitionReader
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.sedona_sql.UDT.RasterUDT
+import 
org.apache.spark.sql.sedona_sql.io.raster.RasterPartitionReader.rasterToInternalRows
+import 
org.apache.spark.sql.sedona_sql.io.raster.RasterTable.{MAX_AUTO_TILE_SIZE, 
RASTER, RASTER_NAME, TILE_X, TILE_Y}
+import org.apache.spark.sql.types.StructType
+import org.geotools.coverage.grid.GridCoverage2D
+
+import java.net.URI
+import scala.collection.JavaConverters._
+
+class RasterPartitionReader(
+    configuration: Configuration,
+    partitionedFiles: Array[PartitionedFile],
+    dataSchema: StructType,
+    rasterOptions: RasterOptions)
+    extends PartitionReader[InternalRow] {
+
+  // Track the current file index we're processing
+  private var currentFileIndex = 0
+
+  // Current raster being processed
+  private var currentRaster: GridCoverage2D = _
+
+  // Current image input stream (must be kept open while the raster is in use)
+  private var currentImageStream: HadoopImageInputStream = _
+
+  // Current row
+  private var currentRow: InternalRow = _
+
+  // Iterator for the current file's tiles
+  private var currentIterator: Iterator[InternalRow] = Iterator.empty
+
+  override def next(): Boolean = {
+    // If current iterator has more elements, return true
+    if (currentIterator.hasNext) {
+      currentRow = currentIterator.next()
+      return true
+    }
+
+    // If current iterator is exhausted, but we have more files, load the next 
file
+    if (currentFileIndex < partitionedFiles.length) {
+      loadNextFile()
+      if (currentIterator.hasNext) {
+        currentRow = currentIterator.next()
+        return true
+      }
+    }
+
+    // No more data
+    false
+  }
+
+  override def get(): InternalRow = {
+    currentRow
+  }
+
+  override def close(): Unit = {
+    if (currentRaster != null) {
+      currentRaster.dispose(true)
+      currentRaster = null
+    }
+    if (currentImageStream != null) {
+      currentImageStream.close()
+      currentImageStream = null
+    }
+  }
+
+  private def loadNextFile(): Unit = {
+    // Clean up previous raster and stream if exists
+    if (currentRaster != null) {
+      currentRaster.dispose(true)
+      currentRaster = null
+    }
+    if (currentImageStream != null) {
+      currentImageStream.close()
+      currentImageStream = null
+    }
+
+    if (currentFileIndex >= partitionedFiles.length) {
+      currentIterator = Iterator.empty
+      return
+    }
+
+    val partition = partitionedFiles(currentFileIndex)
+    val path = new Path(new URI(partition.filePath.toString()))
+
+    try {
+      // Open a stream-based reader instead of materializing the entire file 
as byte[].
+      // This avoids the 2 GB byte[] limit and reduces memory pressure for 
large files.
+      currentImageStream = new HadoopImageInputStream(path, configuration)
+
+      // Create in-db GridCoverage2D from GeoTiff stream. The RenderedImage is 
lazy -
+      // pixel data will only be decoded when accessed via 
image.getData(Rectangle).
+      currentRaster = RasterConstructors.fromGeoTiff(currentImageStream)
+      currentIterator = rasterToInternalRows(currentRaster, dataSchema, 
rasterOptions, path)
+      currentFileIndex += 1
+    } catch {
+      case e: Exception =>
+        if (currentRaster != null) {
+          currentRaster.dispose(true)
+          currentRaster = null
+        }
+        if (currentImageStream != null) {
+          currentImageStream.close()
+          currentImageStream = null
+        }
+        throw e
+    }
+  }
+}
+
+object RasterPartitionReader {
+  def rasterToInternalRows(
+      currentRaster: GridCoverage2D,
+      dataSchema: StructType,
+      rasterOptions: RasterOptions,
+      path: Path): Iterator[InternalRow] = {
+    val retile = rasterOptions.retile
+    val tileWidth = rasterOptions.tileWidth
+    val tileHeight = rasterOptions.tileHeight
+    val padWithNoData = rasterOptions.padWithNoData
+
+    val writer = new UnsafeRowWriter(dataSchema.length)
+    writer.resetRowWriter()
+
+    // Extract the file name from the path
+    val fileName = path.getName
+
+    if (retile) {
+      val (tw, th) = (tileWidth, tileHeight) match {
+        case (Some(tw), Some(th)) => (tw, th)
+        case (None, None) =>
+          // Use the internal tile size of the input raster
+          val tw = currentRaster.getRenderedImage.getTileWidth
+          val th = currentRaster.getRenderedImage.getTileHeight
+          val tileSizeError = {
+            """To resolve this issue, you can try one of the following methods:
+            |  1. Disable retile by setting `.option("retile", "false")`.
+            |  2. Explicitly set `tileWidth` and `tileHeight`.
+            |  3. Convert the raster to a Cloud Optimized GeoTIFF (COG) using 
tools like `gdal_translate`.
+            |""".stripMargin
+          }
+          if (tw >= MAX_AUTO_TILE_SIZE || th >= MAX_AUTO_TILE_SIZE) {
+            throw new IllegalArgumentException(
+              s"Internal tile size of $path is too large ($tw x $th). " + 
tileSizeError)
+          }
+          if (tw == 0 || th == 0) {
+            throw new IllegalArgumentException(
+              s"Internal tile size of $path contains zero ($tw x $th). " + 
tileSizeError)
+          }
+          if (tw.toDouble / th > 10.0 || th.toDouble / tw > 10.0) {
+            throw new IllegalArgumentException(
+              s"Internal tile shape of $path is too thin ($tw x $th). " + 
tileSizeError)
+          }
+          (tw, th)
+        case _ =>
+          throw new IllegalArgumentException("Both tileWidth and tileHeight 
must be set")
+      }
+
+      val iter =
+        RasterConstructors.generateTiles(currentRaster, null, tw, th, 
padWithNoData, Double.NaN)
+      iter.asScala.map { tile =>
+        val tileRaster = tile.getCoverage
+        writer.reset()
+        writeRaster(writer, dataSchema, tileRaster, tile.getTileX, 
tile.getTileY, fileName)
+        tileRaster.dispose(true)
+        writer.getRow
+      }
+    } else {
+      writeRaster(writer, dataSchema, currentRaster, 0, 0, fileName)
+      Iterator.single(writer.getRow)

Review Comment:
   `UnsafeRowWriter.getRow` returns an `UnsafeRow` backed by the writer's 
internal buffer. Since the same writer instance is reused for every tile, 
returning `writer.getRow` directly can corrupt previously emitted rows when the 
next tile overwrites the buffer. Return a stable row per record (e.g., copy the 
row or allocate a new row) before yielding it from the iterator.
   ```suggestion
           writer.getRow.copy()
         }
       } else {
         writeRaster(writer, dataSchema, currentRaster, 0, 0, fileName)
         Iterator.single(writer.getRow.copy())
   ```



##########
spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/raster/RasterPartitionReader.scala:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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.raster
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.sedona.common.raster.RasterConstructors
+import org.apache.sedona.common.raster.inputstream.HadoopImageInputStream
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter
+import org.apache.spark.sql.connector.read.PartitionReader
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.sedona_sql.UDT.RasterUDT
+import 
org.apache.spark.sql.sedona_sql.io.raster.RasterPartitionReader.rasterToInternalRows
+import 
org.apache.spark.sql.sedona_sql.io.raster.RasterTable.{MAX_AUTO_TILE_SIZE, 
RASTER, RASTER_NAME, TILE_X, TILE_Y}
+import org.apache.spark.sql.types.StructType
+import org.geotools.coverage.grid.GridCoverage2D
+
+import java.net.URI
+import scala.collection.JavaConverters._
+
+class RasterPartitionReader(
+    configuration: Configuration,
+    partitionedFiles: Array[PartitionedFile],
+    dataSchema: StructType,
+    rasterOptions: RasterOptions)
+    extends PartitionReader[InternalRow] {
+
+  // Track the current file index we're processing
+  private var currentFileIndex = 0
+
+  // Current raster being processed
+  private var currentRaster: GridCoverage2D = _
+
+  // Current image input stream (must be kept open while the raster is in use)
+  private var currentImageStream: HadoopImageInputStream = _
+
+  // Current row
+  private var currentRow: InternalRow = _
+
+  // Iterator for the current file's tiles
+  private var currentIterator: Iterator[InternalRow] = Iterator.empty
+
+  override def next(): Boolean = {
+    // If current iterator has more elements, return true
+    if (currentIterator.hasNext) {
+      currentRow = currentIterator.next()
+      return true
+    }
+
+    // If current iterator is exhausted, but we have more files, load the next 
file
+    if (currentFileIndex < partitionedFiles.length) {
+      loadNextFile()
+      if (currentIterator.hasNext) {
+        currentRow = currentIterator.next()
+        return true
+      }
+    }
+
+    // No more data
+    false
+  }
+
+  override def get(): InternalRow = {
+    currentRow
+  }
+
+  override def close(): Unit = {
+    if (currentRaster != null) {
+      currentRaster.dispose(true)
+      currentRaster = null
+    }
+    if (currentImageStream != null) {
+      currentImageStream.close()
+      currentImageStream = null
+    }
+  }
+
+  private def loadNextFile(): Unit = {
+    // Clean up previous raster and stream if exists
+    if (currentRaster != null) {
+      currentRaster.dispose(true)
+      currentRaster = null
+    }
+    if (currentImageStream != null) {
+      currentImageStream.close()
+      currentImageStream = null
+    }
+
+    if (currentFileIndex >= partitionedFiles.length) {
+      currentIterator = Iterator.empty
+      return
+    }
+
+    val partition = partitionedFiles(currentFileIndex)
+    val path = new Path(new URI(partition.filePath.toString()))
+
+    try {
+      // Open a stream-based reader instead of materializing the entire file 
as byte[].
+      // This avoids the 2 GB byte[] limit and reduces memory pressure for 
large files.
+      currentImageStream = new HadoopImageInputStream(path, configuration)
+
+      // Create in-db GridCoverage2D from GeoTiff stream. The RenderedImage is 
lazy -
+      // pixel data will only be decoded when accessed via 
image.getData(Rectangle).
+      currentRaster = RasterConstructors.fromGeoTiff(currentImageStream)
+      currentIterator = rasterToInternalRows(currentRaster, dataSchema, 
rasterOptions, path)
+      currentFileIndex += 1
+    } catch {
+      case e: Exception =>
+        if (currentRaster != null) {
+          currentRaster.dispose(true)
+          currentRaster = null
+        }
+        if (currentImageStream != null) {
+          currentImageStream.close()
+          currentImageStream = null
+        }
+        throw e
+    }
+  }
+}
+
+object RasterPartitionReader {
+  def rasterToInternalRows(
+      currentRaster: GridCoverage2D,
+      dataSchema: StructType,
+      rasterOptions: RasterOptions,
+      path: Path): Iterator[InternalRow] = {
+    val retile = rasterOptions.retile
+    val tileWidth = rasterOptions.tileWidth
+    val tileHeight = rasterOptions.tileHeight
+    val padWithNoData = rasterOptions.padWithNoData
+
+    val writer = new UnsafeRowWriter(dataSchema.length)
+    writer.resetRowWriter()
+
+    // Extract the file name from the path
+    val fileName = path.getName
+
+    if (retile) {
+      val (tw, th) = (tileWidth, tileHeight) match {
+        case (Some(tw), Some(th)) => (tw, th)
+        case (None, None) =>
+          // Use the internal tile size of the input raster
+          val tw = currentRaster.getRenderedImage.getTileWidth
+          val th = currentRaster.getRenderedImage.getTileHeight
+          val tileSizeError = {
+            """To resolve this issue, you can try one of the following methods:
+            |  1. Disable retile by setting `.option("retile", "false")`.
+            |  2. Explicitly set `tileWidth` and `tileHeight`.
+            |  3. Convert the raster to a Cloud Optimized GeoTIFF (COG) using 
tools like `gdal_translate`.
+            |""".stripMargin
+          }
+          if (tw >= MAX_AUTO_TILE_SIZE || th >= MAX_AUTO_TILE_SIZE) {
+            throw new IllegalArgumentException(
+              s"Internal tile size of $path is too large ($tw x $th). " + 
tileSizeError)
+          }
+          if (tw == 0 || th == 0) {
+            throw new IllegalArgumentException(
+              s"Internal tile size of $path contains zero ($tw x $th). " + 
tileSizeError)
+          }
+          if (tw.toDouble / th > 10.0 || th.toDouble / tw > 10.0) {
+            throw new IllegalArgumentException(
+              s"Internal tile shape of $path is too thin ($tw x $th). " + 
tileSizeError)
+          }
+          (tw, th)
+        case _ =>
+          throw new IllegalArgumentException("Both tileWidth and tileHeight 
must be set")
+      }
+
+      val iter =
+        RasterConstructors.generateTiles(currentRaster, null, tw, th, 
padWithNoData, Double.NaN)
+      iter.asScala.map { tile =>
+        val tileRaster = tile.getCoverage
+        writer.reset()
+        writeRaster(writer, dataSchema, tileRaster, tile.getTileX, 
tile.getTileY, fileName)
+        tileRaster.dispose(true)
+        writer.getRow
+      }
+    } else {
+      writeRaster(writer, dataSchema, currentRaster, 0, 0, fileName)
+      Iterator.single(writer.getRow)

Review Comment:
   Same buffer-reuse issue for the non-retiled path: 
`Iterator.single(writer.getRow)` returns an `UnsafeRow` backed by the writer's 
mutable buffer. Ensure the returned row is not mutated after emission (e.g., 
return a copy).
   ```suggestion
         val row = writer.getRow.copy()
         Iterator.single(row)
   ```



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