Skip to content

Commit

Permalink
Add dynamic ZFactor for slope calculation (#3014)
Browse files Browse the repository at this point in the history
* Add dynamic ZFactor for slope calculation

Signed-off-by: Jacob Bouffard <jbouffard@azavea.com>
  • Loading branch information
jbouffard authored and pomadchin committed Jul 19, 2019
1 parent 0e60b86 commit c6f424d
Show file tree
Hide file tree
Showing 11 changed files with 230 additions and 29 deletions.
3 changes: 3 additions & 0 deletions docs/CHANGELOG.rst
Original file line number Diff line number Diff line change
Expand Up @@ -15,11 +15,13 @@ API Changes & Project structure changes
- ``geotrellis-raster``

- **Change:** ``geotrellis.raster.summary.polygonal.[Multi]TilePolygonalSummaryHandler`` replaced with ``geotrellis.raster.summary.polygonal.PolygonalSummary. Users should expect to implement concrete subclasses of ``geotrellis.raster.summary.GridVisitor`` and pass those to the new polygonalSummary methods. There are a number of default GridVisitor implementations provided for simple operations in `geotrellis.raster.summary.visitors``
- **New:** Created the ``geotrellis.raster.mapalgebra.focal.ZFactor`` to improve slope calculations (`#3014 <https://github.com/locationtech/geotrellis/pull/3014>`_).

- ``geotrellis.layer``

- **Change:** Replaced ``Mergable`` with cats' ``Semigroup``
- **New:** A sparseStitch method on ``geotrellis.layer.stitch.SpatialTileLayoutCollectionStitchMethods``. Note that ``SpatialTileLayoutCollectionStitchMethods`` now has the additional constraint ``geotrellis.raster.prototype.TilePrototypeMethods`` on type ``V``. This should be transparent for users of the ``geotrellis.raster.Tile`` and ``geotrellis.raster.MultibandTile`` types.
- **Change:** The ``slope`` focal method now requires a ``ZFactor`` instance.

- ``geotrellis.util``, ``geotrellis.store``, ``geotrellis.store.s3``

Expand Down Expand Up @@ -76,6 +78,7 @@ API Changes & Project structure changes
- ``SpatialComponent``
- ``TemporalComponent``
- **Change:** Polygonal summaries on raster RDDs of ``RDD[(SpatialKey, T <: Grid[Int])] with Metadata[TileLayerMetadata[SpatialKey]]`` can now be performed with far less boilerplate using the same visitor pattern as the new raster polygonal summary API. See ``RDDPolygonalSummary.scala`` for additional details.
- **Change:** The ``slope`` focal method now requires a ``ZFactor`` instance.

- ``geotrellis.spark.tiling``

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,12 +16,13 @@

package geotrellis.layer.mapalgebra.focal

import geotrellis.layer.{SpatialComponent, TileBounds}
import geotrellis.raster.buffer.BufferedTile
import geotrellis.layer._
import geotrellis.vector._
import geotrellis.raster._
import geotrellis.raster.buffer.BufferedTile
import geotrellis.raster.mapalgebra.focal._
import geotrellis.layer._
import geotrellis.util.MethodExtensions
import geotrellis.util._


object CollectionFocalOperation {
private def mapOverBufferedTiles[K: SpatialComponent](
Expand All @@ -31,7 +32,7 @@ object CollectionFocalOperation {
calc: (Tile, Option[GridBounds[Int]]) => Tile
): Seq[(K, Tile)] =
bufferedTiles
.map { case (key, BufferedTile(tile, gridBounds)) => key -> calc(tile, Some(gridBounds)) }
.mapValues { case BufferedTile(tile, gridBounds) => calc(tile, Some(gridBounds)) }

def apply[K: SpatialComponent](
seq: Seq[(K, Tile)],
Expand All @@ -44,22 +45,65 @@ object CollectionFocalOperation {
mapOverBufferedTiles(seq.bufferTiles(neighborhood.extent), neighborhood)(calc)

def apply[K: SpatialComponent](
rdd: Seq[(K, Tile)],
collection: Seq[(K, Tile)],
neighborhood: Neighborhood,
layerBounds: TileBounds
)(
calc: (Tile, Option[GridBounds[Int]]) => Tile
): Seq[(K, Tile)] =
mapOverBufferedTiles(rdd.bufferTiles(neighborhood.extent, layerBounds), neighborhood)(calc)
mapOverBufferedTiles(collection.bufferTiles(neighborhood.extent, layerBounds), neighborhood)(calc)

def apply[K: SpatialComponent](
rasterCollection: TileLayerCollection[K],
neighborhood: Neighborhood
)(
calc: (Tile, Option[GridBounds[Int]]) => Tile
): TileLayerCollection[K] =
rasterCollection.withContext { rdd =>
apply(rdd, neighborhood, rasterCollection.metadata.tileBounds)(calc)
rasterCollection.withContext { collection =>
apply(collection, neighborhood, rasterCollection.metadata.tileBounds)(calc)
}

private def applyOnRaster[K: SpatialComponent: GetComponent[?, SpatialKey]](
bufferedTiles: Seq[(K, BufferedTile[Tile])],
neighborhood: Neighborhood,
keyToExtent: SpatialKey => Extent
)(
calc: (Raster[Tile], Option[GridBounds[Int]]) => Tile
): Seq[(K, Tile)] =
bufferedTiles
.map { case (key, BufferedTile(tile, gridBounds)) =>
val spatialKey = key.getComponent[SpatialKey]

key -> calc(Raster(tile, keyToExtent(spatialKey)), Some(gridBounds))
}

def applyOnRaster[K: SpatialComponent: GetComponent[?, SpatialKey]](
collection: Seq[(K, Tile)],
neighborhood: Neighborhood,
layerBounds: TileBounds,
keyToExtent: SpatialKey => Extent
)(
calc: (Raster[Tile], Option[GridBounds[Int]]) => Tile
): Seq[(K, Tile)] =
applyOnRaster(
collection.bufferTiles(neighborhood.extent, layerBounds),
neighborhood,
keyToExtent
)(calc)

def applyOnRaster[K: SpatialComponent: GetComponent[?, SpatialKey]](
rasterCollection: TileLayerCollection[K],
neighborhood: Neighborhood
)(
calc: (Raster[Tile], Option[GridBounds[Int]]) => Tile
): TileLayerCollection[K] =
rasterCollection.withContext { collection =>
applyOnRaster(
collection,
neighborhood,
rasterCollection.metadata.tileBounds,
(key: SpatialKey) => rasterCollection.metadata.mapTransform.keyToExtent(key)
)(calc)
}
}

Expand All @@ -74,4 +118,12 @@ abstract class CollectionFocalOperation[K: SpatialComponent] extends MethodExten
calc(tile, bounds, cellSize)
}
}

def focalWithExtents(n: Neighborhood)(calc: (Raster[Tile], Option[GridBounds[Int]], CellSize) => Tile): TileLayerCollection[K] = {
val cellSize = self.metadata.layout.cellSize

CollectionFocalOperation.applyOnRaster(self, n){ (raster, bounds) =>
calc(raster, bounds, cellSize)
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ package geotrellis.layer.mapalgebra.focal

import geotrellis.raster._
import geotrellis.raster.Neighborhoods.Square
import geotrellis.raster.mapalgebra.focal.{Kernel, Aspect, Sum, Max, Min, Mean, Median, Mode, StandardDeviation, Convolve, Slope}
import geotrellis.raster.mapalgebra.focal.{Kernel, Aspect, Sum, Max, Min, Mean, Median, Mode, StandardDeviation, Convolve, Slope, ZFactor}

trait FocalTileLayerCollectionMethods[K] extends CollectionFocalOperation[K] {

Expand Down Expand Up @@ -47,10 +47,12 @@ trait FocalTileLayerCollectionMethods[K] extends CollectionFocalOperation[K] {
*
* @see [[geotrellis.raster.mapalgebra.focal.Slope]]
*/
def slope(zFactor: Double = 1.0, target: TargetCell = TargetCell.All) = {
def slope(zFactor: ZFactor, target: TargetCell = TargetCell.All) = {
val n = Square(1)
focalWithCellSize(n) { (tile, bounds, cellSize) =>
Slope(tile, n, bounds, cellSize, zFactor, target)
focalWithExtents(n) { (raster, bounds, cellSize) =>
val zValue = zFactor.fromExtent(raster.extent)

Slope(raster.tile, n, bounds, cellSize, zValue, target)
}.mapContext(_.copy(cellType = DoubleConstantNoDataCellType))
}
}
2 changes: 2 additions & 0 deletions project/Dependencies.scala
Original file line number Diff line number Diff line change
Expand Up @@ -115,4 +115,6 @@ object Dependencies {
val scalaj = "org.scalaj" %% "scalaj-http" % "2.4.1"

val scalapbRuntime = "com.thesamet.scalapb" %% "scalapb-runtime" % scalapb.compiler.Version.scalapbVersion

val squants = "org.typelevel" %% "squants" % "1.4.0"
}
1 change: 1 addition & 0 deletions project/Settings.scala
Original file line number Diff line number Diff line change
Expand Up @@ -413,6 +413,7 @@ object Settings {
jts,
catsCore,
spire,
squants,
monocleCore,
monocleMacro,
scalaXml,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,93 @@
/*
* Copyright 2019 Azavea
*
* Licensed 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 geotrellis.raster.mapalgebra.focal

import geotrellis.vector.{Extent, Point}

import org.apache.commons.math3.analysis.interpolation._

import squants.space.{LengthUnit, Feet, Meters}


/** Produces a ZFactor for a given point using the prescribed
* conversion method.
*
* @param produceZFactor A function that takes a latitude point and
* converts it to its corresponding ZFactor.
*/
class ZFactor(produceZFactor: Double => Double) extends Serializable {
/** Produces the associated ZFactor for the given location.
*
* @param extent The center point of this [[Extent]] will be used to
* determine the ZFactor.
*/
def fromExtent(extent: Extent): Double =
fromPoint(extent.center)

/** Produces the associated ZFactor for the given `Point`'s Y coordinate. */
def fromPoint(point: Point): Double =
fromLatitude(point.y)

def fromLatitude(lat: Double): Double =
produceZFactor(lat)
}

/** When creating a ZFactor instance, the projection of the target
* Tile(s) needs to be taken into account. If the Tiles are in
* LatLng, then the conversion between Latitude and ZFactor can
* already be calculated. Otherwise, one will need to supply the
* transformation required to produce the ZFactor.
*/
object ZFactor {
final val LAT_LNG_FEET_AT_EQUATOR = 365217.6
final val LAT_LNG_METERS_AT_EQUATOR = 11320

/** Creates a [[ZFactor]] instance. specifically for Tiles
* that are in LatLng.
*
* @param unit The unit of measure that the Tile(s) are in. Only two
* `LengthUnit`s are supported, `Feet` and `Meters`.
*/
def forLatLng(unit: LengthUnit): ZFactor =
unit match {
case Feet =>
ZFactor((lat: Double) => 1 / (LAT_LNG_FEET_AT_EQUATOR * math.cos(math.toRadians(lat))))
case Meters =>
ZFactor((lat: Double) => 1 / (LAT_LNG_METERS_AT_EQUATOR * math.cos(math.toRadians(lat))))
case _ =>
throw new Exception(s"Could not create a ZFactor instance from the following unit: ${unit}. Please use either Feet or Meters")
}

/** Creates a [[ZFactor]] instance which uses linear interplation to calculate
* ZFactors. The linear interploation itself is derived from the values given.
*
* @param mappedLatitudes A Map that maps latitudes to ZFactors. It is not required
* to supply a ZFactor for every latitude intersected by the Tile(s). Rather,
* based on the values given, a linear interpolation will be created and
* any latitude not mapped will have its associated ZFactor derived from
* that interpolation.
*/
def interpolateFromTable(mappedLatitudes: Map[Double, Double]): ZFactor = {
val interp = new LinearInterpolator()
val spline = interp.interpolate(mappedLatitudes.keys.toArray, mappedLatitudes.values.toArray)

ZFactor((lat: Double) => spline.value(lat))
}

def apply(produceZFactor: Double => Double): ZFactor =
new ZFactor(produceZFactor)
}
2 changes: 2 additions & 0 deletions raster/src/main/scala/geotrellis/raster/package.scala
Original file line number Diff line number Diff line change
Expand Up @@ -99,6 +99,8 @@ package object raster extends Implicits {

val Stitcher = stitch.Stitcher

val ZFactor = mapalgebra.focal.ZFactor

// Keep constant values in sync with macro functions
@inline final val byteNODATA = Byte.MinValue
@inline final val ubyteNODATA = 0.toByte
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,13 +16,14 @@

package geotrellis.spark.mapalgebra.focal

import geotrellis.layer.{SpatialComponent, TileBounds}
import geotrellis.layer.{SpatialComponent, TileBounds, SpatialKey}
import geotrellis.vector.Extent
import geotrellis.raster._
import geotrellis.raster.buffer.BufferedTile
import geotrellis.raster.mapalgebra.focal._
import geotrellis.spark._
import geotrellis.spark.buffer._
import geotrellis.util.MethodExtensions
import geotrellis.util._

import org.apache.spark.rdd.RDD
import org.apache.spark.Partitioner
Expand Down Expand Up @@ -55,6 +56,41 @@ object FocalOperation {
rasterRDD.withContext { rdd =>
apply(rdd, neighborhood, rasterRDD.metadata.tileBounds, partitioner)(calc)
}

private def applyOnRaster[K: SpatialComponent: ClassTag: GetComponent[?, SpatialKey]]
(bufferedTiles: RDD[(K, BufferedTile[Tile])], neighborhood: Neighborhood, keyToExtent: SpatialKey => Extent)
(calc: (Raster[Tile], Option[GridBounds[Int]]) => Tile): RDD[(K, Tile)] =
bufferedTiles
.mapPartitions({ case partition =>
partition.map { case (k, BufferedTile(tile, gridBounds)) =>
val spatialKey = k.getComponent[SpatialKey]

k -> calc(Raster(tile, keyToExtent(spatialKey)), Some(gridBounds))
}
}, preservesPartitioning = true)

def applyOnRaster[K: SpatialComponent: ClassTag: GetComponent[?, SpatialKey]](
rdd: RDD[(K, Tile)],
neighborhood: Neighborhood,
layerBounds: TileBounds,
partitioner: Option[Partitioner],
keyToExtent: SpatialKey => Extent
)(calc: (Raster[Tile], Option[GridBounds[Int]]) => Tile): RDD[(K, Tile)] =
applyOnRaster(rdd.bufferTiles(neighborhood.extent, layerBounds, partitioner), neighborhood, keyToExtent)(calc)

def applyOnRaster[
K: SpatialComponent: ClassTag: GetComponent[?, SpatialKey]
](rasterRDD: TileLayerRDD[K], neighborhood: Neighborhood, partitioner: Option[Partitioner])
(calc: (Raster[Tile], Option[GridBounds[Int]]) => Tile): TileLayerRDD[K] =
rasterRDD.withContext { rdd =>
applyOnRaster(
rdd,
neighborhood,
rasterRDD.metadata.tileBounds,
partitioner,
(key: SpatialKey) => rasterRDD.metadata.mapTransform.keyToExtent(key)
)(calc)
}
}

abstract class FocalOperation[K: SpatialComponent: ClassTag] extends MethodExtensions[TileLayerRDD[K]] {
Expand All @@ -68,4 +104,11 @@ abstract class FocalOperation[K: SpatialComponent: ClassTag] extends MethodExten
val cellSize = self.metadata.layout.cellSize
FocalOperation(self, n, partitioner){ (tile, bounds) => calc(tile, bounds, cellSize) }
}

def focalWithExtents(n: Neighborhood, partitioner: Option[Partitioner])
(calc: (Raster[Tile], Option[GridBounds[Int]], CellSize) => Tile): TileLayerRDD[K] = {
val cellSize = self.metadata.layout.cellSize

FocalOperation.applyOnRaster(self, n, partitioner){ (raster, bounds) => calc(raster, bounds, cellSize) }
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ package geotrellis.spark.mapalgebra.focal

import geotrellis.raster._
import geotrellis.raster.Neighborhoods.Square
import geotrellis.raster.mapalgebra.focal.{Kernel, Aspect, Sum, Max, Min, Mean, Median, Mode, StandardDeviation, Convolve, Slope}
import geotrellis.raster.mapalgebra.focal.{Kernel, Aspect, Sum, Max, Min, Mean, Median, Mode, StandardDeviation, Convolve, Slope, ZFactor}
import geotrellis.spark._

import org.apache.spark.Partitioner
Expand Down Expand Up @@ -106,13 +106,15 @@ trait FocalTileLayerRDDMethods[K] extends FocalOperation[K] {
* @see [[geotrellis.raster.mapalgebra.focal.Slope]]
*/
def slope(
zFactor: Double = 1.0,
zFactor: ZFactor,
target: TargetCell = TargetCell.All,
partitioner: Option[Partitioner] = None
) = {
val n = Square(1)
focalWithCellSize(n, partitioner) { (tile, bounds, cellSize) =>
Slope(tile, n, bounds, cellSize, zFactor, target)
focalWithExtents(n, partitioner) { (raster, bounds, cellSize) =>
val zValue = zFactor.fromExtent(raster.extent)

Slope(raster.tile, n, bounds, cellSize, zValue, target)
}.mapContext(_.copy(cellType = DoubleConstantNoDataCellType))
}
}
Loading

0 comments on commit c6f424d

Please sign in to comment.