Skip to content

Commit 9a9a806

Browse files
committed
RasterSourceRDD.tiledLayerRDD should have a configurable partition transform function
1 parent d0c4af6 commit 9a9a806

2 files changed

Lines changed: 16 additions & 7 deletions

File tree

CHANGELOG.md

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -9,6 +9,7 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0
99
### Added
1010
- Add new shading rules to make GT work with Spark 3.x [#3397](https://github.com/locationtech/geotrellis/pull/3397)
1111
- Add Buffer Tile [#3419](https://github.com/locationtech/geotrellis/pull/3419)
12+
- RasterSourceRDD.tiledLayerRDD should have a configurable partition transform function [#3450](https://github.com/locationtech/geotrellis/pull/3450)
1213

1314
### Changed
1415
- Disambiguate withBufferTileFocalMethods implicit preserving bin compatibility [#3422](https://github.com/locationtech/geotrellis/pull/3422)

spark/src/main/scala/geotrellis/spark/RasterSourceRDD.scala

Lines changed: 15 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -208,13 +208,25 @@ object RasterSourceRDD {
208208
)(implicit sc: SparkContext): MultibandTileLayerRDD[SpatialKey] =
209209
tiledLayerRDD(sources, layout, KeyExtractor.spatialKeyExtractor, resampleMethod, None, None)
210210

211+
/**
212+
* On tiling more than a single MultibandTile may get into a group that correspond to the same key.
213+
* By default the tiledLayerRDD function flattens all bands and converts every group into a single MultibandTile.
214+
* To override this behavior it is possible to set the partitionTransform function, i.e.:
215+
* {{{
216+
* partitionTransform = {
217+
* case iter if iter.nonEmpty => iter.map(_.tile).reduce(_ merge _)
218+
* case _ => MultibandTile(Nil)
219+
* }
220+
* }}}
221+
* */
211222
def tiledLayerRDD[K: SpatialComponent: Boundable: ClassTag, M: Boundable](
212223
sources: RDD[RasterSource],
213224
layout: LayoutDefinition,
214225
keyExtractor: KeyExtractor.Aux[K, M],
215226
resampleMethod: ResampleMethod = NearestNeighbor,
216227
rasterSummary: Option[RasterSummary[M]] = None,
217-
partitioner: Option[Partitioner] = None
228+
partitioner: Option[Partitioner] = None,
229+
partitionTransform: Iterable[Raster[MultibandTile]] => MultibandTile = { iter => MultibandTile(iter.flatMap { _.tile.bands }) }
218230
)(implicit sc: SparkContext): MultibandTileLayerRDD[K] = {
219231
val summary = rasterSummary.getOrElse(RasterSummary.fromRDD(sources, keyExtractor.getMetadata))
220232
val layerMetadata = summary.toTileLayerMetadata(layout, keyExtractor.getKey)
@@ -223,7 +235,7 @@ object RasterSourceRDD {
223235
sources.map { rs =>
224236
val m = keyExtractor.getMetadata(rs)
225237
val tileKeyTransform: SpatialKey => K = { sk => keyExtractor.getKey(m, sk) }
226-
rs.tileToLayout(layout, tileKeyTransform)
238+
rs.tileToLayout(layout, tileKeyTransform, resampleMethod)
227239
}
228240

229241
val rasterRegionRDD: RDD[(K, RasterRegion)] =
@@ -238,11 +250,7 @@ object RasterSourceRDD {
238250
val tiledRDD: RDD[(K, MultibandTile)] =
239251
rasterRegionRDD
240252
.groupByKey(partitioner.getOrElse(SpatialPartitioner[K](partitionCount)))
241-
.mapValues { iter =>
242-
MultibandTile(
243-
iter.flatMap { _.raster.toSeq.flatMap { _.tile.bands } }
244-
)
245-
}
253+
.mapValues { iter => partitionTransform(iter.flatMap(_.raster.toSeq)) }
246254

247255
ContextRDD(tiledRDD, layerMetadata)
248256
}

0 commit comments

Comments
 (0)