Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

RasterSourceRDD.tiledLayerRDD should have a configurable partition transform function #3450

Merged
merged 1 commit into from
Mar 12, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0
### Added
- Add new shading rules to make GT work with Spark 3.x [#3397](https://github.com/locationtech/geotrellis/pull/3397)
- Add Buffer Tile [#3419](https://github.com/locationtech/geotrellis/pull/3419)
- RasterSourceRDD.tiledLayerRDD should have a configurable partition transform function [#3450](https://github.com/locationtech/geotrellis/pull/3450)

### Changed
- Disambiguate withBufferTileFocalMethods implicit preserving bin compatibility [#3422](https://github.com/locationtech/geotrellis/pull/3422)
Expand Down
22 changes: 15 additions & 7 deletions spark/src/main/scala/geotrellis/spark/RasterSourceRDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -208,13 +208,25 @@ object RasterSourceRDD {
)(implicit sc: SparkContext): MultibandTileLayerRDD[SpatialKey] =
tiledLayerRDD(sources, layout, KeyExtractor.spatialKeyExtractor, resampleMethod, None, None)

/**
* On tiling more than a single MultibandTile may get into a group that correspond to the same key.
* By default the tiledLayerRDD function flattens all bands and converts every group into a single MultibandTile.
* To override this behavior it is possible to set the partitionTransform function, i.e.:
* {{{
* partitionTransform = {
* case iter if iter.nonEmpty => iter.map(_.tile).reduce(_ merge _)
* case _ => MultibandTile(Nil)
* }
* }}}
* */
def tiledLayerRDD[K: SpatialComponent: Boundable: ClassTag, M: Boundable](
sources: RDD[RasterSource],
layout: LayoutDefinition,
keyExtractor: KeyExtractor.Aux[K, M],
resampleMethod: ResampleMethod = NearestNeighbor,
rasterSummary: Option[RasterSummary[M]] = None,
partitioner: Option[Partitioner] = None
partitioner: Option[Partitioner] = None,
partitionTransform: Iterable[Raster[MultibandTile]] => MultibandTile = { iter => MultibandTile(iter.flatMap(_.tile.bands)) }
)(implicit sc: SparkContext): MultibandTileLayerRDD[K] = {
val summary = rasterSummary.getOrElse(RasterSummary.fromRDD(sources, keyExtractor.getMetadata))
val layerMetadata = summary.toTileLayerMetadata(layout, keyExtractor.getKey)
Expand All @@ -223,7 +235,7 @@ object RasterSourceRDD {
sources.map { rs =>
val m = keyExtractor.getMetadata(rs)
val tileKeyTransform: SpatialKey => K = { sk => keyExtractor.getKey(m, sk) }
rs.tileToLayout(layout, tileKeyTransform)
rs.tileToLayout(layout, tileKeyTransform, resampleMethod)
}

val rasterRegionRDD: RDD[(K, RasterRegion)] =
Expand All @@ -238,11 +250,7 @@ object RasterSourceRDD {
val tiledRDD: RDD[(K, MultibandTile)] =
rasterRegionRDD
.groupByKey(partitioner.getOrElse(SpatialPartitioner[K](partitionCount)))
.mapValues { iter =>
MultibandTile(
iter.flatMap { _.raster.toSeq.flatMap { _.tile.bands } }
)
}
.mapValues { iter => partitionTransform(iter.flatMap(_.raster.toSeq)) }

ContextRDD(tiledRDD, layerMetadata)
}
Expand Down