Conversation
- Add TileGenerator.java with lazy InDbTileIterator for memory-efficient tiling - Update RasterConstructors.generateTiles() to return lazy TileIterator - Update RS_TileExplode to use lazy iterator with autoDispose - Add Spark DataSourceV2 raster reader: RasterDataSource, RasterTable, RasterScanBuilder, RasterInputPartition, RasterPartitionReaderFactory, RasterPartitionReader - Support GeoTiff, AsciiGrid, and NetCDF format detection by extension - Add read options: retile, tileWidth, tileHeight, padWithNoData - Support limit/sample pushdown, glob path rewriting, recursive directory loading - Register V2 RasterDataSource (remove V1 RasterFileFormat from META-INF) - Port 11 read tests from enterprise, adapted for in-db (no out-db references) - All 42 rasterIOTest, 314 rasteralgebraTest, 19 RasterConstructorsTest pass
There was a problem hiding this comment.
Pull request overview
This PR introduces a Spark DataSourceV2-based raster reader (format("raster")) that reads GeoTIFFs directly and can optionally retile them during read, aiming to simplify the legacy binaryFile + RS_FromGeoTiff + RS_TileExplode workflow while enabling limit/sample pushdown when retile=false.
Changes:
- Added a new
rasterDataSourceV2 implementation (table/scan/partition/reader) with optional retile behavior and recursive/glob path handling. - Introduced a new lazy tiling engine (
TileGenerator) and updated raster tiling APIs to return a lazy iterator. - Expanded raster IO tests and updated SQL docs to document the new data source and deprecate the legacy binaryFile-based section.
Reviewed changes
Copilot reviewed 14 out of 14 changed files in this pull request and generated 5 comments.
Show a summary per file
| File | Description |
|---|---|
| spark/common/src/test/scala/org/apache/sedona/sql/rasterIOTest.scala | Adds read-path tests for the new raster data source, including pushdown checks and path handling. |
| spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/raster/RasterDataSource.scala | New V2 entry point (shortName = "raster") and path rewrite/recursive-loading behavior. |
| spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/raster/RasterTable.scala | New FileTable for raster reads (schema + scan builder wiring). |
| spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/raster/RasterScanBuilder.scala | Implements limit + table sample pushdown for file selection (primarily when retile=false). |
| spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/raster/RasterInputPartition.scala | Defines input partitions used by the raster scan. |
| spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/raster/RasterPartitionReaderFactory.scala | Creates partition readers and wraps them with partition values. |
| spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/raster/RasterPartitionReader.scala | Implements GeoTIFF reading + optional tiling and writes rows via UnsafeRowWriter. |
| spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/raster/RasterOptions.scala | Adds read options (retile, tile sizes, padding) and exposes options class. |
| spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/raster/RasterConstructors.scala | Updates RS_TileExplode to use the new lazy tile iterator API. |
| spark/common/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister | Registers the new RasterDataSource for format("raster"). |
| docs/api/sql/Raster-loader.md | Documents the new raster data source API and deprecates the binaryFile-based approach. |
| common/src/main/java/org/apache/sedona/common/raster/TileGenerator.java | New lazy in-db tiling engine (tile + iterator). |
| common/src/main/java/org/apache/sedona/common/raster/RasterConstructors.java | Changes tiling API to return a lazy iterator and updates rsTile accordingly. |
| common/src/test/java/org/apache/sedona/common/raster/RasterConstructorsTest.java | Updates tests to consume the new lazy tile iterator API. |
💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.
.../common/src/main/scala/org/apache/spark/sql/sedona_sql/io/raster/RasterPartitionReader.scala
Outdated
Show resolved
Hide resolved
spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/raster/RasterScanBuilder.scala
Show resolved
Hide resolved
common/src/main/java/org/apache/sedona/common/raster/TileGenerator.java
Outdated
Show resolved
Hide resolved
...n/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/raster/RasterConstructors.scala
Show resolved
Hide resolved
spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/raster/RasterTable.scala
Show resolved
Hide resolved
…w comments - Replace byte-array-based raster loading with stream-based HadoopImageInputStream to support GeoTIFF files larger than 2GB (fixes getLen.toInt overflow) - Add HadoopImageInputStream adapter (common module) wrapping Hadoop FSDataInputStream into javax.imageio.stream.ImageInputStream for GeoTiffReader - Add fromGeoTiff(ImageInputStream) overload to RasterConstructors - Add hadoop-client as provided dependency in common/pom.xml - Add HadoopImageInputStreamTest with sequential, random, and unstable stream tests - Fix table sample pushdown to respect lowerBound (PR review comment #2) - Fix tile rect computation for images with non-zero minX/minY (PR review comment #3)
There was a problem hiding this comment.
Pull request overview
Copilot reviewed 17 out of 17 changed files in this pull request and generated 6 comments.
💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.
spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/raster/RasterOptions.scala
Show resolved
Hide resolved
spark/common/src/test/scala/org/apache/sedona/sql/rasterIOTest.scala
Outdated
Show resolved
Hide resolved
common/src/main/java/org/apache/sedona/common/raster/inputstream/HadoopImageInputStream.java
Show resolved
Hide resolved
...on/src/test/java/org/apache/sedona/common/raster/inputstream/HadoopImageInputStreamTest.java
Outdated
Show resolved
Hide resolved
...n/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/raster/RasterConstructors.scala
Show resolved
Hide resolved
.../common/src/main/scala/org/apache/spark/sql/sedona_sql/io/raster/RasterPartitionReader.scala
Outdated
Show resolved
Hide resolved
…fixes - RS_TileExplode: return Iterator.empty when raster is null instead of NPE - HadoopImageInputStream: guard against non-progressing stream (ret_len == 0) - HadoopImageInputStreamTest: fix UnstableInputStream to return at least 1 byte - RasterOptions: only enforce tileWidth/tileHeight pairing when retile=true - RasterPartitionReader: use floating-point division for aspect ratio check - rasterIOTest: remove explain(true) noise from CI logs
There was a problem hiding this comment.
Pull request overview
Copilot reviewed 17 out of 17 changed files in this pull request and generated 1 comment.
💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.
spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/raster/RasterDataSource.scala
Outdated
Show resolved
Hide resolved
There was a problem hiding this comment.
Pull request overview
Copilot reviewed 17 out of 17 changed files in this pull request and generated no new comments.
💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.
There was a problem hiding this comment.
Pull request overview
Copilot reviewed 17 out of 17 changed files in this pull request and generated 4 comments.
💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.
Did you read the Contributor Guide?
Is this PR related to a ticket?
Yes, this resolves #2672
What changes were proposed in this PR?
This PR adds a new Spark DataSourceV2-based raster data source that can load GeoTIFF files and optionally tile them on read. It replaces the legacy workflow of
binaryFile+RS_FromGeoTiff+RS_TileExplodewith a simpler one-liner API.Usage
Key benefits over the legacy binaryFile + RS_TileExplode approach
binaryFileapproach, the entire raster file is loaded as acontentcolumn (Array[Byte]). Spark has a hard 2 GB limit onByteArrayvalues inUnsafeRow, so rasters larger than 2 GB cannot be loaded at all. The new raster data source usesHadoopImageInputStreamto stream GeoTIFF data directly from Hadoop FS into GeoTools'GeoTiffReader, never materializing the entire file into a byte array. This removes the 2 GB limitation entirely.binaryFile+RS_TileExplode, the full raster is serialized into Spark'sUnsafeRow(viaRasterUDT.serialize, which deep-copies all pixel data), then deserialized byRS_TileExplode, then each tile is re-serialized. The new data source tiles rasters directly inside the partition reader - the full raster never entersUnsafeRow, so only small tiles are serialized. This reduces peak memory per task by approximately 50% (e.g., from 64 MB to 32 MB for each 2048x2048 double raster tiled into 256x256 tiles).SupportsPushDownLimitandSupportsPushDownTableSample. Whenretile=false,.limit(N)creates only N input partitions and reads only N files. Whenretile=true, limit is partially pushed (Spark'sLimitExecstops callingnext()after N tiles, so remaining tiles are never decoded). The old approach creates partitions for all files and relies on task cancellation.Large raster benchmark
Tested with a 3.2 GB global nightlight GeoTIFF (86,400 x 33,601 pixels, Float32, 256x256 internal tiles):
The old
binaryFileapproach fails withNegativeArraySizeExceptionon this file becausegetLen.toIntoverflows for files > 2.1 GB.Files changed
commonmodule):ImageInputStreamadapter wrapping Hadoop'sFSDataInputStreamfor GeoTiffReader. Supports(Path, Configuration)and(FSDataInputStream)constructors. Handles S3/cloud partial reads via robust read loop.generateTiles()returnsTileGenerator.TileIterator; addedfromGeoTiff(ImageInputStream)overload.HadoopImageInputStream->GeoTiffReader, createsGridCoverage2D, generates tiles lazily.hadoop-clientasprovidedscope dependency.How was this patch tested?
rasterIOTest.scala(20/20 pass including 10 existing write tests)HadoopImageInputStreamTest.java(sequential, random, unstable stream)Test coverage includes:
Did this PR include necessary documentation updates?
Yes. The Raster-loader.md page now documents the new raster data source API with Scala/Java/Python examples, available options, and deprecates the old binaryFile-based approach.