From e2b55725baefd3ae6a8221e0ab75faa3d701e333 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Mon, 21 Oct 2019 12:11:42 -0400 Subject: [PATCH 01/94] Initial port to Geotrellis 3.0 --- .../bench/CatalystSerializerBench.scala | 2 +- .../bench/TileCellScanBench.scala | 3 +- build.sbt | 7 +- .../rasterframes/PairRDDConverter.scala | 2 +- .../rasterframes/StandardColumns.scala | 2 +- .../encoders/ProjectedExtentEncoder.scala | 2 +- .../encoders/StandardEncoders.scala | 5 +- .../encoders/StandardSerializers.scala | 10 +- .../TemporalProjectedExtentEncoder.scala | 5 +- .../encoders/TileLayerMetadataEncoder.scala | 4 +- .../expressions/DynamicExtractors.scala | 2 +- .../expressions/OnCellGridExpression.scala | 2 +- .../expressions/SpatialRelation.scala | 2 +- .../expressions/accessors/GetCRS.scala | 2 +- .../expressions/accessors/GetCellType.scala | 2 +- .../expressions/accessors/GetDimensions.scala | 2 +- .../expressions/accessors/GetGeometry.scala | 2 +- .../ProjectedLayerMetadataAggregate.scala | 3 +- .../aggregates/TileRasterizerAggregate.scala | 2 +- .../expressions/generators/ExplodeTiles.scala | 2 +- .../generators/RasterSourceToRasterRefs.scala | 2 +- .../transformers/ExtentToGeometry.scala | 2 +- .../extensions/ContextRDDMethods.scala | 19 +- .../extensions/DataFrameMethods.scala | 5 +- .../rasterframes/extensions/Implicits.scala | 11 +- .../extensions/ProjectedRasterMethods.scala | 19 +- .../extensions/RFSpatialColumnMethods.scala | 9 +- .../extensions/RasterFrameLayerMethods.scala | 6 +- .../extensions/ReprojectToLayer.scala | 7 +- .../rasterframes/functions/package.scala | 3 +- .../rasterframes/jts/Implicits.scala | 2 +- .../jts/ReprojectionTransformer.scala | 2 +- .../model/FixedRasterExtent.scala | 278 ------------- .../rasterframes/model/LazyCRS.scala | 13 +- .../rasterframes/model/TileDimensions.scala | 5 +- .../rasterframes/rasterframes.scala | 5 +- .../ref/DelegatingRasterSource.scala | 14 +- .../rasterframes/ref/GDALRasterSource.scala | 11 +- .../ref/HadoopGeoTiffRasterSource.scala | 2 +- .../ref/InMemoryRasterSource.scala | 2 +- .../ref/JVMGeoTiffRasterSource.scala | 3 +- .../ref/ProjectedRasterLike.scala | 2 +- .../ref/RangeReaderRasterSource.scala | 10 +- .../rasterframes/ref/RasterRef.scala | 10 +- .../rasterframes/ref/RasterSource.scala | 16 +- .../rasterframes/ref/SimpleRasterInfo.scala | 17 +- .../util/GeoTiffInfoSupport.scala | 10 +- .../rasterframes/util/JsonCodecs.scala | 373 ++++++++++++++++++ .../rasterframes/util/RFKryoRegistrator.scala | 7 +- .../rasterframes/util/SubdivideSupport.scala | 8 +- .../rasterframes/util/debug/package.scala | 27 -- .../rasterframes/util/package.scala | 104 ++--- .../scala/examples/CreatingRasterFrames.scala | 4 +- core/src/test/scala/examples/Exporting.scala | 16 +- .../test/scala/examples/LocalArithmetic.scala | 9 +- .../scala/examples/MakeTargetRaster.scala | 6 +- core/src/test/scala/examples/Masking.scala | 11 +- core/src/test/scala/examples/NDVI.scala | 10 +- .../rasterframes/ExplodeSpec.scala | 2 +- .../rasterframes/ExtensionMethodSpec.scala | 5 +- .../rasterframes/GeometryFunctionsSpec.scala | 19 +- .../rasterframes/RasterFrameSpec.scala | 8 +- .../rasterframes/RasterFunctionsSpec.scala | 2 +- .../rasterframes/ReprojectGeometrySpec.scala | 4 +- .../rasterframes/SpatialKeySpec.scala | 6 +- .../locationtech/rasterframes/TestData.scala | 14 +- .../rasterframes/TileAssemblerSpec.scala | 2 +- .../rasterframes/TileUDTSpec.scala | 8 +- .../encoders/CatalystSerializerSpec.scala | 3 +- .../rasterframes/encoders/EncodingSpec.scala | 4 +- .../ProjectedLayerMetadataAggregateSpec.scala | 2 +- .../expressions/XZ2IndexerSpec.scala | 2 +- .../rasterframes/ref/RasterRefSpec.scala | 2 +- .../rasterframes/ref/RasterSourceSpec.scala | 10 +- .../datasource/geotiff/GeoTiffRelation.scala | 19 +- .../geotrellis/GeoTrellisCatalog.scala | 21 +- .../GeoTrellisLayerDataSource.scala | 6 +- .../geotrellis/GeoTrellisRelation.scala | 22 +- .../datasource/geotrellis/Layer.scala | 2 +- .../geotrellis/TileFeatureSupport.scala | 6 +- .../datasource/geotrellis/package.scala | 2 +- .../geotiff/GeoTiffDataSourceSpec.scala | 4 +- .../geotrellis/GeoTrellisCatalogSpec.scala | 6 +- .../geotrellis/GeoTrellisDataSourceSpec.scala | 22 +- .../geotrellis/TileFeatureSupportSpec.scala | 9 +- .../awspds/L8CatalogRelationTest.scala | 2 +- project/RFDependenciesPlugin.scala | 14 +- .../rasterframes/py/PyRFContext.scala | 6 +- 88 files changed, 725 insertions(+), 632 deletions(-) delete mode 100644 core/src/main/scala/org/locationtech/rasterframes/model/FixedRasterExtent.scala create mode 100644 core/src/main/scala/org/locationtech/rasterframes/util/JsonCodecs.scala diff --git a/bench/src/main/scala/org/locationtech/rasterframes/bench/CatalystSerializerBench.scala b/bench/src/main/scala/org/locationtech/rasterframes/bench/CatalystSerializerBench.scala index 12a6b0486..b24042b56 100644 --- a/bench/src/main/scala/org/locationtech/rasterframes/bench/CatalystSerializerBench.scala +++ b/bench/src/main/scala/org/locationtech/rasterframes/bench/CatalystSerializerBench.scala @@ -46,7 +46,7 @@ class CatalystSerializerBench extends SparkEnv { @Setup(Level.Trial) def setupData(): Unit = { - crsEnc = StandardEncoders.crsEncoder.resolveAndBind() + crsEnc = StandardEncoders.crsSparkEncoder.resolveAndBind() } @Benchmark diff --git a/bench/src/main/scala/org/locationtech/rasterframes/bench/TileCellScanBench.scala b/bench/src/main/scala/org/locationtech/rasterframes/bench/TileCellScanBench.scala index 350ac811a..8de95f56c 100644 --- a/bench/src/main/scala/org/locationtech/rasterframes/bench/TileCellScanBench.scala +++ b/bench/src/main/scala/org/locationtech/rasterframes/bench/TileCellScanBench.scala @@ -23,6 +23,7 @@ package org.locationtech.rasterframes.bench import java.util.concurrent.TimeUnit +import geotrellis.raster.Dimensions import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.rf.TileUDT import org.locationtech.rasterframes.tiles.InternalRowTile @@ -56,7 +57,7 @@ class TileCellScanBench extends SparkEnv { @Benchmark def deserializeRead(): Double = { val tile = TileType.deserialize(tileRow) - val (cols, rows) = tile.dimensions + val Dimensions(cols, rows) = tile.dimensions tile.getDouble(cols - 1, rows - 1) + tile.getDouble(cols/2, rows/2) + tile.getDouble(0, 0) diff --git a/build.sbt b/build.sbt index f941ea060..32c7eb531 100644 --- a/build.sbt +++ b/build.sbt @@ -50,17 +50,20 @@ lazy val core = project .settings( moduleName := "rasterframes", libraryDependencies ++= Seq( + `slf4j-api`, shapeless, `jts-core`, + `spray-json`, geomesa("z3").value, geomesa("spark-jts").value, - `geotrellis-contrib-vlm`, - `geotrellis-contrib-gdal`, +// `geotrellis-contrib-vlm`, +// `geotrellis-contrib-gdal`, spark("core").value % Provided, spark("mllib").value % Provided, spark("sql").value % Provided, geotrellis("spark").value, geotrellis("raster").value, + geotrellis("gdal").value, geotrellis("s3").value, geotrellis("spark-testkit").value % Test excludeAll ( ExclusionRule(organization = "org.scalastic"), diff --git a/core/src/main/scala/org/locationtech/rasterframes/PairRDDConverter.scala b/core/src/main/scala/org/locationtech/rasterframes/PairRDDConverter.scala index 658c0d65d..b4a2fe8f0 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/PairRDDConverter.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/PairRDDConverter.scala @@ -23,7 +23,7 @@ package org.locationtech.rasterframes import org.locationtech.rasterframes.util._ import geotrellis.raster.{MultibandTile, Tile, TileFeature} -import geotrellis.spark.{SpaceTimeKey, SpatialKey} +import geotrellis.layer._ import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.rf.TileUDT diff --git a/core/src/main/scala/org/locationtech/rasterframes/StandardColumns.scala b/core/src/main/scala/org/locationtech/rasterframes/StandardColumns.scala index 2e82ab356..4ae29f1d3 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/StandardColumns.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/StandardColumns.scala @@ -25,7 +25,7 @@ import java.sql.Timestamp import geotrellis.proj4.CRS import geotrellis.raster.Tile -import geotrellis.spark.{SpatialKey, TemporalKey} +import geotrellis.layer._ import geotrellis.vector.{Extent, ProjectedExtent} import org.apache.spark.sql.functions.col import org.locationtech.jts.geom.{Point => jtsPoint, Polygon => jtsPolygon} diff --git a/core/src/main/scala/org/locationtech/rasterframes/encoders/ProjectedExtentEncoder.scala b/core/src/main/scala/org/locationtech/rasterframes/encoders/ProjectedExtentEncoder.scala index f5b078159..d366adbd2 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/encoders/ProjectedExtentEncoder.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/encoders/ProjectedExtentEncoder.scala @@ -32,6 +32,6 @@ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder */ object ProjectedExtentEncoder { def apply(): ExpressionEncoder[ProjectedExtent] = { - DelegatingSubfieldEncoder("extent" -> extentEncoder, "crs" -> crsEncoder) + DelegatingSubfieldEncoder("extent" -> extentEncoder, "crs" -> crsSparkEncoder) } } diff --git a/core/src/main/scala/org/locationtech/rasterframes/encoders/StandardEncoders.scala b/core/src/main/scala/org/locationtech/rasterframes/encoders/StandardEncoders.scala index 256da58d8..b7b3211f5 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/encoders/StandardEncoders.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/encoders/StandardEncoders.scala @@ -28,8 +28,7 @@ import org.locationtech.rasterframes.stats.{CellHistogram, CellStatistics, Local import org.locationtech.jts.geom.Envelope import geotrellis.proj4.CRS import geotrellis.raster.{CellSize, CellType, Raster, Tile, TileLayout} -import geotrellis.spark.tiling.LayoutDefinition -import geotrellis.spark.{KeyBounds, SpaceTimeKey, SpatialKey, TemporalKey, TemporalProjectedExtent, TileLayerMetadata} +import geotrellis.layer._ import geotrellis.vector.{Extent, ProjectedExtent} import org.apache.spark.sql.{Encoder, Encoders} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder @@ -53,7 +52,7 @@ trait StandardEncoders extends SpatialEncoders { implicit def singlebandTileEncoder: ExpressionEncoder[Tile] = ExpressionEncoder() implicit def rasterEncoder: ExpressionEncoder[Raster[Tile]] = ExpressionEncoder() implicit def tileLayerMetadataEncoder[K: TypeTag]: ExpressionEncoder[TileLayerMetadata[K]] = TileLayerMetadataEncoder() - implicit def crsEncoder: ExpressionEncoder[CRS] = CRSEncoder() + implicit def crsSparkEncoder: ExpressionEncoder[CRS] = CRSEncoder() implicit def projectedExtentEncoder: ExpressionEncoder[ProjectedExtent] = ProjectedExtentEncoder() implicit def temporalProjectedExtentEncoder: ExpressionEncoder[TemporalProjectedExtent] = TemporalProjectedExtentEncoder() implicit def cellTypeEncoder: ExpressionEncoder[CellType] = CellTypeEncoder() diff --git a/core/src/main/scala/org/locationtech/rasterframes/encoders/StandardSerializers.scala b/core/src/main/scala/org/locationtech/rasterframes/encoders/StandardSerializers.scala index 1983f8bb9..79eb65255 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/encoders/StandardSerializers.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/encoders/StandardSerializers.scala @@ -24,8 +24,8 @@ package org.locationtech.rasterframes.encoders import com.github.blemale.scaffeine.Scaffeine import geotrellis.proj4.CRS import geotrellis.raster._ -import geotrellis.spark._ -import geotrellis.spark.tiling.LayoutDefinition +import geotrellis.layer._ + import geotrellis.vector._ import org.apache.spark.sql.types._ import org.locationtech.jts.geom.Envelope @@ -71,7 +71,7 @@ trait StandardSerializers { ) } - implicit val gridBoundsSerializer: CatalystSerializer[GridBounds] = new CatalystSerializer[GridBounds] { + implicit val gridBoundsSerializer: CatalystSerializer[GridBounds[Int]] = new CatalystSerializer[GridBounds[Int]] { override val schema: StructType = StructType(Seq( StructField("colMin", IntegerType, false), StructField("rowMin", IntegerType, false), @@ -79,11 +79,11 @@ trait StandardSerializers { StructField("rowMax", IntegerType, false) )) - override protected def to[R](t: GridBounds, io: CatalystIO[R]): R = io.create( + override protected def to[R](t: GridBounds[Int], io: CatalystIO[R]): R = io.create( t.colMin, t.rowMin, t.colMax, t.rowMax ) - override protected def from[R](t: R, io: CatalystIO[R]): GridBounds = GridBounds( + override protected def from[R](t: R, io: CatalystIO[R]): GridBounds[Int] = GridBounds[Int]( io.getInt(t, 0), io.getInt(t, 1), io.getInt(t, 2), diff --git a/core/src/main/scala/org/locationtech/rasterframes/encoders/TemporalProjectedExtentEncoder.scala b/core/src/main/scala/org/locationtech/rasterframes/encoders/TemporalProjectedExtentEncoder.scala index f69f7f160..5d41e6386 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/encoders/TemporalProjectedExtentEncoder.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/encoders/TemporalProjectedExtentEncoder.scala @@ -22,7 +22,7 @@ package org.locationtech.rasterframes.encoders import org.locationtech.rasterframes._ -import geotrellis.spark.TemporalProjectedExtent +import geotrellis.layer._ import org.apache.spark.sql.Encoders import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder @@ -34,9 +34,10 @@ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder */ object TemporalProjectedExtentEncoder { def apply(): ExpressionEncoder[TemporalProjectedExtent] = { + import StandardEncoders.crsSparkEncoder DelegatingSubfieldEncoder( "extent" -> extentEncoder, - "crs" -> crsEncoder, + "crs" -> crsSparkEncoder, "instant" -> Encoders.scalaLong.asInstanceOf[ExpressionEncoder[Long]] ) } diff --git a/core/src/main/scala/org/locationtech/rasterframes/encoders/TileLayerMetadataEncoder.scala b/core/src/main/scala/org/locationtech/rasterframes/encoders/TileLayerMetadataEncoder.scala index 2f59ea451..56f845db3 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/encoders/TileLayerMetadataEncoder.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/encoders/TileLayerMetadataEncoder.scala @@ -21,7 +21,7 @@ package org.locationtech.rasterframes.encoders -import geotrellis.spark.{KeyBounds, TileLayerMetadata} +import geotrellis.layer._ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import scala.reflect.runtime.universe._ @@ -39,7 +39,7 @@ object TileLayerMetadataEncoder { "cellType" -> cellTypeEncoder, "layout" -> layoutDefinitionEncoder, "extent" -> extentEncoder, - "crs" -> crsEncoder + "crs" -> crsSparkEncoder ) def apply[K: TypeTag](): ExpressionEncoder[TileLayerMetadata[K]] = { diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala index e72f158aa..09cd22997 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala @@ -79,7 +79,7 @@ object DynamicExtractors { } /** Partial function for pulling a CellGrid from an input row. */ - lazy val gridExtractor: PartialFunction[DataType, InternalRow ⇒ CellGrid] = { + lazy val gridExtractor: PartialFunction[DataType, InternalRow ⇒ CellGrid[Int]] = { case _: TileUDT => (row: InternalRow) => row.to[Tile](TileUDT.tileSerializer) case _: RasterSourceUDT => diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/OnCellGridExpression.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/OnCellGridExpression.scala index 05d56f7d1..62dac78c1 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/OnCellGridExpression.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/OnCellGridExpression.scala @@ -52,6 +52,6 @@ trait OnCellGridExpression extends UnaryExpression { } /** Implemented by subtypes to process incoming ProjectedRasterLike entity. */ - def eval(grid: CellGrid): Any + def eval(grid: CellGrid[Int]): Any } diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/SpatialRelation.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/SpatialRelation.scala index 1d6697048..9f4d19725 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/SpatialRelation.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/SpatialRelation.scala @@ -49,7 +49,7 @@ abstract class SpatialRelation extends BinaryExpression case udt: AbstractGeometryUDT[_] ⇒ udt.deserialize(r) case dt if dt.conformsTo[Extent] => val extent = r.to[Extent] - extent.jtsGeom + extent.toPolygon() } } } diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetCRS.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetCRS.scala index ae166a51d..468fae7d9 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetCRS.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetCRS.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.types.{DataType, StringType} import org.apache.spark.sql.{Column, TypedColumn} import org.apache.spark.unsafe.types.UTF8String import org.locationtech.rasterframes.encoders.CatalystSerializer._ -import org.locationtech.rasterframes.encoders.StandardEncoders.crsEncoder +import org.locationtech.rasterframes.encoders.StandardEncoders.crsSparkEncoder import org.locationtech.rasterframes.expressions.DynamicExtractors.projectedRasterLikeExtractor import org.locationtech.rasterframes.model.LazyCRS diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetCellType.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetCellType.scala index 869835c5f..bb7cdf233 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetCellType.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetCellType.scala @@ -39,7 +39,7 @@ case class GetCellType(child: Expression) extends OnCellGridExpression with Code def dataType: DataType = schemaOf[CellType] /** Implemented by subtypes to process incoming ProjectedRasterLike entity. */ - override def eval(cg: CellGrid): Any = cg.cellType.toInternalRow + override def eval(cg: CellGrid[Int]): Any = cg.cellType.toInternalRow } object GetCellType { diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetDimensions.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetDimensions.scala index dffdfdecb..e4db95f40 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetDimensions.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetDimensions.scala @@ -45,7 +45,7 @@ case class GetDimensions(child: Expression) extends OnCellGridExpression with Co def dataType = schemaOf[TileDimensions] - override def eval(grid: CellGrid): Any = TileDimensions(grid.cols, grid.rows).toInternalRow + override def eval(grid: CellGrid[Int]): Any = TileDimensions(grid.cols, grid.rows).toInternalRow } object GetDimensions { diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetGeometry.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetGeometry.scala index 7ff3bcfc7..e099cca04 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetGeometry.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetGeometry.scala @@ -48,7 +48,7 @@ case class GetGeometry(child: Expression) extends OnTileContextExpression with C override def dataType: DataType = JTSTypes.GeometryTypeInstance override def nodeName: String = "rf_geometry" override def eval(ctx: TileContext): InternalRow = - JTSTypes.GeometryTypeInstance.serialize(ctx.extent.jtsGeom) + JTSTypes.GeometryTypeInstance.serialize(ctx.extent.toPolygon()) } object GetGeometry { diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/aggregates/ProjectedLayerMetadataAggregate.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/aggregates/ProjectedLayerMetadataAggregate.scala index 267393f79..2bc89e592 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/aggregates/ProjectedLayerMetadataAggregate.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/aggregates/ProjectedLayerMetadataAggregate.scala @@ -28,8 +28,7 @@ import org.locationtech.rasterframes.model.TileDimensions import geotrellis.proj4.{CRS, Transform} import geotrellis.raster._ import geotrellis.raster.reproject.{Reproject, ReprojectRasterExtent} -import geotrellis.spark.tiling.LayoutDefinition -import geotrellis.spark.{KeyBounds, SpatialKey, TileLayerMetadata} +import geotrellis.layer._ import geotrellis.vector.Extent import org.apache.spark.sql.expressions.{MutableAggregationBuffer, UserDefinedAggregateFunction} import org.apache.spark.sql.types.{DataType, StructField, StructType} diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/aggregates/TileRasterizerAggregate.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/aggregates/TileRasterizerAggregate.scala index 360ef93dd..0ab2b0519 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/aggregates/TileRasterizerAggregate.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/aggregates/TileRasterizerAggregate.scala @@ -25,7 +25,7 @@ import geotrellis.proj4.CRS import geotrellis.raster.reproject.Reproject import geotrellis.raster.resample.ResampleMethod import geotrellis.raster.{ArrayTile, CellType, MultibandTile, ProjectedRaster, Raster, Tile} -import geotrellis.spark.{SpatialKey, TileLayerMetadata} +import geotrellis.layer._ import geotrellis.vector.Extent import org.apache.spark.sql.expressions.{MutableAggregationBuffer, UserDefinedAggregateFunction} import org.apache.spark.sql.types.{DataType, StructField, StructType} diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/generators/ExplodeTiles.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/generators/ExplodeTiles.scala index 2a70be585..ef1c51400 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/generators/ExplodeTiles.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/generators/ExplodeTiles.scala @@ -81,7 +81,7 @@ case class ExplodeTiles( ) val numOutCols = tiles.length + 2 - val (cols, rows) = dims.head + val Dimensions(cols, rows) = dims.head val retval = Array.ofDim[InternalRow](cols * rows) cfor(0)(_ < rows, _ + 1) { row => diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/generators/RasterSourceToRasterRefs.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/generators/RasterSourceToRasterRefs.scala index d90d790b5..a514b3560 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/generators/RasterSourceToRasterRefs.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/generators/RasterSourceToRasterRefs.scala @@ -55,7 +55,7 @@ case class RasterSourceToRasterRefs(children: Seq[Expression], bandIndexes: Seq[ name <- bandNames(basename, bandIndexes) } yield StructField(name, schemaOf[RasterRef], true)) - private def band2ref(src: RasterSource, e: Option[(GridBounds, Extent)])(b: Int): RasterRef = + private def band2ref(src: RasterSource, e: Option[(GridBounds[Int], Extent)])(b: Int): RasterRef = if (b < src.bandCount) RasterRef(src, b, e.map(_._2), e.map(_._1)) else null diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/ExtentToGeometry.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/ExtentToGeometry.scala index 9d2d12d2f..50f06c7ce 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/ExtentToGeometry.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/ExtentToGeometry.scala @@ -65,7 +65,7 @@ case class ExtentToGeometry(child: Expression) extends UnaryExpression with Code else { r.to[Extent] } - val geom = extent.jtsGeom + val geom = extent.toPolygon() JTSTypes.GeometryTypeInstance.serialize(geom) } } diff --git a/core/src/main/scala/org/locationtech/rasterframes/extensions/ContextRDDMethods.scala b/core/src/main/scala/org/locationtech/rasterframes/extensions/ContextRDDMethods.scala index 7bf3230b3..de7170d42 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/extensions/ContextRDDMethods.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/extensions/ContextRDDMethods.scala @@ -21,24 +21,23 @@ package org.locationtech.rasterframes.extensions -import org.locationtech.rasterframes.PairRDDConverter._ -import org.locationtech.rasterframes.StandardColumns._ -import Implicits._ -import org.locationtech.rasterframes.util._ -import org.locationtech.rasterframes.RasterFrameLayer +import geotrellis.layer._ import geotrellis.raster.CellGrid -import geotrellis.spark._ -import geotrellis.spark.io._ import geotrellis.util.MethodExtensions import org.apache.spark.rdd.RDD import org.apache.spark.sql.SparkSession -import org.locationtech.rasterframes.PairRDDConverter +import org.locationtech.rasterframes.PairRDDConverter._ +import org.locationtech.rasterframes.{PairRDDConverter, RasterFrameLayer} +import org.locationtech.rasterframes.StandardColumns._ +import org.locationtech.rasterframes.extensions.Implicits._ +import org.locationtech.rasterframes.util.JsonCodecs._ +import org.locationtech.rasterframes.util._ /** * Extension method on `ContextRDD`-shaped RDDs with appropriate context bounds to create a RasterFrameLayer. * @since 7/18/17 */ -abstract class SpatialContextRDDMethods[T <: CellGrid](implicit spark: SparkSession) +abstract class SpatialContextRDDMethods[T <: CellGrid[Int]](implicit spark: SparkSession) extends MethodExtensions[RDD[(SpatialKey, T)] with Metadata[TileLayerMetadata[SpatialKey]]] { import PairRDDConverter._ @@ -56,7 +55,7 @@ abstract class SpatialContextRDDMethods[T <: CellGrid](implicit spark: SparkSess * Extension method on `ContextRDD`-shaped `Tile` RDDs keyed with [[SpaceTimeKey]], with appropriate context bounds to create a RasterFrameLayer. * @since 9/11/17 */ -abstract class SpatioTemporalContextRDDMethods[T <: CellGrid]( +abstract class SpatioTemporalContextRDDMethods[T <: CellGrid[Int]]( implicit spark: SparkSession) extends MethodExtensions[RDD[(SpaceTimeKey, T)] with Metadata[TileLayerMetadata[SpaceTimeKey]]] { diff --git a/core/src/main/scala/org/locationtech/rasterframes/extensions/DataFrameMethods.scala b/core/src/main/scala/org/locationtech/rasterframes/extensions/DataFrameMethods.scala index 9a57b9dd8..c40c628dc 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/extensions/DataFrameMethods.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/extensions/DataFrameMethods.scala @@ -22,8 +22,7 @@ package org.locationtech.rasterframes.extensions import geotrellis.proj4.CRS -import geotrellis.spark.io._ -import geotrellis.spark.{SpaceTimeKey, SpatialComponent, SpatialKey, TemporalKey, TileLayerMetadata} +import geotrellis.layer._ import geotrellis.util.MethodExtensions import geotrellis.vector.Extent import org.apache.spark.sql.catalyst.expressions.Attribute @@ -37,7 +36,7 @@ import org.locationtech.rasterframes.tiles.ProjectedRasterTile import org.locationtech.rasterframes.util._ import org.locationtech.rasterframes.{MetadataKeys, RasterFrameLayer} import spray.json.JsonFormat - +import org.locationtech.rasterframes.util.JsonCodecs._ import scala.util.Try /** diff --git a/core/src/main/scala/org/locationtech/rasterframes/extensions/Implicits.scala b/core/src/main/scala/org/locationtech/rasterframes/extensions/Implicits.scala index 563e03e87..f8c4e134a 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/extensions/Implicits.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/extensions/Implicits.scala @@ -25,14 +25,13 @@ import org.locationtech.rasterframes.RasterFrameLayer import org.locationtech.rasterframes.util.{WithMergeMethods, WithPrototypeMethods} import geotrellis.raster._ import geotrellis.raster.io.geotiff.SinglebandGeoTiff -import geotrellis.spark.{Metadata, SpaceTimeKey, SpatialKey, TileLayerMetadata} +import geotrellis.layer._ import geotrellis.util.MethodExtensions import org.apache.spark.SparkConf import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.types.{MetadataBuilder, Metadata => SMetadata} -import spray.json.JsonFormat - +import spray.json._ import scala.reflect.runtime.universe._ /** @@ -49,7 +48,7 @@ trait Implicits { implicit class WithSKryoMethods(val self: SparkConf) extends KryoMethods.SparkConfKryoMethods - implicit class WithProjectedRasterMethods[T <: CellGrid: WithMergeMethods: WithPrototypeMethods: TypeTag]( + implicit class WithProjectedRasterMethods[T <: CellGrid[Int]: WithMergeMethods: WithPrototypeMethods: TypeTag]( val self: ProjectedRaster[T]) extends ProjectedRasterMethods[T] implicit class WithSinglebandGeoTiffMethods(val self: SinglebandGeoTiff) extends SinglebandGeoTiffMethods @@ -58,11 +57,11 @@ trait Implicits { implicit class WithRasterFrameLayerMethods(val self: RasterFrameLayer) extends RasterFrameLayerMethods - implicit class WithSpatialContextRDDMethods[T <: CellGrid]( + implicit class WithSpatialContextRDDMethods[T <: CellGrid[Int]]( val self: RDD[(SpatialKey, T)] with Metadata[TileLayerMetadata[SpatialKey]] )(implicit spark: SparkSession) extends SpatialContextRDDMethods[T] - implicit class WithSpatioTemporalContextRDDMethods[T <: CellGrid]( + implicit class WithSpatioTemporalContextRDDMethods[T <: CellGrid[Int]]( val self: RDD[(SpaceTimeKey, T)] with Metadata[TileLayerMetadata[SpaceTimeKey]] )(implicit spark: SparkSession) extends SpatioTemporalContextRDDMethods[T] diff --git a/core/src/main/scala/org/locationtech/rasterframes/extensions/ProjectedRasterMethods.scala b/core/src/main/scala/org/locationtech/rasterframes/extensions/ProjectedRasterMethods.scala index 81f5054f9..e23ca3ca4 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/extensions/ProjectedRasterMethods.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/extensions/ProjectedRasterMethods.scala @@ -23,9 +23,9 @@ package org.locationtech.rasterframes.extensions import java.time.ZonedDateTime -import geotrellis.raster.{CellGrid, ProjectedRaster} +import geotrellis.raster.{CellGrid, Dimensions, ProjectedRaster} import geotrellis.spark._ -import geotrellis.spark.tiling._ +import geotrellis.layer._ import geotrellis.util.MethodExtensions import org.apache.spark.rdd.RDD import org.apache.spark.sql.SparkSession @@ -39,7 +39,7 @@ import scala.reflect.runtime.universe._ * * @since 8/10/17 */ -abstract class ProjectedRasterMethods[T <: CellGrid: WithMergeMethods: WithPrototypeMethods: TypeTag] +abstract class ProjectedRasterMethods[T <: CellGrid[Int]: WithMergeMethods: WithPrototypeMethods: TypeTag] extends MethodExtensions[ProjectedRaster[T]] with StandardColumns { import Implicits.{WithSpatialContextRDDMethods, WithSpatioTemporalContextRDDMethods} type XTileLayerRDD[K] = RDD[(K, T)] with Metadata[TileLayerMetadata[K]] @@ -61,7 +61,7 @@ abstract class ProjectedRasterMethods[T <: CellGrid: WithMergeMethods: WithProto */ def toLayer(tileColName: String) (implicit spark: SparkSession, schema: PairRDDConverter[SpatialKey, T]): RasterFrameLayer = { - val (cols, rows) = self.raster.dimensions + val Dimensions(cols, rows) = self.raster.dimensions toLayer(cols, rows, tileColName) } @@ -114,11 +114,18 @@ abstract class ProjectedRasterMethods[T <: CellGrid: WithMergeMethods: WithProto */ def toTileLayerRDD(tileCols: Int, tileRows: Int)(implicit spark: SparkSession): XTileLayerRDD[SpatialKey] = { + + // TODO: get rid of this sloppy type leakage hack. Might not be necessary anyway. + def toArrayTile[T <: CellGrid[Int]](tile: T): T = + tile.getClass.getMethods + .find(_.getName == "toArrayTile") + .map(_.invoke(tile).asInstanceOf[T]) + .getOrElse(tile) + val layout = LayoutDefinition(self.raster.rasterExtent, tileCols, tileRows) val kb = KeyBounds(SpatialKey(0, 0), SpatialKey(layout.layoutCols - 1, layout.layoutRows - 1)) val tlm = TileLayerMetadata(self.tile.cellType, layout, self.extent, self.crs, kb) - - val rdd = spark.sparkContext.makeRDD(Seq((self.projectedExtent, Shims.toArrayTile(self.tile)))) + val rdd = spark.sparkContext.makeRDD(Seq((self.projectedExtent, toArrayTile(self.tile)))) implicit val tct = typeTag[T].asClassTag diff --git a/core/src/main/scala/org/locationtech/rasterframes/extensions/RFSpatialColumnMethods.scala b/core/src/main/scala/org/locationtech/rasterframes/extensions/RFSpatialColumnMethods.scala index af79c1c05..f4da5d664 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/extensions/RFSpatialColumnMethods.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/extensions/RFSpatialColumnMethods.scala @@ -25,10 +25,9 @@ import org.locationtech.rasterframes.util._ import org.locationtech.rasterframes.RasterFrameLayer import org.locationtech.jts.geom.Point import geotrellis.proj4.LatLng -import geotrellis.spark.SpatialKey -import geotrellis.spark.tiling.MapKeyTransform +import geotrellis.layer._ import geotrellis.util.MethodExtensions -import geotrellis.vector.Extent +import geotrellis.vector._ import org.apache.spark.sql.Row import org.apache.spark.sql.functions.{asc, udf => sparkUdf} import org.apache.spark.sql.types.{DoubleType, StructField, StructType} @@ -89,7 +88,7 @@ trait RFSpatialColumnMethods extends MethodExtensions[RasterFrameLayer] with Sta * @return updated RasterFrameLayer */ def withGeometry(colName: String = GEOMETRY_COLUMN.columnName): RasterFrameLayer = { - val key2Bounds = sparkUdf(keyCol2Extent andThen (_.jtsGeom)) + val key2Bounds = sparkUdf(keyCol2Extent andThen (_.toPolygon())) self.withColumn(colName, key2Bounds(self.spatialKeyColumn)).certify } @@ -100,7 +99,7 @@ trait RFSpatialColumnMethods extends MethodExtensions[RasterFrameLayer] with Sta * @return updated RasterFrameLayer */ def withCenter(colName: String = CENTER_COLUMN.columnName): RasterFrameLayer = { - val key2Center = sparkUdf(keyCol2Extent andThen (_.center.jtsGeom)) + val key2Center = sparkUdf(keyCol2Extent andThen (_.center)) self.withColumn(colName, key2Center(self.spatialKeyColumn).as[Point]).certify } diff --git a/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterFrameLayerMethods.scala b/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterFrameLayerMethods.scala index e9d375f12..3225ae373 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterFrameLayerMethods.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterFrameLayerMethods.scala @@ -27,9 +27,10 @@ import com.typesafe.scalalogging.Logger import geotrellis.proj4.CRS import geotrellis.raster.resample.{NearestNeighbor, ResampleMethod} import geotrellis.raster.{MultibandTile, ProjectedRaster, Tile, TileLayout} +import geotrellis.layer._ import geotrellis.spark._ -import geotrellis.spark.io._ -import geotrellis.spark.tiling.{LayoutDefinition, Tiler} +import geotrellis.spark.tiling.Tiler +import geotrellis.spark.{ContextRDD, MultibandTileLayerRDD, TileLayerRDD} import geotrellis.util.MethodExtensions import geotrellis.vector.ProjectedExtent import org.apache.spark.annotation.Experimental @@ -41,6 +42,7 @@ import org.locationtech.rasterframes.encoders.StandardEncoders.PrimitiveEncoders import org.locationtech.rasterframes.encoders.StandardEncoders._ import org.locationtech.rasterframes.tiles.ShowableTile import org.locationtech.rasterframes.util._ +import org.locationtech.rasterframes.util.JsonCodecs._ import org.slf4j.LoggerFactory import spray.json._ diff --git a/core/src/main/scala/org/locationtech/rasterframes/extensions/ReprojectToLayer.scala b/core/src/main/scala/org/locationtech/rasterframes/extensions/ReprojectToLayer.scala index d5e6f5e31..2108e6994 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/extensions/ReprojectToLayer.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/extensions/ReprojectToLayer.scala @@ -21,20 +21,21 @@ package org.locationtech.rasterframes.extensions -import geotrellis.spark.{SpatialKey, TileLayerMetadata} +import geotrellis.layer._ import org.apache.spark.sql._ import org.apache.spark.sql.functions.broadcast import org.locationtech.rasterframes._ +import org.locationtech.rasterframes.encoders.StandardEncoders.crsSparkEncoder import org.locationtech.rasterframes.util._ object ReprojectToLayer { def apply(df: DataFrame, tlm: TileLayerMetadata[SpatialKey]): RasterFrameLayer = { // create a destination dataframe with crs and extend columns // use RasterJoin to do the rest. - val gb = tlm.gridBounds + val gb = tlm.tileBounds val crs = tlm.crs import df.sparkSession.implicits._ - implicit val enc = Encoders.tuple(spatialKeyEncoder, extentEncoder, crsEncoder) + implicit val enc = Encoders.tuple(spatialKeyEncoder, extentEncoder, crsSparkEncoder) val gridItems = for { (col, row) <- gb.coordsIter diff --git a/core/src/main/scala/org/locationtech/rasterframes/functions/package.scala b/core/src/main/scala/org/locationtech/rasterframes/functions/package.scala index 0326046f3..521c9822b 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/functions/package.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/functions/package.scala @@ -152,12 +152,11 @@ package object functions { * Rasterize geometry into tiles. */ private[rasterframes] val rasterize: (Geometry, Geometry, Int, Int, Int) ⇒ Tile = { - import geotrellis.vector.{Geometry => GTGeometry} (geom, bounds, value, cols, rows) ⇒ { // We have to do this because (as of spark 2.2.x) Encoder-only types // can't be used as UDF inputs. Only Spark-native types and UDTs. val extent = Extent(bounds.getEnvelopeInternal) - GTGeometry(geom).rasterizeWithValue(RasterExtent(extent, cols, rows), value).tile + geom.rasterizeWithValue(RasterExtent(extent, cols, rows), value).tile } } diff --git a/core/src/main/scala/org/locationtech/rasterframes/jts/Implicits.scala b/core/src/main/scala/org/locationtech/rasterframes/jts/Implicits.scala index 358fdc258..92527abb2 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/jts/Implicits.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/jts/Implicits.scala @@ -45,7 +45,7 @@ trait Implicits extends SpatialConstructors { new Column(Intersects(self.expr, geomLit(geom).expr)).as[Boolean] def intersects(pt: gtPoint): TypedColumn[Any, Boolean] = - new Column(Intersects(self.expr, geomLit(pt.jtsGeom).expr)).as[Boolean] + new Column(Intersects(self.expr, geomLit(pt).expr)).as[Boolean] def containsGeom(geom: Geometry): TypedColumn[Any, Boolean] = new Column(Contains(self.expr, geomLit(geom).expr)).as[Boolean] diff --git a/core/src/main/scala/org/locationtech/rasterframes/jts/ReprojectionTransformer.scala b/core/src/main/scala/org/locationtech/rasterframes/jts/ReprojectionTransformer.scala index 54b45c034..5b3ee536a 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/jts/ReprojectionTransformer.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/jts/ReprojectionTransformer.scala @@ -36,7 +36,7 @@ class ReprojectionTransformer(src: CRS, dst: CRS) extends GeometryTransformer { @transient private lazy val gf = new GeometryFactory() def apply(geometry: Geometry): Geometry = transform(geometry) - def apply(extent: Extent): Geometry = transform(extent.jtsGeom) + def apply(extent: Extent): Geometry = transform(extent.toPolygon()) def apply(env: Envelope): Geometry = transform(gf.toGeometry(env)) override def transformCoordinates(coords: CoordinateSequence, parent: Geometry): CoordinateSequence = { diff --git a/core/src/main/scala/org/locationtech/rasterframes/model/FixedRasterExtent.scala b/core/src/main/scala/org/locationtech/rasterframes/model/FixedRasterExtent.scala deleted file mode 100644 index cdce274bb..000000000 --- a/core/src/main/scala/org/locationtech/rasterframes/model/FixedRasterExtent.scala +++ /dev/null @@ -1,278 +0,0 @@ -/* - * Copyright 2016 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 org.locationtech.rasterframes.model - - -import geotrellis.raster._ -import geotrellis.vector._ - -import scala.math.ceil - -/** - * This class is a copy of the GeoTrellis 2.x `RasterExtent`, - * with [GT 3.0 fixes](https://github.com/locationtech/geotrellis/pull/2953/files) incorporated into the - * new `GridExtent[T]` class. This class should be removed after RasterFrames is upgraded to GT 3.x. - */ -case class FixedRasterExtent( - override val extent: Extent, - override val cellwidth: Double, - override val cellheight: Double, - cols: Int, - rows: Int -) extends GridExtent(extent, cellwidth, cellheight) with Grid { - import FixedRasterExtent._ - - if (cols <= 0) throw GeoAttrsError(s"invalid cols: $cols") - if (rows <= 0) throw GeoAttrsError(s"invalid rows: $rows") - - /** - * Convert map coordinates (x, y) to grid coordinates (col, row). - */ - final def mapToGrid(x: Double, y: Double): (Int, Int) = { - val col = floorWithTolerance((x - extent.xmin) / cellwidth).toInt - val row = floorWithTolerance((extent.ymax - y) / cellheight).toInt - (col, row) - } - - /** - * Convert map coordinate x to grid coordinate column. - */ - final def mapXToGrid(x: Double): Int = floorWithTolerance(mapXToGridDouble(x)).toInt - - /** - * Convert map coordinate x to grid coordinate column. - */ - final def mapXToGridDouble(x: Double): Double = (x - extent.xmin) / cellwidth - - /** - * Convert map coordinate y to grid coordinate row. - */ - final def mapYToGrid(y: Double): Int = floorWithTolerance(mapYToGridDouble(y)).toInt - - /** - * Convert map coordinate y to grid coordinate row. - */ - final def mapYToGridDouble(y: Double): Double = (extent.ymax - y ) / cellheight - - /** - * Convert map coordinate tuple (x, y) to grid coordinates (col, row). - */ - final def mapToGrid(mapCoord: (Double, Double)): (Int, Int) = { - val (x, y) = mapCoord - mapToGrid(x, y) - } - - /** - * Convert a point to grid coordinates (col, row). - */ - final def mapToGrid(p: Point): (Int, Int) = - mapToGrid(p.x, p.y) - - /** - * The map coordinate of a grid cell is the center point. - */ - final def gridToMap(col: Int, row: Int): (Double, Double) = { - val x = col * cellwidth + extent.xmin + (cellwidth / 2) - val y = extent.ymax - (row * cellheight) - (cellheight / 2) - - (x, y) - } - - /** - * For a give column, find the corresponding x-coordinate in the - * grid of the present [[FixedRasterExtent]]. - */ - final def gridColToMap(col: Int): Double = { - col * cellwidth + extent.xmin + (cellwidth / 2) - } - - /** - * For a give row, find the corresponding y-coordinate in the grid - * of the present [[FixedRasterExtent]]. - */ - final def gridRowToMap(row: Int): Double = { - extent.ymax - (row * cellheight) - (cellheight / 2) - } - - /** - * Gets the GridBounds aligned with this FixedRasterExtent that is the - * smallest subgrid of containing all points within the extent. The - * extent is considered inclusive on it's north and west borders, - * exclusive on it's east and south borders. See [[FixedRasterExtent]] - * for a discussion of grid and extent boundary concepts. - * - * The 'clamp' flag determines whether or not to clamp the - * GridBounds to the FixedRasterExtent; defaults to true. If false, - * GridBounds can contain negative values, or values outside of - * this FixedRasterExtent's boundaries. - * - * @param subExtent The extent to get the grid bounds for - * @param clamp A boolean - */ - def gridBoundsFor(subExtent: Extent, clamp: Boolean = true): GridBounds = { - // West and North boundaries are a simple mapToGrid call. - val (colMin, rowMin) = mapToGrid(subExtent.xmin, subExtent.ymax) - - // If South East corner is on grid border lines, we want to still only include - // what is to the West and\or North of the point. However if the border point - // is not directly on a grid division, include the whole row and/or column that - // contains the point. - val colMax = { - val colMaxDouble = mapXToGridDouble(subExtent.xmax) - if(math.abs(colMaxDouble - floorWithTolerance(colMaxDouble)) < FixedRasterExtent.epsilon) colMaxDouble.toInt - 1 - else colMaxDouble.toInt - } - - val rowMax = { - val rowMaxDouble = mapYToGridDouble(subExtent.ymin) - if(math.abs(rowMaxDouble - floorWithTolerance(rowMaxDouble)) < FixedRasterExtent.epsilon) rowMaxDouble.toInt - 1 - else rowMaxDouble.toInt - } - - if(clamp) { - GridBounds(math.min(math.max(colMin, 0), cols - 1), - math.min(math.max(rowMin, 0), rows - 1), - math.min(math.max(colMax, 0), cols - 1), - math.min(math.max(rowMax, 0), rows - 1)) - } else { - GridBounds(colMin, rowMin, colMax, rowMax) - } - } - - /** - * Combine two different [[FixedRasterExtent]]s (which must have the - * same cellsizes). The result is a new extent at the same - * resolution. - */ - def combine (that: FixedRasterExtent): FixedRasterExtent = { - if (cellwidth != that.cellwidth) - throw GeoAttrsError(s"illegal cellwidths: $cellwidth and ${that.cellwidth}") - if (cellheight != that.cellheight) - throw GeoAttrsError(s"illegal cellheights: $cellheight and ${that.cellheight}") - - val newExtent = extent.combine(that.extent) - val newRows = ceil(newExtent.height / cellheight).toInt - val newCols = ceil(newExtent.width / cellwidth).toInt - - FixedRasterExtent(newExtent, cellwidth, cellheight, newCols, newRows) - } - - /** - * Returns a [[RasterExtent]] with the same extent, but a modified - * number of columns and rows based on the given cell height and - * width. - */ - def withResolution(targetCellWidth: Double, targetCellHeight: Double): FixedRasterExtent = { - val newCols = math.ceil((extent.xmax - extent.xmin) / targetCellWidth).toInt - val newRows = math.ceil((extent.ymax - extent.ymin) / targetCellHeight).toInt - FixedRasterExtent(extent, targetCellWidth, targetCellHeight, newCols, newRows) - } - - /** - * Returns a [[FixedRasterExtent]] with the same extent, but a modified - * number of columns and rows based on the given cell height and - * width. - */ - def withResolution(cellSize: CellSize): FixedRasterExtent = - withResolution(cellSize.width, cellSize.height) - - /** - * Returns a [[FixedRasterExtent]] with the same extent and the given - * number of columns and rows. - */ - def withDimensions(targetCols: Int, targetRows: Int): FixedRasterExtent = - FixedRasterExtent(extent, targetCols, targetRows) - - /** - * Adjusts a raster extent so that it can encompass the tile - * layout. Will resample the extent, but keep the resolution, and - * preserve north and west borders - */ - def adjustTo(tileLayout: TileLayout): FixedRasterExtent = { - val totalCols = tileLayout.tileCols * tileLayout.layoutCols - val totalRows = tileLayout.tileRows * tileLayout.layoutRows - - val resampledExtent = Extent(extent.xmin, extent.ymax - (cellheight*totalRows), - extent.xmin + (cellwidth*totalCols), extent.ymax) - - FixedRasterExtent(resampledExtent, cellwidth, cellheight, totalCols, totalRows) - } - - /** - * Returns a new [[FixedRasterExtent]] which represents the GridBounds - * in relation to this FixedRasterExtent. - */ - def rasterExtentFor(gridBounds: GridBounds): FixedRasterExtent = { - val (xminCenter, ymaxCenter) = gridToMap(gridBounds.colMin, gridBounds.rowMin) - val (xmaxCenter, yminCenter) = gridToMap(gridBounds.colMax, gridBounds.rowMax) - val (hcw, hch) = (cellwidth / 2, cellheight / 2) - val e = Extent(xminCenter - hcw, yminCenter - hch, xmaxCenter + hcw, ymaxCenter + hch) - FixedRasterExtent(e, cellwidth, cellheight, gridBounds.width, gridBounds.height) - } -} - -/** - * The companion object for the [[FixedRasterExtent]] type. - */ -object FixedRasterExtent { - final val epsilon = 0.0000001 - - /** - * Create a new [[FixedRasterExtent]] from an Extent, a column, and a - * row. - */ - def apply(extent: Extent, cols: Int, rows: Int): FixedRasterExtent = { - val cw = extent.width / cols - val ch = extent.height / rows - FixedRasterExtent(extent, cw, ch, cols, rows) - } - - /** - * Create a new [[FixedRasterExtent]] from an Extent and a [[CellSize]]. - */ - def apply(extent: Extent, cellSize: CellSize): FixedRasterExtent = { - val cols = (extent.width / cellSize.width).toInt - val rows = (extent.height / cellSize.height).toInt - FixedRasterExtent(extent, cellSize.width, cellSize.height, cols, rows) - } - - /** - * Create a new [[FixedRasterExtent]] from a [[CellGrid]] and an Extent. - */ - def apply(tile: CellGrid, extent: Extent): FixedRasterExtent = - apply(extent, tile.cols, tile.rows) - - /** - * Create a new [[FixedRasterExtent]] from an Extent and a [[CellGrid]]. - */ - def apply(extent: Extent, tile: CellGrid): FixedRasterExtent = - apply(extent, tile.cols, tile.rows) - - - /** - * The same logic is used in QGIS: https://github.com/qgis/QGIS/blob/607664c5a6b47c559ed39892e736322b64b3faa4/src/analysis/raster/qgsalignraster.cpp#L38 - * The search query: https://github.com/qgis/QGIS/search?p=2&q=floor&type=&utf8=%E2%9C%93 - * - * GDAL uses smth like that, however it was a bit hard to track it down: - * https://github.com/OSGeo/gdal/blob/7601a637dfd204948d00f4691c08f02eb7584de5/gdal/frmts/vrt/vrtsources.cpp#L215 - * */ - def floorWithTolerance(value: Double): Double = { - val roundedValue = math.round(value) - if (math.abs(value - roundedValue) < epsilon) roundedValue - else math.floor(value) - } -} - diff --git a/core/src/main/scala/org/locationtech/rasterframes/model/LazyCRS.scala b/core/src/main/scala/org/locationtech/rasterframes/model/LazyCRS.scala index e8540e171..386dad40a 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/model/LazyCRS.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/model/LazyCRS.scala @@ -50,11 +50,18 @@ object LazyCRS { trait ValidatedCRS type EncodedCRS = String with ValidatedCRS + private object WKTCRS { + def unapply(src: String): Option[CRS] = + if (src.toUpperCase().startsWith("GEOGCS")) + CRS.fromWKT(src) + else None + } + @transient private lazy val mapper: PartialFunction[String, CRS] = { - case e if e.toUpperCase().startsWith("EPSG") => CRS.fromName(e) //not case-sensitive - case p if p.startsWith("+proj") => CRS.fromString(p) // case sensitive - case w if w.toUpperCase().startsWith("GEOGCS") => CRS.fromWKT(w) //only case-sensitive inside double quotes + case e if e.toUpperCase().startsWith("EPSG") => CRS.fromName(e) //not case-sensitive + case p if p.startsWith("+proj") => CRS.fromString(p) // case sensitive + case WKTCRS(w) => w } @transient diff --git a/core/src/main/scala/org/locationtech/rasterframes/model/TileDimensions.scala b/core/src/main/scala/org/locationtech/rasterframes/model/TileDimensions.scala index fbbdfebf1..4f2cb0fa2 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/model/TileDimensions.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/model/TileDimensions.scala @@ -22,7 +22,7 @@ package org.locationtech.rasterframes.model import org.locationtech.rasterframes.encoders.CatalystSerializer.CatalystIO -import geotrellis.raster.Grid +import geotrellis.raster.{Dimensions, Grid} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.types.{ShortType, StructField, StructType} import org.locationtech.rasterframes.encoders.CatalystSerializer @@ -32,10 +32,11 @@ import org.locationtech.rasterframes.encoders.CatalystSerializer * * @since 2018-12-12 */ -case class TileDimensions(cols: Int, rows: Int) extends Grid +case class TileDimensions(cols: Int, rows: Int) extends Grid[Int] object TileDimensions { def apply(colsRows: (Int, Int)): TileDimensions = new TileDimensions(colsRows._1, colsRows._2) + def apply(dims: Dimensions[Int]): TileDimensions = new TileDimensions(dims.cols, dims.rows) implicit val serializer: CatalystSerializer[TileDimensions] = new CatalystSerializer[TileDimensions] { override val schema: StructType = StructType(Seq( diff --git a/core/src/main/scala/org/locationtech/rasterframes/rasterframes.scala b/core/src/main/scala/org/locationtech/rasterframes/rasterframes.scala index b1958d36b..19c0fa1c6 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/rasterframes.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/rasterframes.scala @@ -23,7 +23,8 @@ package org.locationtech import com.typesafe.config.ConfigFactory import com.typesafe.scalalogging.Logger import geotrellis.raster.{Tile, TileFeature, isData} -import geotrellis.spark.{ContextRDD, Metadata, SpaceTimeKey, SpatialKey, TileLayerMetadata} +import geotrellis.layer._ +import geotrellis.spark.ContextRDD import org.apache.spark.rdd.RDD import org.apache.spark.sql.rf.{RasterSourceUDT, TileUDT} import org.apache.spark.sql.{DataFrame, SQLContext, rf} @@ -91,7 +92,7 @@ package object rasterframes extends StandardColumns /** * A RasterFrameLayer is just a DataFrame with certain invariants, enforced via the methods that create and transform them: - * 1. One column is a [[geotrellis.spark.SpatialKey]] or [[geotrellis.spark.SpaceTimeKey]] + * 1. One column is a `SpatialKey` or `SpaceTimeKey`` * 2. One or more columns is a [[Tile]] UDT. * 3. The `TileLayerMetadata` is encoded and attached to the key column. */ diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/DelegatingRasterSource.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/DelegatingRasterSource.scala index cff0b0087..d78f4b328 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/DelegatingRasterSource.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/DelegatingRasterSource.scala @@ -23,7 +23,7 @@ package org.locationtech.rasterframes.ref import java.net.URI -import geotrellis.contrib.vlm.{RasterSource => GTRasterSource} +import geotrellis.raster.{RasterSource => GTRasterSource} import geotrellis.proj4.CRS import geotrellis.raster.io.geotiff.Tags import geotrellis.raster.{CellType, GridBounds, MultibandTile, Raster} @@ -66,19 +66,19 @@ abstract class DelegatingRasterSource(source: URI, delegateBuilder: () => GTRast retryableRead(rs => SimpleRasterInfo(rs)) ) - override def cols: Int = info.cols - override def rows: Int = info.rows + override def cols: Int = info.cols.toInt + override def rows: Int = info.rows.toInt override def crs: CRS = info.crs override def extent: Extent = info.extent override def cellType: CellType = info.cellType override def bandCount: Int = info.bandCount override def tags: Tags = info.tags - override protected def readBounds(bounds: Traversable[GridBounds], bands: Seq[Int]): Iterator[Raster[MultibandTile]] = - retryableRead(_.readBounds(bounds, bands)) + override protected def readBounds(bounds: Traversable[GridBounds[Int]], bands: Seq[Int]): Iterator[Raster[MultibandTile]] = + retryableRead(_.readBounds(bounds.map(_.toGridType[Long]), bands)) - override def read(bounds: GridBounds, bands: Seq[Int]): Raster[MultibandTile] = - retryableRead(_.read(bounds, bands) + override def read(bounds: GridBounds[Int], bands: Seq[Int]): Raster[MultibandTile] = + retryableRead(_.read(bounds.toGridType[Long], bands) .getOrElse(throw new IllegalArgumentException(s"Bounds '$bounds' outside of source")) ) diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/GDALRasterSource.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/GDALRasterSource.scala index fe8736a16..d6f7ffbe1 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/GDALRasterSource.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/GDALRasterSource.scala @@ -25,13 +25,14 @@ import java.net.URI import com.azavea.gdal.GDALWarp import com.typesafe.scalalogging.LazyLogging -import geotrellis.contrib.vlm.gdal.{GDALRasterSource => VLMRasterSource} import geotrellis.proj4.CRS +import geotrellis.raster.gdal.{GDALRasterSource => VLMRasterSource} import geotrellis.raster.io.geotiff.Tags import geotrellis.raster.{CellType, GridBounds, MultibandTile, Raster} import geotrellis.vector.Extent import org.locationtech.rasterframes.ref.RasterSource.URIRasterSource + case class GDALRasterSource(source: URI) extends RasterSource with URIRasterSource { @transient @@ -60,14 +61,14 @@ case class GDALRasterSource(source: URI) extends RasterSource with URIRasterSour override def bandCount: Int = tiffInfo.bandCount - override def cols: Int = tiffInfo.cols + override def cols: Int = tiffInfo.cols.toInt - override def rows: Int = tiffInfo.rows + override def rows: Int = tiffInfo.rows.toInt override def tags: Tags = Tags(metadata, List.empty) - override protected def readBounds(bounds: Traversable[GridBounds], bands: Seq[Int]): Iterator[Raster[MultibandTile]] = - gdal.readBounds(bounds, bands) + override protected def readBounds(bounds: Traversable[GridBounds[Int]], bands: Seq[Int]): Iterator[Raster[MultibandTile]] = + gdal.readBounds(bounds.map(_.toGridType[Long]), bands) } object GDALRasterSource extends LazyLogging { diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/HadoopGeoTiffRasterSource.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/HadoopGeoTiffRasterSource.scala index 3249f1bce..ba899ba4c 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/HadoopGeoTiffRasterSource.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/HadoopGeoTiffRasterSource.scala @@ -23,7 +23,7 @@ package org.locationtech.rasterframes.ref import java.net.URI -import geotrellis.spark.io.hadoop.HdfsRangeReader +import geotrellis.store.hadoop.util.HdfsRangeReader import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.locationtech.rasterframes.ref.RasterSource.{URIRasterSource, URIRasterSourceDebugString} diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/InMemoryRasterSource.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/InMemoryRasterSource.scala index 3a6a2f5e1..5d29f0e32 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/InMemoryRasterSource.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/InMemoryRasterSource.scala @@ -41,7 +41,7 @@ case class InMemoryRasterSource(tile: Tile, extent: Extent, crs: CRS) extends Ra override def tags: Tags = EMPTY_TAGS - override protected def readBounds(bounds: Traversable[GridBounds], bands: Seq[Int]): Iterator[Raster[MultibandTile]] = { + override protected def readBounds(bounds: Traversable[GridBounds[Int]], bands: Seq[Int]): Iterator[Raster[MultibandTile]] = { bounds .map(b => { val subext = rasterExtent.extentFor(b) diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/JVMGeoTiffRasterSource.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/JVMGeoTiffRasterSource.scala index cedb81c61..57b8c883d 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/JVMGeoTiffRasterSource.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/JVMGeoTiffRasterSource.scala @@ -23,6 +23,7 @@ package org.locationtech.rasterframes.ref import java.net.URI -import geotrellis.contrib.vlm.geotiff.GeoTiffRasterSource +import geotrellis.raster.geotiff.GeoTiffRasterSource + case class JVMGeoTiffRasterSource(source: URI) extends DelegatingRasterSource(source, () => GeoTiffRasterSource(source.toASCIIString)) diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/ProjectedRasterLike.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/ProjectedRasterLike.scala index 515c47d12..1361381ef 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/ProjectedRasterLike.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/ProjectedRasterLike.scala @@ -30,7 +30,7 @@ import geotrellis.vector.Extent * * @since 11/3/18 */ -trait ProjectedRasterLike extends CellGrid { +trait ProjectedRasterLike extends CellGrid[Int] { def crs: CRS def extent: Extent } diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/RangeReaderRasterSource.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/RangeReaderRasterSource.scala index d4f7aa6b2..1825f6695 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/RangeReaderRasterSource.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/RangeReaderRasterSource.scala @@ -24,8 +24,8 @@ package org.locationtech.rasterframes.ref import com.typesafe.scalalogging.Logger import geotrellis.proj4.CRS import geotrellis.raster.io.geotiff.Tags -import geotrellis.raster.io.geotiff.reader.GeoTiffReader -import geotrellis.raster.{CellType, GridBounds, MultibandTile, Raster} +import geotrellis.raster.io.geotiff.reader.{GeoTiffInfo, GeoTiffReader} +import geotrellis.raster._ import geotrellis.util.RangeReader import geotrellis.vector.Extent import org.locationtech.rasterframes.util.GeoTiffInfoSupport @@ -37,7 +37,7 @@ trait RangeReaderRasterSource extends RasterSource with GeoTiffInfoSupport { protected def rangeReader: RangeReader private def realInfo = - GeoTiffReader.readGeoTiffInfo(rangeReader, streaming = true, withOverviews = false) + GeoTiffInfo.read(rangeReader, streaming = true, withOverviews = false) protected lazy val tiffInfo = SimpleRasterInfo(realInfo) @@ -55,10 +55,10 @@ trait RangeReaderRasterSource extends RasterSource with GeoTiffInfoSupport { override def tags: Tags = tiffInfo.tags - override protected def readBounds(bounds: Traversable[GridBounds], bands: Seq[Int]): Iterator[Raster[MultibandTile]] = { + override protected def readBounds(bounds: Traversable[GridBounds[Int]], bands: Seq[Int]): Iterator[Raster[MultibandTile]] = { val info = realInfo val geoTiffTile = GeoTiffReader.geoTiffMultibandTile(info) - val intersectingBounds = bounds.flatMap(_.intersection(this)).toSeq + val intersectingBounds = bounds.flatMap(_.intersection(this.gridBounds)).toSeq geoTiffTile.crop(intersectingBounds, bands.toArray).map { case (gb, tile) => Raster(tile, rasterExtent.extentFor(gb, clamp = true)) diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/RasterRef.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/RasterRef.scala index 7ca164a2e..5fc89450e 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/RasterRef.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/RasterRef.scala @@ -38,7 +38,7 @@ import org.locationtech.rasterframes.tiles.ProjectedRasterTile * * @since 8/21/18 */ -case class RasterRef(source: RasterSource, bandIndex: Int, subextent: Option[Extent], subgrid: Option[GridBounds]) +case class RasterRef(source: RasterSource, bandIndex: Int, subextent: Option[Extent], subgrid: Option[GridBounds[Int]]) extends ProjectedRasterLike { def crs: CRS = source.crs def extent: Extent = subextent.getOrElse(source.extent) @@ -48,7 +48,7 @@ case class RasterRef(source: RasterSource, bandIndex: Int, subextent: Option[Ext def cellType: CellType = source.cellType def tile: ProjectedRasterTile = RasterRefTile(this) - protected lazy val grid: GridBounds = + protected lazy val grid: GridBounds[Int] = subgrid.getOrElse(source.rasterExtent.gridBoundsFor(extent, true)) protected lazy val realizedTile: Tile = { @@ -80,14 +80,14 @@ object RasterRef extends LazyLogging { StructField("source", rsType.sqlType, false), StructField("bandIndex", IntegerType, false), StructField("subextent", schemaOf[Extent], true), - StructField("subgrid", schemaOf[GridBounds], true) + StructField("subgrid", schemaOf[GridBounds[Int]], true) )) override def to[R](t: RasterRef, io: CatalystIO[R]): R = io.create( io.to(t.source)(RasterSourceUDT.rasterSourceSerializer), t.bandIndex, t.subextent.map(io.to[Extent]).orNull, - t.subgrid.map(io.to[GridBounds]).orNull + t.subgrid.map(io.to[GridBounds[Int]]).orNull ) override def from[R](row: R, io: CatalystIO[R]): RasterRef = RasterRef( @@ -96,7 +96,7 @@ object RasterRef extends LazyLogging { if (io.isNullAt(row, 2)) None else Option(io.get[Extent](row, 2)), if (io.isNullAt(row, 3)) None - else Option(io.get[GridBounds](row, 3)) + else Option(io.get[GridBounds[Int]](row, 3)) ) } diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/RasterSource.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/RasterSource.scala index 8f3502c7d..39a33adb7 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/RasterSource.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/RasterSource.scala @@ -33,7 +33,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.rf.RasterSourceUDT -import org.locationtech.rasterframes.model.{FixedRasterExtent, TileContext, TileDimensions} +import org.locationtech.rasterframes.model.{TileContext, TileDimensions} import org.locationtech.rasterframes.{NOMINAL_TILE_DIMS, rfConfig} import scala.concurrent.duration.Duration @@ -57,7 +57,7 @@ trait RasterSource extends ProjectedRasterLike with Serializable { def tags: Tags - def read(bounds: GridBounds, bands: Seq[Int]): Raster[MultibandTile] = + def read(bounds: GridBounds[Int], bands: Seq[Int]): Raster[MultibandTile] = readBounds(Seq(bounds), bands).next() def read(extent: Extent, bands: Seq[Int] = SINGLEBAND): Raster[MultibandTile] = @@ -66,13 +66,15 @@ trait RasterSource extends ProjectedRasterLike with Serializable { def readAll(dims: TileDimensions = NOMINAL_TILE_DIMS, bands: Seq[Int] = SINGLEBAND): Seq[Raster[MultibandTile]] = layoutBounds(dims).map(read(_, bands)) - protected def readBounds(bounds: Traversable[GridBounds], bands: Seq[Int]): Iterator[Raster[MultibandTile]] + protected def readBounds(bounds: Traversable[GridBounds[Int]], bands: Seq[Int]): Iterator[Raster[MultibandTile]] - def rasterExtent = FixedRasterExtent(extent, cols, rows) + def rasterExtent = RasterExtent(extent, cols, rows) def cellSize = CellSize(extent, cols, rows) - def gridExtent = GridExtent(extent, cellSize) + def gridExtent: GridExtent[Int] = GridExtent[Int](extent, cellSize) + + def gridBounds: GridBounds[Int] = GridBounds(0, 0, cols - 1, rows - 1) def tileContext: TileContext = TileContext(extent, crs) @@ -81,7 +83,7 @@ trait RasterSource extends ProjectedRasterLike with Serializable { layoutBounds(dims).map(re.extentFor(_, clamp = true)) } - def layoutBounds(dims: TileDimensions): Seq[GridBounds] = { + def layoutBounds(dims: TileDimensions): Seq[GridBounds[Int]] = { gridBounds.split(dims.cols, dims.rows).toSeq } } @@ -133,7 +135,7 @@ object RasterSource extends LazyLogging { /** Extractor for determining if a scheme indicates GDAL preference. */ def unapply(source: URI): Boolean = { - lazy val schemeIsGdal = Option(source.getScheme()) + lazy val schemeIsGdal = Option(source.getScheme) .exists(_.startsWith("gdal")) gdalOnly(source) || ((preferGdal || schemeIsGdal) && GDALRasterSource.hasGDAL) diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/SimpleRasterInfo.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/SimpleRasterInfo.scala index 0b38ab650..6df223158 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/SimpleRasterInfo.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/SimpleRasterInfo.scala @@ -22,18 +22,17 @@ package org.locationtech.rasterframes.ref import com.github.blemale.scaffeine.Scaffeine -import geotrellis.contrib.vlm.geotiff.GeoTiffRasterSource -import geotrellis.contrib.vlm.{RasterSource => GTRasterSource} import geotrellis.proj4.CRS +import geotrellis.raster.geotiff.GeoTiffRasterSource import geotrellis.raster.io.geotiff.Tags -import geotrellis.raster.io.geotiff.reader.GeoTiffReader -import geotrellis.raster.{CellType, RasterExtent} +import geotrellis.raster.io.geotiff.reader.GeoTiffInfo +import geotrellis.raster.{CellType, RasterExtent, RasterSource => GTRasterSource} import geotrellis.vector.Extent import org.locationtech.rasterframes.ref.RasterSource.EMPTY_TAGS case class SimpleRasterInfo( - cols: Int, - rows: Int, + cols: Long, + rows: Long, cellType: CellType, extent: Extent, rasterExtent: RasterExtent, @@ -49,7 +48,7 @@ object SimpleRasterInfo { def apply(key: String, builder: String => SimpleRasterInfo): SimpleRasterInfo = cache.get(key, builder) - def apply(info: GeoTiffReader.GeoTiffInfo): SimpleRasterInfo = + def apply(info: GeoTiffInfo): SimpleRasterInfo = SimpleRasterInfo( info.segmentLayout.totalCols, info.segmentLayout.totalRows, @@ -68,12 +67,12 @@ object SimpleRasterInfo { case _ => EMPTY_TAGS } - SimpleRasterInfo( + new SimpleRasterInfo( rs.cols, rs.rows, rs.cellType, rs.extent, - rs.rasterExtent, + rs.gridExtent.toRasterExtent(), rs.crs, fetchTags, rs.bandCount, diff --git a/core/src/main/scala/org/locationtech/rasterframes/util/GeoTiffInfoSupport.scala b/core/src/main/scala/org/locationtech/rasterframes/util/GeoTiffInfoSupport.scala index e24bb8175..9bf0608ec 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/util/GeoTiffInfoSupport.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/util/GeoTiffInfoSupport.scala @@ -21,11 +21,9 @@ package org.locationtech.rasterframes.util +import geotrellis.layer._ import geotrellis.raster.TileLayout -import geotrellis.raster.io.geotiff.reader.GeoTiffReader -import geotrellis.raster.io.geotiff.reader.GeoTiffReader.GeoTiffInfo -import geotrellis.spark.tiling.LayoutDefinition -import geotrellis.spark.{KeyBounds, SpatialKey, TileLayerMetadata} +import geotrellis.raster.io.geotiff.reader.GeoTiffInfo import geotrellis.util.ByteReader /** @@ -47,8 +45,8 @@ trait GeoTiffInfoSupport { TileLayout(layoutCols, layoutRows, tileCols, tileRows) } - def extractGeoTiffLayout(reader: ByteReader): (GeoTiffReader.GeoTiffInfo, TileLayerMetadata[SpatialKey]) = { - val info: GeoTiffInfo = Shims.readGeoTiffInfo(reader, decompress = false, streaming = true, withOverviews = false) + def extractGeoTiffLayout(reader: ByteReader): (GeoTiffInfo, TileLayerMetadata[SpatialKey]) = { + val info: GeoTiffInfo = GeoTiffInfo.read(reader, streaming = true, withOverviews = false) (info, extractGeoTiffLayout(info)) } diff --git a/core/src/main/scala/org/locationtech/rasterframes/util/JsonCodecs.scala b/core/src/main/scala/org/locationtech/rasterframes/util/JsonCodecs.scala new file mode 100644 index 000000000..ec59a636c --- /dev/null +++ b/core/src/main/scala/org/locationtech/rasterframes/util/JsonCodecs.scala @@ -0,0 +1,373 @@ +/* + * Copyright 2016 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. + */ +// Copied from GeoTrellis 2.3 during conversion to 3.0 + + +package org.locationtech.rasterframes.util +import java.net.URI +import java.time.{ZoneOffset, ZonedDateTime} + +import geotrellis.layer._ +import geotrellis.proj4.CRS +import geotrellis.raster._ +import geotrellis.store.LayerId +import geotrellis.vector._ +import org.apache.avro.Schema +import spray.json._ + +import spray.json.DefaultJsonProtocol._ + +trait JsonCodecs { + + implicit object ExtentFormat extends RootJsonFormat[Extent] { + def write(extent: Extent) = + JsObject( + "xmin" -> JsNumber(extent.xmin), + "ymin" -> JsNumber(extent.ymin), + "xmax" -> JsNumber(extent.xmax), + "ymax" -> JsNumber(extent.ymax) + ) + + def read(value: JsValue): Extent = + value.asJsObject.getFields("xmin", "ymin", "xmax", "ymax") match { + case Seq(JsNumber(xmin), JsNumber(ymin), JsNumber(xmax), JsNumber(ymax)) => + Extent(xmin.toDouble, ymin.toDouble, xmax.toDouble, ymax.toDouble) + case _ => + throw new DeserializationException(s"Extent [xmin,ymin,xmax,ymax] expected: $value") + } + } + + implicit object CellTypeFormat extends RootJsonFormat[CellType] { + def write(cellType: CellType) = + JsString(cellType.name) + + def read(value: JsValue): CellType = + value match { + case JsString(name) => CellType.fromName(name) + case _ => + throw new DeserializationException("CellType must be a string") + } + } + + implicit object CellSizeFormat extends RootJsonFormat[CellSize] { + def write(cs: CellSize): JsValue = JsObject( + "width" -> JsNumber(cs.width), + "height" -> JsNumber(cs.height) + ) + def read(value: JsValue): CellSize = + value.asJsObject.getFields("width", "height") match { + case Seq(JsNumber(width), JsNumber(height)) => CellSize(width.toDouble, height.toDouble) + case _ => + throw new DeserializationException("BackendType must be a valid object.") + } + } + + implicit object RasterExtentFormat extends RootJsonFormat[RasterExtent] { + def write(rasterExtent: RasterExtent) = + JsObject( + "extent" -> rasterExtent.extent.toJson, + "cols" -> JsNumber(rasterExtent.cols), + "rows" -> JsNumber(rasterExtent.rows), + "cellwidth" -> JsNumber(rasterExtent.cellwidth), + "cellheight" -> JsNumber(rasterExtent.cellheight) + ) + + def read(value: JsValue): RasterExtent = + value.asJsObject.getFields("extent", "cols", "rows", "cellwidth", "cellheight") match { + case Seq(extent, JsNumber(cols), JsNumber(rows), JsNumber(cellwidth), JsNumber(cellheight)) => + val ext = extent.convertTo[Extent] + RasterExtent(ext, cellwidth.toDouble, cellheight.toDouble, cols.toInt, rows.toInt) + case _ => + throw new DeserializationException("RasterExtent expected.") + } + } + + implicit object TileLayoutFormat extends RootJsonFormat[TileLayout] { + def write(tileLayout: TileLayout) = + JsObject( + "layoutCols" -> JsNumber(tileLayout.layoutCols), + "layoutRows" -> JsNumber(tileLayout.layoutRows), + "tileCols" -> JsNumber(tileLayout.tileCols), + "tileRows" -> JsNumber(tileLayout.tileRows) + ) + + def read(value: JsValue): TileLayout = + value.asJsObject.getFields("layoutCols", "layoutRows", "tileCols", "tileRows") match { + case Seq(JsNumber(layoutCols), JsNumber(layoutRows), JsNumber(tileCols), JsNumber(tileRows)) => + TileLayout(layoutCols.toInt, layoutRows.toInt, tileCols.toInt, tileRows.toInt) + case _ => + throw new DeserializationException("TileLayout expected.") + } + } + + implicit object GridBoundsFormat extends RootJsonFormat[GridBounds[Int]] { + def write(gridBounds: GridBounds[Int]) = + JsObject( + "colMin" -> JsNumber(gridBounds.colMin), + "rowMin" -> JsNumber(gridBounds.rowMin), + "colMax" -> JsNumber(gridBounds.colMax), + "rowMax" -> JsNumber(gridBounds.rowMax) + ) + + def read(value: JsValue): GridBounds[Int] = + value.asJsObject.getFields("colMin", "rowMin", "colMax", "rowMax") match { + case Seq(JsNumber(colMin), JsNumber(rowMin), JsNumber(colMax), JsNumber(rowMax)) => + GridBounds(colMin.toInt, rowMin.toInt, colMax.toInt, rowMax.toInt) + case _ => + throw new DeserializationException("GridBounds expected.") + } + } + + implicit object CRSFormat extends RootJsonFormat[CRS] { + def write(crs: CRS) = + JsString(crs.toProj4String) + + def read(value: JsValue): CRS = + value match { + case JsString(proj4String) => CRS.fromString(proj4String) + case _ => + throw new DeserializationException("CRS must be a proj4 string.") + } + } + + implicit object URIFormat extends RootJsonFormat[URI] { + def write(uri: URI) = + JsString(uri.toString) + + def read(value: JsValue): URI = + value match { + case JsString(str) => new URI(str) + case _ => + throw new DeserializationException("URI must be a string.") + } + } + implicit object SpatialKeyFormat extends RootJsonFormat[SpatialKey] { + def write(key: SpatialKey) = + JsObject( + "col" -> JsNumber(key.col), + "row" -> JsNumber(key.row) + ) + + def read(value: JsValue): SpatialKey = + value.asJsObject.getFields("col", "row") match { + case Seq(JsNumber(col), JsNumber(row)) => + SpatialKey(col.toInt, row.toInt) + case _ => + throw new DeserializationException("SpatialKey expected") + } + } + + implicit object SpaceTimeKeyFormat extends RootJsonFormat[SpaceTimeKey] { + def write(key: SpaceTimeKey) = + JsObject( + "col" -> JsNumber(key.col), + "row" -> JsNumber(key.row), + "instant" -> JsNumber(key.instant) + ) + + def read(value: JsValue): SpaceTimeKey = + value.asJsObject.getFields("col", "row", "instant") match { + case Seq(JsNumber(col), JsNumber(row), JsNumber(time)) => + SpaceTimeKey(col.toInt, row.toInt, time.toLong) + case _ => + throw new DeserializationException("SpaceTimeKey expected") + } + } + + + implicit object TemporalKeyFormat extends RootJsonFormat[TemporalKey] { + def write(key: TemporalKey) = + JsObject( + "instant" -> JsNumber(key.instant) + ) + + def read(value: JsValue): TemporalKey = + value.asJsObject.getFields("instant") match { + case Seq(JsNumber(time)) => + TemporalKey(time.toLong) + case _ => + throw new DeserializationException("TemporalKey expected") + } + } + + implicit def keyBoundsFormat[K: JsonFormat]: RootJsonFormat[KeyBounds[K]] = + new RootJsonFormat[KeyBounds[K]] { + def write(keyBounds: KeyBounds[K]) = + JsObject( + "minKey" -> keyBounds.minKey.toJson, + "maxKey" -> keyBounds.maxKey.toJson + ) + + def read(value: JsValue): KeyBounds[K] = + value.asJsObject.getFields("minKey", "maxKey") match { + case Seq(minKey, maxKey) => + KeyBounds(minKey.convertTo[K], maxKey.convertTo[K]) + case _ => + throw new DeserializationException("${classOf[KeyBounds[K]] expected") + } + } + + implicit object LayerIdFormat extends RootJsonFormat[LayerId] { + def write(id: LayerId) = + JsObject( + "name" -> JsString(id.name), + "zoom" -> JsNumber(id.zoom) + ) + + def read(value: JsValue): LayerId = + value.asJsObject.getFields("name", "zoom") match { + case Seq(JsString(name), JsNumber(zoom)) => + LayerId(name, zoom.toInt) + case _ => + throw new DeserializationException("LayerId expected") + } + } + + implicit object LayoutDefinitionFormat extends RootJsonFormat[LayoutDefinition] { + def write(obj: LayoutDefinition) = + JsObject( + "extent" -> obj.extent.toJson, + "tileLayout" -> obj.tileLayout.toJson + ) + + def read(json: JsValue) = + json.asJsObject.getFields("extent", "tileLayout") match { + case Seq(extent, tileLayout) => + LayoutDefinition(extent.convertTo[Extent], tileLayout.convertTo[TileLayout]) + case _ => + throw new DeserializationException("LayoutDefinition expected") + } + } + + implicit object ZoomedLayoutSchemeFormat extends RootJsonFormat[ZoomedLayoutScheme] { + def write(obj: ZoomedLayoutScheme) = + JsObject( + "crs" -> obj.crs.toJson, + "tileSize" -> obj.tileSize.toJson, + "resolutionThreshold" -> obj.resolutionThreshold.toJson + ) + + def read(json: JsValue) = + json.asJsObject.getFields("crs", "tileSize", "resolutionThreshold") match { + case Seq(crs, tileSize, resolutionThreshold) => + ZoomedLayoutScheme(crs.convertTo[CRS], tileSize.convertTo[Int], resolutionThreshold.convertTo[Double]) + case _ => + throw new DeserializationException("ZoomedLayoutScheme expected") + } + } + + implicit object FloatingLayoutSchemeFormat extends RootJsonFormat[FloatingLayoutScheme] { + def write(obj: FloatingLayoutScheme) = + JsObject( + "tileCols" -> obj.tileCols.toJson, + "tileRows" -> obj.tileRows.toJson + ) + + def read(json: JsValue) = + json.asJsObject.getFields("tileCols", "tileRows") match { + case Seq(tileCols, tileRows) => + FloatingLayoutScheme(tileCols.convertTo[Int], tileRows.convertTo[Int]) + case _ => + throw new DeserializationException("FloatingLayoutScheme expected") + } + } + + /** + * LayoutScheme Format + */ + implicit object LayoutSchemeFormat extends RootJsonFormat[LayoutScheme] { + def write(obj: LayoutScheme) = + obj match { + case scheme: ZoomedLayoutScheme => scheme.toJson + case scheme: FloatingLayoutScheme => scheme.toJson + case _ => + throw new SerializationException("ZoomedLayoutScheme or FloatingLayoutScheme expected") + } + + def read(json: JsValue) = + try { + ZoomedLayoutSchemeFormat.read(json) + } catch { + case _: DeserializationException => + try { + FloatingLayoutSchemeFormat.read(json) + } catch { + case _: Throwable => + throw new DeserializationException("ZoomedLayoutScheme or FloatingLayoutScheme expected") + } + } + } + + implicit def tileLayerMetadataFormat[K: SpatialComponent: JsonFormat] = new RootJsonFormat[TileLayerMetadata[K]] { + def write(metadata: TileLayerMetadata[K]) = + JsObject( + "cellType" -> metadata.cellType.toJson, + "extent" -> metadata.extent.toJson, + "layoutDefinition" -> metadata.layout.toJson, + "crs" -> metadata.crs.toJson, + "bounds" -> metadata.bounds.get.toJson // we will only store non-empty bounds + ) + + def read(value: JsValue): TileLayerMetadata[K] = + value.asJsObject.getFields("cellType", "extent", "layoutDefinition", "crs", "bounds") match { + case Seq(cellType, extent, layoutDefinition, crs, bounds) => + TileLayerMetadata( + cellType.convertTo[CellType], + layoutDefinition.convertTo[LayoutDefinition], + extent.convertTo[Extent], + crs.convertTo[CRS], + bounds.convertTo[KeyBounds[K]] + ) + case _ => + throw new DeserializationException("TileLayerMetadata expected") + } + } + + implicit object RootDateTimeFormat extends RootJsonFormat[ZonedDateTime] { + def write(dt: ZonedDateTime) = JsString(dt.withZoneSameLocal(ZoneOffset.UTC).toString) + + def read(value: JsValue) = + value match { + case JsString(dateStr) => + ZonedDateTime.parse(dateStr) + case _ => + throw new DeserializationException("DateTime expected") + } + } + + implicit object SchemaFormat extends RootJsonFormat[Schema] { + def read(json: JsValue) = (new Schema.Parser).parse(json.toString()) + def write(obj: Schema) = enrichString(obj.toString).parseJson + } + + implicit object ProjectedExtentFormat extends RootJsonFormat[ProjectedExtent] { + def write(projectedExtent: ProjectedExtent) = + JsObject( + "extent" -> projectedExtent.extent.toJson, + "crs" -> projectedExtent.crs.toJson + ) + + def read(value: JsValue): ProjectedExtent = + value.asJsObject.getFields("xmin", "ymin", "xmax", "ymax") match { + case Seq(extent: JsValue, crs: JsValue) => + ProjectedExtent(extent.convertTo[Extent], crs.convertTo[CRS]) + case _ => + throw new DeserializationException(s"ProjectctionExtent [[xmin,ymin,xmax,ymax], crs] expected: $value") + } + } +} + +object JsonCodecs extends JsonCodecs \ No newline at end of file diff --git a/core/src/main/scala/org/locationtech/rasterframes/util/RFKryoRegistrator.scala b/core/src/main/scala/org/locationtech/rasterframes/util/RFKryoRegistrator.scala index 8275c6402..44dd4ca17 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/util/RFKryoRegistrator.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/util/RFKryoRegistrator.scala @@ -25,7 +25,8 @@ import org.locationtech.rasterframes.ref.RasterRef.RasterRefTile import org.locationtech.rasterframes.ref.{DelegatingRasterSource, RasterRef, RasterSource} import org.locationtech.rasterframes.ref._ import com.esotericsoftware.kryo.Kryo - +import geotrellis.raster.io.geotiff.reader.GeoTiffInfo +import geotrellis.spark.store.kryo.KryoRegistrator /** * @@ -33,7 +34,7 @@ import com.esotericsoftware.kryo.Kryo * * @since 10/29/18 */ -class RFKryoRegistrator extends geotrellis.spark.io.kryo.KryoRegistrator { +class RFKryoRegistrator extends KryoRegistrator { override def registerClasses(kryo: Kryo): Unit = { super.registerClasses(kryo) kryo.register(classOf[RasterSource]) @@ -45,6 +46,6 @@ class RFKryoRegistrator extends geotrellis.spark.io.kryo.KryoRegistrator { kryo.register(classOf[HadoopGeoTiffRasterSource]) kryo.register(classOf[GDALRasterSource]) kryo.register(classOf[SimpleRasterInfo]) - kryo.register(classOf[geotrellis.raster.io.geotiff.reader.GeoTiffReader.GeoTiffInfo]) + kryo.register(classOf[GeoTiffInfo]) } } diff --git a/core/src/main/scala/org/locationtech/rasterframes/util/SubdivideSupport.scala b/core/src/main/scala/org/locationtech/rasterframes/util/SubdivideSupport.scala index 24ee2ce2d..cb2f10c14 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/util/SubdivideSupport.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/util/SubdivideSupport.scala @@ -22,8 +22,8 @@ package org.locationtech.rasterframes.util import geotrellis.raster.crop.Crop -import geotrellis.raster.{CellGrid, TileLayout} -import geotrellis.spark.{Bounds, KeyBounds, SpatialComponent, SpatialKey, TileLayerMetadata} +import geotrellis.raster.{CellGrid, Dimensions, TileLayout} +import geotrellis.layer._ import geotrellis.util._ /** @@ -98,9 +98,9 @@ trait SubdivideSupport { } } - implicit class TileHasSubdivide[T <: CellGrid: WithCropMethods](self: T) { + implicit class TileHasSubdivide[T <: CellGrid[Int]: WithCropMethods](self: T) { def subdivide(divs: Int): Seq[T] = { - val (cols, rows) = self.dimensions + val Dimensions(cols, rows) = self.dimensions val (newCols, newRows) = (cols/divs, rows/divs) for { i ← 0 until divs diff --git a/core/src/main/scala/org/locationtech/rasterframes/util/debug/package.scala b/core/src/main/scala/org/locationtech/rasterframes/util/debug/package.scala index e33529b02..f039a4a09 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/util/debug/package.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/util/debug/package.scala @@ -21,37 +21,10 @@ package org.locationtech.rasterframes.util -import org.locationtech.rasterframes._ -import geotrellis.proj4.LatLng -import geotrellis.vector.{Feature, Geometry} -import geotrellis.vector.io.json.JsonFeatureCollection -import spray.json.JsValue - /** * Additional debugging routines. No guarantees these are or will remain stable. * * @since 4/6/18 */ package object debug { - implicit class RasterFrameWithDebug(val self: RasterFrameLayer) { - - /** Renders the whole schema with metadata as a JSON string. */ - def describeFullSchema: String = { - self.schema.prettyJson - } - - /** Renders all the extents in this RasterFrameLayer as GeoJSON in EPSG:4326. This does a full - * table scan and collects **all** the geometry into the driver, and then converts it into a - * Spray JSON data structure. Not performant, and for debugging only. */ - def geoJsonExtents: JsValue = { - import spray.json.DefaultJsonProtocol._ - - val features = self - .select(GEOMETRY_COLUMN, SPATIAL_KEY_COLUMN) - .collect() - .map{ case (p, s) ⇒ Feature(Geometry(p).reproject(self.crs, LatLng), Map("col" -> s.col, "row" -> s.row)) } - - JsonFeatureCollection(features).toJson - } - } } diff --git a/core/src/main/scala/org/locationtech/rasterframes/util/package.scala b/core/src/main/scala/org/locationtech/rasterframes/util/package.scala index 3186c4877..1c5d830dd 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/util/package.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/util/package.scala @@ -22,16 +22,15 @@ package org.locationtech.rasterframes import com.typesafe.scalalogging.Logger -import geotrellis.raster.CellGrid +import geotrellis.layer._ import geotrellis.raster.crop.TileCropMethods -import geotrellis.raster.io.geotiff.reader.GeoTiffReader import geotrellis.raster.mapalgebra.local.LocalTileBinaryOp import geotrellis.raster.mask.TileMaskMethods import geotrellis.raster.merge.TileMergeMethods import geotrellis.raster.prototype.TilePrototypeMethods -import geotrellis.spark.Bounds +import geotrellis.raster.{CellGrid, Grid, GridBounds} import geotrellis.spark.tiling.TilerKeyMethods -import geotrellis.util.{ByteReader, GetComponent} +import geotrellis.util.GetComponent import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression} @@ -40,8 +39,7 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.rf._ import org.apache.spark.sql.types.StringType import org.slf4j.LoggerFactory - -import scala.Boolean.box +import spire.math.Integral /** * Internal utilities. @@ -60,6 +58,12 @@ package object util extends DataFrameRenderers { def asClassTag: ClassTag[T] = ClassTag[T](t.mirror.runtimeClass(t.tpe)) } + implicit class GridHasGridBounds[N: Integral](re: Grid[N]) { + import spire.syntax.integral._ + val in = Integral[N] + def gridBounds: GridBounds[N] = GridBounds(in.zero, in.zero, re.cols - in.one, re.rows - in.one) + } + /** * Type lambda alias for components that have bounds with parameterized key. * @tparam K bounds key type @@ -70,8 +74,8 @@ package object util extends DataFrameRenderers { // Type lambda aliases type WithMergeMethods[V] = V ⇒ TileMergeMethods[V] - type WithPrototypeMethods[V <: CellGrid] = V ⇒ TilePrototypeMethods[V] - type WithCropMethods[V <: CellGrid] = V ⇒ TileCropMethods[V] + type WithPrototypeMethods[V <: CellGrid[Int]] = V ⇒ TilePrototypeMethods[V] + type WithCropMethods[V <: CellGrid[Int]] = V ⇒ TileCropMethods[V] type WithMaskMethods[V] = V ⇒ TileMaskMethods[V] type KeyMethodsProvider[K1, K2] = K1 ⇒ TilerKeyMethods[K1, K2] @@ -157,46 +161,46 @@ package object util extends DataFrameRenderers { analyzer(sqlContext).extendedResolutionRules } - object Shims { - // GT 1.2.1 to 2.0.0 - def toArrayTile[T <: CellGrid](tile: T): T = - tile.getClass.getMethods - .find(_.getName == "toArrayTile") - .map(_.invoke(tile).asInstanceOf[T]) - .getOrElse(tile) - - // GT 1.2.1 to 2.0.0 - def merge[V <: CellGrid: ClassTag: WithMergeMethods](left: V, right: V, col: Int, row: Int): V = { - val merger = implicitly[WithMergeMethods[V]].apply(left) - merger.getClass.getDeclaredMethods - .find(m ⇒ m.getName == "merge" && m.getParameterCount == 3) - .map(_.invoke(merger, right, Int.box(col), Int.box(row)).asInstanceOf[V]) - .getOrElse(merger.merge(right)) - } - - // GT 1.2.1 to 2.0.0 - // only decompress and streaming apply to 1.2.x - // only streaming and withOverviews apply to 2.0.x - // 1.2.x only has a 3-arg readGeoTiffInfo method - // 2.0.x has a 3- and 4-arg readGeoTiffInfo method, but the 3-arg one has different boolean - // parameters than the 1.2.x one - def readGeoTiffInfo(byteReader: ByteReader, - decompress: Boolean, - streaming: Boolean, - withOverviews: Boolean): GeoTiffReader.GeoTiffInfo = { - val reader = GeoTiffReader.getClass.getDeclaredMethods - .find(c ⇒ c.getName == "readGeoTiffInfo" && c.getParameterCount == 4) - .getOrElse( - GeoTiffReader.getClass.getDeclaredMethods - .find(c ⇒ c.getName == "readGeoTiffInfo" && c.getParameterCount == 3) - .getOrElse( - throw new RuntimeException("Could not find method GeoTiffReader.readGeoTiffInfo"))) - - val result = reader.getParameterCount match { - case 3 ⇒ reader.invoke(GeoTiffReader, byteReader, box(decompress), box(streaming)) - case 4 ⇒ reader.invoke(GeoTiffReader, byteReader, box(streaming), box(withOverviews), None) - } - result.asInstanceOf[GeoTiffReader.GeoTiffInfo] - } - } +// object Shims { +// // GT 1.2.1 to 2.0.0 +// def toArrayTile[T <: CellGrid](tile: T): T = +// tile.getClass.getMethods +// .find(_.getName == "toArrayTile") +// .map(_.invoke(tile).asInstanceOf[T]) +// .getOrElse(tile) +// +// // GT 1.2.1 to 2.0.0 +// def merge[V <: CellGrid: ClassTag: WithMergeMethods](left: V, right: V, col: Int, row: Int): V = { +// val merger = implicitly[WithMergeMethods[V]].apply(left) +// merger.getClass.getDeclaredMethods +// .find(m ⇒ m.getName == "merge" && m.getParameterCount == 3) +// .map(_.invoke(merger, right, Int.box(col), Int.box(row)).asInstanceOf[V]) +// .getOrElse(merger.merge(right)) +// } +// +// // GT 1.2.1 to 2.0.0 +// // only decompress and streaming apply to 1.2.x +// // only streaming and withOverviews apply to 2.0.x +// // 1.2.x only has a 3-arg readGeoTiffInfo method +// // 2.0.x has a 3- and 4-arg readGeoTiffInfo method, but the 3-arg one has different boolean +// // parameters than the 1.2.x one +// def readGeoTiffInfo(byteReader: ByteReader, +// decompress: Boolean, +// streaming: Boolean, +// withOverviews: Boolean): GeoTiffReader.GeoTiffInfo = { +// val reader = GeoTiffReader.getClass.getDeclaredMethods +// .find(c ⇒ c.getName == "readGeoTiffInfo" && c.getParameterCount == 4) +// .getOrElse( +// GeoTiffReader.getClass.getDeclaredMethods +// .find(c ⇒ c.getName == "readGeoTiffInfo" && c.getParameterCount == 3) +// .getOrElse( +// throw new RuntimeException("Could not find method GeoTiffReader.readGeoTiffInfo"))) +// +// val result = reader.getParameterCount match { +// case 3 ⇒ reader.invoke(GeoTiffReader, byteReader, box(decompress), box(streaming)) +// case 4 ⇒ reader.invoke(GeoTiffReader, byteReader, box(streaming), box(withOverviews), None) +// } +// result.asInstanceOf[GeoTiffReader.GeoTiffInfo] +// } +// } } diff --git a/core/src/test/scala/examples/CreatingRasterFrames.scala b/core/src/test/scala/examples/CreatingRasterFrames.scala index 8b5c00c72..4e9ca837b 100644 --- a/core/src/test/scala/examples/CreatingRasterFrames.scala +++ b/core/src/test/scala/examples/CreatingRasterFrames.scala @@ -36,8 +36,7 @@ object CreatingRasterFrames extends App { import org.locationtech.rasterframes._ import geotrellis.raster._ import geotrellis.raster.io.geotiff.SinglebandGeoTiff - import geotrellis.spark.io._ - import org.apache.spark.sql._ + import org.apache.spark.sql._ // Next, initialize the `SparkSession`, and call the `withRasterFrames` method on it: @@ -86,6 +85,7 @@ object CreatingRasterFrames extends App { import spray.json._ // The `fold` is required because an `Either` is retured, depending on the key type. + import org.locationtech.rasterframes.util.JsonCodecs._ rf.tileLayerMetadata.fold(_.toJson, _.toJson).prettyPrint spark.stop() diff --git a/core/src/test/scala/examples/Exporting.scala b/core/src/test/scala/examples/Exporting.scala index 25fa321c1..dd6a3d436 100644 --- a/core/src/test/scala/examples/Exporting.scala +++ b/core/src/test/scala/examples/Exporting.scala @@ -20,14 +20,15 @@ package examples import java.nio.file.Files -import org.locationtech.rasterframes._ +import geotrellis.layer._ import geotrellis.raster._ import geotrellis.raster.io.geotiff.SinglebandGeoTiff -import geotrellis.raster.render._ -import geotrellis.spark.{LayerId, SpatialKey} +import geotrellis.spark.store.LayerWriter +import geotrellis.store.LayerId +import geotrellis.store.index.ZCurveKeyIndexMethod import org.apache.spark.sql._ import org.apache.spark.sql.functions._ -import spray.json.JsValue +import org.locationtech.rasterframes._ object Exporting extends App { @@ -152,16 +153,11 @@ object Exporting extends App { val tlRDD = equalized.toTileLayerRDD($"equalized").left.get // First create a GeoTrellis layer writer - import geotrellis.spark.io._ val p = Files.createTempDirectory("gt-store") val writer: LayerWriter[LayerId] = LayerWriter(p.toUri) val layerId = LayerId("equalized", 0) - writer.write(layerId, tlRDD, index.ZCurveKeyIndexMethod) - - // Take a look at the metadata in JSON format: - import spray.json.DefaultJsonProtocol._ - AttributeStore(p.toUri).readMetadata[JsValue](layerId).prettyPrint + writer.write(layerId, tlRDD, ZCurveKeyIndexMethod) spark.stop() } diff --git a/core/src/test/scala/examples/LocalArithmetic.scala b/core/src/test/scala/examples/LocalArithmetic.scala index 428fcc64a..e7b76566e 100644 --- a/core/src/test/scala/examples/LocalArithmetic.scala +++ b/core/src/test/scala/examples/LocalArithmetic.scala @@ -19,11 +19,9 @@ package examples -import org.locationtech.rasterframes._ import geotrellis.raster.io.geotiff.SinglebandGeoTiff -import geotrellis.spark.io.kryo.KryoRegistrator -import org.apache.spark.serializer.KryoSerializer import org.apache.spark.sql._ +import org.locationtech.rasterframes._ /** * Boilerplate test run file @@ -34,10 +32,7 @@ object LocalArithmetic extends App { implicit val spark = SparkSession.builder() .master("local[*]") .appName(getClass.getName) - .config("spark.serializer", classOf[KryoSerializer].getName) - .config("spark.kryoserializer.buffer.max", "500m") - .config("spark.kryo.registrationRequired", "false") - .config("spark.kryo.registrator", classOf[KryoRegistrator].getName) + .withKryoSerialization .getOrCreate() .withRasterFrames diff --git a/core/src/test/scala/examples/MakeTargetRaster.scala b/core/src/test/scala/examples/MakeTargetRaster.scala index f0151c4a1..69f8a9410 100644 --- a/core/src/test/scala/examples/MakeTargetRaster.scala +++ b/core/src/test/scala/examples/MakeTargetRaster.scala @@ -20,13 +20,11 @@ package examples import geotrellis.proj4.CRS +import geotrellis.raster._ import geotrellis.raster.io.geotiff.SinglebandGeoTiff +import geotrellis.raster.mapalgebra.local.TileReducer import geotrellis.util.Filesystem import geotrellis.vector._ -import geotrellis.vector.io._ -import geotrellis.raster._ -import geotrellis.raster.mapalgebra.local.TileReducer -import spray.json.DefaultJsonProtocol._ /** diff --git a/core/src/test/scala/examples/Masking.scala b/core/src/test/scala/examples/Masking.scala index 6270bcef1..11e51c147 100644 --- a/core/src/test/scala/examples/Masking.scala +++ b/core/src/test/scala/examples/Masking.scala @@ -2,7 +2,6 @@ package examples import org.locationtech.rasterframes._ import geotrellis.raster.io.geotiff.SinglebandGeoTiff -import geotrellis.raster.render._ import geotrellis.raster.{mask => _, _} import org.apache.spark.sql._ import org.apache.spark.sql.functions._ @@ -41,11 +40,11 @@ object Masking extends App { val b2 = masked.toRaster(masked("band_2"), 466, 428) val brownToGreen = ColorRamp( - RGBA(166,97,26,255), - RGBA(223,194,125,255), - RGBA(245,245,245,255), - RGBA(128,205,193,255), - RGBA(1,133,113,255) + RGB(166,97,26), + RGB(223,194,125), + RGB(245,245,245), + RGB(128,205,193), + RGB(1,133,113) ).stops(128) val colors = ColorMap.fromQuantileBreaks(maskRF.tile.histogramDouble(), brownToGreen) diff --git a/core/src/test/scala/examples/NDVI.scala b/core/src/test/scala/examples/NDVI.scala index 48a6f6e51..f79a91f05 100644 --- a/core/src/test/scala/examples/NDVI.scala +++ b/core/src/test/scala/examples/NDVI.scala @@ -22,7 +22,6 @@ import java.nio.file.{Files, Paths} import org.locationtech.rasterframes._ import geotrellis.raster._ -import geotrellis.raster.render._ import geotrellis.raster.io.geotiff.{GeoTiff, SinglebandGeoTiff} import org.apache.commons.io.IOUtils import org.apache.spark.sql._ @@ -62,8 +61,13 @@ object NDVI extends App { val pr = rf.toRaster($"ndvi", 233, 214) GeoTiff(pr).write("ndvi.tiff") - val brownToGreen = ColorRamp(RGBA(166, 97, 26, 255), RGBA(223, 194, 125, 255), - RGBA(245, 245, 245, 255), RGBA(128, 205, 193, 255), RGBA(1, 133, 113, 255)) + val brownToGreen = ColorRamp( + RGB(166, 97, 26), + RGB(223, 194, 125), + RGB(245, 245, 245), + RGB(128, 205, 193), + RGB(1, 133, 113) + ) .stops(128) val colors = ColorMap.fromQuantileBreaks(pr.tile.histogramDouble(), brownToGreen) diff --git a/core/src/test/scala/org/locationtech/rasterframes/ExplodeSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/ExplodeSpec.scala index 4768d27b8..3eae60461 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/ExplodeSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/ExplodeSpec.scala @@ -179,7 +179,7 @@ class ExplodeSpec extends TestEnvironment with TestData { val rf = assembled.asLayer(SPATIAL_KEY_COLUMN, tlm) - val (cols, rows) = image.tile.dimensions + val Dimensions(cols, rows) = image.tile.dimensions val recovered = rf.toRaster(TILE_COLUMN, cols, rows, NearestNeighbor) diff --git a/core/src/test/scala/org/locationtech/rasterframes/ExtensionMethodSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/ExtensionMethodSpec.scala index 4f5fe3591..f191f201f 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/ExtensionMethodSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/ExtensionMethodSpec.scala @@ -23,8 +23,7 @@ package org.locationtech.rasterframes import geotrellis.proj4.LatLng import geotrellis.raster.{ByteCellType, GridBounds, TileLayout} -import geotrellis.spark.tiling.{CRSWorldExtent, LayoutDefinition} -import geotrellis.spark.{KeyBounds, SpatialKey, TileLayerMetadata} +import geotrellis.layer._ import org.apache.spark.sql.Encoders import org.locationtech.rasterframes.util._ @@ -67,7 +66,7 @@ class ExtensionMethodSpec extends TestEnvironment with TestData with SubdivideSu it("should find multiple crs columns") { // Not sure why implicit resolution isn't handling this properly. - implicit val enc = Encoders.tuple(crsEncoder, Encoders.STRING, crsEncoder, Encoders.scalaDouble) + implicit val enc = Encoders.tuple(crsSparkEncoder, Encoders.STRING, crsSparkEncoder, Encoders.scalaDouble) val df = Seq((pe.crs, "fred", pe.crs, 34.0)).toDF("c1", "s", "c2", "n") df.crsColumns.size should be(2) } diff --git a/core/src/test/scala/org/locationtech/rasterframes/GeometryFunctionsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/GeometryFunctionsSpec.scala index 2623614bb..9caf47bdb 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/GeometryFunctionsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/GeometryFunctionsSpec.scala @@ -22,8 +22,9 @@ package org.locationtech.rasterframes import geotrellis.proj4.{LatLng, Sinusoidal, WebMercator} -import geotrellis.vector.{Extent, Point => GTPoint} -import org.locationtech.jts.geom._ +import geotrellis.raster.Dimensions +import geotrellis.vector._ +import org.locationtech.jts.geom.{Coordinate, GeometryFactory} import spray.json.JsNumber /** @@ -41,9 +42,9 @@ class GeometryFunctionsSpec extends TestEnvironment with TestData with StandardC val crs = rf.tileLayerMetadata.merge.crs val coords = Seq( - "one" -> GTPoint(-78.6445222907, 38.3957546898).reproject(LatLng, crs).jtsGeom, - "two" -> GTPoint(-78.6601240367, 38.3976614324).reproject(LatLng, crs).jtsGeom, - "three" -> GTPoint( -78.6123381343, 38.4001666769).reproject(LatLng, crs).jtsGeom + "one" -> Point(-78.6445222907, 38.3957546898).reproject(LatLng, crs), + "two" -> Point(-78.6601240367, 38.3976614324).reproject(LatLng, crs), + "three" -> Point( -78.6123381343, 38.4001666769).reproject(LatLng, crs) ) val locs = coords.toDF("id", "point") @@ -57,7 +58,7 @@ class GeometryFunctionsSpec extends TestEnvironment with TestData with StandardC assert(rf.filter(st_contains(GEOMETRY_COLUMN, geomLit(point))).count === 1) assert(rf.filter(st_intersects(GEOMETRY_COLUMN, geomLit(point))).count === 1) assert(rf.filter(GEOMETRY_COLUMN intersects point).count === 1) - assert(rf.filter(GEOMETRY_COLUMN intersects GTPoint(point)).count === 1) + assert(rf.filter(GEOMETRY_COLUMN intersects point).count === 1) assert(rf.filter(GEOMETRY_COLUMN containsGeom point).count === 1) } @@ -138,15 +139,15 @@ class GeometryFunctionsSpec extends TestEnvironment with TestData with StandardC it("should rasterize geometry") { val rf = l8Sample(1).projectedRaster.toLayer.withGeometry() val df = GeomData.features.map(f ⇒ ( - f.geom.reproject(LatLng, rf.crs).jtsGeom, - f.data.fields("id").asInstanceOf[JsNumber].value.intValue() + f.geom.reproject(LatLng, rf.crs), + f.data("id").asInstanceOf[JsNumber].value.intValue() )).toDF("geom", "__fid__") val toRasterize = rf.crossJoin(df) val tlm = rf.tileLayerMetadata.merge - val (cols, rows) = tlm.layout.tileLayout.tileDimensions + val Dimensions(cols, rows) = tlm.layout.tileLayout.tileDimensions val rasterized = toRasterize.withColumn("rasterized", rf_rasterize($"geom", GEOMETRY_COLUMN, $"__fid__", cols, rows)) diff --git a/core/src/test/scala/org/locationtech/rasterframes/RasterFrameSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/RasterFrameSpec.scala index f37c5150a..65e7943bb 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/RasterFrameSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/RasterFrameSpec.scala @@ -29,9 +29,9 @@ import java.time.ZonedDateTime import org.locationtech.rasterframes.util._ import geotrellis.proj4.LatLng import geotrellis.raster.render.{ColorMap, ColorRamp} -import geotrellis.raster.{ProjectedRaster, Tile, TileFeature, TileLayout, UByteCellType} +import geotrellis.raster.{Dimensions, ProjectedRaster, Tile, TileFeature, TileLayout, UByteCellType} import geotrellis.spark._ -import geotrellis.spark.tiling._ +import geotrellis.layer._ import geotrellis.vector.{Extent, ProjectedExtent} import org.apache.spark.sql.functions._ import org.apache.spark.sql.{SQLContext, SparkSession} @@ -207,7 +207,7 @@ class RasterFrameSpec extends TestEnvironment with MetadataKeys it("should convert a GeoTiff to RasterFrameLayer") { val praster: ProjectedRaster[Tile] = sampleGeoTiff.projectedRaster - val (cols, rows) = praster.raster.dimensions + val Dimensions(cols, rows) = praster.raster.dimensions val layoutCols = math.ceil(cols / 128.0).toInt val layoutRows = math.ceil(rows / 128.0).toInt @@ -327,7 +327,7 @@ class RasterFrameSpec extends TestEnvironment with MetadataKeys it("should restitch to raster") { // 774 × 500 val praster: ProjectedRaster[Tile] = sampleGeoTiff.projectedRaster - val (cols, rows) = praster.raster.dimensions + val Dimensions(cols, rows) = praster.raster.dimensions val rf = praster.toLayer(64, 64) val raster = rf.toRaster($"tile", cols, rows) diff --git a/core/src/test/scala/org/locationtech/rasterframes/RasterFunctionsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/RasterFunctionsSpec.scala index f5256a32f..f676e00cb 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/RasterFunctionsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/RasterFunctionsSpec.scala @@ -325,7 +325,7 @@ class RasterFunctionsSpec extends TestEnvironment with RasterMatchers { it("should get the Geometry of a ProjectedRasterTile") { val g = Seq(randPRT).toDF("tile").select(rf_geometry($"tile")).first() - g should be (extent.jtsGeom) + g should be (extent.toPolygon()) checkDocs("rf_geometry") } } diff --git a/core/src/test/scala/org/locationtech/rasterframes/ReprojectGeometrySpec.scala b/core/src/test/scala/org/locationtech/rasterframes/ReprojectGeometrySpec.scala index a58294287..ef677f0e5 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/ReprojectGeometrySpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/ReprojectGeometrySpec.scala @@ -71,7 +71,7 @@ class ReprojectGeometrySpec extends TestEnvironment { } it("should handle one literal crs") { - implicit val enc = Encoders.tuple(jtsGeometryEncoder, jtsGeometryEncoder, crsEncoder) + implicit val enc = Encoders.tuple(jtsGeometryEncoder, jtsGeometryEncoder, crsSparkEncoder) val df = Seq((llLineString, wmLineString, LatLng: CRS)).toDF("ll", "wm", "llCRS") val rp = df.select( @@ -97,7 +97,7 @@ class ReprojectGeometrySpec extends TestEnvironment { } it("should work in SQL") { - implicit val enc = Encoders.tuple(jtsGeometryEncoder, jtsGeometryEncoder, crsEncoder) + implicit val enc = Encoders.tuple(jtsGeometryEncoder, jtsGeometryEncoder, crsSparkEncoder) val df = Seq((llLineString, wmLineString, LatLng: CRS)).toDF("ll", "wm", "llCRS") df.createOrReplaceTempView("geom") diff --git a/core/src/test/scala/org/locationtech/rasterframes/SpatialKeySpec.scala b/core/src/test/scala/org/locationtech/rasterframes/SpatialKeySpec.scala index b99b5c48e..21fc7c886 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/SpatialKeySpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/SpatialKeySpec.scala @@ -22,7 +22,7 @@ package org.locationtech.rasterframes import geotrellis.proj4.LatLng -import geotrellis.vector.Point +import geotrellis.vector._ import org.locationtech.geomesa.curve.Z2SFC /** @@ -41,7 +41,7 @@ class SpatialKeySpec extends TestEnvironment with TestData { val rf = raster.toLayer(raster.tile.cols, raster.tile.rows) it("should add an extent column") { - val expected = raster.extent.jtsGeom + val expected = raster.extent.toPolygon() val result = rf.withGeometry().select(GEOMETRY_COLUMN).first assert(result === expected) } @@ -49,7 +49,7 @@ class SpatialKeySpec extends TestEnvironment with TestData { it("should add a center value") { val expected = raster.extent.center val result = rf.withCenter().select(CENTER_COLUMN).first - assert(result === expected.jtsGeom) + assert(result === expected) } it("should add a center lat/lng value") { diff --git a/core/src/test/scala/org/locationtech/rasterframes/TestData.scala b/core/src/test/scala/org/locationtech/rasterframes/TestData.scala index 1b1fd4022..d65f6e02e 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/TestData.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/TestData.scala @@ -31,8 +31,9 @@ import geotrellis.raster._ import geotrellis.raster.io.geotiff.{MultibandGeoTiff, SinglebandGeoTiff} import geotrellis.spark._ import geotrellis.spark.testkit.TileLayerRDDBuilders -import geotrellis.spark.tiling.LayoutDefinition -import geotrellis.vector.{Extent, ProjectedExtent} +import geotrellis.layer._ +import geotrellis.vector._ +import geotrellis.vector.io.json.JsonFeatureCollection import org.apache.commons.io.IOUtils import org.apache.spark.SparkContext import org.apache.spark.sql.SparkSession @@ -202,13 +203,8 @@ trait TestData { .getResource("/L8-Labels-Elkton-VA.geojson").toURI) Files.readAllLines(p).mkString("\n") } - lazy val features = { - import geotrellis.vector.io._ - import geotrellis.vector.io.json.JsonFeatureCollection - import spray.json.DefaultJsonProtocol._ - import spray.json._ - GeomData.geoJson.parseGeoJson[JsonFeatureCollection].getAllPolygonFeatures[JsObject]() - } + lazy val features = GeomData.geoJson.parseGeoJson[JsonFeatureCollection] + .getAllPolygonFeatures[_root_.io.circe.JsonObject]() } } diff --git a/core/src/test/scala/org/locationtech/rasterframes/TileAssemblerSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/TileAssemblerSpec.scala index 73ba85320..b73beede1 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/TileAssemblerSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/TileAssemblerSpec.scala @@ -43,7 +43,7 @@ class TileAssemblerSpec extends TestEnvironment { val raster = TestData.l8Sample(8).projectedRaster val rf = raster.toLayer(16, 16) val ct = rf.tileLayerMetadata.merge.cellType - val (tileCols, tileRows) = rf.tileLayerMetadata.merge.tileLayout.tileDimensions + val Dimensions(tileCols, tileRows) = rf.tileLayerMetadata.merge.tileLayout.tileDimensions val exploded = rf.select($"spatial_key", rf_explode_tiles($"tile")) diff --git a/core/src/test/scala/org/locationtech/rasterframes/TileUDTSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/TileUDTSpec.scala index a0dd214b7..62ddeeb70 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/TileUDTSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/TileUDTSpec.scala @@ -21,7 +21,7 @@ package org.locationtech.rasterframes import geotrellis.raster -import geotrellis.raster.{CellType, NoNoData, Tile} +import geotrellis.raster.{CellType, Dimensions, NoNoData, Tile} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.rf._ import org.apache.spark.sql.types.StringType @@ -46,9 +46,9 @@ class TileUDTSpec extends TestEnvironment with TestData with Inspectors { val ct = functions.cellTypes().filter(_ != "bool") def forEveryConfig(test: Tile ⇒ Unit): Unit = { - forEvery(tileSizes.combinations(2).toSeq) { case Seq(cols, rows) ⇒ + forEvery(tileSizes.combinations(2).toSeq) { case Seq(tc, tr) ⇒ forEvery(ct) { c ⇒ - val tile = randomTile(cols, rows, CellType.fromName(c)) + val tile = randomTile(tc, tr, CellType.fromName(c)) test(tile) } } @@ -85,7 +85,7 @@ class TileUDTSpec extends TestEnvironment with TestData with Inspectors { forEveryConfig { tile ⇒ val row = TileType.serialize(tile) val wrapper = row.to[Tile] - val (cols,rows) = wrapper.dimensions + val Dimensions(cols,rows) = wrapper.dimensions val indexes = Seq((0, 0), (cols - 1, rows - 1), (cols/2, rows/2), (1, 1)) forAll(indexes) { case (c, r) ⇒ assert(wrapper.get(c, r) === tile.get(c, r)) diff --git a/core/src/test/scala/org/locationtech/rasterframes/encoders/CatalystSerializerSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/encoders/CatalystSerializerSpec.scala index a3f50693b..6eaa38b18 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/encoders/CatalystSerializerSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/encoders/CatalystSerializerSpec.scala @@ -25,8 +25,7 @@ import java.time.ZonedDateTime import geotrellis.proj4._ import geotrellis.raster.{CellSize, CellType, TileLayout, UShortUserDefinedNoDataCellType} -import geotrellis.spark.tiling.LayoutDefinition -import geotrellis.spark.{KeyBounds, SpaceTimeKey, SpatialKey, TileLayerMetadata} +import geotrellis.layer._ import geotrellis.vector.{Extent, ProjectedExtent} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.locationtech.rasterframes.{TestData, TestEnvironment} diff --git a/core/src/test/scala/org/locationtech/rasterframes/encoders/EncodingSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/encoders/EncodingSpec.scala index 421b449f8..bde90fb8e 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/encoders/EncodingSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/encoders/EncodingSpec.scala @@ -28,7 +28,7 @@ import org.locationtech.rasterframes._ import org.locationtech.jts.geom.Envelope import geotrellis.proj4._ import geotrellis.raster.{CellType, Tile, TileFeature} -import geotrellis.spark.{SpaceTimeKey, SpatialKey, TemporalProjectedExtent, TileLayerMetadata} +import geotrellis.layer._ import geotrellis.vector.{Extent, ProjectedExtent} import org.apache.spark.sql.Row import org.apache.spark.sql.functions._ @@ -155,7 +155,7 @@ class EncodingSpec extends TestEnvironment with TestData { describe("Dataframe encoding ops on spatial types") { it("should code RDD[Point]") { - val points = Seq(null, extent.center.jtsGeom, null) + val points = Seq(null, extent.center, null) val ds = points.toDS write(ds) assert(ds.collect().toSeq === points) diff --git a/core/src/test/scala/org/locationtech/rasterframes/expressions/ProjectedLayerMetadataAggregateSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/expressions/ProjectedLayerMetadataAggregateSpec.scala index e33f74c18..09ee27903 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/expressions/ProjectedLayerMetadataAggregateSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/expressions/ProjectedLayerMetadataAggregateSpec.scala @@ -23,7 +23,7 @@ package org.locationtech.rasterframes.expressions import geotrellis.raster.Tile import geotrellis.spark._ -import geotrellis.spark.tiling.FloatingLayoutScheme +import geotrellis.layer._ import geotrellis.vector.{Extent, ProjectedExtent} import org.locationtech.rasterframes._ import org.locationtech.rasterframes.encoders.serialized_literal diff --git a/core/src/test/scala/org/locationtech/rasterframes/expressions/XZ2IndexerSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/expressions/XZ2IndexerSpec.scala index fc62949dd..9048bcbd7 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/expressions/XZ2IndexerSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/expressions/XZ2IndexerSpec.scala @@ -68,7 +68,7 @@ class XZ2IndexerSpec extends TestEnvironment with Inspectors { } it("should create index from Geometry") { val crs: CRS = LatLng - val df = testExtents.map(_.jtsGeom).map(Tuple1.apply).toDF("extent") + val df = testExtents.map(Tuple1.apply).toDF("extent") val indexes = df.select(rf_spatial_index($"extent", serialized_literal(crs))).collect() forEvery(indexes.zip(expected)) { case (i, e) => diff --git a/core/src/test/scala/org/locationtech/rasterframes/ref/RasterRefSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/ref/RasterRefSpec.scala index 80f0a7082..d424bbba4 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/ref/RasterRefSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/ref/RasterRefSpec.scala @@ -24,7 +24,7 @@ package org.locationtech.rasterframes.ref import java.net.URI import geotrellis.raster.{ByteConstantNoDataCellType, Tile} -import geotrellis.vector.Extent +import geotrellis.vector._ import org.apache.spark.SparkException import org.apache.spark.sql.Encoders import org.locationtech.rasterframes.{TestEnvironment, _} diff --git a/core/src/test/scala/org/locationtech/rasterframes/ref/RasterSourceSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/ref/RasterSourceSpec.scala index 6b3371ea3..d16382429 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/ref/RasterSourceSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/ref/RasterSourceSpec.scala @@ -23,10 +23,12 @@ package org.locationtech.rasterframes.ref import java.net.URI -import org.locationtech.rasterframes._ -import geotrellis.vector.Extent +import geotrellis.raster.RasterExtent +import geotrellis.vector._ import org.apache.spark.sql.rf.RasterSourceUDT -import org.locationtech.rasterframes.model.{FixedRasterExtent, TileDimensions} +import org.locationtech.rasterframes._ +import org.locationtech.rasterframes.model._ +import org.locationtech.rasterframes.util.GridHasGridBounds class RasterSourceSpec extends TestEnvironment with TestData { @@ -71,7 +73,7 @@ class RasterSourceSpec extends TestEnvironment with TestData { d._2 should be <= NOMINAL_TILE_SIZE } - val re = FixedRasterExtent( + val re = RasterExtent( Extent(1.4455356755667E7, -3335851.5589995002, 1.55673072753335E7, -2223901.039333), 2400, 2400 ) diff --git a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotiff/GeoTiffRelation.scala b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotiff/GeoTiffRelation.scala index 81aab93af..471b84637 100644 --- a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotiff/GeoTiffRelation.scala +++ b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotiff/GeoTiffRelation.scala @@ -24,11 +24,10 @@ package org.locationtech.rasterframes.datasource.geotiff import java.net.URI import com.typesafe.scalalogging.Logger -import geotrellis.proj4.CRS +import geotrellis.layer._ import geotrellis.spark._ -import geotrellis.spark.io._ -import geotrellis.spark.io.hadoop._ -import geotrellis.util._ +import geotrellis.proj4.CRS +import geotrellis.store.hadoop.util.HdfsRangeReader import geotrellis.vector.Extent import org.apache.hadoop.fs.Path import org.apache.spark.rdd.RDD @@ -41,6 +40,9 @@ import org.locationtech.rasterframes._ import org.locationtech.rasterframes.encoders.CatalystSerializer._ import org.locationtech.rasterframes.util._ import org.slf4j.LoggerFactory +import JsonCodecs._ +import geotrellis.raster.CellGrid +import geotrellis.spark.store.hadoop.{HadoopGeoTiffRDD, HadoopGeoTiffReader} /** * Spark SQL data source over a single GeoTiff file. Works best with CoG compliant ones. @@ -112,9 +114,16 @@ case class GeoTiffRelation(sqlContext: SQLContext, uri: URI) extends BaseRelatio } } else { + // TODO: get rid of this sloppy type leakage hack. Might not be necessary anyway. + def toArrayTile[T <: CellGrid[Int]](tile: T): T = + tile.getClass.getMethods + .find(_.getName == "toArrayTile") + .map(_.invoke(tile).asInstanceOf[T]) + .getOrElse(tile) + //logger.warn("GeoTIFF is not already tiled. In-memory read required: " + uri) val geotiff = HadoopGeoTiffReader.readMultiband(new Path(uri)) - val rdd = sqlContext.sparkContext.makeRDD(Seq((geotiff.projectedExtent, Shims.toArrayTile(geotiff.tile)))) + val rdd = sqlContext.sparkContext.makeRDD(Seq((geotiff.projectedExtent, toArrayTile(geotiff.tile)))) rdd.tileToLayout(tlm) .map { case (sk, tiles) ⇒ diff --git a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisCatalog.scala b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisCatalog.scala index 11edc1d5f..b296f19e6 100644 --- a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisCatalog.scala +++ b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisCatalog.scala @@ -23,7 +23,7 @@ package org.locationtech.rasterframes.datasource.geotrellis import java.net.URI -import geotrellis.spark.io.AttributeStore +import geotrellis.store._ import org.apache.spark.annotation.Experimental import org.apache.spark.rdd.RDD import org.apache.spark.sql._ @@ -32,8 +32,6 @@ import org.apache.spark.sql.rf.VersionShims import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType import org.locationtech.rasterframes.datasource.geotrellis.GeoTrellisCatalog.GeoTrellisCatalogRelation -import spray.json.DefaultJsonProtocol._ -import spray.json._ /** * @@ -64,9 +62,10 @@ object GeoTrellisCatalog { private lazy val layers = { // The attribute groups are processed separately and joined at the end to // maintain a semblance of separation in the resulting schema. - val mergeId = (id: Int, json: JsObject) ⇒ { - val jid = id.toJson - json.copy(fields = json.fields + ("index" -> jid) ) + val mergeId = (id: Int, json: io.circe.JsonObject) ⇒ { + import io.circe.syntax._ + val jid = id.asJson + json.add("index", jid).asJson } implicit val layerStuffEncoder: Encoder[(Int, Layer)] = Encoders.tuple( @@ -82,16 +81,16 @@ object GeoTrellisCatalog { val indexedLayers = layerSpecs .toDF("index", "layer") - val headerRows = layerSpecs - .map{case (index, layer) ⇒(index, attributes.readHeader[JsObject](layer.id))} + val headerRows = layerSpecs + .map{case (index, layer) ⇒(index, attributes.readHeader[io.circe.JsonObject](layer.id))} .map(mergeId.tupled) - .map(_.compactPrint) + .map(io.circe.Printer.noSpaces.pretty) .toDS val metadataRows = layerSpecs - .map{case (index, layer) ⇒ (index, attributes.readMetadata[JsObject](layer.id))} + .map{case (index, layer) ⇒ (index, attributes.readMetadata[io.circe.JsonObject](layer.id))} .map(mergeId.tupled) - .map(_.compactPrint) + .map(io.circe.Printer.noSpaces.pretty) .toDS val headers = VersionShims.readJson(sqlContext, broadcast(headerRows)) diff --git a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisLayerDataSource.scala b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisLayerDataSource.scala index d12ea1e17..f4958a7b6 100644 --- a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisLayerDataSource.scala +++ b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisLayerDataSource.scala @@ -23,11 +23,11 @@ package org.locationtech.rasterframes.datasource.geotrellis import java.net.URI +import geotrellis.spark.store.LayerWriter import org.locationtech.rasterframes._ import org.locationtech.rasterframes.datasource.DataSourceOptions -import geotrellis.spark._ -import geotrellis.spark.io._ -import geotrellis.spark.io.index.ZCurveKeyIndexMethod +import geotrellis.store._ +import geotrellis.store.index.ZCurveKeyIndexMethod import org.apache.spark.annotation.Experimental import org.apache.spark.sql._ import org.apache.spark.sql.sources._ diff --git a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisRelation.scala b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisRelation.scala index 49a7a0af0..ec4f5035c 100644 --- a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisRelation.scala +++ b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisRelation.scala @@ -27,11 +27,12 @@ import java.sql.{Date, Timestamp} import java.time.{ZoneOffset, ZonedDateTime} import com.typesafe.scalalogging.Logger +import geotrellis.layer.{Metadata => LMetadata, _} import geotrellis.raster.{CellGrid, MultibandTile, Tile, TileFeature} -import geotrellis.spark.io._ -import geotrellis.spark.io.avro.AvroRecordCodec +import geotrellis.spark.store.{FilteringLayerReader, LayerReader} import geotrellis.spark.util.KryoWrapper -import geotrellis.spark.{LayerId, Metadata, SpatialKey, TileLayerMetadata, _} +import geotrellis.store._ +import geotrellis.store.avro.AvroRecordCodec import geotrellis.util._ import geotrellis.vector._ import org.apache.avro.Schema @@ -49,6 +50,7 @@ import org.locationtech.rasterframes.datasource.geotrellis.TileFeatureSupport._ import org.locationtech.rasterframes.rules.SpatialFilters.{Contains => sfContains, Intersects => sfIntersects} import org.locationtech.rasterframes.rules.TemporalFilters.{BetweenDates, BetweenTimes} import org.locationtech.rasterframes.rules.{SpatialRelationReceiver, splitFilters} +import org.locationtech.rasterframes.util.JsonCodecs._ import org.locationtech.rasterframes.util.SubdivideSupport._ import org.locationtech.rasterframes.util._ import org.slf4j.LoggerFactory @@ -182,7 +184,7 @@ case class GeoTrellisRelation(sqlContext: SQLContext, StructType((keyFields :+ extentField) ++ tileFields) } - type BLQ[K, T] = BoundLayerQuery[K, TileLayerMetadata[K], RDD[(K, T)] with Metadata[TileLayerMetadata[K]]] + type BLQ[K, T] = BoundLayerQuery[K, TileLayerMetadata[K], RDD[(K, T)] with LMetadata[TileLayerMetadata[K]]] def applyFilter[K: Boundable: SpatialComponent, T](query: BLQ[K, T], predicate: Filter): BLQ[K, T] = { predicate match { @@ -193,9 +195,9 @@ case class GeoTrellisRelation(sqlContext: SQLContext, Intersects(Extent(right.getEnvelopeInternal)) )) case sfIntersects(C.EX, rhs: geom.Point) ⇒ - query.where(Contains(Point(rhs))) + query.where(Contains(rhs)) case sfContains(C.EX, rhs: geom.Point) ⇒ - query.where(Contains(Point(rhs))) + query.where(Contains(rhs)) case sfIntersects(C.EX, rhs) ⇒ query.where(Intersects(Extent(rhs.getEnvelopeInternal))) case _ ⇒ @@ -249,13 +251,13 @@ case class GeoTrellisRelation(sqlContext: SQLContext, } } - private def subdivider[K: SpatialComponent, T <: CellGrid: WithCropMethods](divs: Int) = (p: (K, T)) ⇒ { + private def subdivider[K: SpatialComponent, T <: CellGrid[Int]: WithCropMethods](divs: Int) = (p: (K, T)) ⇒ { val newKeys = p._1.subdivide(divs) val newTiles = p._2.subdivide(divs) newKeys.zip(newTiles) } - private def query[T <: CellGrid: WithCropMethods: WithMergeMethods: AvroRecordCodec: ClassTag](reader: FilteringLayerReader[LayerId], columnIndexes: Seq[Int]) = { + private def query[T <: CellGrid[Int]: WithCropMethods: WithMergeMethods: AvroRecordCodec: ClassTag](reader: FilteringLayerReader[LayerId], columnIndexes: Seq[Int]) = { subdividedTileLayerMetadata.fold( // Without temporal key case (tlm: TileLayerMetadata[SpatialKey]) ⇒ { @@ -277,7 +279,7 @@ case class GeoTrellisRelation(sqlContext: SQLContext, .map { case (sk: SpatialKey, tile: T) ⇒ val entries = columnIndexes.map { case 0 ⇒ sk - case 1 ⇒ trans.keyToExtent(sk).jtsGeom + case 1 ⇒ trans.keyToExtent(sk).toPolygon() case 2 ⇒ tile match { case t: Tile ⇒ t case t: TileFeature[Tile @unchecked, TileFeatureData @unchecked] ⇒ t.tile @@ -313,7 +315,7 @@ case class GeoTrellisRelation(sqlContext: SQLContext, case 0 ⇒ sk case 1 ⇒ stk.temporalKey case 2 ⇒ new Timestamp(stk.temporalKey.instant) - case 3 ⇒ trans.keyToExtent(stk).jtsGeom + case 3 ⇒ trans.keyToExtent(stk).toPolygon() case 4 ⇒ tile match { case t: Tile ⇒ t case t: TileFeature[Tile @unchecked, TileFeatureData @unchecked] ⇒ t.tile diff --git a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/Layer.scala b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/Layer.scala index 9f90c96fd..f7fb8b7d5 100644 --- a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/Layer.scala +++ b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/Layer.scala @@ -24,7 +24,7 @@ package org.locationtech.rasterframes.datasource.geotrellis import java.net.URI import org.locationtech.rasterframes.encoders.DelegatingSubfieldEncoder -import geotrellis.spark.LayerId +import geotrellis.store.LayerId import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.locationtech.rasterframes diff --git a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/TileFeatureSupport.scala b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/TileFeatureSupport.scala index 67ea65510..a2ca0e7de 100644 --- a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/TileFeatureSupport.scala +++ b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/TileFeatureSupport.scala @@ -36,7 +36,7 @@ import scala.reflect.ClassTag trait TileFeatureSupport { - implicit class TileFeatureMethodsWrapper[V <: CellGrid: ClassTag: WithMergeMethods: WithPrototypeMethods: WithCropMethods: WithMaskMethods, D: MergeableData](val self: TileFeature[V, D]) + implicit class TileFeatureMethodsWrapper[V <: CellGrid[Int]: ClassTag: WithMergeMethods: WithPrototypeMethods: WithCropMethods: WithMaskMethods, D: MergeableData](val self: TileFeature[V, D]) extends TileMergeMethods[TileFeature[V, D]] with TilePrototypeMethods[TileFeature[V,D]] with TileCropMethods[TileFeature[V,D]] @@ -47,7 +47,7 @@ trait TileFeatureSupport { TileFeature(self.tile.merge(other.tile), MergeableData[D].merge(self.data,other.data)) def merge(other: TileFeature[V, D], col: Int, row: Int): TileFeature[V, D] = - TileFeature(Shims.merge(self.tile, other.tile, col, row), MergeableData[D].merge(self.data, other.data)) + TileFeature(self.tile.merge(other.tile, col, row), MergeableData[D].merge(self.data, other.data)) override def merge(extent: Extent, otherExtent: Extent, other: TileFeature[V, D], method: ResampleMethod): TileFeature[V, D] = TileFeature(self.tile.merge(extent, otherExtent, other.tile, method), MergeableData[D].merge(self.data,other.data)) @@ -61,7 +61,7 @@ trait TileFeatureSupport { override def crop(srcExtent: Extent, extent: Extent, options: Crop.Options): TileFeature[V, D] = TileFeature(self.tile.crop(srcExtent, extent, options), self.data) - override def crop(gb: GridBounds, options: Crop.Options): TileFeature[V, D] = + override def crop(gb: GridBounds[Int], options: Crop.Options): TileFeature[V, D] = TileFeature(self.tile.crop(gb, options), self.data) override def localMask(r: TileFeature[V, D], readMask: Int, writeMask: Int): TileFeature[V, D] = diff --git a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/package.scala b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/package.scala index c4a7dc425..402805d64 100644 --- a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/package.scala +++ b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/package.scala @@ -24,10 +24,10 @@ import java.net.URI import org.apache.spark.sql._ import org.apache.spark.sql.functions._ -import _root_.geotrellis.spark.LayerId import org.locationtech.rasterframes._ import shapeless.tag.@@ import shapeless.tag +import _root_.geotrellis.store.{Layer => _, _} package object geotrellis extends DataSourceOptions { implicit val layerEncoder = Layer.layerEncoder diff --git a/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotiff/GeoTiffDataSourceSpec.scala b/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotiff/GeoTiffDataSourceSpec.scala index 817d7d5bf..ef4de9624 100644 --- a/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotiff/GeoTiffDataSourceSpec.scala +++ b/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotiff/GeoTiffDataSourceSpec.scala @@ -50,7 +50,7 @@ class GeoTiffDataSourceSpec val rf = spark.read.format("geotiff").load(cogPath.toASCIIString).asLayer val tlm = rf.tileLayerMetadata.left.get - val gb = tlm.gridBounds + val gb = tlm.tileBounds assert(gb.colMax > gb.colMin) assert(gb.rowMax > gb.rowMin) } @@ -71,7 +71,7 @@ class GeoTiffDataSourceSpec ).first.toSeq.toString() ) val tlm = rf.tileLayerMetadata.left.get - val gb = tlm.gridBounds + val gb = tlm.tileBounds assert(gb.rowMax > gb.rowMin) assert(gb.colMax > gb.colMin) diff --git a/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisCatalogSpec.scala b/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisCatalogSpec.scala index 8fea43906..56990bc78 100644 --- a/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisCatalogSpec.scala +++ b/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisCatalogSpec.scala @@ -22,9 +22,9 @@ package org.locationtech.rasterframes.datasource.geotrellis import org.locationtech.rasterframes._ import geotrellis.proj4.LatLng -import geotrellis.spark._ -import geotrellis.spark.io._ -import geotrellis.spark.io.index.ZCurveKeyIndexMethod +import geotrellis.store._ +import geotrellis.spark.store.LayerWriter +import geotrellis.store.index.ZCurveKeyIndexMethod import org.apache.hadoop.fs.FileUtil import org.locationtech.rasterframes.TestEnvironment import org.scalatest.BeforeAndAfter diff --git a/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisDataSourceSpec.scala b/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisDataSourceSpec.scala index 42b5c7c33..907bdf5f6 100644 --- a/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisDataSourceSpec.scala +++ b/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisDataSourceSpec.scala @@ -24,6 +24,7 @@ import java.io.File import java.sql.Timestamp import java.time.ZonedDateTime +import geotrellis.layer._ import org.locationtech.rasterframes._ import org.locationtech.rasterframes.datasource.DataSourceOptions import org.locationtech.rasterframes.rules._ @@ -33,11 +34,10 @@ import geotrellis.raster._ import geotrellis.raster.resample.NearestNeighbor import geotrellis.raster.testkit.RasterMatchers import geotrellis.spark._ -import geotrellis.spark.io._ -import geotrellis.spark.io.avro.AvroRecordCodec -import geotrellis.spark.io.avro.codecs.Implicits._ -import geotrellis.spark.io.index.ZCurveKeyIndexMethod -import geotrellis.spark.tiling.ZoomedLayoutScheme +import geotrellis.spark.store.LayerWriter +import geotrellis.store._ +import geotrellis.store.avro.AvroRecordCodec +import geotrellis.store.index.ZCurveKeyIndexMethod import geotrellis.vector._ import org.apache.avro.generic._ import org.apache.avro.{Schema, SchemaBuilder} @@ -141,7 +141,7 @@ class GeoTrellisDataSourceSpec val boundKeys = KeyBounds(SpatialKey(3, 4), SpatialKey(4, 4)) val bbox = testRdd.metadata.layout .mapTransform(boundKeys.toGridBounds()) - .jtsGeom + .toPolygon() val wc = layerReader.loadLayer(layer).withCenter() withClue("literate API") { @@ -239,7 +239,7 @@ class GeoTrellisDataSourceSpec assert(rf.count === (TestData.sampleTileLayerRDD.count * subs * subs)) - val (width, height) = sampleGeoTiff.tile.dimensions + val Dimensions(width, height) = sampleGeoTiff.tile.dimensions val raster = rf.toRaster(rf.tileColumns.head, width, height, NearestNeighbor) @@ -309,7 +309,7 @@ class GeoTrellisDataSourceSpec it("should *not* support extent filter against a UDF") { val targetKey = testRdd.metadata.mapTransform(pt1) - val mkPtFcn = sparkUdf((_: Row) ⇒ { Point(-88, 60).jtsGeom }) + val mkPtFcn = sparkUdf((_: Row) ⇒ { Point(-88, 60) }) val df = layerReader .loadLayer(layer) @@ -412,7 +412,7 @@ class GeoTrellisDataSourceSpec .loadLayer(layer) .where($"timestamp" >= Timestamp.valueOf(now.minusDays(1).toLocalDateTime)) .where($"timestamp" <= Timestamp.valueOf(now.plusDays(1).toLocalDateTime)) - .where(st_intersects(GEOMETRY_COLUMN, geomLit(pt1.jtsGeom))) + .where(st_intersects(GEOMETRY_COLUMN, geomLit(pt1))) assert(numFilters(df) == 1) } @@ -422,7 +422,7 @@ class GeoTrellisDataSourceSpec it("should handle renamed spatial filter columns") { val df = layerReader .loadLayer(layer) - .where(GEOMETRY_COLUMN intersects region.jtsGeom) + .where(GEOMETRY_COLUMN intersects region) .withColumnRenamed(GEOMETRY_COLUMN.columnName, "foobar") assert(numFilters(df) === 1) @@ -432,7 +432,7 @@ class GeoTrellisDataSourceSpec it("should handle dropped spatial filter columns") { val df = layerReader .loadLayer(layer) - .where(GEOMETRY_COLUMN intersects region.jtsGeom) + .where(GEOMETRY_COLUMN intersects region) .drop(GEOMETRY_COLUMN) assert(numFilters(df) === 1) diff --git a/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotrellis/TileFeatureSupportSpec.scala b/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotrellis/TileFeatureSupportSpec.scala index 0cf7e358c..0c3ed942c 100644 --- a/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotrellis/TileFeatureSupportSpec.scala +++ b/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotrellis/TileFeatureSupportSpec.scala @@ -21,16 +21,17 @@ package org.locationtech.rasterframes.datasource.geotrellis +import geotrellis.layer.LayoutDefinition import org.locationtech.rasterframes._ import org.locationtech.rasterframes.datasource.geotrellis.TileFeatureSupport._ -import org.locationtech.rasterframes.util.{WithCropMethods, WithMaskMethods, WithMergeMethods, WithPrototypeMethods} +import org.locationtech.rasterframes.util._ import geotrellis.proj4.LatLng import geotrellis.raster.crop.Crop import geotrellis.raster.rasterize.Rasterizer import geotrellis.raster.resample.Bilinear -import geotrellis.raster.{CellGrid, GridBounds, IntCellType, ShortCellType, ShortConstantNoDataCellType, Tile, TileFeature, TileLayout} +import geotrellis.raster._ import geotrellis.spark.tiling.Implicits._ -import geotrellis.spark.tiling._ +import geotrellis.layer._ import geotrellis.vector.{Extent, ProjectedExtent} import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD @@ -133,7 +134,7 @@ class TileFeatureSupportSpec extends TestEnvironment } } - private def testAllOps[V <: CellGrid: ClassTag: WithMergeMethods: WithPrototypeMethods: + private def testAllOps[V <: CellGrid[Int]: ClassTag: WithMergeMethods: WithPrototypeMethods: WithCropMethods: WithMaskMethods, D: MergeableData](tf1: TileFeature[V, D], tf2: TileFeature[V, D]) = { assert(tf1.prototype(20, 20) == TileFeature(tf1.tile.prototype(20, 20), MergeableData[D].prototype(tf1.data))) diff --git a/experimental/src/it/scala/org/locationtech/rasterframes/experimental/datasource/awspds/L8CatalogRelationTest.scala b/experimental/src/it/scala/org/locationtech/rasterframes/experimental/datasource/awspds/L8CatalogRelationTest.scala index ea202b726..d5373be34 100644 --- a/experimental/src/it/scala/org/locationtech/rasterframes/experimental/datasource/awspds/L8CatalogRelationTest.scala +++ b/experimental/src/it/scala/org/locationtech/rasterframes/experimental/datasource/awspds/L8CatalogRelationTest.scala @@ -114,7 +114,7 @@ class L8CatalogRelationTest extends TestEnvironment { val scene = catalog .where( to_date($"acquisition_date") === to_date(lit("2019-07-03")) && - st_intersects(st_geometry($"bounds_wgs84"), geomLit(aoiLL.jtsGeom)) + st_intersects(st_geometry($"bounds_wgs84"), geomLit(aoiLL)) ) .orderBy("cloud_cover_pct") .limit(1) diff --git a/project/RFDependenciesPlugin.scala b/project/RFDependenciesPlugin.scala index 20cca567f..47eb4fb6b 100644 --- a/project/RFDependenciesPlugin.scala +++ b/project/RFDependenciesPlugin.scala @@ -41,12 +41,14 @@ object RFDependenciesPlugin extends AutoPlugin { } val scalatest = "org.scalatest" %% "scalatest" % "3.0.3" % Test - val shapeless = "com.chuusai" %% "shapeless" % "2.3.2" - val `jts-core` = "org.locationtech.jts" % "jts-core" % "1.16.0" - val `geotrellis-contrib-vlm` = "com.azavea.geotrellis" %% "geotrellis-contrib-vlm" % "2.12.0" - val `geotrellis-contrib-gdal` = "com.azavea.geotrellis" %% "geotrellis-contrib-gdal" % "2.12.0" + val shapeless = "com.chuusai" %% "shapeless" % "2.3.3" + val `jts-core` = "org.locationtech.jts" % "jts-core" % "1.16.1" + val `slf4j-api` = "org.slf4j" % "slf4j-api" % "1.7.25" + val scaffeine = "com.github.blemale" %% "scaffeine" % "3.1.0" + val `spray-json` = "io.spray" %% "spray-json" % "1.3.4" - val scaffeine = "com.github.blemale" %% "scaffeine" % "2.6.0" + //val `geotrellis-contrib-vlm` = "com.azavea.geotrellis" %% "geotrellis-contrib-vlm" % "2.12.0" + //val `geotrellis-contrib-gdal` = "com.azavea.geotrellis" %% "geotrellis-contrib-gdal" % "2.12.0" } import autoImport._ @@ -60,7 +62,7 @@ object RFDependenciesPlugin extends AutoPlugin { // NB: Make sure to update the Spark version in pyrasterframes/python/setup.py rfSparkVersion := "2.4.4", - rfGeoTrellisVersion := "2.3.3", + rfGeoTrellisVersion := "3.0.0", rfGeoMesaVersion := "2.2.1", ) } diff --git a/pyrasterframes/src/main/scala/org/locationtech/rasterframes/py/PyRFContext.scala b/pyrasterframes/src/main/scala/org/locationtech/rasterframes/py/PyRFContext.scala index c31dccd38..d651f5cb4 100644 --- a/pyrasterframes/src/main/scala/org/locationtech/rasterframes/py/PyRFContext.scala +++ b/pyrasterframes/src/main/scala/org/locationtech/rasterframes/py/PyRFContext.scala @@ -24,8 +24,8 @@ import java.nio.ByteBuffer import geotrellis.proj4.CRS import geotrellis.raster.{CellType, MultibandTile} -import geotrellis.spark.io._ -import geotrellis.spark.{ContextRDD, MultibandTileLayerRDD, SpaceTimeKey, SpatialKey, TileLayerMetadata} +import geotrellis.spark._ +import geotrellis.layer._ import geotrellis.vector.Extent import org.apache.spark.sql._ import org.locationtech.rasterframes @@ -35,7 +35,7 @@ import org.locationtech.rasterframes.ref.{GDALRasterSource, RasterRef, RasterSou import org.locationtech.rasterframes.util.KryoSupport import org.locationtech.rasterframes.{RasterFunctions, _} import spray.json._ - +import org.locationtech.rasterframes.util.JsonCodecs._ import scala.collection.JavaConverters._ /** From 56e04b9678b9de4df8ed31d36b178f60115f996d Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Mon, 21 Oct 2019 16:02:47 -0400 Subject: [PATCH 02/94] Incremental progress commit toward fixing tests. --- .../rasterframes/bench/RasterRefBench.scala | 6 +-- .../rasterframes/bench/TileEncodeBench.scala | 4 +- build.sbt | 19 +++++++-- .../rasterframes/ref/RasterRefIT.scala | 6 +-- .../rasterframes/ref/RasterSourceIT.scala | 20 +++++----- .../apache/spark/sql/rf/RasterSourceUDT.scala | 26 ++++++------ .../expressions/DynamicExtractors.scala | 6 +-- .../generators/RasterSourceToRasterRefs.scala | 4 +- .../transformers/URIToRasterSource.scala | 8 ++-- .../expressions/transformers/XZ2Indexer.scala | 4 +- .../rasterframes/model/TileDimensions.scala | 20 +++++----- .../ref/DelegatingRasterSource.scala | 4 +- .../rasterframes/ref/GDALRasterSource.scala | 4 +- .../ref/HadoopGeoTiffRasterSource.scala | 2 +- .../ref/InMemoryRasterSource.scala | 4 +- ...asterSource.scala => RFRasterSource.scala} | 18 ++++----- .../ref/RangeReaderRasterSource.scala | 2 +- .../rasterframes/ref/RasterRef.scala | 4 +- .../rasterframes/ref/SimpleRasterInfo.scala | 2 +- .../tiles/FixedDelegatingTile.scala | 40 ------------------- .../rasterframes/tiles/InternalRowTile.scala | 2 +- .../tiles/ProjectedRasterTile.scala | 4 +- .../rasterframes/tiles/ShowableTile.scala | 4 +- .../rasterframes/util/DataBiasedOp.scala | 18 ++++++--- .../rasterframes/util/RFKryoRegistrator.scala | 4 +- core/src/test/resources/log4j.properties | 2 + .../rasterframes/ExplodeSpec.scala | 8 ++-- .../rasterframes/ExtensionMethodSpec.scala | 4 +- .../rasterframes/GeometryFunctionsSpec.scala | 2 +- .../rasterframes/RasterFrameSpec.scala | 8 ++-- .../rasterframes/RasterFunctionsSpec.scala | 8 ++-- .../locationtech/rasterframes/TestData.scala | 4 +- .../rasterframes/TileAssemblerSpec.scala | 6 +-- .../rasterframes/TileStatsSpec.scala | 11 ++--- .../encoders/CatalystSerializerSpec.scala | 4 +- .../rasterframes/encoders/EncodingSpec.scala | 22 +++++----- .../expressions/XZ2IndexerSpec.scala | 4 +- .../rasterframes/ref/RasterRefSpec.scala | 10 ++--- .../rasterframes/ref/RasterSourceSpec.scala | 22 +++++----- project/RFAssemblyPlugin.scala | 9 ++++- project/RFDependenciesPlugin.scala | 5 +-- project/RFProjectPlugin.scala | 12 +++--- .../src/main/python/pyrasterframes/version.py | 2 +- .../rasterframes/py/PyRFContext.scala | 4 +- version.sbt | 2 +- 45 files changed, 183 insertions(+), 201 deletions(-) rename core/src/main/scala/org/locationtech/rasterframes/ref/{RasterSource.scala => RFRasterSource.scala} (92%) delete mode 100644 core/src/main/scala/org/locationtech/rasterframes/tiles/FixedDelegatingTile.scala diff --git a/bench/src/main/scala/org/locationtech/rasterframes/bench/RasterRefBench.scala b/bench/src/main/scala/org/locationtech/rasterframes/bench/RasterRefBench.scala index 448fab9c3..a4fd2dfab 100644 --- a/bench/src/main/scala/org/locationtech/rasterframes/bench/RasterRefBench.scala +++ b/bench/src/main/scala/org/locationtech/rasterframes/bench/RasterRefBench.scala @@ -29,7 +29,7 @@ import org.locationtech.rasterframes._ import org.locationtech.rasterframes.expressions.generators.RasterSourceToRasterRefs import org.locationtech.rasterframes.expressions.transformers.RasterRefToTile import org.locationtech.rasterframes.model.TileDimensions -import org.locationtech.rasterframes.ref.RasterSource +import org.locationtech.rasterframes.ref.RFRasterSource import org.openjdk.jmh.annotations._ @BenchmarkMode(Array(Mode.AverageTime)) @@ -43,8 +43,8 @@ class RasterRefBench extends SparkEnv with LazyLogging { @Setup(Level.Trial) def setupData(): Unit = { - val r1 = RasterSource(remoteCOGSingleband1) - val r2 = RasterSource(remoteCOGSingleband2) + val r1 = RFRasterSource(remoteCOGSingleband1) + val r2 = RFRasterSource(remoteCOGSingleband2) singleDF = Seq((r1, r2)).toDF("B1", "B2") .select(RasterRefToTile(RasterSourceToRasterRefs(Some(TileDimensions(r1.dimensions)), Seq(0), $"B1", $"B2"))) diff --git a/bench/src/main/scala/org/locationtech/rasterframes/bench/TileEncodeBench.scala b/bench/src/main/scala/org/locationtech/rasterframes/bench/TileEncodeBench.scala index 20e255b06..5f9982307 100644 --- a/bench/src/main/scala/org/locationtech/rasterframes/bench/TileEncodeBench.scala +++ b/bench/src/main/scala/org/locationtech/rasterframes/bench/TileEncodeBench.scala @@ -30,7 +30,7 @@ import geotrellis.raster.Tile import geotrellis.vector.Extent import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.locationtech.rasterframes.ref.{RasterRef, RasterSource} +import org.locationtech.rasterframes.ref.{RasterRef, RFRasterSource} import org.openjdk.jmh.annotations._ @BenchmarkMode(Array(Mode.AverageTime)) @@ -56,7 +56,7 @@ class TileEncodeBench extends SparkEnv { case "rasterRef" ⇒ val baseCOG = "https://s3-us-west-2.amazonaws.com/landsat-pds/c1/L8/149/039/LC08_L1TP_149039_20170411_20170415_01_T1/LC08_L1TP_149039_20170411_20170415_01_T1_B1.TIF" val extent = Extent(253785.0, 3235185.0, 485115.0, 3471015.0) - tile = RasterRefTile(RasterRef(RasterSource(URI.create(baseCOG)), 0, Some(extent), None)) + tile = RasterRefTile(RasterRef(RFRasterSource(URI.create(baseCOG)), 0, Some(extent), None)) case _ ⇒ tile = randomTile(tileSize, tileSize, cellTypeName) } diff --git a/build.sbt b/build.sbt index 32c7eb531..882ba3b39 100644 --- a/build.sbt +++ b/build.sbt @@ -56,8 +56,6 @@ lazy val core = project `spray-json`, geomesa("z3").value, geomesa("spark-jts").value, -// `geotrellis-contrib-vlm`, -// `geotrellis-contrib-gdal`, spark("core").value % Provided, spark("mllib").value % Provided, spark("sql").value % Provided, @@ -72,14 +70,27 @@ lazy val core = project scaffeine, scalatest ), + /** https://github.com/lucidworks/spark-solr/issues/179 + * Thanks @pomadchin for the tip! */ + dependencyOverrides ++= { + val deps = Seq( + "com.fasterxml.jackson.core" % "jackson-core" % "2.6.7", + "com.fasterxml.jackson.core" % "jackson-databind" % "2.6.7", + "com.fasterxml.jackson.core" % "jackson-annotations" % "2.6.7" + ) + CrossVersion.partialVersion(scalaVersion.value) match { + // if Scala 2.12+ is used + case Some((2, scalaMajor)) if scalaMajor >= 12 => deps + case _ => deps :+ "com.fasterxml.jackson.module" %% "jackson-module-scala" % "2.6.7" + } + }, buildInfoKeys ++= Seq[BuildInfoKey]( - moduleName, version, scalaVersion, sbtVersion, rfGeoTrellisVersion, rfGeoMesaVersion, rfSparkVersion + version, scalaVersion, rfGeoTrellisVersion, rfGeoMesaVersion, rfSparkVersion ), buildInfoPackage := "org.locationtech.rasterframes", buildInfoObject := "RFBuildInfo", buildInfoOptions := Seq( BuildInfoOption.ToMap, - BuildInfoOption.BuildTime, BuildInfoOption.ToJson ) ) diff --git a/core/src/it/scala/org/locationtech/rasterframes/ref/RasterRefIT.scala b/core/src/it/scala/org/locationtech/rasterframes/ref/RasterRefIT.scala index 88b5b8617..a2f238891 100644 --- a/core/src/it/scala/org/locationtech/rasterframes/ref/RasterRefIT.scala +++ b/core/src/it/scala/org/locationtech/rasterframes/ref/RasterRefIT.scala @@ -35,13 +35,13 @@ class RasterRefIT extends TestEnvironment { def scene(idx: Int) = URI.create(s"https://landsat-pds.s3.us-west-2.amazonaws.com" + s"/c1/L8/176/039/LC08_L1TP_176039_20190703_20190718_01_T1/LC08_L1TP_176039_20190703_20190718_01_T1_B$idx.TIF") - val redScene = RasterSource(scene(4)) + val redScene = RFRasterSource(scene(4)) // [west, south, east, north] val area = Extent(31.115, 29.963, 31.148, 29.99).reproject(LatLng, redScene.crs) val red = RasterRef(redScene, 0, Some(area), None) - val green = RasterRef(RasterSource(scene(3)), 0, Some(area), None) - val blue = RasterRef(RasterSource(scene(2)), 0, Some(area), None) + val green = RasterRef(RFRasterSource(scene(3)), 0, Some(area), None) + val blue = RasterRef(RFRasterSource(scene(2)), 0, Some(area), None) val rf = Seq((red, green, blue)).toDF("red", "green", "blue") val df = rf.select( diff --git a/core/src/it/scala/org/locationtech/rasterframes/ref/RasterSourceIT.scala b/core/src/it/scala/org/locationtech/rasterframes/ref/RasterSourceIT.scala index ae8b0b1d4..61a5b5b6b 100644 --- a/core/src/it/scala/org/locationtech/rasterframes/ref/RasterSourceIT.scala +++ b/core/src/it/scala/org/locationtech/rasterframes/ref/RasterSourceIT.scala @@ -44,10 +44,10 @@ class RasterSourceIT extends TestEnvironment with TestData { val bURI = new URI( "https://s3-us-west-2.amazonaws.com/landsat-pds/c1/L8/016/034/LC08_L1TP_016034_20181003_20181003_01_RT/LC08_L1TP_016034_20181003_20181003_01_RT_B2.TIF") val red = time("read B4") { - RasterSource(rURI).readAll() + RFRasterSource(rURI).readAll() } val blue = time("read B2") { - RasterSource(bURI).readAll() + RFRasterSource(bURI).readAll() } time("test empty") { red should not be empty @@ -69,47 +69,47 @@ class RasterSourceIT extends TestEnvironment with TestData { it("should read JPEG2000 scene") { - RasterSource(localSentinel).readAll().flatMap(_.tile.statisticsDouble).size should be(64) + RFRasterSource(localSentinel).readAll().flatMap(_.tile.statisticsDouble).size should be(64) } it("should read small MRF scene with one band converted from MODIS HDF") { val (expectedTileCount, _) = expectedTileCountAndBands(2400, 2400) - RasterSource(modisConvertedMrfPath).readAll().flatMap(_.tile.statisticsDouble).size should be (expectedTileCount) + RFRasterSource(modisConvertedMrfPath).readAll().flatMap(_.tile.statisticsDouble).size should be (expectedTileCount) } it("should read remote HTTP MRF scene") { val (expectedTileCount, bands) = expectedTileCountAndBands(6257, 7584, 4) - RasterSource(remoteHttpMrfPath).readAll(bands = bands).flatMap(_.tile.statisticsDouble).size should be (expectedTileCount) + RFRasterSource(remoteHttpMrfPath).readAll(bands = bands).flatMap(_.tile.statisticsDouble).size should be (expectedTileCount) } it("should read remote S3 MRF scene") { val (expectedTileCount, bands) = expectedTileCountAndBands(6257, 7584, 4) - RasterSource(remoteS3MrfPath).readAll(bands = bands).flatMap(_.tile.statisticsDouble).size should be (expectedTileCount) + RFRasterSource(remoteS3MrfPath).readAll(bands = bands).flatMap(_.tile.statisticsDouble).size should be (expectedTileCount) } } } else { describe("GDAL missing error support") { it("should throw exception reading JPEG2000 scene") { intercept[IllegalArgumentException] { - RasterSource(localSentinel) + RFRasterSource(localSentinel) } } it("should throw exception reading MRF scene with one band converted from MODIS HDF") { intercept[IllegalArgumentException] { - RasterSource(modisConvertedMrfPath) + RFRasterSource(modisConvertedMrfPath) } } it("should throw exception reading remote HTTP MRF scene") { intercept[IllegalArgumentException] { - RasterSource(remoteHttpMrfPath) + RFRasterSource(remoteHttpMrfPath) } } it("should throw exception reading remote S3 MRF scene") { intercept[IllegalArgumentException] { - RasterSource(remoteS3MrfPath) + RFRasterSource(remoteS3MrfPath) } } } diff --git a/core/src/main/scala/org/apache/spark/sql/rf/RasterSourceUDT.scala b/core/src/main/scala/org/apache/spark/sql/rf/RasterSourceUDT.scala index 51d204b58..51acae55b 100644 --- a/core/src/main/scala/org/apache/spark/sql/rf/RasterSourceUDT.scala +++ b/core/src/main/scala/org/apache/spark/sql/rf/RasterSourceUDT.scala @@ -27,7 +27,7 @@ import org.locationtech.rasterframes.encoders.CatalystSerializer._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.types.{DataType, UDTRegistration, UserDefinedType, _} import org.locationtech.rasterframes.encoders.CatalystSerializer -import org.locationtech.rasterframes.ref.RasterSource +import org.locationtech.rasterframes.ref.RFRasterSource import org.locationtech.rasterframes.util.KryoSupport /** @@ -36,25 +36,25 @@ import org.locationtech.rasterframes.util.KryoSupport * @since 9/5/18 */ @SQLUserDefinedType(udt = classOf[RasterSourceUDT]) -class RasterSourceUDT extends UserDefinedType[RasterSource] { +class RasterSourceUDT extends UserDefinedType[RFRasterSource] { import RasterSourceUDT._ override def typeName = "rf_rastersource" override def pyUDT: String = "pyrasterframes.rf_types.RasterSourceUDT" - def userClass: Class[RasterSource] = classOf[RasterSource] + def userClass: Class[RFRasterSource] = classOf[RFRasterSource] - override def sqlType: DataType = schemaOf[RasterSource] + override def sqlType: DataType = schemaOf[RFRasterSource] - override def serialize(obj: RasterSource): InternalRow = + override def serialize(obj: RFRasterSource): InternalRow = Option(obj) .map(_.toInternalRow) .orNull - override def deserialize(datum: Any): RasterSource = + override def deserialize(datum: Any): RFRasterSource = Option(datum) .collect { - case ir: InternalRow ⇒ ir.to[RasterSource] + case ir: InternalRow ⇒ ir.to[RFRasterSource] } .orNull @@ -66,24 +66,24 @@ class RasterSourceUDT extends UserDefinedType[RasterSource] { } object RasterSourceUDT { - UDTRegistration.register(classOf[RasterSource].getName, classOf[RasterSourceUDT].getName) + UDTRegistration.register(classOf[RFRasterSource].getName, classOf[RasterSourceUDT].getName) /** Deserialize a byte array, also used inside the Python API */ - def from(byteArray: Array[Byte]): RasterSource = CatalystSerializer.CatalystIO.rowIO.create(byteArray).to[RasterSource] + def from(byteArray: Array[Byte]): RFRasterSource = CatalystSerializer.CatalystIO.rowIO.create(byteArray).to[RFRasterSource] - implicit val rasterSourceSerializer: CatalystSerializer[RasterSource] = new CatalystSerializer[RasterSource] { + implicit val rasterSourceSerializer: CatalystSerializer[RFRasterSource] = new CatalystSerializer[RFRasterSource] { override val schema: StructType = StructType(Seq( StructField("raster_source_kryo", BinaryType, false) )) - override def to[R](t: RasterSource, io: CatalystIO[R]): R = { + override def to[R](t: RFRasterSource, io: CatalystIO[R]): R = { val buf = KryoSupport.serialize(t) io.create(buf.array()) } - override def from[R](row: R, io: CatalystIO[R]): RasterSource = { - KryoSupport.deserialize[RasterSource](ByteBuffer.wrap(io.getByteArray(row, 0))) + override def from[R](row: R, io: CatalystIO[R]): RFRasterSource = { + KryoSupport.deserialize[RFRasterSource](ByteBuffer.wrap(io.getByteArray(row, 0))) } } } diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala index 09cd22997..0288d7ca1 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala @@ -33,7 +33,7 @@ import org.apache.spark.unsafe.types.UTF8String import org.locationtech.jts.geom.Envelope import org.locationtech.rasterframes.encoders.CatalystSerializer._ import org.locationtech.rasterframes.model.{LazyCRS, TileContext} -import org.locationtech.rasterframes.ref.{ProjectedRasterLike, RasterRef, RasterSource} +import org.locationtech.rasterframes.ref.{ProjectedRasterLike, RasterRef, RFRasterSource} import org.locationtech.rasterframes.tiles.ProjectedRasterTile private[rasterframes] @@ -71,7 +71,7 @@ object DynamicExtractors { /** Partial function for pulling a ProjectedRasterLike an input row. */ lazy val projectedRasterLikeExtractor: PartialFunction[DataType, InternalRow ⇒ ProjectedRasterLike] = { case _: RasterSourceUDT ⇒ - (row: InternalRow) => row.to[RasterSource](RasterSourceUDT.rasterSourceSerializer) + (row: InternalRow) => row.to[RFRasterSource](RasterSourceUDT.rasterSourceSerializer) case t if t.conformsTo[ProjectedRasterTile] => (row: InternalRow) => row.to[ProjectedRasterTile] case t if t.conformsTo[RasterRef] => @@ -83,7 +83,7 @@ object DynamicExtractors { case _: TileUDT => (row: InternalRow) => row.to[Tile](TileUDT.tileSerializer) case _: RasterSourceUDT => - (row: InternalRow) => row.to[RasterSource](RasterSourceUDT.rasterSourceSerializer) + (row: InternalRow) => row.to[RFRasterSource](RasterSourceUDT.rasterSourceSerializer) case t if t.conformsTo[RasterRef] ⇒ (row: InternalRow) => row.to[RasterRef] case t if t.conformsTo[ProjectedRasterTile] => diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/generators/RasterSourceToRasterRefs.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/generators/RasterSourceToRasterRefs.scala index a514b3560..73e8df458 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/generators/RasterSourceToRasterRefs.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/generators/RasterSourceToRasterRefs.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.{Column, TypedColumn} import org.locationtech.rasterframes.encoders.CatalystSerializer._ import org.locationtech.rasterframes.expressions.generators.RasterSourceToRasterRefs.bandNames import org.locationtech.rasterframes.model.TileDimensions -import org.locationtech.rasterframes.ref.{RasterRef, RasterSource} +import org.locationtech.rasterframes.ref.{RasterRef, RFRasterSource} import org.locationtech.rasterframes.util._ import org.locationtech.rasterframes.RasterSourceType @@ -55,7 +55,7 @@ case class RasterSourceToRasterRefs(children: Seq[Expression], bandIndexes: Seq[ name <- bandNames(basename, bandIndexes) } yield StructField(name, schemaOf[RasterRef], true)) - private def band2ref(src: RasterSource, e: Option[(GridBounds[Int], Extent)])(b: Int): RasterRef = + private def band2ref(src: RFRasterSource, e: Option[(GridBounds[Int], Extent)])(b: Int): RasterRef = if (b < src.bandCount) RasterRef(src, b, e.map(_._2), e.map(_._1)) else null diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/URIToRasterSource.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/URIToRasterSource.scala index 96af62149..53f177daa 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/URIToRasterSource.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/URIToRasterSource.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.types.{DataType, StringType} import org.apache.spark.sql.{Column, TypedColumn} import org.apache.spark.unsafe.types.UTF8String import org.locationtech.rasterframes.RasterSourceType -import org.locationtech.rasterframes.ref.RasterSource +import org.locationtech.rasterframes.ref.RFRasterSource import org.slf4j.LoggerFactory /** @@ -53,12 +53,12 @@ case class URIToRasterSource(override val child: Expression) override protected def nullSafeEval(input: Any): Any = { val uriString = input.asInstanceOf[UTF8String].toString val uri = URI.create(uriString) - val ref = RasterSource(uri) + val ref = RFRasterSource(uri) RasterSourceType.serialize(ref) } } object URIToRasterSource { - def apply(rasterURI: Column): TypedColumn[Any, RasterSource] = - new Column(new URIToRasterSource(rasterURI.expr)).as[RasterSource] + def apply(rasterURI: Column): TypedColumn[Any, RFRasterSource] = + new Column(new URIToRasterSource(rasterURI.expr)).as[RFRasterSource] } diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/XZ2Indexer.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/XZ2Indexer.scala index 7acbb3277..bfa687ec7 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/XZ2Indexer.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/XZ2Indexer.scala @@ -38,7 +38,7 @@ import org.locationtech.rasterframes.expressions.DynamicExtractors._ import org.locationtech.rasterframes.expressions.accessors.GetCRS import org.locationtech.rasterframes.expressions.row import org.locationtech.rasterframes.jts.ReprojectionTransformer -import org.locationtech.rasterframes.ref.{RasterRef, RasterSource} +import org.locationtech.rasterframes.ref.{RasterRef, RFRasterSource} import org.locationtech.rasterframes.tiles.ProjectedRasterTile /** @@ -88,7 +88,7 @@ case class XZ2Indexer(left: Expression, right: Expression, indexResolution: Shor case t if t.conformsTo[Envelope] => row(leftInput).to[Envelope] case _: RasterSourceUDT ⇒ - row(leftInput).to[RasterSource](RasterSourceUDT.rasterSourceSerializer).extent + row(leftInput).to[RFRasterSource](RasterSourceUDT.rasterSourceSerializer).extent case t if t.conformsTo[ProjectedRasterTile] => row(leftInput).to[ProjectedRasterTile].extent case t if t.conformsTo[RasterRef] => diff --git a/core/src/main/scala/org/locationtech/rasterframes/model/TileDimensions.scala b/core/src/main/scala/org/locationtech/rasterframes/model/TileDimensions.scala index 4f2cb0fa2..683f5fb27 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/model/TileDimensions.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/model/TileDimensions.scala @@ -21,18 +21,18 @@ package org.locationtech.rasterframes.model -import org.locationtech.rasterframes.encoders.CatalystSerializer.CatalystIO -import geotrellis.raster.{Dimensions, Grid} +import geotrellis.raster.Dimensions import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.types.{ShortType, StructField, StructType} +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} import org.locationtech.rasterframes.encoders.CatalystSerializer +import org.locationtech.rasterframes.encoders.CatalystSerializer.CatalystIO /** * Typed wrapper for tile size information. * * @since 2018-12-12 */ -case class TileDimensions(cols: Int, rows: Int) extends Grid[Int] +case class TileDimensions(cols: Int, rows: Int) object TileDimensions { def apply(colsRows: (Int, Int)): TileDimensions = new TileDimensions(colsRows._1, colsRows._2) @@ -40,18 +40,18 @@ object TileDimensions { implicit val serializer: CatalystSerializer[TileDimensions] = new CatalystSerializer[TileDimensions] { override val schema: StructType = StructType(Seq( - StructField("cols", ShortType, false), - StructField("rows", ShortType, false) + StructField("cols", IntegerType, false), + StructField("rows", IntegerType, false) )) override protected def to[R](t: TileDimensions, io: CatalystIO[R]): R = io.create( - t.cols.toShort, - t.rows.toShort + t.cols, + t.rows ) override protected def from[R](t: R, io: CatalystIO[R]): TileDimensions = TileDimensions( - io.getShort(t, 0).toInt, - io.getShort(t, 1).toInt + io.getInt(t, 0), + io.getInt(t, 1) ) } diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/DelegatingRasterSource.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/DelegatingRasterSource.scala index d78f4b328..9eb2633a6 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/DelegatingRasterSource.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/DelegatingRasterSource.scala @@ -28,10 +28,10 @@ import geotrellis.proj4.CRS import geotrellis.raster.io.geotiff.Tags import geotrellis.raster.{CellType, GridBounds, MultibandTile, Raster} import geotrellis.vector.Extent -import org.locationtech.rasterframes.ref.RasterSource.URIRasterSource +import org.locationtech.rasterframes.ref.RFRasterSource.URIRasterSource /** A RasterFrames RasterSource which delegates most operations to a geotrellis-contrib RasterSource */ -abstract class DelegatingRasterSource(source: URI, delegateBuilder: () => GTRasterSource) extends RasterSource with URIRasterSource { +abstract class DelegatingRasterSource(source: URI, delegateBuilder: () => GTRasterSource) extends RFRasterSource with URIRasterSource { @transient @volatile private var _delRef: GTRasterSource = _ diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/GDALRasterSource.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/GDALRasterSource.scala index d6f7ffbe1..382844012 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/GDALRasterSource.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/GDALRasterSource.scala @@ -30,10 +30,10 @@ import geotrellis.raster.gdal.{GDALRasterSource => VLMRasterSource} import geotrellis.raster.io.geotiff.Tags import geotrellis.raster.{CellType, GridBounds, MultibandTile, Raster} import geotrellis.vector.Extent -import org.locationtech.rasterframes.ref.RasterSource.URIRasterSource +import org.locationtech.rasterframes.ref.RFRasterSource.URIRasterSource -case class GDALRasterSource(source: URI) extends RasterSource with URIRasterSource { +case class GDALRasterSource(source: URI) extends RFRasterSource with URIRasterSource { @transient private lazy val gdal: VLMRasterSource = { diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/HadoopGeoTiffRasterSource.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/HadoopGeoTiffRasterSource.scala index ba899ba4c..a222485b8 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/HadoopGeoTiffRasterSource.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/HadoopGeoTiffRasterSource.scala @@ -26,7 +26,7 @@ import java.net.URI import geotrellis.store.hadoop.util.HdfsRangeReader import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.locationtech.rasterframes.ref.RasterSource.{URIRasterSource, URIRasterSourceDebugString} +import org.locationtech.rasterframes.ref.RFRasterSource.{URIRasterSource, URIRasterSourceDebugString} case class HadoopGeoTiffRasterSource(source: URI, config: () => Configuration) extends RangeReaderRasterSource with URIRasterSource with URIRasterSourceDebugString { self => diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/InMemoryRasterSource.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/InMemoryRasterSource.scala index 5d29f0e32..4bb6b7d0b 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/InMemoryRasterSource.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/InMemoryRasterSource.scala @@ -25,10 +25,10 @@ import geotrellis.proj4.CRS import geotrellis.raster.{CellType, GridBounds, MultibandTile, Raster, Tile} import geotrellis.raster.io.geotiff.Tags import geotrellis.vector.Extent -import org.locationtech.rasterframes.ref.RasterSource.EMPTY_TAGS +import org.locationtech.rasterframes.ref.RFRasterSource.EMPTY_TAGS import org.locationtech.rasterframes.tiles.ProjectedRasterTile -case class InMemoryRasterSource(tile: Tile, extent: Extent, crs: CRS) extends RasterSource { +case class InMemoryRasterSource(tile: Tile, extent: Extent, crs: CRS) extends RFRasterSource { def this(prt: ProjectedRasterTile) = this(prt, prt.extent, prt.crs) override def rows: Int = tile.rows diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/RasterSource.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/RFRasterSource.scala similarity index 92% rename from core/src/main/scala/org/locationtech/rasterframes/ref/RasterSource.scala rename to core/src/main/scala/org/locationtech/rasterframes/ref/RFRasterSource.scala index 39a33adb7..5deff0344 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/RasterSource.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/RFRasterSource.scala @@ -44,8 +44,8 @@ import scala.concurrent.duration.Duration * @since 8/21/18 */ @Experimental -trait RasterSource extends ProjectedRasterLike with Serializable { - import RasterSource._ +trait RFRasterSource extends ProjectedRasterLike with Serializable { + import RFRasterSource._ def crs: CRS @@ -88,7 +88,7 @@ trait RasterSource extends ProjectedRasterLike with Serializable { } } -object RasterSource extends LazyLogging { +object RFRasterSource extends LazyLogging { final val SINGLEBAND = Seq(0) final val EMPTY_TAGS = Tags(Map.empty, List.empty) @@ -96,17 +96,17 @@ object RasterSource extends LazyLogging { private[ref] val rsCache = Scaffeine() .recordStats() - .expireAfterAccess(RasterSource.cacheTimeout) - .build[String, RasterSource] + .expireAfterAccess(RFRasterSource.cacheTimeout) + .build[String, RFRasterSource] def cacheStats = rsCache.stats() - implicit def rsEncoder: ExpressionEncoder[RasterSource] = { + implicit def rsEncoder: ExpressionEncoder[RFRasterSource] = { RasterSourceUDT // Makes sure UDT is registered first ExpressionEncoder() } - def apply(source: URI): RasterSource = + def apply(source: URI): RFRasterSource = rsCache.get( source.toASCIIString, _ => source match { case IsGDAL() => GDALRasterSource(source) @@ -157,14 +157,14 @@ object RasterSource extends LazyLogging { } } - trait URIRasterSource { _: RasterSource => + trait URIRasterSource { _: RFRasterSource => def source: URI abstract override def toString: String = { s"${getClass.getSimpleName}(${source})" } } - trait URIRasterSourceDebugString { _: RasterSource with URIRasterSource with Product => + trait URIRasterSourceDebugString { _: RFRasterSource with URIRasterSource with Product => def toDebugString: String = { val buf = new StringBuilder() buf.append(productPrefix) diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/RangeReaderRasterSource.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/RangeReaderRasterSource.scala index 1825f6695..aaf1ddad2 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/RangeReaderRasterSource.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/RangeReaderRasterSource.scala @@ -31,7 +31,7 @@ import geotrellis.vector.Extent import org.locationtech.rasterframes.util.GeoTiffInfoSupport import org.slf4j.LoggerFactory -trait RangeReaderRasterSource extends RasterSource with GeoTiffInfoSupport { +trait RangeReaderRasterSource extends RFRasterSource with GeoTiffInfoSupport { @transient protected lazy val logger = Logger(LoggerFactory.getLogger(getClass.getName)) protected def rangeReader: RangeReader diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/RasterRef.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/RasterRef.scala index 5fc89450e..c239ed3b6 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/RasterRef.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/RasterRef.scala @@ -38,7 +38,7 @@ import org.locationtech.rasterframes.tiles.ProjectedRasterTile * * @since 8/21/18 */ -case class RasterRef(source: RasterSource, bandIndex: Int, subextent: Option[Extent], subgrid: Option[GridBounds[Int]]) +case class RasterRef(source: RFRasterSource, bandIndex: Int, subextent: Option[Extent], subgrid: Option[GridBounds[Int]]) extends ProjectedRasterLike { def crs: CRS = source.crs def extent: Extent = subextent.getOrElse(source.extent) @@ -91,7 +91,7 @@ object RasterRef extends LazyLogging { ) override def from[R](row: R, io: CatalystIO[R]): RasterRef = RasterRef( - io.get[RasterSource](row, 0)(RasterSourceUDT.rasterSourceSerializer), + io.get[RFRasterSource](row, 0)(RasterSourceUDT.rasterSourceSerializer), io.getInt(row, 1), if (io.isNullAt(row, 2)) None else Option(io.get[Extent](row, 2)), diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/SimpleRasterInfo.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/SimpleRasterInfo.scala index 6df223158..501e17639 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/SimpleRasterInfo.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/SimpleRasterInfo.scala @@ -28,7 +28,7 @@ import geotrellis.raster.io.geotiff.Tags import geotrellis.raster.io.geotiff.reader.GeoTiffInfo import geotrellis.raster.{CellType, RasterExtent, RasterSource => GTRasterSource} import geotrellis.vector.Extent -import org.locationtech.rasterframes.ref.RasterSource.EMPTY_TAGS +import org.locationtech.rasterframes.ref.RFRasterSource.EMPTY_TAGS case class SimpleRasterInfo( cols: Long, diff --git a/core/src/main/scala/org/locationtech/rasterframes/tiles/FixedDelegatingTile.scala b/core/src/main/scala/org/locationtech/rasterframes/tiles/FixedDelegatingTile.scala deleted file mode 100644 index 52bfa5c1d..000000000 --- a/core/src/main/scala/org/locationtech/rasterframes/tiles/FixedDelegatingTile.scala +++ /dev/null @@ -1,40 +0,0 @@ -/* - * This software is licensed under the Apache 2 license, quoted below. - * - * Copyright 2019 Astraea, Inc. - * - * 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. - * - * SPDX-License-Identifier: Apache-2.0 - * - */ - -package org.locationtech.rasterframes.tiles -import geotrellis.raster.{ArrayTile, DelegatingTile, Tile} - -/** - * Temporary workaroud for https://github.com/locationtech/geotrellis/issues/2907 - * - * @since 8/22/18 - */ -trait FixedDelegatingTile extends DelegatingTile { - override def combine(r2: Tile)(f: (Int, Int) ⇒ Int): Tile = (delegate, r2) match { - case (del: ArrayTile, r2: DelegatingTile) ⇒ del.combine(r2.toArrayTile())(f) - case _ ⇒ delegate.combine(r2)(f) - } - - override def combineDouble(r2: Tile)(f: (Double, Double) ⇒ Double): Tile = (delegate, r2) match { - case (del: ArrayTile, r2: DelegatingTile) ⇒ del.combineDouble(r2.toArrayTile())(f) - case _ ⇒ delegate.combineDouble(r2)(f) - } -} diff --git a/core/src/main/scala/org/locationtech/rasterframes/tiles/InternalRowTile.scala b/core/src/main/scala/org/locationtech/rasterframes/tiles/InternalRowTile.scala index 98be22446..ba0be15ac 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/tiles/InternalRowTile.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/tiles/InternalRowTile.scala @@ -34,7 +34,7 @@ import org.locationtech.rasterframes.model.{Cells, TileDataContext} * * @since 11/29/17 */ -class InternalRowTile(val mem: InternalRow) extends FixedDelegatingTile { +class InternalRowTile(val mem: InternalRow) extends DelegatingTile { import InternalRowTile._ override def toArrayTile(): ArrayTile = realizedTile.toArrayTile() diff --git a/core/src/main/scala/org/locationtech/rasterframes/tiles/ProjectedRasterTile.scala b/core/src/main/scala/org/locationtech/rasterframes/tiles/ProjectedRasterTile.scala index ec490edfc..4121892ca 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/tiles/ProjectedRasterTile.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/tiles/ProjectedRasterTile.scala @@ -23,7 +23,7 @@ package org.locationtech.rasterframes.tiles import geotrellis.proj4.CRS import geotrellis.raster.io.geotiff.SinglebandGeoTiff -import geotrellis.raster.{CellType, ProjectedRaster, Tile} +import geotrellis.raster.{CellType, DelegatingTile, ProjectedRaster, Tile} import geotrellis.vector.{Extent, ProjectedExtent} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.rf.TileUDT @@ -40,7 +40,7 @@ import org.locationtech.rasterframes.ref.RasterRef.RasterRefTile * * @since 9/5/18 */ -trait ProjectedRasterTile extends FixedDelegatingTile with ProjectedRasterLike { +trait ProjectedRasterTile extends DelegatingTile with ProjectedRasterLike { def extent: Extent def crs: CRS def projectedExtent: ProjectedExtent = ProjectedExtent(extent, crs) diff --git a/core/src/main/scala/org/locationtech/rasterframes/tiles/ShowableTile.scala b/core/src/main/scala/org/locationtech/rasterframes/tiles/ShowableTile.scala index ccec3a340..7255f4f73 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/tiles/ShowableTile.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/tiles/ShowableTile.scala @@ -21,9 +21,9 @@ package org.locationtech.rasterframes.tiles import org.locationtech.rasterframes._ -import geotrellis.raster.{Tile, isNoData} +import geotrellis.raster.{DelegatingTile, Tile, isNoData} -class ShowableTile(val delegate: Tile) extends FixedDelegatingTile { +class ShowableTile(val delegate: Tile) extends DelegatingTile { override def equals(obj: Any): Boolean = obj match { case st: ShowableTile => delegate.equals(st.delegate) case o => delegate.equals(o) diff --git a/core/src/main/scala/org/locationtech/rasterframes/util/DataBiasedOp.scala b/core/src/main/scala/org/locationtech/rasterframes/util/DataBiasedOp.scala index 83e5fe76c..b286fde0f 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/util/DataBiasedOp.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/util/DataBiasedOp.scala @@ -52,13 +52,19 @@ trait DataBiasedOp extends LocalTileBinaryOp { def combine(z1: Int, z2: Int): Int = if (isNoData(z1) && isNoData(z2)) raster.NODATA - else if (isNoData(z1)) z2 - else if (isNoData(z2)) z1 - else op(z1, z2) + else if (isNoData(z1)) + z2 + else if (isNoData(z2)) + z1 + else + op(z1, z2) def combine(z1: Double, z2: Double): Double = if (isNoData(z1) && isNoData(z2)) raster.doubleNODATA - else if (isNoData(z1)) z2 - else if (isNoData(z2)) z1 - else op(z1, z2) + else if (isNoData(z1)) + z2 + else if (isNoData(z2)) + z1 + else + op(z1, z2) } diff --git a/core/src/main/scala/org/locationtech/rasterframes/util/RFKryoRegistrator.scala b/core/src/main/scala/org/locationtech/rasterframes/util/RFKryoRegistrator.scala index 44dd4ca17..b2ae4e1d5 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/util/RFKryoRegistrator.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/util/RFKryoRegistrator.scala @@ -22,7 +22,7 @@ package org.locationtech.rasterframes.util import org.locationtech.rasterframes.ref.RasterRef.RasterRefTile -import org.locationtech.rasterframes.ref.{DelegatingRasterSource, RasterRef, RasterSource} +import org.locationtech.rasterframes.ref.{DelegatingRasterSource, RasterRef, RFRasterSource} import org.locationtech.rasterframes.ref._ import com.esotericsoftware.kryo.Kryo import geotrellis.raster.io.geotiff.reader.GeoTiffInfo @@ -37,7 +37,7 @@ import geotrellis.spark.store.kryo.KryoRegistrator class RFKryoRegistrator extends KryoRegistrator { override def registerClasses(kryo: Kryo): Unit = { super.registerClasses(kryo) - kryo.register(classOf[RasterSource]) + kryo.register(classOf[RFRasterSource]) kryo.register(classOf[RasterRef]) kryo.register(classOf[RasterRefTile]) kryo.register(classOf[DelegatingRasterSource]) diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 39e791fa3..9e3b08ac5 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -44,3 +44,5 @@ log4j.logger.org.apache.parquet.hadoop.ParquetRecordReader=OFF # SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR + +log4j.logger.org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator=ERROR \ No newline at end of file diff --git a/core/src/test/scala/org/locationtech/rasterframes/ExplodeSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/ExplodeSpec.scala index 3eae60461..cb483ef32 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/ExplodeSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/ExplodeSpec.scala @@ -24,16 +24,16 @@ package org.locationtech.rasterframes import geotrellis.raster._ import geotrellis.raster.resample.NearestNeighbor - /** * Test rig for Tile operations associated with converting to/from * exploded/long form representations of the tile's data. * * @since 9/18/17 */ -class ExplodeSpec extends TestEnvironment with TestData { +class ExplodeSpec extends TestEnvironment { describe("conversion to/from exploded representation of tiles") { import spark.implicits._ + import TestData._ it("should explode tiles") { val query = sql( @@ -129,7 +129,7 @@ class ExplodeSpec extends TestEnvironment with TestData { val assembledSqlExpr = df.selectExpr("rf_assemble_tile(column_index, row_index, tile, 10, 10)") val resultSql = assembledSqlExpr.as[Tile].first() - assert(resultSql === tile) + assertEqual(resultSql, tile) checkDocs("rf_assemble_tile") } @@ -185,7 +185,7 @@ class ExplodeSpec extends TestEnvironment with TestData { //GeoTiff(recovered).write("foo.tiff") - assert(image.tile.toArrayTile() === recovered.tile.toArrayTile()) + assertEqual(image.tile.toArrayTile(), recovered.tile.toArrayTile()) } } } diff --git a/core/src/test/scala/org/locationtech/rasterframes/ExtensionMethodSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/ExtensionMethodSpec.scala index f191f201f..26a4a0e29 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/ExtensionMethodSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/ExtensionMethodSpec.scala @@ -22,7 +22,7 @@ package org.locationtech.rasterframes import geotrellis.proj4.LatLng -import geotrellis.raster.{ByteCellType, GridBounds, TileLayout} +import geotrellis.raster.{ByteCellType, Dimensions, GridBounds, TileLayout} import geotrellis.layer._ import org.apache.spark.sql.Encoders import org.locationtech.rasterframes.util._ @@ -108,7 +108,7 @@ class ExtensionMethodSpec extends TestEnvironment with TestData with SubdivideSu val divided = tlm.subdivide(2) - assert(divided.tileLayout.tileDimensions === (tileSize / 2, tileSize / 2)) + assert(divided.tileLayout.tileDimensions === Dimensions(tileSize / 2, tileSize / 2)) } it("should render Markdown") { diff --git a/core/src/test/scala/org/locationtech/rasterframes/GeometryFunctionsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/GeometryFunctionsSpec.scala index 9caf47bdb..7aaffba83 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/GeometryFunctionsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/GeometryFunctionsSpec.scala @@ -140,7 +140,7 @@ class GeometryFunctionsSpec extends TestEnvironment with TestData with StandardC val rf = l8Sample(1).projectedRaster.toLayer.withGeometry() val df = GeomData.features.map(f ⇒ ( f.geom.reproject(LatLng, rf.crs), - f.data("id").asInstanceOf[JsNumber].value.intValue() + f.data("id").flatMap(_.asNumber).flatMap(_.toInt).getOrElse(0) )).toDF("geom", "__fid__") val toRasterize = rf.crossJoin(df) diff --git a/core/src/test/scala/org/locationtech/rasterframes/RasterFrameSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/RasterFrameSpec.scala index 65e7943bb..4d95d6e6a 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/RasterFrameSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/RasterFrameSpec.scala @@ -332,19 +332,19 @@ class RasterFrameSpec extends TestEnvironment with MetadataKeys val raster = rf.toRaster($"tile", cols, rows) render(raster.tile, "normal") - assert(raster.raster.dimensions === (cols, rows)) + assert(raster.raster.dimensions === Dimensions(cols, rows)) val smaller = rf.toRaster($"tile", cols/4, rows/4) render(smaller.tile, "smaller") - assert(smaller.raster.dimensions === (cols/4, rows/4)) + assert(smaller.raster.dimensions === Dimensions(cols/4, rows/4)) val bigger = rf.toRaster($"tile", cols*4, rows*4) render(bigger.tile, "bigger") - assert(bigger.raster.dimensions === (cols*4, rows*4)) + assert(bigger.raster.dimensions === Dimensions(cols*4, rows*4)) val squished = rf.toRaster($"tile", cols*5/4, rows*3/4) render(squished.tile, "squished") - assert(squished.raster.dimensions === (cols*5/4, rows*3/4)) + assert(squished.raster.dimensions === Dimensions(cols*5/4, rows*3/4)) } it("shouldn't restitch raster that's has derived tiles") { diff --git a/core/src/test/scala/org/locationtech/rasterframes/RasterFunctionsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/RasterFunctionsSpec.scala index f676e00cb..136bdcba8 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/RasterFunctionsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/RasterFunctionsSpec.scala @@ -535,9 +535,11 @@ class RasterFunctionsSpec extends TestEnvironment with RasterMatchers { val df = Seq(two, three, one, six).toDF("tile") .withColumn("id", monotonically_increasing_id()) - df.select(rf_agg_local_mean($"tile")).first() should be(three.toArrayTile()) + val expected = three.toArrayTile().convert(DoubleConstantNoDataCellType) - df.selectExpr("rf_agg_local_mean(tile)").as[Tile].first() should be(three.toArrayTile()) + df.select(rf_agg_local_mean($"tile")).first() should be(expected) + + df.selectExpr("rf_agg_local_mean(tile)").as[Tile].first() should be(expected) noException should be thrownBy { df.groupBy($"id") @@ -560,7 +562,7 @@ class RasterFunctionsSpec extends TestEnvironment with RasterMatchers { val t2 = df.selectExpr("rf_agg_local_no_data_cells(tile) as cnt").select($"cnt".as[Tile]).first() t1 should be (t2) val t3 = df.select(rf_local_add(rf_agg_local_data_cells($"tile"), rf_agg_local_no_data_cells($"tile"))).as[Tile].first() - t3 should be(three.toArrayTile()) + t3 should be(three.toArrayTile().convert(IntConstantNoDataCellType)) checkDocs("rf_agg_local_no_data_cells") } } diff --git a/core/src/test/scala/org/locationtech/rasterframes/TestData.scala b/core/src/test/scala/org/locationtech/rasterframes/TestData.scala index d65f6e02e..6431528a7 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/TestData.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/TestData.scala @@ -39,7 +39,7 @@ import org.apache.spark.SparkContext import org.apache.spark.sql.SparkSession import org.locationtech.jts.geom.{Coordinate, GeometryFactory} import org.locationtech.rasterframes.expressions.tilestats.NoDataCells -import org.locationtech.rasterframes.ref.{RasterRef, RasterSource} +import org.locationtech.rasterframes.ref.{RasterRef, RFRasterSource} import org.locationtech.rasterframes.tiles.ProjectedRasterTile import scala.reflect.ClassTag @@ -182,7 +182,7 @@ trait TestData { TestData.randomTile(cols, rows, UByteConstantNoDataCellType) )).map(ProjectedRasterTile(_, extent, crs)) :+ null - def lazyPRT = RasterRef(RasterSource(TestData.l8samplePath), 0, None, None).tile + def lazyPRT = RasterRef(RFRasterSource(TestData.l8samplePath), 0, None, None).tile object GeomData { val fact = new GeometryFactory() diff --git a/core/src/test/scala/org/locationtech/rasterframes/TileAssemblerSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/TileAssemblerSpec.scala index b73beede1..aef04ae9d 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/TileAssemblerSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/TileAssemblerSpec.scala @@ -27,7 +27,7 @@ import geotrellis.raster._ import geotrellis.raster.render.ColorRamps import geotrellis.vector.Extent import org.apache.spark.sql.{functions => F, _} -import org.locationtech.rasterframes.ref.{InMemoryRasterSource, RasterSource} +import org.locationtech.rasterframes.ref.{InMemoryRasterSource, RFRasterSource} /** * @@ -84,7 +84,7 @@ class TileAssemblerSpec extends TestEnvironment { it("should reassemble a realistic scene") { val df = util.time("read scene") { - RasterSource(TestData.remoteMODIS).toDF + RFRasterSource(TestData.remoteMODIS).toDF } val exploded = util.time("exploded") { @@ -131,7 +131,7 @@ object TileAssemblerSpec extends LazyLogging { } } - implicit class WithToDF(val rs: RasterSource) { + implicit class WithToDF(val rs: RFRasterSource) { def toDF(implicit spark: SparkSession): DataFrame = { import spark.implicits._ rs.readAll() diff --git a/core/src/test/scala/org/locationtech/rasterframes/TileStatsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/TileStatsSpec.scala index 90aef8244..ac2118c0c 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/TileStatsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/TileStatsSpec.scala @@ -49,7 +49,8 @@ class TileStatsSpec extends TestEnvironment with TestData { assert(dims.as[(Int, Int)].first() === (3, 3)) assert(dims.schema.head.name === "cols") - val query = sql("""|select dims.* from ( + val query = sql( + """|select dims.* from ( |select rf_dimensions(tiles) as dims from ( |select rf_make_constant_tile(1, 10, 10, 'int8raw') as tiles)) |""".stripMargin) @@ -282,14 +283,14 @@ class TileStatsSpec extends TestEnvironment with TestData { val countNodataArray = dsNd.select(rf_agg_local_no_data_cells($"tiles")).first().toArray assert(countNodataArray === incompleteTile.localUndefined().toArray) - val minTile = dsNd.select(rf_agg_local_min($"tiles")).first() - assert(minTile.toArray() === completeTile.toArray()) +// val meanTile = dsNd.select(rf_agg_local_mean($"tiles")).first() +// assert(meanTile.toArray() === completeTile.toArray()) val maxTile = dsNd.select(rf_agg_local_max($"tiles")).first() assert(maxTile.toArray() === completeTile.toArray()) - val meanTile = dsNd.select(rf_agg_local_mean($"tiles")).first() - assert(meanTile.toArray() === completeTile.toArray()) + val minTile = dsNd.select(rf_agg_local_min($"tiles")).first() + assert(minTile.toArray() === completeTile.toArray()) } } describe("NoData handling") { diff --git a/core/src/test/scala/org/locationtech/rasterframes/encoders/CatalystSerializerSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/encoders/CatalystSerializerSpec.scala index 6eaa38b18..cfe1b81a5 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/encoders/CatalystSerializerSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/encoders/CatalystSerializerSpec.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.locationtech.rasterframes.{TestData, TestEnvironment} import org.locationtech.rasterframes.encoders.StandardEncoders._ import org.locationtech.rasterframes.model.{CellContext, TileContext, TileDataContext, TileDimensions} -import org.locationtech.rasterframes.ref.{RasterRef, RasterSource} +import org.locationtech.rasterframes.ref.{RasterRef, RFRasterSource} import org.scalatest.Assertion class CatalystSerializerSpec extends TestEnvironment { @@ -105,7 +105,7 @@ class CatalystSerializerSpec extends TestEnvironment { it("should serialize RasterRef") { // TODO: Decide if RasterRef should be encoded 'flat', non-'flat', or depends - val src = RasterSource(remoteCOGSingleband1) + val src = RFRasterSource(remoteCOGSingleband1) val ext = src.extent.buffer(-3.0) val value = RasterRef(src, 0, Some(ext), Some(src.rasterExtent.gridBoundsFor(ext))) assertConsistent(value) diff --git a/core/src/test/scala/org/locationtech/rasterframes/encoders/EncodingSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/encoders/EncodingSpec.scala index bde90fb8e..bd4b1a8e5 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/encoders/EncodingSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/encoders/EncodingSpec.scala @@ -24,16 +24,16 @@ package org.locationtech.rasterframes.encoders import java.io.File import java.net.URI -import org.locationtech.rasterframes._ -import org.locationtech.jts.geom.Envelope -import geotrellis.proj4._ -import geotrellis.raster.{CellType, Tile, TileFeature} import geotrellis.layer._ +import geotrellis.proj4._ +import geotrellis.raster.{CellType, Tile} import geotrellis.vector.{Extent, ProjectedExtent} +import org.apache.spark.SparkConf import org.apache.spark.sql.Row import org.apache.spark.sql.functions._ import org.apache.spark.sql.rf.TileUDT -import org.locationtech.rasterframes.TestEnvironment +import org.locationtech.jts.geom.Envelope +import org.locationtech.rasterframes.{TestEnvironment, _} import org.locationtech.rasterframes.tiles.ProjectedRasterTile /** @@ -43,6 +43,7 @@ import org.locationtech.rasterframes.tiles.ProjectedRasterTile */ class EncodingSpec extends TestEnvironment with TestData { + import spark.implicits._ describe("Spark encoding on standard types") { @@ -70,13 +71,6 @@ class EncodingSpec extends TestEnvironment with TestData { assert(ds.toDF.as[(Int, Tile)].collect().head === ((1, byteArrayTile))) } - it("should code RDD[TileFeature]") { - val thing = TileFeature(byteArrayTile: Tile, "meta") - val ds = Seq(thing).toDS() - write(ds) - assert(ds.toDF.as[TileFeature[Tile, String]].collect().head === thing) - } - it("should code RDD[ProjectedRasterTile]") { val tile = TestData.projectedRasterTile(20, 30, -1.2, extent) val ds = Seq(tile).toDS() @@ -161,4 +155,8 @@ class EncodingSpec extends TestEnvironment with TestData { assert(ds.collect().toSeq === points) } } + + override def additionalConf: SparkConf = { + super.additionalConf.set("spark.sql.codegen.logging.maxLines", Int.MaxValue.toString) + } } diff --git a/core/src/test/scala/org/locationtech/rasterframes/expressions/XZ2IndexerSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/expressions/XZ2IndexerSpec.scala index 9048bcbd7..c2a267f4b 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/expressions/XZ2IndexerSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/expressions/XZ2IndexerSpec.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.Encoders import org.locationtech.geomesa.curve.XZ2SFC import org.locationtech.rasterframes.{TestEnvironment, _} import org.locationtech.rasterframes.encoders.serialized_literal -import org.locationtech.rasterframes.ref.{InMemoryRasterSource, RasterSource} +import org.locationtech.rasterframes.ref.{InMemoryRasterSource, RFRasterSource} import org.locationtech.rasterframes.tiles.ProjectedRasterTile import org.scalatest.Inspectors @@ -92,7 +92,7 @@ class XZ2IndexerSpec extends TestEnvironment with Inspectors { it("should create index from RasterSource") { val crs: CRS = WebMercator val tile = TestData.randomTile(2, 2, CellType.fromName("uint8")) - val srcs = testExtents.map(reproject(crs)).map(InMemoryRasterSource(tile, _, crs): RasterSource).toDF("src") + val srcs = testExtents.map(reproject(crs)).map(InMemoryRasterSource(tile, _, crs): RFRasterSource).toDF("src") val indexes = srcs.select(rf_spatial_index($"src")).collect() forEvery(indexes.zip(expected)) { case (i, e) => diff --git a/core/src/test/scala/org/locationtech/rasterframes/ref/RasterRefSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/ref/RasterRefSpec.scala index d424bbba4..25171c8b4 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/ref/RasterRefSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/ref/RasterRefSpec.scala @@ -49,7 +49,7 @@ class RasterRefSpec extends TestEnvironment with TestData { } trait Fixture { - val src = RasterSource(remoteCOGSingleband1) + val src = RFRasterSource(remoteCOGSingleband1) val fullRaster = RasterRef(src, 0, None, None) val subExtent = sub(src.extent) val subRaster = RasterRef(src, 0, Some(subExtent), Some(src.rasterExtent.gridBoundsFor(subExtent))) @@ -171,7 +171,7 @@ class RasterRefSpec extends TestEnvironment with TestData { describe("RasterRef creation") { it("should realize subiles of proper size") { - val src = RasterSource(remoteMODIS) + val src = RFRasterSource(remoteMODIS) val dims = src .layoutExtents(NOMINAL_TILE_DIMS) .map(e => RasterRef(src, 0, Some(e), None)) @@ -187,7 +187,7 @@ class RasterRefSpec extends TestEnvironment with TestData { describe("RasterSourceToRasterRefs") { it("should convert and expand RasterSource") { - val src = RasterSource(remoteMODIS) + val src = RFRasterSource(remoteMODIS) import spark.implicits._ val df = Seq(src).toDF("src") val refs = df.select(RasterSourceToRasterRefs(None, Seq(0), $"src")) @@ -195,7 +195,7 @@ class RasterRefSpec extends TestEnvironment with TestData { } it("should properly realize subtiles") { - val src = RasterSource(remoteMODIS) + val src = RFRasterSource(remoteMODIS) import spark.implicits._ val df = Seq(src).toDF("src") val refs = df.select(RasterSourceToRasterRefs(Some(NOMINAL_TILE_DIMS), Seq(0), $"src") as "proj_raster") @@ -209,7 +209,7 @@ class RasterRefSpec extends TestEnvironment with TestData { } } it("should throw exception on invalid URI") { - val src = RasterSource(URI.create("http://foo/bar")) + val src = RFRasterSource(URI.create("http://foo/bar")) import spark.implicits._ val df = Seq(src).toDF("src") val refs = df.select(RasterSourceToRasterRefs($"src") as "proj_raster") diff --git a/core/src/test/scala/org/locationtech/rasterframes/ref/RasterSourceSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/ref/RasterSourceSpec.scala index d16382429..d11832f21 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/ref/RasterSourceSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/ref/RasterSourceSpec.scala @@ -43,7 +43,7 @@ class RasterSourceSpec extends TestEnvironment with TestData { it("should identify as UDT") { assert(new RasterSourceUDT() === new RasterSourceUDT()) } - val rs = RasterSource(getClass.getResource("/L8-B8-Robinson-IL.tiff").toURI) + val rs = RFRasterSource(getClass.getResource("/L8-B8-Robinson-IL.tiff").toURI) it("should compute nominal tile layout bounds") { val bounds = rs.layoutBounds(TileDimensions(65, 60)) val agg = bounds.reduce(_ combine _) @@ -62,7 +62,7 @@ class RasterSourceSpec extends TestEnvironment with TestData { } it("should compute layout extents from scene with fractional gsd") { - val rs = RasterSource(remoteMODIS) + val rs = RFRasterSource(remoteMODIS) val dims = rs.layoutExtents(NOMINAL_TILE_DIMS) .map(e => rs.rasterExtent.gridBoundsFor(e, false)) @@ -93,29 +93,29 @@ class RasterSourceSpec extends TestEnvironment with TestData { describe("HTTP RasterSource") { it("should support metadata querying over HTTP") { withClue("remoteCOGSingleband") { - val src = RasterSource(remoteCOGSingleband1) + val src = RFRasterSource(remoteCOGSingleband1) assert(!src.extent.isEmpty) } withClue("remoteCOGMultiband") { - val src = RasterSource(remoteCOGMultiband) + val src = RFRasterSource(remoteCOGMultiband) assert(!src.extent.isEmpty) } } it("should read sub-tile") { withClue("remoteCOGSingleband") { - val src = RasterSource(remoteCOGSingleband1) + val src = RFRasterSource(remoteCOGSingleband1) val raster = src.read(sub(src.extent)) assert(raster.size > 0 && raster.size < src.size) } withClue("remoteCOGMultiband") { - val src = RasterSource(remoteCOGMultiband) + val src = RFRasterSource(remoteCOGMultiband) val raster = src.read(sub(src.extent)) assert(raster.size > 0 && raster.size < src.size) } } it("should Java serialize") { import java.io._ - val src = RasterSource(remoteCOGSingleband1) + val src = RFRasterSource(remoteCOGSingleband1) val buf = new java.io.ByteArrayOutputStream() val out = new ObjectOutputStream(buf) out.writeObject(src) @@ -123,21 +123,21 @@ class RasterSourceSpec extends TestEnvironment with TestData { val data = buf.toByteArray val in = new ObjectInputStream(new ByteArrayInputStream(data)) - val recovered = in.readObject().asInstanceOf[RasterSource] + val recovered = in.readObject().asInstanceOf[RFRasterSource] assert(src.toString === recovered.toString) } } describe("File RasterSource") { it("should support metadata querying of file") { val localSrc = geotiffDir.resolve("LC08_B7_Memphis_COG.tiff").toUri - val src = RasterSource(localSrc) + val src = RFRasterSource(localSrc) assert(!src.extent.isEmpty) } it("should interpret no scheme as file://"){ val localSrc = geotiffDir.resolve("LC08_B7_Memphis_COG.tiff").toString val schemelessUri = new URI(localSrc) schemelessUri.getScheme should be (null) - val src = RasterSource(schemelessUri) + val src = RFRasterSource(schemelessUri) assert(!src.extent.isEmpty) } } @@ -180,7 +180,7 @@ class RasterSourceSpec extends TestEnvironment with TestData { describe("RasterSource tile construction") { it("should read all tiles") { - val src = RasterSource(remoteMODIS) + val src = RFRasterSource(remoteMODIS) val subrasters = src.readAll() diff --git a/project/RFAssemblyPlugin.scala b/project/RFAssemblyPlugin.scala index cbde26437..3a39bc917 100644 --- a/project/RFAssemblyPlugin.scala +++ b/project/RFAssemblyPlugin.scala @@ -55,10 +55,13 @@ object RFAssemblyPlugin extends AutoPlugin { "org.apache.avro", "org.apache.http", "com.google.guava", + "com.google.common", "com.typesafe.scalalogging", - "com.typesafe.config" + "com.typesafe.config", + "com.fasterxml.jackson", + "io.netty" ) - shadePrefixes.map(p ⇒ ShadeRule.rename(s"$p.**" -> s"rf.shaded.$p.@1").inAll) + shadePrefixes.map(p ⇒ ShadeRule.rename(s"$p.**" -> s"shaded.rasterframes.$p.@1").inAll) }, assemblyOption in assembly := (assemblyOption in assembly).value.copy(includeScala = false), @@ -81,6 +84,8 @@ object RFAssemblyPlugin extends AutoPlugin { xs map { _.toLowerCase } match { case "manifest.mf" :: Nil | "index.list" :: Nil | "dependencies" :: Nil ⇒ MergeStrategy.discard + case "io.netty.versions.properties" :: Nil => + MergeStrategy.concat case ps @ x :: _ if ps.last.endsWith(".sf") || ps.last.endsWith(".dsa") ⇒ MergeStrategy.discard case "plexus" :: _ ⇒ diff --git a/project/RFDependenciesPlugin.scala b/project/RFDependenciesPlugin.scala index 47eb4fb6b..ba4e8f748 100644 --- a/project/RFDependenciesPlugin.scala +++ b/project/RFDependenciesPlugin.scala @@ -46,16 +46,13 @@ object RFDependenciesPlugin extends AutoPlugin { val `slf4j-api` = "org.slf4j" % "slf4j-api" % "1.7.25" val scaffeine = "com.github.blemale" %% "scaffeine" % "3.1.0" val `spray-json` = "io.spray" %% "spray-json" % "1.3.4" - - //val `geotrellis-contrib-vlm` = "com.azavea.geotrellis" %% "geotrellis-contrib-vlm" % "2.12.0" - //val `geotrellis-contrib-gdal` = "com.azavea.geotrellis" %% "geotrellis-contrib-gdal" % "2.12.0" } import autoImport._ override def projectSettings = Seq( resolvers ++= Seq( - "locationtech-releases" at "https://repo.locationtech.org/content/groups/releases", "Azavea Public Builds" at "https://dl.bintray.com/azavea/geotrellis", + "locationtech-releases" at "https://repo.locationtech.org/content/groups/releases", "boundless-releases" at "https://repo.boundlessgeo.com/main/", "Open Source Geospatial Foundation Repository" at "http://download.osgeo.org/webdav/geotools/" ), diff --git a/project/RFProjectPlugin.scala b/project/RFProjectPlugin.scala index b7c904416..864f3b9c8 100644 --- a/project/RFProjectPlugin.scala +++ b/project/RFProjectPlugin.scala @@ -46,6 +46,12 @@ object RFProjectPlugin extends AutoPlugin { email = "fitch@astraea.earth", url = url("http://www.astraea.earth") ), + Developer( + id = "vpipkt", + name = "Jason Brown", + email = "jbrown@astraea.earth", + url = url("http://www.astraea.earth") + ), Developer( id = "mteldridge", name = "Matt Eldridge", @@ -58,12 +64,6 @@ object RFProjectPlugin extends AutoPlugin { email = "bguseman@astraea.earth", url = url("http://www.astraea.earth") ), - Developer( - id = "vpipkt", - name = "Jason Brown", - email = "jbrown@astraea.earth", - url = url("http://www.astraea.earth") - ) ), initialCommands in console := """ diff --git a/pyrasterframes/src/main/python/pyrasterframes/version.py b/pyrasterframes/src/main/python/pyrasterframes/version.py index 0a09a6338..7253bac59 100644 --- a/pyrasterframes/src/main/python/pyrasterframes/version.py +++ b/pyrasterframes/src/main/python/pyrasterframes/version.py @@ -20,4 +20,4 @@ # # Translating Java version from version.sbt to PEP440 norms -__version__ = '0.8.4.dev0' +__version__ = '0.9.0.dev0' diff --git a/pyrasterframes/src/main/scala/org/locationtech/rasterframes/py/PyRFContext.scala b/pyrasterframes/src/main/scala/org/locationtech/rasterframes/py/PyRFContext.scala index d651f5cb4..dfb530e26 100644 --- a/pyrasterframes/src/main/scala/org/locationtech/rasterframes/py/PyRFContext.scala +++ b/pyrasterframes/src/main/scala/org/locationtech/rasterframes/py/PyRFContext.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql._ import org.locationtech.rasterframes import org.locationtech.rasterframes.extensions.RasterJoin import org.locationtech.rasterframes.model.LazyCRS -import org.locationtech.rasterframes.ref.{GDALRasterSource, RasterRef, RasterSource} +import org.locationtech.rasterframes.ref.{GDALRasterSource, RasterRef, RFRasterSource} import org.locationtech.rasterframes.util.KryoSupport import org.locationtech.rasterframes.{RasterFunctions, _} import spray.json._ @@ -226,7 +226,7 @@ class PyRFContext(implicit sparkSession: SparkSession) extends RasterFunctions type jDouble = java.lang.Double // NB: Tightly coupled to the `RFContext.resolve_raster_ref` method in `pyrasterframes.rf_context`. */ def _resolveRasterRef(srcBin: Array[Byte], bandIndex: jInt, xmin: jDouble, ymin: jDouble, xmax: jDouble, ymax: jDouble): AnyRef = { - val src = KryoSupport.deserialize[RasterSource](ByteBuffer.wrap(srcBin)) + val src = KryoSupport.deserialize[RFRasterSource](ByteBuffer.wrap(srcBin)) val extent = Extent(xmin, ymin, xmax, ymax) RasterRef(src, bandIndex, Some(extent), None) } diff --git a/version.sbt b/version.sbt index 58771512b..338b0ba29 100644 --- a/version.sbt +++ b/version.sbt @@ -1 +1 @@ -version in ThisBuild := "0.8.4-SNAPSHOT" +version in ThisBuild := "0.9.0-SNAPSHOT" From a16ddb49c353113c38fd8d925033614c369500af Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Tue, 22 Oct 2019 10:50:32 -0400 Subject: [PATCH 03/94] Incremental updates toward GT 3.0 compatibility. --- .../tiles/FixedDelegatingTile.scala | 39 +++++++++++++++++++ .../rasterframes/tiles/InternalRowTile.scala | 4 +- .../tiles/ProjectedRasterTile.scala | 4 +- .../rasterframes/tiles/ShowableTile.scala | 4 +- core/src/test/resources/log4j.properties | 3 +- .../rasterframes/GeometryFunctionsSpec.scala | 1 - .../rasterframes/RasterJoinSpec.scala | 12 ++++-- .../rasterframes/encoders/EncodingSpec.scala | 12 +++++- project/RFProjectPlugin.scala | 2 +- 9 files changed, 68 insertions(+), 13 deletions(-) create mode 100644 core/src/main/scala/org/locationtech/rasterframes/tiles/FixedDelegatingTile.scala diff --git a/core/src/main/scala/org/locationtech/rasterframes/tiles/FixedDelegatingTile.scala b/core/src/main/scala/org/locationtech/rasterframes/tiles/FixedDelegatingTile.scala new file mode 100644 index 000000000..3ba0aa541 --- /dev/null +++ b/core/src/main/scala/org/locationtech/rasterframes/tiles/FixedDelegatingTile.scala @@ -0,0 +1,39 @@ +/* + * This software is licensed under the Apache 2 license, quoted below. + * + * Copyright 2019 Astraea, Inc. + * + * 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. + * + * SPDX-License-Identifier: Apache-2.0 + * + */ + +package org.locationtech.rasterframes.tiles +import geotrellis.raster.{ArrayTile, DelegatingTile, Tile} + +/** + * Workaround for case where `combine` is invoked on two delegating tiles. + * @since 8/22/18 + */ +trait FixedDelegatingTile extends DelegatingTile { + override def combine(r2: Tile)(f: (Int, Int) ⇒ Int): Tile = (delegate, r2) match { + case (del: ArrayTile, r2: DelegatingTile) ⇒ del.combine(r2.toArrayTile())(f) + case _ ⇒ delegate.combine(r2)(f) + } + + override def combineDouble(r2: Tile)(f: (Double, Double) ⇒ Double): Tile = (delegate, r2) match { + case (del: ArrayTile, r2: DelegatingTile) ⇒ del.combineDouble(r2.toArrayTile())(f) + case _ ⇒ delegate.combineDouble(r2)(f) + } +} diff --git a/core/src/main/scala/org/locationtech/rasterframes/tiles/InternalRowTile.scala b/core/src/main/scala/org/locationtech/rasterframes/tiles/InternalRowTile.scala index ba0be15ac..72f5631ae 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/tiles/InternalRowTile.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/tiles/InternalRowTile.scala @@ -23,9 +23,9 @@ package org.locationtech.rasterframes.tiles import java.nio.ByteBuffer -import org.locationtech.rasterframes.encoders.CatalystSerializer.CatalystIO import geotrellis.raster._ import org.apache.spark.sql.catalyst.InternalRow +import org.locationtech.rasterframes.encoders.CatalystSerializer.CatalystIO import org.locationtech.rasterframes.model.{Cells, TileDataContext} /** @@ -34,7 +34,7 @@ import org.locationtech.rasterframes.model.{Cells, TileDataContext} * * @since 11/29/17 */ -class InternalRowTile(val mem: InternalRow) extends DelegatingTile { +class InternalRowTile(val mem: InternalRow) extends FixedDelegatingTile { import InternalRowTile._ override def toArrayTile(): ArrayTile = realizedTile.toArrayTile() diff --git a/core/src/main/scala/org/locationtech/rasterframes/tiles/ProjectedRasterTile.scala b/core/src/main/scala/org/locationtech/rasterframes/tiles/ProjectedRasterTile.scala index 4121892ca..ec490edfc 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/tiles/ProjectedRasterTile.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/tiles/ProjectedRasterTile.scala @@ -23,7 +23,7 @@ package org.locationtech.rasterframes.tiles import geotrellis.proj4.CRS import geotrellis.raster.io.geotiff.SinglebandGeoTiff -import geotrellis.raster.{CellType, DelegatingTile, ProjectedRaster, Tile} +import geotrellis.raster.{CellType, ProjectedRaster, Tile} import geotrellis.vector.{Extent, ProjectedExtent} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.rf.TileUDT @@ -40,7 +40,7 @@ import org.locationtech.rasterframes.ref.RasterRef.RasterRefTile * * @since 9/5/18 */ -trait ProjectedRasterTile extends DelegatingTile with ProjectedRasterLike { +trait ProjectedRasterTile extends FixedDelegatingTile with ProjectedRasterLike { def extent: Extent def crs: CRS def projectedExtent: ProjectedExtent = ProjectedExtent(extent, crs) diff --git a/core/src/main/scala/org/locationtech/rasterframes/tiles/ShowableTile.scala b/core/src/main/scala/org/locationtech/rasterframes/tiles/ShowableTile.scala index 7255f4f73..ccec3a340 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/tiles/ShowableTile.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/tiles/ShowableTile.scala @@ -21,9 +21,9 @@ package org.locationtech.rasterframes.tiles import org.locationtech.rasterframes._ -import geotrellis.raster.{DelegatingTile, Tile, isNoData} +import geotrellis.raster.{Tile, isNoData} -class ShowableTile(val delegate: Tile) extends DelegatingTile { +class ShowableTile(val delegate: Tile) extends FixedDelegatingTile { override def equals(obj: Any): Boolean = obj match { case st: ShowableTile => delegate.equals(st.delegate) case o => delegate.equals(o) diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 9e3b08ac5..e17586b72 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -45,4 +45,5 @@ log4j.logger.org.apache.parquet.hadoop.ParquetRecordReader=OFF log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR -log4j.logger.org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator=ERROR \ No newline at end of file +log4j.logger.org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator=ERROR +log4j.logger.org.apache.spark.sql.execution.WholeStageCodegenExec=ERROR \ No newline at end of file diff --git a/core/src/test/scala/org/locationtech/rasterframes/GeometryFunctionsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/GeometryFunctionsSpec.scala index 7aaffba83..cf0217229 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/GeometryFunctionsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/GeometryFunctionsSpec.scala @@ -25,7 +25,6 @@ import geotrellis.proj4.{LatLng, Sinusoidal, WebMercator} import geotrellis.raster.Dimensions import geotrellis.vector._ import org.locationtech.jts.geom.{Coordinate, GeometryFactory} -import spray.json.JsNumber /** * Test rig for operations providing interop with JTS types. diff --git a/core/src/test/scala/org/locationtech/rasterframes/RasterJoinSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/RasterJoinSpec.scala index b2cd5d8ce..1a9560a39 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/RasterJoinSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/RasterJoinSpec.scala @@ -24,6 +24,8 @@ package org.locationtech.rasterframes import geotrellis.raster.resample.Bilinear import geotrellis.raster.testkit.RasterMatchers import geotrellis.raster.{IntConstantNoDataCellType, Raster, Tile} +import geotrellis.vector.Extent +import org.apache.spark.SparkConf import org.apache.spark.sql.functions._ import org.locationtech.rasterframes.expressions.aggregates.TileRasterizerAggregate import org.locationtech.rasterframes.expressions.aggregates.TileRasterizerAggregate.ProjectedRasterDefinition @@ -74,14 +76,16 @@ class RasterJoinSpec extends TestEnvironment with TestData with RasterMatchers { } it("should join same scene in two projections, same tile size") { - // b4warpedRf source data is gdal warped b4nativeRf data; join them together. val joined = b4nativeRf.rasterJoin(b4warpedRf) // create a Raster from tile2 which should be almost equal to b4nativeTif - val result = joined.agg(TileRasterizerAggregate( + val agg = joined.agg(TileRasterizerAggregate( ProjectedRasterDefinition(b4nativeTif.cols, b4nativeTif.rows, b4nativeTif.cellType, b4nativeTif.crs, b4nativeTif.extent, Bilinear), $"crs", $"extent", $"tile2") as "raster" - ).select(col("raster").as[Raster[Tile]]).first() + ).select(col("raster").as[Raster[Tile]]) + + agg.printSchema() + val result = agg.first() result.extent shouldBe b4nativeTif.extent @@ -165,4 +169,6 @@ class RasterJoinSpec extends TestEnvironment with TestData with RasterMatchers { joined.columns should contain allElementsOf Seq("left_id", "right_id_agg") } } + + override def additionalConf: SparkConf = super.additionalConf.set("spark.sql.codegen.comments", "true") } diff --git a/core/src/test/scala/org/locationtech/rasterframes/encoders/EncodingSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/encoders/EncodingSpec.scala index bd4b1a8e5..38758eaff 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/encoders/EncodingSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/encoders/EncodingSpec.scala @@ -26,7 +26,7 @@ import java.net.URI import geotrellis.layer._ import geotrellis.proj4._ -import geotrellis.raster.{CellType, Tile} +import geotrellis.raster.{ArrayTile, CellType, Raster, Tile} import geotrellis.vector.{Extent, ProjectedExtent} import org.apache.spark.SparkConf import org.apache.spark.sql.Row @@ -145,6 +145,16 @@ class EncodingSpec extends TestEnvironment with TestData { write(ds) assert(ds.first === env) } + + it("should code RDD[Raster[Tile]]") { + import spark.implicits._ + val t: Tile = ArrayTile(Array.emptyDoubleArray, 0, 0) + val e = Extent(1, 2 ,3, 4) + val r = Raster(t, e) + val ds = Seq(r).toDS() + println(ds.first()) + } + } describe("Dataframe encoding ops on spatial types") { diff --git a/project/RFProjectPlugin.scala b/project/RFProjectPlugin.scala index 864f3b9c8..f15e88dda 100644 --- a/project/RFProjectPlugin.scala +++ b/project/RFProjectPlugin.scala @@ -28,7 +28,7 @@ object RFProjectPlugin extends AutoPlugin { "-Ywarn-unused-import" ), scalacOptions in (Compile, doc) ++= Seq("-no-link-warnings"), - console / scalacOptions := Seq("-feature"), + Compile / console / scalacOptions := Seq("-feature"), javacOptions ++= Seq("-source", "1.8", "-target", "1.8"), cancelable in Global := true, publishTo in ThisBuild := sonatypePublishTo.value, From 89e2b70aed3ccc37c58ace86c6eba9bff3614e7d Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Tue, 22 Oct 2019 16:25:51 -0400 Subject: [PATCH 04/94] Incremental updates to type structure. --- build.sbt | 14 -------------- core/src/main/resources/reference.conf | 3 +-- .../rasterframes/ref/ProjectedRasterLike.scala | 2 +- .../rasterframes/ref/RFRasterSource.scala | 2 +- .../locationtech/rasterframes/ref/RasterRef.scala | 4 ++-- .../rasterframes/tiles/FixedDelegatingTile.scala | 2 +- .../rasterframes/tiles/ProjectedRasterTile.scala | 2 +- .../locationtech/rasterframes/RasterJoinSpec.scala | 3 --- .../datasource/geotiff/GeoTiffDataSourceSpec.scala | 4 ++-- project/RFDependenciesPlugin.scala | 14 ++++++++++++++ 10 files changed, 23 insertions(+), 27 deletions(-) diff --git a/build.sbt b/build.sbt index 882ba3b39..19b8cd852 100644 --- a/build.sbt +++ b/build.sbt @@ -70,20 +70,6 @@ lazy val core = project scaffeine, scalatest ), - /** https://github.com/lucidworks/spark-solr/issues/179 - * Thanks @pomadchin for the tip! */ - dependencyOverrides ++= { - val deps = Seq( - "com.fasterxml.jackson.core" % "jackson-core" % "2.6.7", - "com.fasterxml.jackson.core" % "jackson-databind" % "2.6.7", - "com.fasterxml.jackson.core" % "jackson-annotations" % "2.6.7" - ) - CrossVersion.partialVersion(scalaVersion.value) match { - // if Scala 2.12+ is used - case Some((2, scalaMajor)) if scalaMajor >= 12 => deps - case _ => deps :+ "com.fasterxml.jackson.module" %% "jackson-module-scala" % "2.6.7" - } - }, buildInfoKeys ++= Seq[BuildInfoKey]( version, scalaVersion, rfGeoTrellisVersion, rfGeoMesaVersion, rfSparkVersion ), diff --git a/core/src/main/resources/reference.conf b/core/src/main/resources/reference.conf index bcdca6aa3..c677e0aaf 100644 --- a/core/src/main/resources/reference.conf +++ b/core/src/main/resources/reference.conf @@ -6,8 +6,7 @@ rasterframes { max-truncate-row-element-length = 40 raster-source-cache-timeout = 120 seconds } - -vlm.gdal { +geotrellis.raster.gdal { options { // See https://trac.osgeo.org/gdal/wiki/ConfigOptions for options //CPL_DEBUG = "OFF" diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/ProjectedRasterLike.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/ProjectedRasterLike.scala index 1361381ef..5d16c0d2d 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/ProjectedRasterLike.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/ProjectedRasterLike.scala @@ -30,7 +30,7 @@ import geotrellis.vector.Extent * * @since 11/3/18 */ -trait ProjectedRasterLike extends CellGrid[Int] { +trait ProjectedRasterLike { _: CellGrid[Int] => def crs: CRS def extent: Extent } diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/RFRasterSource.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/RFRasterSource.scala index 5deff0344..4db8e8aef 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/RFRasterSource.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/RFRasterSource.scala @@ -44,7 +44,7 @@ import scala.concurrent.duration.Duration * @since 8/21/18 */ @Experimental -trait RFRasterSource extends ProjectedRasterLike with Serializable { +abstract class RFRasterSource extends CellGrid[Int] with ProjectedRasterLike with Serializable { import RFRasterSource._ def crs: CRS diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/RasterRef.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/RasterRef.scala index c239ed3b6..87e811f3e 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/RasterRef.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/RasterRef.scala @@ -23,7 +23,7 @@ package org.locationtech.rasterframes.ref import com.typesafe.scalalogging.LazyLogging import geotrellis.proj4.CRS -import geotrellis.raster.{CellType, GridBounds, Tile} +import geotrellis.raster.{CellGrid, CellType, GridBounds, Tile} import geotrellis.vector.{Extent, ProjectedExtent} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.rf.RasterSourceUDT @@ -39,7 +39,7 @@ import org.locationtech.rasterframes.tiles.ProjectedRasterTile * @since 8/21/18 */ case class RasterRef(source: RFRasterSource, bandIndex: Int, subextent: Option[Extent], subgrid: Option[GridBounds[Int]]) - extends ProjectedRasterLike { + extends CellGrid[Int] with ProjectedRasterLike { def crs: CRS = source.crs def extent: Extent = subextent.getOrElse(source.extent) def projectedExtent: ProjectedExtent = ProjectedExtent(extent, crs) diff --git a/core/src/main/scala/org/locationtech/rasterframes/tiles/FixedDelegatingTile.scala b/core/src/main/scala/org/locationtech/rasterframes/tiles/FixedDelegatingTile.scala index 3ba0aa541..742617abb 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/tiles/FixedDelegatingTile.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/tiles/FixedDelegatingTile.scala @@ -26,7 +26,7 @@ import geotrellis.raster.{ArrayTile, DelegatingTile, Tile} * Workaround for case where `combine` is invoked on two delegating tiles. * @since 8/22/18 */ -trait FixedDelegatingTile extends DelegatingTile { +abstract class FixedDelegatingTile extends DelegatingTile { override def combine(r2: Tile)(f: (Int, Int) ⇒ Int): Tile = (delegate, r2) match { case (del: ArrayTile, r2: DelegatingTile) ⇒ del.combine(r2.toArrayTile())(f) case _ ⇒ delegate.combine(r2)(f) diff --git a/core/src/main/scala/org/locationtech/rasterframes/tiles/ProjectedRasterTile.scala b/core/src/main/scala/org/locationtech/rasterframes/tiles/ProjectedRasterTile.scala index ec490edfc..9a822cebc 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/tiles/ProjectedRasterTile.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/tiles/ProjectedRasterTile.scala @@ -40,7 +40,7 @@ import org.locationtech.rasterframes.ref.RasterRef.RasterRefTile * * @since 9/5/18 */ -trait ProjectedRasterTile extends FixedDelegatingTile with ProjectedRasterLike { +abstract class ProjectedRasterTile extends FixedDelegatingTile with ProjectedRasterLike { def extent: Extent def crs: CRS def projectedExtent: ProjectedExtent = ProjectedExtent(extent, crs) diff --git a/core/src/test/scala/org/locationtech/rasterframes/RasterJoinSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/RasterJoinSpec.scala index 1a9560a39..57ac9418a 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/RasterJoinSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/RasterJoinSpec.scala @@ -24,7 +24,6 @@ package org.locationtech.rasterframes import geotrellis.raster.resample.Bilinear import geotrellis.raster.testkit.RasterMatchers import geotrellis.raster.{IntConstantNoDataCellType, Raster, Tile} -import geotrellis.vector.Extent import org.apache.spark.SparkConf import org.apache.spark.sql.functions._ import org.locationtech.rasterframes.expressions.aggregates.TileRasterizerAggregate @@ -158,8 +157,6 @@ class RasterJoinSpec extends TestEnvironment with TestData with RasterMatchers { total18 should be > 0.0 total18 should be < total17 - - } it("should pass through ancillary columns") { diff --git a/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotiff/GeoTiffDataSourceSpec.scala b/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotiff/GeoTiffDataSourceSpec.scala index ef4de9624..b840307cb 100644 --- a/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotiff/GeoTiffDataSourceSpec.scala +++ b/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotiff/GeoTiffDataSourceSpec.scala @@ -23,7 +23,7 @@ package org.locationtech.rasterframes.datasource.geotiff import java.nio.file.{Path, Paths} import geotrellis.proj4._ -import geotrellis.raster.CellType +import geotrellis.raster.{CellType, Dimensions} import geotrellis.raster.io.geotiff.{MultibandGeoTiff, SinglebandGeoTiff} import geotrellis.vector.Extent import org.locationtech.rasterframes._ @@ -93,7 +93,7 @@ class GeoTiffDataSourceSpec def checkTiff(file: Path, cols: Int, rows: Int, extent: Extent, cellType: Option[CellType] = None) = { val outputTif = SinglebandGeoTiff(file.toString) - outputTif.tile.dimensions should be ((cols, rows)) + outputTif.tile.dimensions should be (Dimensions(cols, rows)) outputTif.extent should be (extent) cellType.foreach(ct => outputTif.cellType should be (ct) diff --git a/project/RFDependenciesPlugin.scala b/project/RFDependenciesPlugin.scala index ba4e8f748..658147555 100644 --- a/project/RFDependenciesPlugin.scala +++ b/project/RFDependenciesPlugin.scala @@ -57,6 +57,20 @@ object RFDependenciesPlugin extends AutoPlugin { "Open Source Geospatial Foundation Repository" at "http://download.osgeo.org/webdav/geotools/" ), + /** https://github.com/lucidworks/spark-solr/issues/179 + * Thanks @pomadchin for the tip! */ + dependencyOverrides ++= { + val deps = Seq( + "com.fasterxml.jackson.core" % "jackson-core" % "2.6.7", + "com.fasterxml.jackson.core" % "jackson-databind" % "2.6.7", + "com.fasterxml.jackson.core" % "jackson-annotations" % "2.6.7" + ) + CrossVersion.partialVersion(scalaVersion.value) match { + // if Scala 2.12+ is used + case Some((2, scalaMajor)) if scalaMajor >= 12 => deps + case _ => deps :+ "com.fasterxml.jackson.module" %% "jackson-module-scala" % "2.6.7" + } + }, // NB: Make sure to update the Spark version in pyrasterframes/python/setup.py rfSparkVersion := "2.4.4", rfGeoTrellisVersion := "3.0.0", From 2341ad65dd484daac4a693c43ede5bf06d92de0d Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Tue, 5 Nov 2019 12:45:22 -0500 Subject: [PATCH 05/94] Backup commit. --- .../rasterframes/ref/ProjectedRasterLike.scala | 7 +++++-- project/RFDependenciesPlugin.scala | 6 +++--- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/ProjectedRasterLike.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/ProjectedRasterLike.scala index 5d16c0d2d..a36796e51 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/ProjectedRasterLike.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/ProjectedRasterLike.scala @@ -22,7 +22,7 @@ package org.locationtech.rasterframes.ref import geotrellis.proj4.CRS -import geotrellis.raster.CellGrid +import geotrellis.raster.{CellGrid, CellType} import geotrellis.vector.Extent /** @@ -30,7 +30,10 @@ import geotrellis.vector.Extent * * @since 11/3/18 */ -trait ProjectedRasterLike { _: CellGrid[Int] => +trait ProjectedRasterLike { def crs: CRS def extent: Extent + def cellType: CellType + def cols: Int + def rows: Int } diff --git a/project/RFDependenciesPlugin.scala b/project/RFDependenciesPlugin.scala index 658147555..cc68991b4 100644 --- a/project/RFDependenciesPlugin.scala +++ b/project/RFDependenciesPlugin.scala @@ -51,12 +51,12 @@ object RFDependenciesPlugin extends AutoPlugin { override def projectSettings = Seq( resolvers ++= Seq( + Resolver.mavenLocal, "Azavea Public Builds" at "https://dl.bintray.com/azavea/geotrellis", "locationtech-releases" at "https://repo.locationtech.org/content/groups/releases", "boundless-releases" at "https://repo.boundlessgeo.com/main/", "Open Source Geospatial Foundation Repository" at "http://download.osgeo.org/webdav/geotools/" ), - /** https://github.com/lucidworks/spark-solr/issues/179 * Thanks @pomadchin for the tip! */ dependencyOverrides ++= { @@ -73,7 +73,7 @@ object RFDependenciesPlugin extends AutoPlugin { }, // NB: Make sure to update the Spark version in pyrasterframes/python/setup.py rfSparkVersion := "2.4.4", - rfGeoTrellisVersion := "3.0.0", - rfGeoMesaVersion := "2.2.1", + rfGeoTrellisVersion := "3.0.0-SNAPSHOT", + rfGeoMesaVersion := "2.2.1" ) } From 88fec1a7b7fc309ecda70629f138c552c9fd59f5 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Fri, 8 Nov 2019 13:18:44 -0500 Subject: [PATCH 06/94] Updated to GT 3.1.0. --- .../org/locationtech/rasterframes/ref/ProjectedRasterLike.scala | 2 +- .../rasterframes/datasource/geotiff/GeoTiffDataSourceSpec.scala | 2 +- project/RFDependenciesPlugin.scala | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/ProjectedRasterLike.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/ProjectedRasterLike.scala index a36796e51..7c4eb0193 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/ProjectedRasterLike.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/ProjectedRasterLike.scala @@ -22,7 +22,7 @@ package org.locationtech.rasterframes.ref import geotrellis.proj4.CRS -import geotrellis.raster.{CellGrid, CellType} +import geotrellis.raster.CellType import geotrellis.vector.Extent /** diff --git a/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotiff/GeoTiffDataSourceSpec.scala b/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotiff/GeoTiffDataSourceSpec.scala index 1de613989..f8d4ebcbb 100644 --- a/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotiff/GeoTiffDataSourceSpec.scala +++ b/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotiff/GeoTiffDataSourceSpec.scala @@ -196,7 +196,7 @@ class GeoTiffDataSourceSpec val sample = rgbCogSample val expectedExtent = sample.extent - val (expCols, expRows) = sample.tile.dimensions + val Dimensions(expCols, expRows) = sample.tile.dimensions val rf = spark.read.raster.withBandIndexes(0, 1, 2).load(rgbCogSamplePath.toASCIIString) diff --git a/project/RFDependenciesPlugin.scala b/project/RFDependenciesPlugin.scala index cc68991b4..7d9311ff3 100644 --- a/project/RFDependenciesPlugin.scala +++ b/project/RFDependenciesPlugin.scala @@ -73,7 +73,7 @@ object RFDependenciesPlugin extends AutoPlugin { }, // NB: Make sure to update the Spark version in pyrasterframes/python/setup.py rfSparkVersion := "2.4.4", - rfGeoTrellisVersion := "3.0.0-SNAPSHOT", + rfGeoTrellisVersion := "3.1.0", rfGeoMesaVersion := "2.2.1" ) } From 9a4f1565af9e43e8e2bb048a9ac717cea019622c Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Thu, 14 Nov 2019 15:55:14 -0500 Subject: [PATCH 07/94] Replaced `TileDimensions` with `Dimension[Int]`. --- .../rasterframes/bench/RasterRefBench.scala | 3 +- build.sbt | 7 ++- .../apache/spark/sql/rf/VersionShims.scala | 7 +-- .../rasterframes/RasterFunctions.scala | 7 +-- .../encoders/StandardEncoders.scala | 5 +- .../encoders/StandardSerializers.scala | 30 +++++++++- .../expressions/accessors/GetDimensions.scala | 12 ++-- .../ProjectedLayerMetadataAggregate.scala | 13 ++-- .../aggregates/TileRasterizerAggregate.scala | 5 +- .../generators/RasterSourceToRasterRefs.scala | 9 ++- .../generators/RasterSourceToTiles.scala | 6 +- .../extensions/SinglebandGeoTiffMethods.scala | 4 +- .../rasterframes/functions/package.scala | 3 +- .../rasterframes/model/TileDataContext.scala | 10 ++-- .../rasterframes/model/TileDimensions.scala | 59 ------------------- .../rasterframes/rasterframes.scala | 5 +- .../rasterframes/ref/RFRasterSource.scala | 8 +-- .../rasterframes/RasterFrameSpec.scala | 3 +- .../rasterframes/RasterFunctionsSpec.scala | 7 +-- .../rasterframes/RasterJoinSpec.scala | 18 +++--- .../encoders/CatalystSerializerSpec.scala | 8 +-- .../rasterframes/encoders/EncodingSpec.scala | 4 +- .../ProjectedLayerMetadataAggregateSpec.scala | 5 +- .../rasterframes/ref/RasterSourceSpec.scala | 11 ++-- .../geotiff/GeoTiffDataSource.scala | 7 ++- .../geotrellis/GeoTrellisCatalog.scala | 6 +- .../raster/RasterSourceDataSource.scala | 6 +- .../raster/RasterSourceRelation.scala | 4 +- .../raster/RasterSourceDataSourceSpec.scala | 17 +++--- docs/src/main/paradox/release-notes.md | 23 ++++++++ project/RFDependenciesPlugin.scala | 3 - .../src/main/python/docs/getting-started.pymd | 6 +- 32 files changed, 146 insertions(+), 175 deletions(-) delete mode 100644 core/src/main/scala/org/locationtech/rasterframes/model/TileDimensions.scala diff --git a/bench/src/main/scala/org/locationtech/rasterframes/bench/RasterRefBench.scala b/bench/src/main/scala/org/locationtech/rasterframes/bench/RasterRefBench.scala index a4fd2dfab..c7e36d985 100644 --- a/bench/src/main/scala/org/locationtech/rasterframes/bench/RasterRefBench.scala +++ b/bench/src/main/scala/org/locationtech/rasterframes/bench/RasterRefBench.scala @@ -28,7 +28,6 @@ import org.apache.spark.sql._ import org.locationtech.rasterframes._ import org.locationtech.rasterframes.expressions.generators.RasterSourceToRasterRefs import org.locationtech.rasterframes.expressions.transformers.RasterRefToTile -import org.locationtech.rasterframes.model.TileDimensions import org.locationtech.rasterframes.ref.RFRasterSource import org.openjdk.jmh.annotations._ @@ -47,7 +46,7 @@ class RasterRefBench extends SparkEnv with LazyLogging { val r2 = RFRasterSource(remoteCOGSingleband2) singleDF = Seq((r1, r2)).toDF("B1", "B2") - .select(RasterRefToTile(RasterSourceToRasterRefs(Some(TileDimensions(r1.dimensions)), Seq(0), $"B1", $"B2"))) + .select(RasterRefToTile(RasterSourceToRasterRefs(Some(r1.dimensions), Seq(0), $"B1", $"B2"))) expandedDF = Seq((r1, r2)).toDF("B1", "B2") .select(RasterRefToTile(RasterSourceToRasterRefs($"B1", $"B2"))) diff --git a/build.sbt b/build.sbt index 19b8cd852..1941be5dd 100644 --- a/build.sbt +++ b/build.sbt @@ -90,9 +90,11 @@ lazy val pyrasterframes = project spark("core").value % Provided, spark("mllib").value % Provided, spark("sql").value % Provided - ) + ), + Test / test := (Test / test).dependsOn(experimental / Test / test).value ) + lazy val datasource = project .configs(IntegrationTest) .settings(Defaults.itSettings) @@ -105,6 +107,7 @@ lazy val datasource = project spark("mllib").value % Provided, spark("sql").value % Provided ), + Test / test := (Test / test).dependsOn(core / Test / test).value, initialCommands in console := (initialCommands in console).value + """ |import org.locationtech.rasterframes.datasource.geotrellis._ @@ -127,7 +130,7 @@ lazy val experimental = project ), fork in IntegrationTest := true, javaOptions in IntegrationTest := Seq("-Xmx2G"), - parallelExecution in IntegrationTest := false + Test / test := (Test / test).dependsOn(datasource / Test / test).value ) lazy val docs = project diff --git a/core/src/main/scala/org/apache/spark/sql/rf/VersionShims.scala b/core/src/main/scala/org/apache/spark/sql/rf/VersionShims.scala index 81418d466..a75932886 100644 --- a/core/src/main/scala/org/apache/spark/sql/rf/VersionShims.scala +++ b/core/src/main/scala/org/apache/spark/sql/rf/VersionShims.scala @@ -2,6 +2,7 @@ package org.apache.spark.sql.rf import java.lang.reflect.Constructor +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder @@ -12,7 +13,6 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types.DataType -import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, SQLContext} import scala.reflect._ import scala.util.{Failure, Success, Try} @@ -23,11 +23,6 @@ import scala.util.{Failure, Success, Try} * @since 2/13/18 */ object VersionShims { - def readJson(sqlContext: SQLContext, rows: Dataset[String]): DataFrame = { - // NB: Will get a deprecation warning for Spark 2.2.x - sqlContext.read.json(rows.rdd) // <-- deprecation warning expected - } - def updateRelation(lr: LogicalRelation, base: BaseRelation): LogicalPlan = { val lrClazz = classOf[LogicalRelation] val ctor = lrClazz.getConstructors.head.asInstanceOf[Constructor[LogicalRelation]] diff --git a/core/src/main/scala/org/locationtech/rasterframes/RasterFunctions.scala b/core/src/main/scala/org/locationtech/rasterframes/RasterFunctions.scala index 94dcef333..b9f3fa27a 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/RasterFunctions.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/RasterFunctions.scala @@ -23,7 +23,7 @@ package org.locationtech.rasterframes import geotrellis.proj4.CRS import geotrellis.raster.mapalgebra.local.LocalTileBinaryOp import geotrellis.raster.render.ColorRamp -import geotrellis.raster.{CellType, Tile} +import geotrellis.raster.{CellType, Dimensions, Tile} import geotrellis.vector.Extent import org.apache.spark.annotation.Experimental import org.apache.spark.sql.functions.{lit, udf} @@ -35,9 +35,8 @@ import org.locationtech.rasterframes.expressions.aggregates._ import org.locationtech.rasterframes.expressions.generators._ import org.locationtech.rasterframes.expressions.localops._ import org.locationtech.rasterframes.expressions.tilestats._ -import org.locationtech.rasterframes.expressions.transformers.RenderPNG.{RenderCompositePNG, RenderColorRampPNG} +import org.locationtech.rasterframes.expressions.transformers.RenderPNG.{RenderColorRampPNG, RenderCompositePNG} import org.locationtech.rasterframes.expressions.transformers._ -import org.locationtech.rasterframes.model.TileDimensions import org.locationtech.rasterframes.stats._ import org.locationtech.rasterframes.{functions => F} @@ -51,7 +50,7 @@ trait RasterFunctions { // format: off /** Query the number of (cols, rows) in a Tile. */ - def rf_dimensions(col: Column): TypedColumn[Any, TileDimensions] = GetDimensions(col) + def rf_dimensions(col: Column): TypedColumn[Any, Dimensions[Int]] = GetDimensions(col) /** Extracts the bounding box of a geometry as an Extent */ def st_extent(col: Column): TypedColumn[Any, Extent] = GeometryToExtent(col) diff --git a/core/src/main/scala/org/locationtech/rasterframes/encoders/StandardEncoders.scala b/core/src/main/scala/org/locationtech/rasterframes/encoders/StandardEncoders.scala index b7b3211f5..302262768 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/encoders/StandardEncoders.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/encoders/StandardEncoders.scala @@ -27,7 +27,7 @@ import java.sql.Timestamp import org.locationtech.rasterframes.stats.{CellHistogram, CellStatistics, LocalCellStatistics} import org.locationtech.jts.geom.Envelope import geotrellis.proj4.CRS -import geotrellis.raster.{CellSize, CellType, Raster, Tile, TileLayout} +import geotrellis.raster.{CellSize, CellType, Dimensions, Raster, Tile, TileLayout} import geotrellis.layer._ import geotrellis.vector.{Extent, ProjectedExtent} import org.apache.spark.sql.{Encoder, Encoders} @@ -70,8 +70,7 @@ trait StandardEncoders extends SpatialEncoders { implicit def tileContextEncoder: ExpressionEncoder[TileContext] = TileContext.encoder implicit def tileDataContextEncoder: ExpressionEncoder[TileDataContext] = TileDataContext.encoder implicit def extentTilePairEncoder: Encoder[(ProjectedExtent, Tile)] = Encoders.tuple(projectedExtentEncoder, singlebandTileEncoder) - - + implicit def tileDimensionsEncoder: Encoder[Dimensions[Int]] = CatalystSerializerEncoder[Dimensions[Int]](true) } object StandardEncoders extends StandardEncoders diff --git a/core/src/main/scala/org/locationtech/rasterframes/encoders/StandardSerializers.scala b/core/src/main/scala/org/locationtech/rasterframes/encoders/StandardSerializers.scala index 79eb65255..a1815d7c7 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/encoders/StandardSerializers.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/encoders/StandardSerializers.scala @@ -25,7 +25,6 @@ import com.github.blemale.scaffeine.Scaffeine import geotrellis.proj4.CRS import geotrellis.raster._ import geotrellis.layer._ - import geotrellis.vector._ import org.apache.spark.sql.types._ import org.locationtech.jts.geom.Envelope @@ -60,9 +59,11 @@ trait StandardSerializers { StructField("xmax", DoubleType, false), StructField("ymax", DoubleType, false) )) + override def to[R](t: Extent, io: CatalystIO[R]): R = io.create( t.xmin, t.ymin, t.xmax, t.ymax ) + override def from[R](row: R, io: CatalystIO[R]): Extent = Extent( io.getDouble(row, 0), io.getDouble(row, 1), @@ -95,6 +96,7 @@ trait StandardSerializers { override val schema: StructType = StructType(Seq( StructField("crsProj4", StringType, false) )) + override def to[R](t: CRS, io: CatalystIO[R]): R = io.create( io.encode( // Don't do this... it's 1000x slower to decode. @@ -102,18 +104,23 @@ trait StandardSerializers { t.toProj4String ) ) + override def from[R](row: R, io: CatalystIO[R]): CRS = LazyCRS(io.getString(row, 0)) } implicit val cellTypeSerializer: CatalystSerializer[CellType] = new CatalystSerializer[CellType] { + import StandardSerializers._ + override val schema: StructType = StructType(Seq( StructField("cellTypeName", StringType, false) )) + override def to[R](t: CellType, io: CatalystIO[R]): R = io.create( io.encode(ct2sCache.get(t)) ) + override def from[R](row: R, io: CatalystIO[R]): CellType = s2ctCache.get(io.getString(row, 0)) } @@ -229,7 +236,7 @@ trait StandardSerializers { ) } - implicit def boundsSerializer[T >: Null: CatalystSerializer]: CatalystSerializer[KeyBounds[T]] = new CatalystSerializer[KeyBounds[T]] { + implicit def boundsSerializer[T >: Null : CatalystSerializer]: CatalystSerializer[KeyBounds[T]] = new CatalystSerializer[KeyBounds[T]] { override val schema: StructType = StructType(Seq( StructField("minKey", schemaOf[T], true), StructField("maxKey", schemaOf[T], true) @@ -246,7 +253,7 @@ trait StandardSerializers { ) } - def tileLayerMetadataSerializer[T >: Null: CatalystSerializer]: CatalystSerializer[TileLayerMetadata[T]] = new CatalystSerializer[TileLayerMetadata[T]] { + def tileLayerMetadataSerializer[T >: Null : CatalystSerializer]: CatalystSerializer[TileLayerMetadata[T]] = new CatalystSerializer[TileLayerMetadata[T]] { override val schema: StructType = StructType(Seq( StructField("cellType", schemaOf[CellType], false), StructField("layout", schemaOf[LayoutDefinition], false), @@ -273,6 +280,7 @@ trait StandardSerializers { } implicit def rasterSerializer: CatalystSerializer[Raster[Tile]] = new CatalystSerializer[Raster[Tile]] { + import org.apache.spark.sql.rf.TileUDT.tileSerializer override val schema: StructType = StructType(Seq( @@ -294,6 +302,22 @@ trait StandardSerializers { implicit val spatialKeyTLMSerializer = tileLayerMetadataSerializer[SpatialKey] implicit val spaceTimeKeyTLMSerializer = tileLayerMetadataSerializer[SpaceTimeKey] + implicit val tileDimensionsSerializer: CatalystSerializer[Dimensions[Int]] = new CatalystSerializer[Dimensions[Int]] { + override val schema: StructType = StructType(Seq( + StructField("cols", IntegerType, false), + StructField("rows", IntegerType, false) + )) + + override protected def to[R](t: Dimensions[Int], io: CatalystIO[R]): R = io.create( + t.cols, + t.rows + ) + + override protected def from[R](t: R, io: CatalystIO[R]): Dimensions[Int] = Dimensions[Int]( + io.getInt(t, 0), + io.getInt(t, 1) + ) + } } object StandardSerializers { diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetDimensions.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetDimensions.scala index e4db95f40..2e8c71ded 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetDimensions.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetDimensions.scala @@ -23,11 +23,10 @@ package org.locationtech.rasterframes.expressions.accessors import org.locationtech.rasterframes.encoders.CatalystSerializer._ import org.locationtech.rasterframes.expressions.OnCellGridExpression -import geotrellis.raster.CellGrid +import geotrellis.raster.{CellGrid, Dimensions} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionDescription} import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback -import org.locationtech.rasterframes.model.TileDimensions /** * Extract a raster's dimensions @@ -43,12 +42,13 @@ import org.locationtech.rasterframes.model.TileDimensions case class GetDimensions(child: Expression) extends OnCellGridExpression with CodegenFallback { override def nodeName: String = "rf_dimensions" - def dataType = schemaOf[TileDimensions] + def dataType = schemaOf[Dimensions[Int]] - override def eval(grid: CellGrid[Int]): Any = TileDimensions(grid.cols, grid.rows).toInternalRow + override def eval(grid: CellGrid[Int]): Any = Dimensions[Int](grid.cols, grid.rows).toInternalRow } object GetDimensions { - def apply(col: Column): TypedColumn[Any, TileDimensions] = - new Column(new GetDimensions(col.expr)).as[TileDimensions] + import org.locationtech.rasterframes.encoders.StandardEncoders.tileDimensionsEncoder + def apply(col: Column): TypedColumn[Any, Dimensions[Int]] = + new Column(new GetDimensions(col.expr)).as[Dimensions[Int]] } diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/aggregates/ProjectedLayerMetadataAggregate.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/aggregates/ProjectedLayerMetadataAggregate.scala index 2bc89e592..ca9c8f58f 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/aggregates/ProjectedLayerMetadataAggregate.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/aggregates/ProjectedLayerMetadataAggregate.scala @@ -24,7 +24,6 @@ package org.locationtech.rasterframes.expressions.aggregates import org.locationtech.rasterframes._ import org.locationtech.rasterframes.encoders.CatalystSerializer import org.locationtech.rasterframes.encoders.CatalystSerializer._ -import org.locationtech.rasterframes.model.TileDimensions import geotrellis.proj4.{CRS, Transform} import geotrellis.raster._ import geotrellis.raster.reproject.{Reproject, ReprojectRasterExtent} @@ -34,7 +33,7 @@ import org.apache.spark.sql.expressions.{MutableAggregationBuffer, UserDefinedAg import org.apache.spark.sql.types.{DataType, StructField, StructType} import org.apache.spark.sql.{Column, Row, TypedColumn} -class ProjectedLayerMetadataAggregate(destCRS: CRS, destDims: TileDimensions) extends UserDefinedAggregateFunction { +class ProjectedLayerMetadataAggregate(destCRS: CRS, destDims: Dimensions[Int]) extends UserDefinedAggregateFunction { import ProjectedLayerMetadataAggregate._ override def inputSchema: StructType = CatalystSerializer[InputRecord].schema @@ -94,14 +93,14 @@ object ProjectedLayerMetadataAggregate { /** Primary user facing constructor */ def apply(destCRS: CRS, extent: Column, crs: Column, cellType: Column, tileSize: Column): TypedColumn[Any, TileLayerMetadata[SpatialKey]] = // Ordering must match InputRecord schema - new ProjectedLayerMetadataAggregate(destCRS, TileDimensions(NOMINAL_TILE_SIZE, NOMINAL_TILE_SIZE))(extent, crs, cellType, tileSize).as[TileLayerMetadata[SpatialKey]] + new ProjectedLayerMetadataAggregate(destCRS, Dimensions(NOMINAL_TILE_SIZE, NOMINAL_TILE_SIZE))(extent, crs, cellType, tileSize).as[TileLayerMetadata[SpatialKey]] - def apply(destCRS: CRS, destDims: TileDimensions, extent: Column, crs: Column, cellType: Column, tileSize: Column): TypedColumn[Any, TileLayerMetadata[SpatialKey]] = + def apply(destCRS: CRS, destDims: Dimensions[Int], extent: Column, crs: Column, cellType: Column, tileSize: Column): TypedColumn[Any, TileLayerMetadata[SpatialKey]] = // Ordering must match InputRecord schema new ProjectedLayerMetadataAggregate(destCRS, destDims)(extent, crs, cellType, tileSize).as[TileLayerMetadata[SpatialKey]] private[expressions] - case class InputRecord(extent: Extent, crs: CRS, cellType: CellType, tileSize: TileDimensions) { + case class InputRecord(extent: Extent, crs: CRS, cellType: CellType, tileSize: Dimensions[Int]) { def toBufferRecord(destCRS: CRS): BufferRecord = { val transform = Transform(crs, destCRS) @@ -125,7 +124,7 @@ object ProjectedLayerMetadataAggregate { StructField("extent", CatalystSerializer[Extent].schema, false), StructField("crs", CatalystSerializer[CRS].schema, false), StructField("cellType", CatalystSerializer[CellType].schema, false), - StructField("tileSize", CatalystSerializer[TileDimensions].schema, false) + StructField("tileSize", CatalystSerializer[Dimensions[Int]].schema, false) )) override protected def to[R](t: InputRecord, io: CatalystIO[R]): R = @@ -135,7 +134,7 @@ object ProjectedLayerMetadataAggregate { io.get[Extent](t, 0), io.get[CRS](t, 1), io.get[CellType](t, 2), - io.get[TileDimensions](t, 3) + io.get[Dimensions[Int]](t, 3) ) } } diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/aggregates/TileRasterizerAggregate.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/aggregates/TileRasterizerAggregate.scala index e4e2884fe..9eed70f0f 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/aggregates/TileRasterizerAggregate.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/aggregates/TileRasterizerAggregate.scala @@ -24,7 +24,7 @@ package org.locationtech.rasterframes.expressions.aggregates import geotrellis.proj4.CRS import geotrellis.raster.reproject.Reproject import geotrellis.raster.resample.ResampleMethod -import geotrellis.raster.{ArrayTile, CellType, MultibandTile, ProjectedRaster, Raster, Tile} +import geotrellis.raster.{ArrayTile, CellType, Dimensions, MultibandTile, ProjectedRaster, Raster, Tile} import geotrellis.layer._ import geotrellis.vector.Extent import org.apache.spark.sql.expressions.{MutableAggregationBuffer, UserDefinedAggregateFunction} @@ -34,7 +34,6 @@ import org.locationtech.rasterframes._ import org.locationtech.rasterframes.util._ import org.locationtech.rasterframes.encoders.CatalystSerializer._ import org.locationtech.rasterframes.expressions.aggregates.TileRasterizerAggregate.ProjectedRasterDefinition -import org.locationtech.rasterframes.model.TileDimensions import org.slf4j.LoggerFactory /** @@ -119,7 +118,7 @@ object TileRasterizerAggregate { new TileRasterizerAggregate(prd)(crsCol, extentCol, tileCol).as(nodeName).as[Raster[Tile]] } - def collect(df: DataFrame, destCRS: CRS, destExtent: Option[Extent], rasterDims: Option[TileDimensions]): ProjectedRaster[MultibandTile] = { + def collect(df: DataFrame, destCRS: CRS, destExtent: Option[Extent], rasterDims: Option[Dimensions[Int]]): ProjectedRaster[MultibandTile] = { val tileCols = WithDataFrameMethods(df).tileColumns require(tileCols.nonEmpty, "need at least one tile column") // Select the anchoring Tile, Extent and CRS columns diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/generators/RasterSourceToRasterRefs.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/generators/RasterSourceToRasterRefs.scala index 73e8df458..7022f75db 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/generators/RasterSourceToRasterRefs.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/generators/RasterSourceToRasterRefs.scala @@ -21,7 +21,7 @@ package org.locationtech.rasterframes.expressions.generators -import geotrellis.raster.GridBounds +import geotrellis.raster.{Dimensions, GridBounds} import geotrellis.vector.Extent import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ @@ -30,8 +30,7 @@ import org.apache.spark.sql.types.{DataType, StructField, StructType} import org.apache.spark.sql.{Column, TypedColumn} import org.locationtech.rasterframes.encoders.CatalystSerializer._ import org.locationtech.rasterframes.expressions.generators.RasterSourceToRasterRefs.bandNames -import org.locationtech.rasterframes.model.TileDimensions -import org.locationtech.rasterframes.ref.{RasterRef, RFRasterSource} +import org.locationtech.rasterframes.ref.{RFRasterSource, RasterRef} import org.locationtech.rasterframes.util._ import org.locationtech.rasterframes.RasterSourceType @@ -43,7 +42,7 @@ import scala.util.control.NonFatal * * @since 9/6/18 */ -case class RasterSourceToRasterRefs(children: Seq[Expression], bandIndexes: Seq[Int], subtileDims: Option[TileDimensions] = None) extends Expression +case class RasterSourceToRasterRefs(children: Seq[Expression], bandIndexes: Seq[Int], subtileDims: Option[Dimensions[Int]] = None) extends Expression with Generator with CodegenFallback with ExpectsInputTypes { override def inputTypes: Seq[DataType] = Seq.fill(children.size)(RasterSourceType) @@ -86,7 +85,7 @@ case class RasterSourceToRasterRefs(children: Seq[Expression], bandIndexes: Seq[ object RasterSourceToRasterRefs { def apply(rrs: Column*): TypedColumn[Any, RasterRef] = apply(None, Seq(0), rrs: _*) - def apply(subtileDims: Option[TileDimensions], bandIndexes: Seq[Int], rrs: Column*): TypedColumn[Any, RasterRef] = + def apply(subtileDims: Option[Dimensions[Int]], bandIndexes: Seq[Int], rrs: Column*): TypedColumn[Any, RasterRef] = new Column(new RasterSourceToRasterRefs(rrs.map(_.expr), bandIndexes, subtileDims)).as[RasterRef] private[rasterframes] def bandNames(basename: String, bandIndexes: Seq[Int]): Seq[String] = bandIndexes match { diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/generators/RasterSourceToTiles.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/generators/RasterSourceToTiles.scala index 595bac20d..309d306ae 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/generators/RasterSourceToTiles.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/generators/RasterSourceToTiles.scala @@ -22,6 +22,7 @@ package org.locationtech.rasterframes.expressions.generators import com.typesafe.scalalogging.Logger +import geotrellis.raster.Dimensions import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback @@ -31,7 +32,6 @@ import org.locationtech.rasterframes import org.locationtech.rasterframes.RasterSourceType import org.locationtech.rasterframes.encoders.CatalystSerializer._ import org.locationtech.rasterframes.expressions.generators.RasterSourceToRasterRefs.bandNames -import org.locationtech.rasterframes.model.TileDimensions import org.locationtech.rasterframes.tiles.ProjectedRasterTile import org.locationtech.rasterframes.util._ import org.slf4j.LoggerFactory @@ -45,7 +45,7 @@ import scala.util.control.NonFatal * * @since 9/6/18 */ -case class RasterSourceToTiles(children: Seq[Expression], bandIndexes: Seq[Int], subtileDims: Option[TileDimensions] = None) extends Expression +case class RasterSourceToTiles(children: Seq[Expression], bandIndexes: Seq[Int], subtileDims: Option[Dimensions[Int]] = None) extends Expression with Generator with CodegenFallback with ExpectsInputTypes { @transient protected lazy val logger = Logger(LoggerFactory.getLogger(getClass.getName)) @@ -84,7 +84,7 @@ case class RasterSourceToTiles(children: Seq[Expression], bandIndexes: Seq[Int], object RasterSourceToTiles { def apply(rrs: Column*): TypedColumn[Any, ProjectedRasterTile] = apply(None, Seq(0), rrs: _*) - def apply(subtileDims: Option[TileDimensions], bandIndexes: Seq[Int], rrs: Column*): TypedColumn[Any, ProjectedRasterTile] = + def apply(subtileDims: Option[Dimensions[Int]], bandIndexes: Seq[Int], rrs: Column*): TypedColumn[Any, ProjectedRasterTile] = new Column(new RasterSourceToTiles(rrs.map(_.expr), bandIndexes, subtileDims)).as[ProjectedRasterTile] } diff --git a/core/src/main/scala/org/locationtech/rasterframes/extensions/SinglebandGeoTiffMethods.scala b/core/src/main/scala/org/locationtech/rasterframes/extensions/SinglebandGeoTiffMethods.scala index 168444efe..7815a0ecc 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/extensions/SinglebandGeoTiffMethods.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/extensions/SinglebandGeoTiffMethods.scala @@ -22,6 +22,7 @@ package org.locationtech.rasterframes.extensions import geotrellis.proj4.CRS +import geotrellis.raster.Dimensions import geotrellis.raster.io.geotiff.SinglebandGeoTiff import geotrellis.util.MethodExtensions import geotrellis.vector.Extent @@ -29,11 +30,10 @@ import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.sql.{DataFrame, Row, SparkSession} import org.locationtech.rasterframes._ import org.locationtech.rasterframes.encoders.CatalystSerializer._ -import org.locationtech.rasterframes.model.TileDimensions import org.locationtech.rasterframes.tiles.ProjectedRasterTile trait SinglebandGeoTiffMethods extends MethodExtensions[SinglebandGeoTiff] { - def toDF(dims: TileDimensions = NOMINAL_TILE_DIMS)(implicit spark: SparkSession): DataFrame = { + def toDF(dims: Dimensions[Int] = NOMINAL_TILE_DIMS)(implicit spark: SparkSession): DataFrame = { val segmentLayout = self.imageData.segmentLayout val re = self.rasterExtent diff --git a/core/src/main/scala/org/locationtech/rasterframes/functions/package.scala b/core/src/main/scala/org/locationtech/rasterframes/functions/package.scala index 521c9822b..ebd06c48f 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/functions/package.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/functions/package.scala @@ -27,7 +27,6 @@ import org.apache.spark.sql.functions.udf import org.apache.spark.sql.{Row, SQLContext} import org.locationtech.jts.geom.Geometry import org.locationtech.rasterframes.encoders.CatalystSerializer._ -import org.locationtech.rasterframes.model.TileDimensions /** * Module utils. @@ -104,7 +103,7 @@ package object functions { require(tiles.length == rightExtentEnc.length && tiles.length == rightCRSEnc.length, "size mismatch") val leftExtent = leftExtentEnc.to[Extent] - val leftDims = leftDimsEnc.to[TileDimensions] + val leftDims = leftDimsEnc.to[Dimensions[Int]] val leftCRS = leftCRSEnc.to[CRS] val rightExtents = rightExtentEnc.map(_.to[Extent]) val rightCRSs = rightCRSEnc.map(_.to[CRS]) diff --git a/core/src/main/scala/org/locationtech/rasterframes/model/TileDataContext.scala b/core/src/main/scala/org/locationtech/rasterframes/model/TileDataContext.scala index addc4aee5..9d0d5f387 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/model/TileDataContext.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/model/TileDataContext.scala @@ -22,13 +22,13 @@ package org.locationtech.rasterframes.model import org.locationtech.rasterframes.encoders.CatalystSerializer._ -import geotrellis.raster.{CellType, Tile} +import geotrellis.raster.{CellType, Dimensions, Tile} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.types.{StructField, StructType} import org.locationtech.rasterframes.encoders.{CatalystSerializer, CatalystSerializerEncoder} /** Encapsulates all information about a tile aside from actual cell values. */ -case class TileDataContext(cellType: CellType, dimensions: TileDimensions) +case class TileDataContext(cellType: CellType, dimensions: Dimensions[Int]) object TileDataContext { /** Extracts the TileDataContext from a Tile. */ @@ -36,14 +36,14 @@ object TileDataContext { require(t.cols <= Short.MaxValue, s"RasterFrames doesn't support tiles of size ${t.cols}") require(t.rows <= Short.MaxValue, s"RasterFrames doesn't support tiles of size ${t.rows}") TileDataContext( - t.cellType, TileDimensions(t.dimensions) + t.cellType, t.dimensions ) } implicit val serializer: CatalystSerializer[TileDataContext] = new CatalystSerializer[TileDataContext] { override val schema: StructType = StructType(Seq( StructField("cellType", schemaOf[CellType], false), - StructField("dimensions", schemaOf[TileDimensions], false) + StructField("dimensions", schemaOf[Dimensions[Int]], false) )) override protected def to[R](t: TileDataContext, io: CatalystIO[R]): R = io.create( @@ -52,7 +52,7 @@ object TileDataContext { ) override protected def from[R](t: R, io: CatalystIO[R]): TileDataContext = TileDataContext( io.get[CellType](t, 0), - io.get[TileDimensions](t, 1) + io.get[Dimensions[Int]](t, 1) ) } diff --git a/core/src/main/scala/org/locationtech/rasterframes/model/TileDimensions.scala b/core/src/main/scala/org/locationtech/rasterframes/model/TileDimensions.scala deleted file mode 100644 index 683f5fb27..000000000 --- a/core/src/main/scala/org/locationtech/rasterframes/model/TileDimensions.scala +++ /dev/null @@ -1,59 +0,0 @@ -/* - * This software is licensed under the Apache 2 license, quoted below. - * - * Copyright 2019 Astraea, Inc. - * - * 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. - * - * SPDX-License-Identifier: Apache-2.0 - * - */ - -package org.locationtech.rasterframes.model - -import geotrellis.raster.Dimensions -import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.types.{IntegerType, StructField, StructType} -import org.locationtech.rasterframes.encoders.CatalystSerializer -import org.locationtech.rasterframes.encoders.CatalystSerializer.CatalystIO - -/** - * Typed wrapper for tile size information. - * - * @since 2018-12-12 - */ -case class TileDimensions(cols: Int, rows: Int) - -object TileDimensions { - def apply(colsRows: (Int, Int)): TileDimensions = new TileDimensions(colsRows._1, colsRows._2) - def apply(dims: Dimensions[Int]): TileDimensions = new TileDimensions(dims.cols, dims.rows) - - implicit val serializer: CatalystSerializer[TileDimensions] = new CatalystSerializer[TileDimensions] { - override val schema: StructType = StructType(Seq( - StructField("cols", IntegerType, false), - StructField("rows", IntegerType, false) - )) - - override protected def to[R](t: TileDimensions, io: CatalystIO[R]): R = io.create( - t.cols, - t.rows - ) - - override protected def from[R](t: R, io: CatalystIO[R]): TileDimensions = TileDimensions( - io.getInt(t, 0), - io.getInt(t, 1) - ) - } - - implicit def encoder: ExpressionEncoder[TileDimensions] = ExpressionEncoder[TileDimensions]() -} diff --git a/core/src/main/scala/org/locationtech/rasterframes/rasterframes.scala b/core/src/main/scala/org/locationtech/rasterframes/rasterframes.scala index 19c0fa1c6..d39ee9359 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/rasterframes.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/rasterframes.scala @@ -22,7 +22,7 @@ package org.locationtech import com.typesafe.config.ConfigFactory import com.typesafe.scalalogging.Logger -import geotrellis.raster.{Tile, TileFeature, isData} +import geotrellis.raster.{Dimensions, Tile, TileFeature, isData} import geotrellis.layer._ import geotrellis.spark.ContextRDD import org.apache.spark.rdd.RDD @@ -31,7 +31,6 @@ import org.apache.spark.sql.{DataFrame, SQLContext, rf} import org.locationtech.geomesa.spark.jts.DataFrameFunctions import org.locationtech.rasterframes.encoders.StandardEncoders import org.locationtech.rasterframes.extensions.Implicits -import org.locationtech.rasterframes.model.TileDimensions import org.slf4j.LoggerFactory import shapeless.tag.@@ @@ -53,7 +52,7 @@ package object rasterframes extends StandardColumns /** The generally expected tile size, as defined by configuration property `rasterframes.nominal-tile-size`.*/ @transient final val NOMINAL_TILE_SIZE: Int = rfConfig.getInt("nominal-tile-size") - final val NOMINAL_TILE_DIMS: TileDimensions = TileDimensions(NOMINAL_TILE_SIZE, NOMINAL_TILE_SIZE) + final val NOMINAL_TILE_DIMS: Dimensions[Int] = Dimensions(NOMINAL_TILE_SIZE, NOMINAL_TILE_SIZE) /** * Initialization injection point. Must be called before any RasterFrameLayer diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/RFRasterSource.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/RFRasterSource.scala index 4db8e8aef..ec4053ec9 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/RFRasterSource.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/RFRasterSource.scala @@ -33,7 +33,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.rf.RasterSourceUDT -import org.locationtech.rasterframes.model.{TileContext, TileDimensions} +import org.locationtech.rasterframes.model.TileContext import org.locationtech.rasterframes.{NOMINAL_TILE_DIMS, rfConfig} import scala.concurrent.duration.Duration @@ -63,7 +63,7 @@ abstract class RFRasterSource extends CellGrid[Int] with ProjectedRasterLike wit def read(extent: Extent, bands: Seq[Int] = SINGLEBAND): Raster[MultibandTile] = read(rasterExtent.gridBoundsFor(extent, clamp = true), bands) - def readAll(dims: TileDimensions = NOMINAL_TILE_DIMS, bands: Seq[Int] = SINGLEBAND): Seq[Raster[MultibandTile]] = + def readAll(dims: Dimensions[Int] = NOMINAL_TILE_DIMS, bands: Seq[Int] = SINGLEBAND): Seq[Raster[MultibandTile]] = layoutBounds(dims).map(read(_, bands)) protected def readBounds(bounds: Traversable[GridBounds[Int]], bands: Seq[Int]): Iterator[Raster[MultibandTile]] @@ -78,12 +78,12 @@ abstract class RFRasterSource extends CellGrid[Int] with ProjectedRasterLike wit def tileContext: TileContext = TileContext(extent, crs) - def layoutExtents(dims: TileDimensions): Seq[Extent] = { + def layoutExtents(dims: Dimensions[Int]): Seq[Extent] = { val re = rasterExtent layoutBounds(dims).map(re.extentFor(_, clamp = true)) } - def layoutBounds(dims: TileDimensions): Seq[GridBounds[Int]] = { + def layoutBounds(dims: Dimensions[Int]): Seq[GridBounds[Int]] = { gridBounds.split(dims.cols, dims.rows).toSeq } } diff --git a/core/src/test/scala/org/locationtech/rasterframes/RasterFrameSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/RasterFrameSpec.scala index 4d95d6e6a..eb081a4f7 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/RasterFrameSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/RasterFrameSpec.scala @@ -35,7 +35,6 @@ import geotrellis.layer._ import geotrellis.vector.{Extent, ProjectedExtent} import org.apache.spark.sql.functions._ import org.apache.spark.sql.{SQLContext, SparkSession} -import org.locationtech.rasterframes.model.TileDimensions import scala.util.control.NonFatal @@ -90,7 +89,7 @@ class RasterFrameSpec extends TestEnvironment with MetadataKeys assert( rf.select(rf_dimensions($"tile")) .collect() - .forall(_ == TileDimensions(10, 10)) + .forall(_ == Dimensions(10, 10)) ) assert(rf.count() === 4) diff --git a/core/src/test/scala/org/locationtech/rasterframes/RasterFunctionsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/RasterFunctionsSpec.scala index bc0f7ce1d..dd0158401 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/RasterFunctionsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/RasterFunctionsSpec.scala @@ -28,10 +28,9 @@ import geotrellis.raster._ import geotrellis.raster.render.ColorRamps import geotrellis.raster.testkit.RasterMatchers import javax.imageio.ImageIO -import org.apache.spark.sql.{Column, Encoders, TypedColumn} +import org.apache.spark.sql.Encoders import org.apache.spark.sql.functions._ import org.locationtech.rasterframes.expressions.accessors.ExtractTile -import org.locationtech.rasterframes.model.TileDimensions import org.locationtech.rasterframes.stats._ import org.locationtech.rasterframes.tiles.ProjectedRasterTile @@ -303,7 +302,7 @@ class RasterFunctionsSpec extends TestEnvironment with RasterMatchers { describe("raster metadata") { it("should get the TileDimensions of a Tile") { val t = Seq(randPRT).toDF("tile").select(rf_dimensions($"tile")).first() - t should be (TileDimensions(randPRT.dimensions)) + t should be (randPRT.dimensions) checkDocs("rf_dimensions") } it("should get the Extent of a ProjectedRasterTile") { @@ -703,7 +702,7 @@ class RasterFunctionsSpec extends TestEnvironment with RasterMatchers { val withMasked = withMask.withColumn("masked", rf_inverse_mask_by_value($"tile", $"mask", mask_value)) .withColumn("masked2", rf_mask_by_value($"tile", $"mask", lit(mask_value), true)) - withMasked.explain(true) + val result = withMasked.agg(rf_agg_no_data_cells($"tile") < rf_agg_no_data_cells($"masked")).as[Boolean] result.first() should be(true) diff --git a/core/src/test/scala/org/locationtech/rasterframes/RasterJoinSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/RasterJoinSpec.scala index 57ac9418a..b86a511d0 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/RasterJoinSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/RasterJoinSpec.scala @@ -23,12 +23,11 @@ package org.locationtech.rasterframes import geotrellis.raster.resample.Bilinear import geotrellis.raster.testkit.RasterMatchers -import geotrellis.raster.{IntConstantNoDataCellType, Raster, Tile} +import geotrellis.raster.{Dimensions, IntConstantNoDataCellType, Raster, Tile} import org.apache.spark.SparkConf import org.apache.spark.sql.functions._ import org.locationtech.rasterframes.expressions.aggregates.TileRasterizerAggregate import org.locationtech.rasterframes.expressions.aggregates.TileRasterizerAggregate.ProjectedRasterDefinition -import org.locationtech.rasterframes.model.TileDimensions class RasterJoinSpec extends TestEnvironment with TestData with RasterMatchers { @@ -38,13 +37,13 @@ class RasterJoinSpec extends TestEnvironment with TestData with RasterMatchers { // Same data, reprojected to EPSG:4326 val b4warpedTif = readSingleband("L8-B4-Elkton-VA-4326.tiff") - val b4nativeRf = b4nativeTif.toDF(TileDimensions(10, 10)) - val b4warpedRf = b4warpedTif.toDF(TileDimensions(10, 10)) + val b4nativeRf = b4nativeTif.toDF(Dimensions(10, 10)) + val b4warpedRf = b4warpedTif.toDF(Dimensions(10, 10)) .withColumnRenamed("tile", "tile2") it("should join the same scene correctly") { - val b4nativeRfPrime = b4nativeTif.toDF(TileDimensions(10, 10)) + val b4nativeRfPrime = b4nativeTif.toDF(Dimensions(10, 10)) .withColumnRenamed("tile", "tile2") val joined = b4nativeRf.rasterJoin(b4nativeRfPrime) @@ -59,7 +58,7 @@ class RasterJoinSpec extends TestEnvironment with TestData with RasterMatchers { } it("should join same scene in different tile sizes"){ - val r1prime = b4nativeTif.toDF(TileDimensions(25, 25)).withColumnRenamed("tile", "tile2") + val r1prime = b4nativeTif.toDF(Dimensions(25, 25)).withColumnRenamed("tile", "tile2") r1prime.select(rf_dimensions($"tile2").getField("rows")).as[Int].first() should be (25) val joined = b4nativeRf.rasterJoin(r1prime) @@ -83,7 +82,6 @@ class RasterJoinSpec extends TestEnvironment with TestData with RasterMatchers { $"crs", $"extent", $"tile2") as "raster" ).select(col("raster").as[Raster[Tile]]) - agg.printSchema() val result = agg.first() result.extent shouldBe b4nativeTif.extent @@ -130,11 +128,11 @@ class RasterJoinSpec extends TestEnvironment with TestData with RasterMatchers { it("should join with heterogeneous LHS CRS and coverages"){ val df17 = readSingleband("m_3607824_se_17_1_20160620_subset.tif") - .toDF(TileDimensions(50, 50)) + .toDF(Dimensions(50, 50)) .withColumn("utm", lit(17)) // neighboring and slightly overlapping NAIP scene val df18 = readSingleband("m_3607717_sw_18_1_20160620_subset.tif") - .toDF(TileDimensions(60, 60)) + .toDF(Dimensions(60, 60)) .withColumn("utm", lit(18)) df17.count() should be (6 * 6) // file is 300 x 300 @@ -146,7 +144,7 @@ class RasterJoinSpec extends TestEnvironment with TestData with RasterMatchers { df.select($"crs".getField("crsProj4")).distinct().as[String].collect() should contain theSameElementsAs expectCrs // read a third source to join. burned in box that intersects both above subsets; but more so on the df17 - val box = readSingleband("m_3607_box.tif").toDF(TileDimensions(4,4)).withColumnRenamed("tile", "burned") + val box = readSingleband("m_3607_box.tif").toDF(Dimensions(4,4)).withColumnRenamed("tile", "burned") val joined = df.rasterJoin(box) joined.count() should be (df.count) diff --git a/core/src/test/scala/org/locationtech/rasterframes/encoders/CatalystSerializerSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/encoders/CatalystSerializerSpec.scala index cfe1b81a5..dc8a60f22 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/encoders/CatalystSerializerSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/encoders/CatalystSerializerSpec.scala @@ -24,20 +24,20 @@ package org.locationtech.rasterframes.encoders import java.time.ZonedDateTime import geotrellis.proj4._ -import geotrellis.raster.{CellSize, CellType, TileLayout, UShortUserDefinedNoDataCellType} +import geotrellis.raster.{CellSize, CellType, Dimensions, TileLayout, UShortUserDefinedNoDataCellType} import geotrellis.layer._ import geotrellis.vector.{Extent, ProjectedExtent} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.locationtech.rasterframes.{TestData, TestEnvironment} import org.locationtech.rasterframes.encoders.StandardEncoders._ -import org.locationtech.rasterframes.model.{CellContext, TileContext, TileDataContext, TileDimensions} -import org.locationtech.rasterframes.ref.{RasterRef, RFRasterSource} +import org.locationtech.rasterframes.model.{CellContext, TileContext, TileDataContext} +import org.locationtech.rasterframes.ref.{RFRasterSource, RasterRef} import org.scalatest.Assertion class CatalystSerializerSpec extends TestEnvironment { import TestData._ - val dc = TileDataContext(UShortUserDefinedNoDataCellType(3), TileDimensions(12, 23)) + val dc = TileDataContext(UShortUserDefinedNoDataCellType(3), Dimensions(12, 23)) val tc = TileContext(Extent(1, 2, 3, 4), WebMercator) val cc = CellContext(tc, dc, 34, 45) val ext = Extent(1.2, 2.3, 3.4, 4.5) diff --git a/core/src/test/scala/org/locationtech/rasterframes/encoders/EncodingSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/encoders/EncodingSpec.scala index 38758eaff..97a833a46 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/encoders/EncodingSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/encoders/EncodingSpec.scala @@ -152,9 +152,9 @@ class EncodingSpec extends TestEnvironment with TestData { val e = Extent(1, 2 ,3, 4) val r = Raster(t, e) val ds = Seq(r).toDS() - println(ds.first()) + ds.first().tile should be (t) + ds.first().extent should be (e) } - } describe("Dataframe encoding ops on spatial types") { diff --git a/core/src/test/scala/org/locationtech/rasterframes/expressions/ProjectedLayerMetadataAggregateSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/expressions/ProjectedLayerMetadataAggregateSpec.scala index 09ee27903..00154c9a9 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/expressions/ProjectedLayerMetadataAggregateSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/expressions/ProjectedLayerMetadataAggregateSpec.scala @@ -21,14 +21,13 @@ package org.locationtech.rasterframes.expressions +import geotrellis.layer._ import geotrellis.raster.Tile import geotrellis.spark._ -import geotrellis.layer._ import geotrellis.vector.{Extent, ProjectedExtent} import org.locationtech.rasterframes._ import org.locationtech.rasterframes.encoders.serialized_literal import org.locationtech.rasterframes.expressions.aggregates.ProjectedLayerMetadataAggregate -import org.locationtech.rasterframes.model.TileDimensions class ProjectedLayerMetadataAggregateSpec extends TestEnvironment { @@ -49,7 +48,7 @@ class ProjectedLayerMetadataAggregateSpec extends TestEnvironment { .map { case (ext, tile) => (ProjectedExtent(ext, crs), tile) } .rdd.collectMetadata[SpatialKey](FloatingLayoutScheme(tileDims._1, tileDims._2)) - val md = df.select(ProjectedLayerMetadataAggregate(crs, TileDimensions(tileDims), $"extent", + val md = df.select(ProjectedLayerMetadataAggregate(crs, tileDims, $"extent", serialized_literal(crs), rf_cell_type($"tile"), rf_dimensions($"tile"))) val tlm2 = md.first() diff --git a/core/src/test/scala/org/locationtech/rasterframes/ref/RasterSourceSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/ref/RasterSourceSpec.scala index d11832f21..54c8f3a47 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/ref/RasterSourceSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/ref/RasterSourceSpec.scala @@ -23,11 +23,10 @@ package org.locationtech.rasterframes.ref import java.net.URI -import geotrellis.raster.RasterExtent +import geotrellis.raster.{Dimensions, RasterExtent} import geotrellis.vector._ import org.apache.spark.sql.rf.RasterSourceUDT import org.locationtech.rasterframes._ -import org.locationtech.rasterframes.model._ import org.locationtech.rasterframes.util.GridHasGridBounds @@ -45,17 +44,17 @@ class RasterSourceSpec extends TestEnvironment with TestData { } val rs = RFRasterSource(getClass.getResource("/L8-B8-Robinson-IL.tiff").toURI) it("should compute nominal tile layout bounds") { - val bounds = rs.layoutBounds(TileDimensions(65, 60)) + val bounds = rs.layoutBounds(Dimensions(65, 60)) val agg = bounds.reduce(_ combine _) agg should be (rs.gridBounds) } it("should compute nominal tile layout extents") { - val extents = rs.layoutExtents(TileDimensions(63, 63)) + val extents = rs.layoutExtents(Dimensions(63, 63)) val agg = extents.reduce(_ combine _) agg should be (rs.extent) } it("should reassemble correct grid from extents") { - val dims = TileDimensions(63, 63) + val dims = Dimensions(63, 63) val ext = rs.layoutExtents(dims).head val bounds = rs.layoutBounds(dims).head rs.rasterExtent.gridBoundsFor(ext) should be (bounds) @@ -150,7 +149,7 @@ class RasterSourceSpec extends TestEnvironment with TestData { gdal.cellType should be(jvm.cellType) } it("should compute the same dimensions as JVM RasterSource") { - val dims = TileDimensions(128, 128) + val dims = Dimensions(128, 128) gdal.extent should be(jvm.extent) gdal.rasterExtent should be(jvm.rasterExtent) gdal.cellSize should be(jvm.cellSize) diff --git a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotiff/GeoTiffDataSource.scala b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotiff/GeoTiffDataSource.scala index d236449ed..e25ef20c0 100644 --- a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotiff/GeoTiffDataSource.scala +++ b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotiff/GeoTiffDataSource.scala @@ -24,6 +24,7 @@ package org.locationtech.rasterframes.datasource.geotiff import java.net.URI import _root_.geotrellis.proj4.CRS +import _root_.geotrellis.raster.Dimensions import _root_.geotrellis.raster.io.geotiff.compression._ import _root_.geotrellis.raster.io.geotiff.tags.codes.ColorSpace import _root_.geotrellis.raster.io.geotiff.{GeoTiffOptions, MultibandGeoTiff, Tags, Tiled} @@ -33,7 +34,7 @@ import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, Da import org.locationtech.rasterframes._ import org.locationtech.rasterframes.datasource._ import org.locationtech.rasterframes.expressions.aggregates.TileRasterizerAggregate -import org.locationtech.rasterframes.model.{LazyCRS, TileDimensions} +import org.locationtech.rasterframes.model.LazyCRS import org.locationtech.rasterframes.util._ import org.slf4j.LoggerFactory @@ -117,13 +118,13 @@ object GeoTiffDataSource { def path: Option[URI] = uriParam(PATH_PARAM, parameters) def compress: Boolean = parameters.get(COMPRESS_PARAM).exists(_.toBoolean) def crs: Option[CRS] = parameters.get(CRS_PARAM).map(s => LazyCRS(s)) - def rasterDimensions: Option[TileDimensions] = { + def rasterDimensions: Option[Dimensions[Int]] = { numParam(IMAGE_WIDTH_PARAM, parameters) .zip(numParam(IMAGE_HEIGHT_PARAM, parameters)) .map { case (cols, rows) => require(cols <= Int.MaxValue && rows <= Int.MaxValue, s"Can't construct a GeoTIFF of size $cols x $rows. (Too big!)") - TileDimensions(cols.toInt, rows.toInt) + Dimensions(cols.toInt, rows.toInt) } .headOption } diff --git a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisCatalog.scala b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisCatalog.scala index b296f19e6..0cfd9e134 100644 --- a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisCatalog.scala +++ b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisCatalog.scala @@ -28,7 +28,6 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ -import org.apache.spark.sql.rf.VersionShims import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType import org.locationtech.rasterframes.datasource.geotrellis.GeoTrellisCatalog.GeoTrellisCatalogRelation @@ -93,8 +92,9 @@ object GeoTrellisCatalog { .map(io.circe.Printer.noSpaces.pretty) .toDS - val headers = VersionShims.readJson(sqlContext, broadcast(headerRows)) - val metadata = VersionShims.readJson(sqlContext, broadcast(metadataRows)) + + val headers = sqlContext.read.json(headerRows) + val metadata = sqlContext.read.json(metadataRows) broadcast(indexedLayers).join(broadcast(headers), Seq("index")).join(broadcast(metadata), Seq("index")) } diff --git a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/raster/RasterSourceDataSource.scala b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/raster/RasterSourceDataSource.scala index 03b2fd0da..de6d6531e 100644 --- a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/raster/RasterSourceDataSource.scala +++ b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/raster/RasterSourceDataSource.scala @@ -24,11 +24,11 @@ package org.locationtech.rasterframes.datasource.raster import java.net.URI import java.util.UUID +import geotrellis.raster.Dimensions import org.locationtech.rasterframes._ import org.locationtech.rasterframes.util._ import org.apache.spark.sql.{DataFrame, DataFrameReader, SQLContext} import org.apache.spark.sql.sources.{BaseRelation, DataSourceRegister, RelationProvider} -import org.locationtech.rasterframes.model.TileDimensions import shapeless.tag import shapeless.tag.@@ @@ -105,10 +105,10 @@ object RasterSourceDataSource { implicit class ParamsDictAccessors(val parameters: Map[String, String]) extends AnyVal { def tokenize(csv: String): Seq[String] = csv.split(',').map(_.trim) - def tileDims: Option[TileDimensions] = + def tileDims: Option[Dimensions[Int]] = parameters.get(TILE_DIMS_PARAM) .map(tokenize(_).map(_.toInt)) - .map { case Seq(cols, rows) => TileDimensions(cols, rows)} + .map { case Seq(cols, rows) => Dimensions(cols, rows)} def bandIndexes: Seq[Int] = parameters .get(BAND_INDEXES_PARAM) diff --git a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/raster/RasterSourceRelation.scala b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/raster/RasterSourceRelation.scala index 9b381d3a6..2bb1a8758 100644 --- a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/raster/RasterSourceRelation.scala +++ b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/raster/RasterSourceRelation.scala @@ -21,6 +21,7 @@ package org.locationtech.rasterframes.datasource.raster +import geotrellis.raster.Dimensions import org.apache.spark.rdd.RDD import org.apache.spark.sql.functions._ import org.apache.spark.sql.sources.{BaseRelation, TableScan} @@ -31,7 +32,6 @@ import org.locationtech.rasterframes.encoders.CatalystSerializer._ import org.locationtech.rasterframes.expressions.generators.{RasterSourceToRasterRefs, RasterSourceToTiles} import org.locationtech.rasterframes.expressions.generators.RasterSourceToRasterRefs.bandNames import org.locationtech.rasterframes.expressions.transformers.{RasterRefToTile, URIToRasterSource} -import org.locationtech.rasterframes.model.TileDimensions import org.locationtech.rasterframes.tiles.ProjectedRasterTile /** @@ -45,7 +45,7 @@ case class RasterSourceRelation( sqlContext: SQLContext, catalogTable: RasterSourceCatalogRef, bandIndexes: Seq[Int], - subtileDims: Option[TileDimensions], + subtileDims: Option[Dimensions[Int]], lazyTiles: Boolean ) extends BaseRelation with TableScan { diff --git a/datasource/src/test/scala/org/locationtech/rasterframes/datasource/raster/RasterSourceDataSourceSpec.scala b/datasource/src/test/scala/org/locationtech/rasterframes/datasource/raster/RasterSourceDataSourceSpec.scala index 7bd46ce37..fc107e07c 100644 --- a/datasource/src/test/scala/org/locationtech/rasterframes/datasource/raster/RasterSourceDataSourceSpec.scala +++ b/datasource/src/test/scala/org/locationtech/rasterframes/datasource/raster/RasterSourceDataSourceSpec.scala @@ -20,11 +20,10 @@ */ package org.locationtech.rasterframes.datasource.raster -import geotrellis.raster.Tile -import org.apache.spark.sql.functions.{lit, udf, round} +import geotrellis.raster.{Dimensions, Tile} +import org.apache.spark.sql.functions.{lit, round, udf} import org.locationtech.rasterframes.{TestEnvironment, _} import org.locationtech.rasterframes.datasource.raster.RasterSourceDataSource.{RasterSourceCatalog, _} -import org.locationtech.rasterframes.model.TileDimensions import org.locationtech.rasterframes.ref.RasterRef.RasterRefTile import org.locationtech.rasterframes.util._ @@ -59,7 +58,7 @@ class RasterSourceDataSourceSpec extends TestEnvironment with TestData { } it("should parse tile dimensions") { val p = Map(TILE_DIMS_PARAM -> "4, 5") - p.tileDims should be (Some(TileDimensions(4, 5))) + p.tileDims should be (Some(Dimensions(4, 5))) } it("should parse path table specification") { @@ -120,9 +119,9 @@ class RasterSourceDataSourceSpec extends TestEnvironment with TestData { df.count() should be(math.ceil(1028.0 / 128).toInt * math.ceil(989.0 / 128).toInt) - val dims = df.select(rf_dimensions($"$b").as[TileDimensions]).distinct().collect() + val dims = df.select(rf_dimensions($"$b").as[Dimensions[Int]]).distinct().collect() dims should contain allElementsOf - Seq(TileDimensions(4,128), TileDimensions(128,128), TileDimensions(128,93), TileDimensions(4,93)) + Seq(Dimensions(4,128), Dimensions(128,128), Dimensions(128,93), Dimensions(4,93)) df.select($"${b}_path").distinct().count() should be(1) } @@ -281,13 +280,13 @@ class RasterSourceDataSourceSpec extends TestEnvironment with TestData { .fromCatalog(cat, "red", "green", "blue").load() val dims = df.select(rf_dimensions($"red")).first() - dims should be (TileDimensions(l8Sample(1).tile.dimensions)) + dims should be (l8Sample(1).tile.dimensions) } it("should provide MODIS tiles with requested size") { val res = modis_df .withColumn("dims", rf_dimensions($"proj_raster")) - .select($"dims".as[TileDimensions]).distinct().collect() + .select($"dims".as[Dimensions[Int]]).distinct().collect() forEvery(res) { r => r.cols should be <= 256 @@ -298,7 +297,7 @@ class RasterSourceDataSourceSpec extends TestEnvironment with TestData { it("should provide Landsat tiles with requested size") { val dims = l8_df .withColumn("dims", rf_dimensions($"proj_raster")) - .select($"dims".as[TileDimensions]).distinct().collect() + .select($"dims".as[Dimensions[Int]]).distinct().collect() forEvery(dims) { d => d.cols should be <= 32 diff --git a/docs/src/main/paradox/release-notes.md b/docs/src/main/paradox/release-notes.md index c181d55da..3e007baf7 100644 --- a/docs/src/main/paradox/release-notes.md +++ b/docs/src/main/paradox/release-notes.md @@ -1,5 +1,28 @@ # Release Notes +## 0.9.x + +### 0.9.0 + +* Upgraded to GeoTrellis 3.1.0. This includes a number of _breaking_ changes enumerated as a part of the [PR's](https://github.com/locationtech/rasterframes/pull/398) change log. These include: + - Add `Int` type parameter to `Grid` + - Add `Int` type parameter to `CellGrid` + - Add `Int` type parameter to `GridBounds`... or `TileBounds` + - Use `GridBounds.toGridType` to coerce from `Int` to `Long` type parameter + - Update imports for layers, particularly `geotrellis.spark.tiling` to `geotrellis.layer` + - Update imports for `geotrellis.spark.io` to `geotrellis.spark.store...` + - Removed `FixedRasterExtent` + - Removed `org.locationtech.rasterframes.util.Shims` + - Change `Extent.jtsGeom` to `Extent.toPolygon` + - Change `TileLayerMetadata.gridBounds` to `TileLayerMetadata.tileBounds` + - Add `geotrellis-gdal` dependency + - Remove any conversions between JTS geometry and old `geotrellis.vector` geometry + - Changed `org.locationtech.rasterframes.encoders.StandardEncoders.crsEncoder` to `crsSparkEncoder` + - Change `(cols, rows)` dimension destructuring to `Dimensions(cols, rows)` + - Revisit use of `Tile` equality since [it's more strict](https://github.com/locationtech/geotrellis/pull/2991) + - Update `reference.conf` to use `geotrellis.raster.gdal` namespace. + - Replace all uses of `TileDimensions` with `geotrellis.raster.Dimensions[Int]`. + ## 0.8.x ### 0.8.4 diff --git a/project/RFDependenciesPlugin.scala b/project/RFDependenciesPlugin.scala index 7d9311ff3..d911e9316 100644 --- a/project/RFDependenciesPlugin.scala +++ b/project/RFDependenciesPlugin.scala @@ -51,11 +51,8 @@ object RFDependenciesPlugin extends AutoPlugin { override def projectSettings = Seq( resolvers ++= Seq( - Resolver.mavenLocal, "Azavea Public Builds" at "https://dl.bintray.com/azavea/geotrellis", "locationtech-releases" at "https://repo.locationtech.org/content/groups/releases", - "boundless-releases" at "https://repo.boundlessgeo.com/main/", - "Open Source Geospatial Foundation Repository" at "http://download.osgeo.org/webdav/geotools/" ), /** https://github.com/lucidworks/spark-solr/issues/179 * Thanks @pomadchin for the tip! */ diff --git a/pyrasterframes/src/main/python/docs/getting-started.pymd b/pyrasterframes/src/main/python/docs/getting-started.pymd index 748070eee..c04044a34 100644 --- a/pyrasterframes/src/main/python/docs/getting-started.pymd +++ b/pyrasterframes/src/main/python/docs/getting-started.pymd @@ -116,8 +116,8 @@ If you would like to use RasterFrames in Scala, you'll need to add the following ```scala resolvers ++= Seq( - "locationtech-releases" at "https://repo.locationtech.org/content/groups/releases", - "Azavea Public Builds" at "https://dl.bintray.com/azavea/geotrellis" + "Azavea Public Builds" at "https://dl.bintray.com/azavea/geotrellis", + "locationtech-releases" at "https://repo.locationtech.org/content/groups/releases" ) libraryDependencies ++= Seq( "org.locationtech.rasterframes" %% "rasterframes" % ${VERSION}, @@ -127,6 +127,8 @@ libraryDependencies ++= Seq( ) ``` +RasterFrames is compatible with Spark 2.4.4. + ## Installing GDAL Support GDAL provides a wide variety of drivers to read data from many different raster formats. If GDAL is installed in the environment, RasterFrames will be able to @ref:[read](raster-read.md) those formats. If you are using the @ref:[Jupyter Notebook image](getting-started.md#jupyter-notebook), GDAL is already installed for you. Otherwise follow the instructions below. Version 2.4.1 or greater is required. From 73452516e7af9b1d6d7ff1d93f0d344a90aea243 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Fri, 15 Nov 2019 10:31:41 -0500 Subject: [PATCH 08/94] Evaluated removal of `FixedDelegationTile`, created (ignored) test to verify behavior, and filed locationtech/geotrellis#3153. --- build.sbt | 10 +--- .../tiles/FixedDelegatingTile.scala | 1 + .../rasterframes/TileStatsSpec.scala | 53 ++++++++++++++++++- 3 files changed, 55 insertions(+), 9 deletions(-) diff --git a/build.sbt b/build.sbt index 1941be5dd..12b98a0a8 100644 --- a/build.sbt +++ b/build.sbt @@ -90,11 +90,9 @@ lazy val pyrasterframes = project spark("core").value % Provided, spark("mllib").value % Provided, spark("sql").value % Provided - ), - Test / test := (Test / test).dependsOn(experimental / Test / test).value + ) ) - lazy val datasource = project .configs(IntegrationTest) .settings(Defaults.itSettings) @@ -107,7 +105,6 @@ lazy val datasource = project spark("mllib").value % Provided, spark("sql").value % Provided ), - Test / test := (Test / test).dependsOn(core / Test / test).value, initialCommands in console := (initialCommands in console).value + """ |import org.locationtech.rasterframes.datasource.geotrellis._ @@ -129,8 +126,7 @@ lazy val experimental = project spark("sql").value % Provided ), fork in IntegrationTest := true, - javaOptions in IntegrationTest := Seq("-Xmx2G"), - Test / test := (Test / test).dependsOn(datasource / Test / test).value + javaOptions in IntegrationTest := Seq("-Xmx2G") ) lazy val docs = project @@ -171,8 +167,6 @@ lazy val docs = project addMappingsToSiteDir(Compile / paradox / mappings, paradox / siteSubdirName) ) -//ParadoxMaterialThemePlugin.paradoxMaterialThemeSettings(Paradox) - lazy val bench = project .dependsOn(core % "compile->test") .settings(publish / skip := true) diff --git a/core/src/main/scala/org/locationtech/rasterframes/tiles/FixedDelegatingTile.scala b/core/src/main/scala/org/locationtech/rasterframes/tiles/FixedDelegatingTile.scala index 742617abb..5bdb7d258 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/tiles/FixedDelegatingTile.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/tiles/FixedDelegatingTile.scala @@ -24,6 +24,7 @@ import geotrellis.raster.{ArrayTile, DelegatingTile, Tile} /** * Workaround for case where `combine` is invoked on two delegating tiles. + * Remove after https://github.com/locationtech/geotrellis/issues/3153 is fixed and integrated * @since 8/22/18 */ abstract class FixedDelegatingTile extends DelegatingTile { diff --git a/core/src/test/scala/org/locationtech/rasterframes/TileStatsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/TileStatsSpec.scala index ac2118c0c..6ae3b9e62 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/TileStatsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/TileStatsSpec.scala @@ -22,12 +22,13 @@ package org.locationtech.rasterframes import geotrellis.raster._ -import geotrellis.raster.mapalgebra.local.{Max, Min} +import geotrellis.raster.mapalgebra.local.{Add, Max, Min} import geotrellis.spark._ import org.apache.spark.sql.Column import org.apache.spark.sql.functions._ import org.locationtech.rasterframes.TestData.randomTile import org.locationtech.rasterframes.stats.CellHistogram +import org.locationtech.rasterframes.util.DataBiasedOp.BiasedAdd /** * Test rig associated with computing statistics and other descriptive @@ -318,6 +319,56 @@ class TileStatsSpec extends TestEnvironment with TestData { val ndCount2 = ndTiles.select("*").where(rf_is_no_data_tile($"tiles")).count() ndCount2 should be(count + 1) } + + // Awaiting https://github.com/locationtech/geotrellis/issues/3153 to be fixed and integrated + ignore("should allow NoData algebra to be changed via delegating tile") { + val t1 = ArrayTile(Array.fill(4)(1), 2, 2) + val t2 = { + val d = Array.fill(4)(2) + d(1) = geotrellis.raster.NODATA + ArrayTile(d, 2, 2) + } + + val d1 = new DelegatingTile { + override def delegate: Tile = t1 + } + val d2 = new DelegatingTile { + override def delegate: Tile = t2 + } + + /** Counts the number of non-NoData cells in a tile */ + case object CountData { + def apply(t: Tile) = { + var count: Long = 0 + t.dualForeach( + z ⇒ if(isData(z)) count = count + 1 + ) ( + z ⇒ if(isData(z)) count = count + 1 + ) + count + } + } + + // Confirm counts + CountData(t1) should be (4L) + CountData(t2) should be (3L) + CountData(d1) should be (4L) + CountData(d2) should be (3L) + + // Standard Add evaluates `x + NoData` as `NoData` + CountData(Add(t1, t2)) should be (3L) + CountData(Add(d1, d2)) should be (3L) + // Is commutative + CountData(Add(t2, t1)) should be (3L) + CountData(Add(d2, d1)) should be (3L) + + // With BiasedAdd, all cells should be data cells + CountData(BiasedAdd(t1, t2)) should be (4L) // <-- passes + CountData(BiasedAdd(d1, d2)) should be (4L) // <-- fails + // Should be commutative. + CountData(BiasedAdd(t2, t1)) should be (4L) // <-- passes + CountData(BiasedAdd(d2, d1)) should be (4L) // <-- fails + } } describe("proj_raster handling") { From 6333ef2a45adc6de9d5880af7583aeefac4fabb8 Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Tue, 7 Jan 2020 15:52:59 -0500 Subject: [PATCH 09/94] Formally adopting Python 3 Signed-off-by: Jason T. Brown --- docs/src/main/paradox/release-notes.md | 1 + .../main/python/pyrasterframes/__init__.py | 51 +-- .../python/pyrasterframes/rasterfunctions.py | 358 ++++++++++-------- .../main/python/pyrasterframes/rf_context.py | 13 +- .../main/python/pyrasterframes/rf_ipython.py | 23 +- .../main/python/pyrasterframes/rf_types.py | 36 +- .../src/main/python/pyrasterframes/utils.py | 33 +- .../src/main/python/pyrasterframes/version.py | 2 +- pyrasterframes/src/main/python/setup.py | 7 +- .../main/python/tests/PyRasterFramesTests.py | 4 +- .../main/python/tests/RasterFunctionsTests.py | 7 + .../src/main/python/tests/__init__.py | 8 +- 12 files changed, 308 insertions(+), 235 deletions(-) diff --git a/docs/src/main/paradox/release-notes.md b/docs/src/main/paradox/release-notes.md index 801d5686b..78848473d 100644 --- a/docs/src/main/paradox/release-notes.md +++ b/docs/src/main/paradox/release-notes.md @@ -22,6 +22,7 @@ - Revisit use of `Tile` equality since [it's more strict](https://github.com/locationtech/geotrellis/pull/2991) - Update `reference.conf` to use `geotrellis.raster.gdal` namespace. - Replace all uses of `TileDimensions` with `geotrellis.raster.Dimensions[Int]`. +* Formally abandon support for Python 2. Python 2 is dead. Long live Python 2. ## 0.8.x diff --git a/pyrasterframes/src/main/python/pyrasterframes/__init__.py b/pyrasterframes/src/main/python/pyrasterframes/__init__.py index 5f89508b1..d2d698627 100644 --- a/pyrasterframes/src/main/python/pyrasterframes/__init__.py +++ b/pyrasterframes/src/main/python/pyrasterframes/__init__.py @@ -23,11 +23,9 @@ appended to PySpark classes. """ -from __future__ import absolute_import from pyspark import SparkContext from pyspark.sql import SparkSession, DataFrame, DataFrameReader, DataFrameWriter from pyspark.sql.column import _to_java_column -from geomesa_pyspark import types # <-- required to ensure Shapely UDTs get registered. # Import RasterFrameLayer types and functions from .rf_context import RFContext @@ -35,10 +33,12 @@ from .rf_types import RasterFrameLayer, TileExploder, TileUDT, RasterSourceUDT import geomesa_pyspark.types # enable vector integrations +from typing import Dict, Tuple, List, Optional, Union + __all__ = ['RasterFrameLayer', 'TileExploder'] -def _rf_init(spark_session): +def _rf_init(spark_session: SparkSession) -> SparkSession: """ Adds RasterFrames functionality to PySpark session.""" if not hasattr(spark_session, "rasterframes"): spark_session.rasterframes = RFContext(spark_session) @@ -47,7 +47,7 @@ def _rf_init(spark_session): return spark_session -def _kryo_init(builder): +def _kryo_init(builder: SparkSession.Builder) -> SparkSession.Builder: """Registers Kryo Serializers for better performance.""" # NB: These methods need to be kept up-to-date wit those in `org.locationtech.rasterframes.extensions.KryoMethods` builder \ @@ -56,7 +56,9 @@ def _kryo_init(builder): .config("spark.kryoserializer.buffer.max", "500m") return builder -def _convert_df(df, sp_key=None, metadata=None): + +def _convert_df(df: DataFrame, sp_key=None, metadata=None) -> RasterFrameLayer: + """ Internal function to convert a DataFrame to a RasterFrameLayer. """ ctx = SparkContext._active_spark_context._rf_context if sp_key is None: @@ -67,7 +69,10 @@ def _convert_df(df, sp_key=None, metadata=None): df._jdf, _to_java_column(sp_key), json.dumps(metadata)), ctx._spark_session) -def _raster_join(df, other, left_extent=None, left_crs=None, right_extent=None, right_crs=None, join_exprs=None): +def _raster_join(df: DataFrame, other: DataFrame, + left_extent=None, left_crs=None, + right_extent=None, right_crs=None, + join_exprs=None) -> DataFrame: ctx = SparkContext._active_spark_context._rf_context if join_exprs is not None: assert left_extent is not None and left_crs is not None and right_extent is not None and right_crs is not None @@ -86,31 +91,31 @@ def _raster_join(df, other, left_extent=None, left_crs=None, right_extent=None, return RasterFrameLayer(jdf, ctx._spark_session) -def _layer_reader(df_reader, format_key, path, **options): +def _layer_reader(df_reader: DataFrameReader, format_key: str, path: Optional[str], **options: str) -> RasterFrameLayer: """ Loads the file of the given type at the given path.""" df = df_reader.format(format_key).load(path, **options) return _convert_df(df) -def _aliased_reader(df_reader, format_key, path, **options): +def _aliased_reader(df_reader: DataFrameReader, format_key: str, path: Optional[str], **options: str) -> DataFrame: """ Loads the file of the given type at the given path.""" return df_reader.format(format_key).load(path, **options) -def _aliased_writer(df_writer, format_key, path, **options): +def _aliased_writer(df_writer: DataFrameWriter, format_key: str, path: Optional[str], **options: str): """ Saves the dataframe to a file of the given type at the given path.""" return df_writer.format(format_key).save(path, **options) def _raster_reader( - df_reader, + df_reader: DataFrameReader, source=None, - catalog_col_names=None, - band_indexes=None, - tile_dimensions=(256, 256), - lazy_tiles=True, + catalog_col_names: Optional[List[str]] = None, + band_indexes: Optional[List[int]] = None, + tile_dimensions: Tuple[int] = (256, 256), + lazy_tiles: bool = True, spatial_index_partitions=None, - **options): + **options: str) -> DataFrame: """ Returns a Spark DataFrame from raster data files specified by URIs. Each row in the returned DataFrame will contain a column with struct of (CRS, Extent, Tile) for each item in @@ -166,7 +171,7 @@ def temp_name(): options.update({ "band_indexes": to_csv(band_indexes), "tile_dimensions": to_csv(tile_dimensions), - "lazy_tiles": lazy_tiles + "lazy_tiles": str(lazy_tiles) }) # Parse the `source` argument @@ -241,19 +246,19 @@ def temp_name(): def _geotiff_writer( - df_writer, - path=None, - crs=None, - raster_dimensions=None, - **options): + df_writer: DataFrameWriter, + path: str, + crs: Optional[str] = None, + raster_dimensions: Tuple[int] = None, + **options: str): def set_dims(parts): parts = [int(p) for p in parts] assert len(parts) == 2, "Expected dimensions specification to have exactly two components" assert all([p > 0 for p in parts]), "Expected all components in dimensions to be positive integers" options.update({ - "imageWidth": parts[0], - "imageHeight": parts[1] + "imageWidth": str(parts[0]), + "imageHeight": str(parts[1]) }) parts = [int(p) for p in parts] assert all([p > 0 for p in parts]), 'nice message' diff --git a/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py b/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py index 6e71f9ed2..f6031ba45 100644 --- a/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py +++ b/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py @@ -23,21 +23,38 @@ implementations. Most functions are standard Column functions, but those with unique signatures are handled here as well. """ -from __future__ import absolute_import from pyspark.sql.column import Column, _to_java_column from pyspark.sql.functions import lit from .rf_context import RFContext from .rf_types import CellType +from .version import __version__ + +from deprecation import deprecated +from typing import Union, List, Optional, Iterable +from py4j.java_gateway import JavaObject THIS_MODULE = 'pyrasterframes' +Column_type = Union[str, Column] + -def _context_call(name, *args): +def _context_call(name: str, *args): f = RFContext.active().lookup(name) return f(*args) -def _parse_cell_type(cell_type_arg): +def _apply_column_function(name: str, *args: Column_type) -> Column: + jfcn = RFContext.active().lookup(name) + jcols = [_to_java_column(arg) for arg in args] + return Column(jfcn(*jcols)) + + +def _apply_scalar_to_tile(name: str, tile_col: Column_type, scalar: Union[int, float]) -> Column: + jfcn = RFContext.active().lookup(name) + return Column(jfcn(_to_java_column(tile_col), scalar)) + + +def _parse_cell_type(cell_type_arg: Union[str, CellType]) -> JavaObject: """ Convert the cell type representation to the expected JVM CellType object.""" def to_jvm(ct): @@ -49,12 +66,14 @@ def to_jvm(ct): return to_jvm(cell_type_arg.cell_type_name) -def rf_cell_types(): +def rf_cell_types() -> List[CellType]: """Return a list of standard cell types""" return [CellType(str(ct)) for ct in _context_call('rf_cell_types')] -def rf_assemble_tile(col_index, row_index, cell_data_col, num_cols, num_rows, cell_type=None): +def rf_assemble_tile(col_index: Column_type, row_index: Column_type, cell_data_col: Column_type, + num_cols: Union[int, Column_type], num_rows: Union[int, Column_type], + cell_type: Optional[Union[str, CellType]] = None) -> Column: """Create a Tile from a column of cell data with location indices""" jfcn = RFContext.active().lookup('rf_assemble_tile') @@ -76,189 +95,275 @@ def rf_assemble_tile(col_index, row_index, cell_data_col, num_cols, num_rows, ce num_cols, num_rows, _parse_cell_type(cell_type) )) -def rf_array_to_tile(array_col, num_cols, num_rows): + +def rf_array_to_tile(array_col: Column_type, num_cols: int, num_rows: int) -> Column: """Convert array in `array_col` into a Tile of dimensions `num_cols` and `num_rows'""" jfcn = RFContext.active().lookup('rf_array_to_tile') return Column(jfcn(_to_java_column(array_col), num_cols, num_rows)) -def rf_convert_cell_type(tile_col, cell_type): +def rf_convert_cell_type(tile_col: Column_type, cell_type: Union[str, CellType]) -> Column: """Convert the numeric type of the Tiles in `tileCol`""" jfcn = RFContext.active().lookup('rf_convert_cell_type') return Column(jfcn(_to_java_column(tile_col), _parse_cell_type(cell_type))) -def rf_interpret_cell_type_as(tile_col, cell_type): + +def rf_interpret_cell_type_as(tile_col: Column_type, cell_type: Union[str, CellType]) -> Column: """Change the interpretation of the tile_col's cell values according to specified cell_type""" jfcn = RFContext.active().lookup('rf_interpret_cell_type_as') return Column(jfcn(_to_java_column(tile_col), _parse_cell_type(cell_type))) -def rf_make_constant_tile(scalar_value, num_cols, num_rows, cell_type=CellType.float64()): +def rf_make_constant_tile(scalar_value: Union[int, float], num_cols: int, num_rows: int, + cell_type: Union[str, CellType] = CellType.float64()) -> Column: """Constructor for constant tile column""" jfcn = RFContext.active().lookup('rf_make_constant_tile') return Column(jfcn(scalar_value, num_cols, num_rows, _parse_cell_type(cell_type))) -def rf_make_zeros_tile(num_cols, num_rows, cell_type=CellType.float64()): +def rf_make_zeros_tile(num_cols: int, num_rows: int, cell_type: Union[str, CellType] = CellType.float64()) -> Column: """Create column of constant tiles of zero""" jfcn = RFContext.active().lookup('rf_make_zeros_tile') return Column(jfcn(num_cols, num_rows, _parse_cell_type(cell_type))) -def rf_make_ones_tile(num_cols, num_rows, cell_type=CellType.float64()): +def rf_make_ones_tile(num_cols: int, num_rows: int, cell_type: Union[str, CellType] = CellType.float64()) -> Column: """Create column of constant tiles of one""" jfcn = RFContext.active().lookup('rf_make_ones_tile') return Column(jfcn(num_cols, num_rows, _parse_cell_type(cell_type))) -def rf_rasterize(geometry_col, bounds_col, value_col, num_cols_col, num_rows_col): +def rf_rasterize(geometry_col: Column_type, bounds_col: Column_type, value_col: Column_type, num_cols_col: Column_type, + num_rows_col: Column_type) -> Column: """Create a tile where cells in the grid defined by cols, rows, and bounds are filled with the given value.""" - jfcn = RFContext.active().lookup('rf_rasterize') - return Column(jfcn(_to_java_column(geometry_col), _to_java_column(bounds_col), - _to_java_column(value_col), _to_java_column(num_cols_col), _to_java_column(num_rows_col))) + return _apply_column_function('rf_rasterize', geometry_col, bounds_col, value_col, num_cols_col, num_rows_col) -def st_reproject(geometry_col, src_crs, dst_crs): +def st_reproject(geometry_col: Column_type, src_crs: Column_type, dst_crs: Column_type) -> Column: """Reproject a column of geometry given the CRSs of the source and destination.""" - jfcn = RFContext.active().lookup('st_reproject') - return Column(jfcn(_to_java_column(geometry_col), _to_java_column(src_crs), _to_java_column(dst_crs))) + return _apply_column_function('st_reproject', geometry_col, src_crs, dst_crs) -def rf_explode_tiles(*tile_cols): +def rf_explode_tiles(*tile_cols: Column_type) -> Column: """Create a row for each cell in Tile.""" jfcn = RFContext.active().lookup('rf_explode_tiles') jcols = [_to_java_column(arg) for arg in tile_cols] return Column(jfcn(RFContext.active().list_to_seq(jcols))) -def rf_explode_tiles_sample(sample_frac, seed, *tile_cols): +def rf_explode_tiles_sample(sample_frac: float, seed: int, *tile_cols: Column_type) -> Column: """Create a row for a sample of cells in Tile columns.""" jfcn = RFContext.active().lookup('rf_explode_tiles_sample') jcols = [_to_java_column(arg) for arg in tile_cols] return Column(jfcn(sample_frac, seed, RFContext.active().list_to_seq(jcols))) -def _apply_scalar_to_tile(name, tile_col, scalar): - jfcn = RFContext.active().lookup(name) - return Column(jfcn(_to_java_column(tile_col), scalar)) - - -def rf_with_no_data(tile_col, scalar): +def rf_with_no_data(tile_col: Column_type, scalar: Union[int, float]) -> Column: """Assign a `NoData` value to the Tiles in the given Column.""" return _apply_scalar_to_tile('rf_with_no_data', tile_col, scalar) -def rf_local_add_double(tile_col, scalar): +def rf_local_add(left_tile_col: Column_type, rhs: Union[float, int, Column_type]) -> Column: + """Add two Tiles, or add a scalar to a Tile""" + if isinstance(rhs, (float, int)): + rhs = lit(rhs) + return _apply_column_function('rf_local_add', left_tile_col, rhs) + + +@deprecated(deprecated_in='0.9.0', removed_in='1.0.0', current_version=__version__) +def rf_local_add_double(tile_col: Column_type, scalar: float) -> Column: """Add a floating point scalar to a Tile""" return _apply_scalar_to_tile('rf_local_add_double', tile_col, scalar) -def rf_local_add_int(tile_col, scalar): +@deprecated(deprecated_in='0.9.0', removed_in='1.0.0', current_version=__version__) +def rf_local_add_int(tile_col, scalar) -> Column: """Add an integral scalar to a Tile""" return _apply_scalar_to_tile('rf_local_add_int', tile_col, scalar) +def rf_local_subtract(left_tile_col: Column_type, rhs: Union[float, int, Column_type]) -> Column: + """Subtract two Tiles, or subtract a scalar from a Tile""" + if isinstance(rhs, (float, int)): + rhs = lit(rhs) + return _apply_column_function('rf_local_subtract', left_tile_col, rhs) + + +@deprecated(deprecated_in='0.9.0', removed_in='1.0.0', current_version=__version__) def rf_local_subtract_double(tile_col, scalar): """Subtract a floating point scalar from a Tile""" return _apply_scalar_to_tile('rf_local_subtract_double', tile_col, scalar) +@deprecated(deprecated_in='0.9.0', removed_in='1.0.0', current_version=__version__) def rf_local_subtract_int(tile_col, scalar): """Subtract an integral scalar from a Tile""" return _apply_scalar_to_tile('rf_local_subtract_int', tile_col, scalar) +def rf_local_multiply(left_tile_col: Column_type, rhs: Union[float, int, Column_type]) -> Column: + """Multiply two Tiles cell-wise, or multiply Tile cells by a scalar""" + if isinstance(rhs, (float, int)): + rhs = lit(rhs) + return _apply_column_function('rf_local_multiply', left_tile_col, rhs) + + +@deprecated(deprecated_in='0.9.0', removed_in='1.0.0', current_version=__version__) def rf_local_multiply_double(tile_col, scalar): """Multiply a Tile by a float point scalar""" return _apply_scalar_to_tile('rf_local_multiply_double', tile_col, scalar) +@deprecated(deprecated_in='0.9.0', removed_in='1.0.0', current_version=__version__) def rf_local_multiply_int(tile_col, scalar): """Multiply a Tile by an integral scalar""" return _apply_scalar_to_tile('rf_local_multiply_int', tile_col, scalar) +def rf_local_divide(left_tile_col: Column_type, rhs: Union[float, int, Column_type]) -> Column: + """Divide two Tiles cell-wise, or divide a Tile's cell values by a scalar""" + if isinstance(rhs, (float, int)): + rhs = lit(rhs) + return _apply_column_function('rf_local_divide', left_tile_col, rhs) + + +@deprecated(deprecated_in='0.9.0', removed_in='1.0.0', current_version=__version__) def rf_local_divide_double(tile_col, scalar): """Divide a Tile by a floating point scalar""" return _apply_scalar_to_tile('rf_local_divide_double', tile_col, scalar) +@deprecated(deprecated_in='0.9.0', removed_in='1.0.0', current_version=__version__) def rf_local_divide_int(tile_col, scalar): """Divide a Tile by an integral scalar""" return _apply_scalar_to_tile('rf_local_divide_int', tile_col, scalar) +def rf_local_less(left_tile_col: Column_type, rhs: Union[float, int, Column_type]) -> Column: + """Cellwise less than comparison between two tiles, or with a scalar value""" + if isinstance(rhs, (float, int)): + rhs = lit(rhs) + return _apply_column_function('rf_local_less', left_tile_col, rhs) + + +@deprecated(deprecated_in='0.9.0', removed_in='1.0.0', current_version=__version__) def rf_local_less_double(tile_col, scalar): """Return a Tile with values equal 1 if the cell is less than a scalar, otherwise 0""" return _apply_scalar_to_tile('foo', tile_col, scalar) +@deprecated(deprecated_in='0.9.0', removed_in='1.0.0', current_version=__version__) def rf_local_less_int(tile_col, scalar): """Return a Tile with values equal 1 if the cell is less than a scalar, otherwise 0""" return _apply_scalar_to_tile('rf_local_less_double', tile_col, scalar) +def rf_local_less_equal(left_tile_col: Column_type, rhs: Union[float, int, Column_type]) -> Column: + """Cellwise less than or equal to comparison between two tiles, or with a scalar value""" + if isinstance(rhs, (float, int)): + rhs = lit(rhs) + return _apply_column_function('rf_local_less_equal', left_tile_col, rhs) + + +@deprecated(deprecated_in='0.9.0', removed_in='1.0.0', current_version=__version__) def rf_local_less_equal_double(tile_col, scalar): """Return a Tile with values equal 1 if the cell is less than or equal to a scalar, otherwise 0""" return _apply_scalar_to_tile('rf_local_less_equal_double', tile_col, scalar) +@deprecated(deprecated_in='0.9.0', removed_in='1.0.0', current_version=__version__) def rf_local_less_equal_int(tile_col, scalar): """Return a Tile with values equal 1 if the cell is less than or equal to a scalar, otherwise 0""" return _apply_scalar_to_tile('rf_local_less_equal_int', tile_col, scalar) +def rf_local_greater(left_tile_col: Column, rhs: Union[float, int, Column_type]) -> Column: + """Cellwise greater than comparison between two tiles, or with a scalar value""" + if isinstance(rhs, (float, int)): + rhs = lit(rhs) + return _apply_column_function('rf_local_greater', left_tile_col, rhs) + + +@deprecated(deprecated_in='0.9.0', removed_in='1.0.0', current_version=__version__) def rf_local_greater_double(tile_col, scalar): """Return a Tile with values equal 1 if the cell is greater than a scalar, otherwise 0""" return _apply_scalar_to_tile('rf_local_greater_double', tile_col, scalar) +@deprecated(deprecated_in='0.9.0', removed_in='1.0.0', current_version=__version__) def rf_local_greater_int(tile_col, scalar): """Return a Tile with values equal 1 if the cell is greater than a scalar, otherwise 0""" return _apply_scalar_to_tile('rf_local_greater_int', tile_col, scalar) +def rf_local_greater_equal(left_tile_col: Column, rhs: Union[float, int, Column_type]) -> Column: + """Cellwise greater than or equal to comparison between two tiles, or with a scalar value""" + if isinstance(rhs, (float, int)): + rhs = lit(rhs) + return _apply_column_function('rf_local_greater_equal', left_tile_col, rhs) + + +@deprecated(deprecated_in='0.9.0', removed_in='1.0.0', current_version=__version__) def rf_local_greater_equal_double(tile_col, scalar): """Return a Tile with values equal 1 if the cell is greater than or equal to a scalar, otherwise 0""" return _apply_scalar_to_tile('rf_local_greater_equal_double', tile_col, scalar) +@deprecated(deprecated_in='0.9.0', removed_in='1.0.0', current_version=__version__) def rf_local_greater_equal_int(tile_col, scalar): """Return a Tile with values equal 1 if the cell is greater than or equal to a scalar, otherwise 0""" return _apply_scalar_to_tile('rf_local_greater_equal_int', tile_col, scalar) +def rf_local_equal(left_tile_col, rhs: Union[float, int, Column_type]) -> Column: + """Cellwise equality comparison between two tiles, or with a scalar value""" + if isinstance(rhs, (float, int)): + rhs = lit(rhs) + return _apply_column_function('rf_local_equal', left_tile_col, rhs) + + +@deprecated(deprecated_in='0.9.0', removed_in='1.0.0', current_version=__version__) def rf_local_equal_double(tile_col, scalar): """Return a Tile with values equal 1 if the cell is equal to a scalar, otherwise 0""" return _apply_scalar_to_tile('rf_local_equal_double', tile_col, scalar) +@deprecated(deprecated_in='0.9.0', removed_in='1.0.0', current_version=__version__) def rf_local_equal_int(tile_col, scalar): """Return a Tile with values equal 1 if the cell is equal to a scalar, otherwise 0""" return _apply_scalar_to_tile('rf_local_equal_int', tile_col, scalar) +def rf_local_unequal(left_tile_col, rhs: Union[float, int, Column_type]) -> Column: + """Cellwise inequality comparison between two tiles, or with a scalar value""" + if isinstance(rhs, (float, int)): + rhs = lit(rhs) + return _apply_column_function('rf_local_unequal', left_tile_col, rhs) + + +@deprecated(deprecated_in='0.9.0', removed_in='1.0.0', current_version=__version__) def rf_local_unequal_double(tile_col, scalar): """Return a Tile with values equal 1 if the cell is not equal to a scalar, otherwise 0""" return _apply_scalar_to_tile('rf_local_unequal_double', tile_col, scalar) +@deprecated(deprecated_in='0.9.0', removed_in='1.0.0', current_version=__version__) def rf_local_unequal_int(tile_col, scalar): """Return a Tile with values equal 1 if the cell is not equal to a scalar, otherwise 0""" return _apply_scalar_to_tile('rf_local_unequal_int', tile_col, scalar) -def rf_local_no_data(tile_col): +def rf_local_no_data(tile_col: Column_type) -> Column: """Return a tile with ones where the input is NoData, otherwise zero.""" return _apply_column_function('rf_local_no_data', tile_col) -def rf_local_data(tile_col): +def rf_local_data(tile_col: Column_type) -> Column: """Return a tile with zeros where the input is NoData, otherwise one.""" return _apply_column_function('rf_local_data', tile_col) -def rf_local_is_in(tile_col, array): +def rf_local_is_in(tile_col: Column_type, array: Union[Column_type, List]) -> Column: """Return a tile with cell values of 1 where the `tile_col` cell is in the provided array.""" from pyspark.sql.functions import array as sql_array if isinstance(array, list): @@ -267,188 +372,162 @@ def rf_local_is_in(tile_col, array): return _apply_column_function('rf_local_is_in', tile_col, array) -def _apply_column_function(name, *args): - jfcn = RFContext.active().lookup(name) - jcols = [_to_java_column(arg) for arg in args] - return Column(jfcn(*jcols)) - - -def rf_dimensions(tile_col): +def rf_dimensions(tile_col: Column_type) -> Column: """Query the number of (cols, rows) in a Tile.""" return _apply_column_function('rf_dimensions', tile_col) -def rf_tile_to_array_int(tile_col): +def rf_tile_to_array_int(tile_col: Column_type) -> Column: """Flattens Tile into an array of integers.""" return _apply_column_function('rf_tile_to_array_int', tile_col) -def rf_tile_to_array_double(tile_col): +def rf_tile_to_array_double(tile_col: Column_type) -> Column: """Flattens Tile into an array of doubles.""" return _apply_column_function('rf_tile_to_array_double', tile_col) -def rf_cell_type(tile_col): +def rf_cell_type(tile_col: Column_type) -> Column: """Extract the Tile's cell type""" return _apply_column_function('rf_cell_type', tile_col) -def rf_is_no_data_tile(tile_col): +def rf_is_no_data_tile(tile_col: Column_type) -> Column: """Report if the Tile is entirely NODDATA cells""" return _apply_column_function('rf_is_no_data_tile', tile_col) -def rf_exists(tile_col): +def rf_exists(tile_col: Column_type) -> Column: """Returns true if any cells in the tile are true (non-zero and not NoData)""" return _apply_column_function('rf_exists', tile_col) -def rf_for_all(tile_col): +def rf_for_all(tile_col: Column_type) -> Column: """Returns true if all cells in the tile are true (non-zero and not NoData).""" return _apply_column_function('rf_for_all', tile_col) -def rf_agg_approx_histogram(tile_col): +def rf_agg_approx_histogram(tile_col: Column_type) -> Column: """Compute the full column aggregate floating point histogram""" return _apply_column_function('rf_agg_approx_histogram', tile_col) -def rf_agg_stats(tile_col): +def rf_agg_stats(tile_col: Column_type) -> Column: """Compute the full column aggregate floating point statistics""" return _apply_column_function('rf_agg_stats', tile_col) -def rf_agg_mean(tile_col): +def rf_agg_mean(tile_col: Column_type) -> Column: """Computes the column aggregate mean""" return _apply_column_function('rf_agg_mean', tile_col) -def rf_agg_data_cells(tile_col): +def rf_agg_data_cells(tile_col: Column_type) -> Column: """Computes the number of non-NoData cells in a column""" return _apply_column_function('rf_agg_data_cells', tile_col) -def rf_agg_no_data_cells(tile_col): +def rf_agg_no_data_cells(tile_col: Column_type) -> Column: """Computes the number of NoData cells in a column""" return _apply_column_function('rf_agg_no_data_cells', tile_col) -def rf_tile_histogram(tile_col): +def rf_tile_histogram(tile_col: Column_type) -> Column: """Compute the Tile-wise histogram""" return _apply_column_function('rf_tile_histogram', tile_col) -def rf_tile_mean(tile_col): +def rf_tile_mean(tile_col: Column_type) -> Column: """Compute the Tile-wise mean""" return _apply_column_function('rf_tile_mean', tile_col) -def rf_tile_sum(tile_col): +def rf_tile_sum(tile_col: Column_type) -> Column: """Compute the Tile-wise sum""" return _apply_column_function('rf_tile_sum', tile_col) -def rf_tile_min(tile_col): +def rf_tile_min(tile_col: Column_type) -> Column: """Compute the Tile-wise minimum""" return _apply_column_function('rf_tile_min', tile_col) -def rf_tile_max(tile_col): +def rf_tile_max(tile_col: Column_type) -> Column: """Compute the Tile-wise maximum""" return _apply_column_function('rf_tile_max', tile_col) -def rf_tile_stats(tile_col): +def rf_tile_stats(tile_col: Column_type) -> Column: """Compute the Tile-wise floating point statistics""" return _apply_column_function('rf_tile_stats', tile_col) -def rf_render_ascii(tile_col): +def rf_render_ascii(tile_col: Column_type) -> Column: """Render ASCII art of tile""" return _apply_column_function('rf_render_ascii', tile_col) -def rf_render_matrix(tile_col): +def rf_render_matrix(tile_col: Column_type) -> Column: """Render Tile cell values as numeric values, for debugging purposes""" return _apply_column_function('rf_render_matrix', tile_col) -def rf_render_png(red_tile_col, green_tile_col, blue_tile_col): +def rf_render_png(red_tile_col: Column_type, green_tile_col: Column_type, blue_tile_col: Column_type) -> Column: """Converts columns of tiles representing RGB channels into a PNG encoded byte array.""" return _apply_column_function('rf_render_png', red_tile_col, green_tile_col, blue_tile_col) -def rf_rgb_composite(red_tile_col, green_tile_col, blue_tile_col): +def rf_rgb_composite(red_tile_col: Column_type, green_tile_col: Column_type, blue_tile_col: Column_type) -> Column: """Converts columns of tiles representing RGB channels into a single RGB packaged tile.""" return _apply_column_function('rf_rgb_composite', red_tile_col, green_tile_col, blue_tile_col) -def rf_no_data_cells(tile_col): +def rf_no_data_cells(tile_col: Column_type) -> Column: """Count of NODATA cells""" return _apply_column_function('rf_no_data_cells', tile_col) -def rf_data_cells(tile_col): +def rf_data_cells(tile_col: Column_type) -> Column: """Count of cells with valid data""" return _apply_column_function('rf_data_cells', tile_col) -def rf_local_add(left_tile_col, right_tile_col): - """Add two Tiles""" - return _apply_column_function('rf_local_add', left_tile_col, right_tile_col) - - -def rf_local_subtract(left_tile_col, right_tile_col): - """Subtract two Tiles""" - return _apply_column_function('rf_local_subtract', left_tile_col, right_tile_col) - - -def rf_local_multiply(left_tile_col, right_tile_col): - """Multiply two Tiles""" - return _apply_column_function('rf_local_multiply', left_tile_col, right_tile_col) - - -def rf_local_divide(left_tile_col, right_tile_col): - """Divide two Tiles""" - return _apply_column_function('rf_local_divide', left_tile_col, right_tile_col) - - -def rf_normalized_difference(left_tile_col, right_tile_col): +def rf_normalized_difference(left_tile_col: Column_type, right_tile_col: Column_type) -> Column: """Compute the normalized difference of two tiles""" return _apply_column_function('rf_normalized_difference', left_tile_col, right_tile_col) -def rf_agg_local_max(tile_col): +def rf_agg_local_max(tile_col: Column_type) -> Column: """Compute the cell-wise/local max operation between Tiles in a column.""" return _apply_column_function('rf_agg_local_max', tile_col) -def rf_agg_local_min(tile_col): +def rf_agg_local_min(tile_col: Column_type) -> Column: """Compute the cellwise/local min operation between Tiles in a column.""" return _apply_column_function('rf_agg_local_min', tile_col) -def rf_agg_local_mean(tile_col): +def rf_agg_local_mean(tile_col: Column_type) -> Column: """Compute the cellwise/local mean operation between Tiles in a column.""" return _apply_column_function('rf_agg_local_mean', tile_col) -def rf_agg_local_data_cells(tile_col): +def rf_agg_local_data_cells(tile_col: Column_type) -> Column: """Compute the cellwise/local count of non-NoData cells for all Tiles in a column.""" return _apply_column_function('rf_agg_local_data_cells', tile_col) -def rf_agg_local_no_data_cells(tile_col): +def rf_agg_local_no_data_cells(tile_col: Column_type) -> Column: """Compute the cellwise/local count of NoData cells for all Tiles in a column.""" return _apply_column_function('rf_agg_local_no_data_cells', tile_col) -def rf_agg_local_stats(tile_col): +def rf_agg_local_stats(tile_col: Column_type) -> Column: """Compute cell-local aggregate descriptive statistics for a column of Tiles.""" return _apply_column_function('rf_agg_local_stats', tile_col) -def rf_mask(src_tile_col, mask_tile_col, inverse=False): +def rf_mask(src_tile_col: Column_type, mask_tile_col: Column_type, inverse: bool = False) -> Column: """Where the rf_mask (second) tile contains NODATA, replace values in the source (first) tile with NODATA. If `inverse` is true, replaces values in the source tile with NODATA where the mask tile contains valid data. """ @@ -458,13 +537,14 @@ def rf_mask(src_tile_col, mask_tile_col, inverse=False): rf_inverse_mask(src_tile_col, mask_tile_col) -def rf_inverse_mask(src_tile_col, mask_tile_col): +def rf_inverse_mask(src_tile_col: Column_type, mask_tile_col: Column_type) -> Column: """Where the rf_mask (second) tile DOES NOT contain NODATA, replace values in the source (first) tile with NODATA.""" return _apply_column_function('rf_inverse_mask', src_tile_col, mask_tile_col) -def rf_mask_by_value(data_tile, mask_tile, mask_value, inverse=False): +def rf_mask_by_value(data_tile: Column_type, mask_tile: Column_type, mask_value: Union[int, float, Column_type], + inverse: bool = False) -> Column: """Generate a tile with the values from the data tile, but where cells in the masking tile contain the masking value, replace the data value with NODATA. """ if isinstance(mask_value, (int, float)): @@ -474,7 +554,8 @@ def rf_mask_by_value(data_tile, mask_tile, mask_value, inverse=False): return Column(jfcn(_to_java_column(data_tile), _to_java_column(mask_tile), _to_java_column(mask_value), inverse)) -def rf_mask_by_values(data_tile, mask_tile, mask_values): +def rf_mask_by_values(data_tile: Column_type, mask_tile: Column_type, + mask_values: Union[List[Union[int, float]], Column_type]) -> Column: """Generate a tile with the values from `data_tile`, but where cells in the `mask_tile` are in the `mask_values` list, replace the value with NODATA. """ @@ -487,7 +568,8 @@ def rf_mask_by_values(data_tile, mask_tile, mask_values): return Column(jfcn(*col_args)) -def rf_inverse_mask_by_value(data_tile, mask_tile, mask_value): +def rf_inverse_mask_by_value(data_tile: Column_type, mask_tile: Column_type, + mask_value: Union[int, float, Column_type]) -> Column: """Generate a tile with the values from the data tile, but where cells in the masking tile do not contain the masking value, replace the data value with NODATA. """ if isinstance(mask_value, (int, float)): @@ -495,7 +577,9 @@ def rf_inverse_mask_by_value(data_tile, mask_tile, mask_value): return _apply_column_function('rf_inverse_mask_by_value', data_tile, mask_tile, mask_value) -def rf_mask_by_bit(data_tile, mask_tile, bit_position, value_to_mask): +def rf_mask_by_bit(data_tile: Column_type, mask_tile: Column_type, + bit_position: Union[int, Column_type], + value_to_mask: Union[int, float, bool, Column_type]) -> Column: """Applies a mask using bit values in the `mask_tile`. Working from the right, extract the bit at `bitPosition` from the `maskTile`. In all locations where these are equal to the `valueToMask`, the returned tile is set to NoData, else the original `dataTile` cell value.""" if isinstance(bit_position, int): bit_position = lit(bit_position) @@ -504,7 +588,9 @@ def rf_mask_by_bit(data_tile, mask_tile, bit_position, value_to_mask): return _apply_column_function('rf_mask_by_bit', data_tile, mask_tile, bit_position, value_to_mask) -def rf_mask_by_bits(data_tile, mask_tile, start_bit, num_bits, values_to_mask): +def rf_mask_by_bits(data_tile: Column_type, mask_tile: Column_type, start_bit: Union[int, Column_type], + num_bits: Union[int, Column_type], + values_to_mask: Union[Iterable[Union[int, float]], Column_type]) -> Column: """Applies a mask from blacklisted bit values in the `mask_tile`. Working from the right, the bits from `start_bit` to `start_bit + num_bits` are @ref:[extracted](reference.md#rf_local_extract_bits) from cell values of the `mask_tile`. In all locations where these are in the `mask_values`, the returned tile is set to NoData; otherwise the original `tile` cell value is returned.""" if isinstance(start_bit, int): start_bit = lit(start_bit) @@ -517,145 +603,118 @@ def rf_mask_by_bits(data_tile, mask_tile, start_bit, num_bits, values_to_mask): return _apply_column_function('rf_mask_by_bits', data_tile, mask_tile, start_bit, num_bits, values_to_mask) -def rf_local_extract_bits(tile, start_bit, num_bits=1): +def rf_local_extract_bits(tile: Column_type, start_bit: Union[int, Column_type], + num_bits: Union[int, Column_type] = 1) -> Column: """Extract value from specified bits of the cells' underlying binary data. * `startBit` is the first bit to consider, working from the right. It is zero indexed. * `numBits` is the number of bits to take moving further to the left. """ if isinstance(start_bit, int): - start_bit = lit(bit_position) + start_bit = lit(start_bit) if isinstance(num_bits, int): num_bits = lit(num_bits) return _apply_column_function('rf_local_extract_bits', tile, start_bit, num_bits) -def rf_local_less(left_tile_col, right_tile_col): - """Cellwise less than comparison between two tiles""" - return _apply_column_function('rf_local_less', left_tile_col, right_tile_col) - - -def rf_local_less_equal(left_tile_col, right_tile_col): - """Cellwise less than or equal to comparison between two tiles""" - return _apply_column_function('rf_local_less_equal', left_tile_col, right_tile_col) - - -def rf_local_greater(left_tile_col, right_tile_col): - """Cellwise greater than comparison between two tiles""" - return _apply_column_function('rf_local_greater', left_tile_col, right_tile_col) - - -def rf_local_greater_equal(left_tile_col, right_tile_col): - """Cellwise greater than or equal to comparison between two tiles""" - return _apply_column_function('rf_local_greater_equal', left_tile_col, right_tile_col) - - -def rf_local_equal(left_tile_col, right_tile_col): - """Cellwise equality comparison between two tiles""" - return _apply_column_function('rf_local_equal', left_tile_col, right_tile_col) - - -def rf_local_unequal(left_tile_col, right_tile_col): - """Cellwise inequality comparison between two tiles""" - return _apply_column_function('rf_local_unequal', left_tile_col, right_tile_col) - - -def rf_round(tile_col): +def rf_round(tile_col: Column_type) -> Column: """Round cell values to the nearest integer without changing the cell type""" return _apply_column_function('rf_round', tile_col) -def rf_abs(tile_col): +def rf_abs(tile_col: Column_type) -> Column: """Compute the absolute value of each cell""" return _apply_column_function('rf_abs', tile_col) -def rf_log(tile_col): +def rf_log(tile_col: Column_type) -> Column: """Performs cell-wise natural logarithm""" return _apply_column_function('rf_log', tile_col) -def rf_log10(tile_col): +def rf_log10(tile_col: Column_type) -> Column: """Performs cell-wise logartithm with base 10""" return _apply_column_function('rf_log10', tile_col) -def rf_log2(tile_col): +def rf_log2(tile_col: Column_type) -> Column: """Performs cell-wise logartithm with base 2""" return _apply_column_function('rf_log2', tile_col) -def rf_log1p(tile_col): +def rf_log1p(tile_col: Column_type) -> Column: """Performs natural logarithm of cell values plus one""" return _apply_column_function('rf_log1p', tile_col) -def rf_exp(tile_col): +def rf_exp(tile_col: Column_type) -> Column: """Performs cell-wise exponential""" return _apply_column_function('rf_exp', tile_col) -def rf_exp2(tile_col): +def rf_exp2(tile_col: Column_type) -> Column: """Compute 2 to the power of cell values""" return _apply_column_function('rf_exp2', tile_col) -def rf_exp10(tile_col): +def rf_exp10(tile_col: Column_type) -> Column: """Compute 10 to the power of cell values""" return _apply_column_function('rf_exp10', tile_col) -def rf_expm1(tile_col): +def rf_expm1(tile_col: Column_type) -> Column: """Performs cell-wise exponential, then subtract one""" return _apply_column_function('rf_expm1', tile_col) -def rf_identity(tile_col): +def rf_identity(tile_col: Column_type) -> Column: """Pass tile through unchanged""" return _apply_column_function('rf_identity', tile_col) -def rf_resample(tile_col, scale_factor_col): +def rf_resample(tile_col: Column_type, scale_factor: Union[int, float, Column_type]) -> Column: """Resample tile to different size based on scalar factor or tile whose dimension to match Scalar less than one will downsample tile; greater than one will upsample. Uses nearest-neighbor.""" - return _apply_column_function('rf_resample', tile_col, scale_factor_col) + if isinstance(scale_factor, (int, float)): + scale_factor = lit(scale_factor) + return _apply_column_function('rf_resample', tile_col, scale_factor) -def rf_crs(tile_col): +def rf_crs(tile_col: Column_type) -> Column: """Get the CRS of a RasterSource or ProjectedRasterTile""" return _apply_column_function('rf_crs', tile_col) -def rf_mk_crs(crs_text): +def rf_mk_crs(crs_text: str) -> Column: """Resolve CRS from text identifier. Supported registries are EPSG, ESRI, WORLD, NAD83, & NAD27. An example of a valid CRS name is EPSG:3005.""" return Column(_context_call('_make_crs_literal', crs_text)) -def st_extent(geom_col): +def st_extent(geom_col: Column_type) -> Column: """Compute the extent/bbox of a Geometry (a tile with embedded extent and CRS)""" return _apply_column_function('st_extent', geom_col) -def rf_extent(proj_raster_col): +def rf_extent(proj_raster_col: Column_type) -> Column: """Get the extent of a RasterSource or ProjectedRasterTile (a tile with embedded extent and CRS)""" return _apply_column_function('rf_extent', proj_raster_col) -def rf_tile(proj_raster_col): +def rf_tile(proj_raster_col: Column_type) -> Column: """Extracts the Tile component of a ProjectedRasterTile (or Tile).""" return _apply_column_function('rf_tile', proj_raster_col) -def st_geometry(geom_col): +def st_geometry(geom_col: Column_type) -> Column: """Convert the given extent/bbox to a polygon""" return _apply_column_function('st_geometry', geom_col) -def rf_geometry(proj_raster_col): +def rf_geometry(proj_raster_col: Column_type) -> Column: """Get the extent of a RasterSource or ProjectdRasterTile as a Geometry""" return _apply_column_function('rf_geometry', proj_raster_col) -def rf_xz2_index(geom_col, crs_col=None, index_resolution = 18): +def rf_xz2_index(geom_col: Column_type, crs_col: Optional[Column_type] = None, index_resolution: int = 18) -> Column: """Constructs a XZ2 index in WGS84 from either a Geometry, Extent, ProjectedRasterTile, or RasterSource and its CRS. For details: https://www.geomesa.org/documentation/user/datastores/index_overview.html """ @@ -666,7 +725,8 @@ def rf_xz2_index(geom_col, crs_col=None, index_resolution = 18): else: return Column(jfcn(_to_java_column(geom_col), index_resolution)) -def rf_z2_index(geom_col, crs_col=None, index_resolution = 18): + +def rf_z2_index(geom_col: Column_type, crs_col: Optional[Column_type] = None, index_resolution: int = 18) -> Column: """Constructs a Z2 index in WGS84 from either a Geometry, Extent, ProjectedRasterTile, or RasterSource and its CRS. First the native extent is extracted or computed, and then center is used as the indexing location. For details: https://www.geomesa.org/documentation/user/datastores/index_overview.html """ diff --git a/pyrasterframes/src/main/python/pyrasterframes/rf_context.py b/pyrasterframes/src/main/python/pyrasterframes/rf_context.py index 39a470697..4e8e91a4b 100644 --- a/pyrasterframes/src/main/python/pyrasterframes/rf_context.py +++ b/pyrasterframes/src/main/python/pyrasterframes/rf_context.py @@ -23,6 +23,11 @@ """ from pyspark import SparkContext +from pyspark.sql import SparkSession + +from typing import Any, List +from py4j.java_gateway import JavaMember +from py4j.java_collections import JavaList, JavaMap __all__ = ['RFContext'] @@ -31,21 +36,21 @@ class RFContext(object): """ Entrypoint to RasterFrames services """ - def __init__(self, spark_session): + def __init__(self, spark_session: SparkSession): self._spark_session = spark_session self._gateway = spark_session.sparkContext._gateway self._jvm = self._gateway.jvm jsess = self._spark_session._jsparkSession self._jrfctx = self._jvm.org.locationtech.rasterframes.py.PyRFContext(jsess) - def list_to_seq(self, py_list): + def list_to_seq(self, py_list: List[Any]) -> JavaList: conv = self.lookup('_listToSeq') return conv(py_list) - def lookup(self, function_name): + def lookup(self, function_name: str) -> JavaMember: return getattr(self._jrfctx, function_name) - def build_info(self): + def build_info(self) -> JavaMap: return self._jrfctx.buildInfo() # NB: Tightly coupled to `org.locationtech.rasterframes.py.PyRFContext._resolveRasterRef` diff --git a/pyrasterframes/src/main/python/pyrasterframes/rf_ipython.py b/pyrasterframes/src/main/python/pyrasterframes/rf_ipython.py index 0ae23d4ab..e6da8b553 100644 --- a/pyrasterframes/src/main/python/pyrasterframes/rf_ipython.py +++ b/pyrasterframes/src/main/python/pyrasterframes/rf_ipython.py @@ -19,12 +19,16 @@ # import pyrasterframes.rf_types +from pyrasterframes.rf_types import Tile from shapely.geometry.base import BaseGeometry - +import matplotlib.axes.Axes import numpy as np +from pandas import DataFrame +from typing import Optional, Tuple -def plot_tile(tile, normalize=True, lower_percentile=1, upper_percentile=99, axis=None, **imshow_args): +def plot_tile(tile: Tile, normalize: bool = True, lower_percentile: float = 1., upper_percentile: float = 99., + axis: Optional[matplotlib.axis.Axes] = None, **imshow_args): """ Display an image of the tile @@ -50,7 +54,7 @@ def plot_tile(tile, normalize=True, lower_percentile=1, upper_percentile=99, axi arr = tile.cells - def normalize_cells(cells): + def normalize_cells(cells: np.ndarray) -> np.ndarray: assert upper_percentile > lower_percentile, 'invalid upper and lower percentiles {}, {}'.format(lower_percentile, upper_percentile) sans_mask = np.array(cells) lower = np.nanpercentile(sans_mask, lower_percentile) @@ -72,7 +76,8 @@ def normalize_cells(cells): return axis -def tile_to_png(tile, lower_percentile=1, upper_percentile=99, title=None, fig_size=None): +def tile_to_png(tile: Tile, lower_percentile: float = 1., upper_percentile: float = 99., title: Optional[str] = None, + fig_size: Optional[Tuple[int, int]] = None) -> bytes: """ Provide image of Tile.""" if tile.cells is None: return None @@ -106,7 +111,7 @@ def tile_to_png(tile, lower_percentile=1, upper_percentile=99, title=None, fig_s return output.getvalue() -def tile_to_html(tile, fig_size=None): +def tile_to_html(tile: Tile, fig_size: Optional[Tuple[int, int]] = None) -> str: """ Provide HTML string representation of Tile image.""" import base64 b64_img_html = '' @@ -115,7 +120,7 @@ def tile_to_html(tile, fig_size=None): return b64_img_html.format(b64_png) -def pandas_df_to_html(df): +def pandas_df_to_html(df: DataFrame) -> str: """Provide HTML formatting for pandas.DataFrame with rf_types.Tile in the columns. """ import pandas as pd # honor the existing options on display @@ -170,17 +175,17 @@ def _safe_geom_to_html(g): return return_html -def spark_df_to_markdown(df, num_rows=5, truncate=False): +def spark_df_to_markdown(df: DataFrame, num_rows: int = 5, truncate: bool = False) -> str: from pyrasterframes import RFContext return RFContext.active().call("_dfToMarkdown", df._jdf, num_rows, truncate) -def spark_df_to_html(df, num_rows=5, truncate=False): +def spark_df_to_html(df: DataFrame, num_rows: int = 5, truncate: bool = False) -> str: from pyrasterframes import RFContext return RFContext.active().call("_dfToHTML", df._jdf, num_rows, truncate) -def _folium_map_formatter(map): +def _folium_map_formatter(map) -> str: """ inputs a folium.Map object and returns html of rendered map """ import base64 diff --git a/pyrasterframes/src/main/python/pyrasterframes/rf_types.py b/pyrasterframes/src/main/python/pyrasterframes/rf_types.py index d76e3832c..53ca0f27d 100644 --- a/pyrasterframes/src/main/python/pyrasterframes/rf_types.py +++ b/pyrasterframes/src/main/python/pyrasterframes/rf_types.py @@ -34,18 +34,22 @@ class here provides the PyRasterFrames entry point. from pyspark.ml.util import DefaultParamsReadable, DefaultParamsWritable from pyrasterframes.rf_context import RFContext +from pyspark.sql import SparkSession +from py4j.java_collections import Sequence import numpy as np +from typing import List, Tuple + __all__ = ['RasterFrameLayer', 'Tile', 'TileUDT', 'CellType', 'RasterSourceUDT', 'TileExploder', 'NoDataFilter'] class RasterFrameLayer(DataFrame): - def __init__(self, jdf, spark_session): + def __init__(self, jdf: DataFrame, spark_session: SparkSession): DataFrame.__init__(self, jdf, spark_session._wrapped) self._jrfctx = spark_session.rasterframes._jrfctx - def tile_columns(self): + def tile_columns(self) -> List[Column]: """ Fetches columns of type Tile. :return: One or more Column instances associated with Tiles. @@ -53,7 +57,7 @@ def tile_columns(self): cols = self._jrfctx.tileColumns(self._jdf) return [Column(c) for c in cols] - def spatial_key_column(self): + def spatial_key_column(self) -> Column: """ Fetch the tagged spatial key column. :return: Spatial key column @@ -61,7 +65,7 @@ def spatial_key_column(self): col = self._jrfctx.spatialKeyColumn(self._jdf) return Column(col) - def temporal_key_column(self): + def temporal_key_column(self) -> Column: """ Fetch the temporal key column, if any. :return: Temporal key column, or None. @@ -77,7 +81,7 @@ def tile_layer_metadata(self): import json return json.loads(str(self._jrfctx.tileLayerMetadata(self._jdf))) - def spatial_join(self, other_df): + def spatial_join(self, other_df: DataFrame): """ Spatially join this RasterFrameLayer to the given RasterFrameLayer. :return: Joined RasterFrameLayer. @@ -86,7 +90,7 @@ def spatial_join(self, other_df): df = ctx._jrfctx.spatialJoin(self._jdf, other_df._jdf) return RasterFrameLayer(df, ctx._spark_session) - def to_int_raster(self, colname, cols, rows): + def to_int_raster(self, colname: str, cols: int, rows: int) -> Sequence: """ Convert a tile to an Int raster :return: array containing values of the tile's cells @@ -94,7 +98,7 @@ def to_int_raster(self, colname, cols, rows): resArr = self._jrfctx.toIntRaster(self._jdf, colname, cols, rows) return resArr - def to_double_raster(self, colname, cols, rows): + def to_double_raster(self, colname: str, cols: int, rows: int) -> Sequence: """ Convert a tile to an Double raster :return: array containing values of the tile's cells @@ -170,7 +174,7 @@ def __init__(self, cell_type_name): self.cell_type_name = cell_type_name @classmethod - def from_numpy_dtype(cls, np_dtype): + def from_numpy_dtype(cls, np_dtype: np.dtype): return CellType(str(np_dtype.name)) @classmethod @@ -205,19 +209,19 @@ def float32(cls): def float64(cls): return CellType('float64') - def is_raw(self): + def is_raw(self) -> bool: return self.cell_type_name.endswith('raw') - def is_user_defined_no_data(self): + def is_user_defined_no_data(self) -> bool: return "ud" in self.cell_type_name - def is_default_no_data(self): + def is_default_no_data(self) -> bool: return not (self.is_raw() or self.is_user_defined_no_data()) - def is_floating_point(self): + def is_floating_point(self) -> bool: return self.cell_type_name.startswith('float') - def base_cell_type_name(self): + def base_cell_type_name(self) -> str: if self.is_raw(): return self.cell_type_name[:-3] elif self.is_user_defined_no_data(): @@ -225,7 +229,7 @@ def base_cell_type_name(self): else: return self.cell_type_name - def has_no_data(self): + def has_no_data(self) -> bool: return not self.is_raw() def no_data_value(self): @@ -254,7 +258,7 @@ def no_data_value(self): return None raise Exception("Unable to determine no_data_value from '{}'".format(n)) - def to_numpy_dtype(self): + def to_numpy_dtype(self) -> np.dtype: n = self.base_cell_type_name() return np.dtype(n).newbyteorder('>') @@ -354,7 +358,7 @@ def __matmul__(self, right): other = right return Tile(np.matmul(self.cells, other)) - def dimensions(self): + def dimensions(self) -> Tuple[int, int]: """ Return a list of cols, rows as is conventional in GeoTrellis and RasterFrames.""" return [self.cells.shape[1], self.cells.shape[0]] diff --git a/pyrasterframes/src/main/python/pyrasterframes/utils.py b/pyrasterframes/src/main/python/pyrasterframes/utils.py index b87dfd581..806d7015d 100644 --- a/pyrasterframes/src/main/python/pyrasterframes/utils.py +++ b/pyrasterframes/src/main/python/pyrasterframes/utils.py @@ -22,34 +22,25 @@ from pyspark.sql import SparkSession from pyspark import SparkConf import os -import sys from . import RFContext +from typing import Union, Dict __all__ = ["create_rf_spark_session", "find_pyrasterframes_jar_dir", "find_pyrasterframes_assembly", "gdal_version"] -def find_pyrasterframes_jar_dir(): +def find_pyrasterframes_jar_dir() -> str: """ Locates the directory where JVM libraries for Spark are stored. :return: path to jar director as a string """ jar_dir = None - if sys.version < "3": - import imp - try: - module_home = imp.find_module("pyrasterframes")[1] # path - jar_dir = os.path.join(module_home, 'jars') - except ImportError: - pass - - else: - from importlib.util import find_spec - try: - module_home = find_spec("pyrasterframes").origin - jar_dir = os.path.join(os.path.dirname(module_home), 'jars') - except ImportError: - pass + from importlib.util import find_spec + try: + module_home = find_spec("pyrasterframes").origin + jar_dir = os.path.join(os.path.dirname(module_home), 'jars') + except ImportError: + pass # Case for when we're running from source build if jar_dir is None or not os.path.exists(jar_dir): @@ -66,7 +57,7 @@ def pdir(curr): return os.path.realpath(jar_dir) -def find_pyrasterframes_assembly(): +def find_pyrasterframes_assembly() -> Union[bytes, str]: jar_dir = find_pyrasterframes_jar_dir() jarpath = glob.glob(os.path.join(jar_dir, 'pyrasterframes-assembly*.jar')) @@ -77,7 +68,7 @@ def find_pyrasterframes_assembly(): return jarpath[0] -def create_rf_spark_session(master="local[*]", **kwargs): +def create_rf_spark_session(master="local[*]", **kwargs: str) -> SparkSession: """ Create a SparkSession with pyrasterframes enabled and configured. """ jar_path = find_pyrasterframes_assembly() @@ -103,11 +94,11 @@ def create_rf_spark_session(master="local[*]", **kwargs): return None -def gdal_version(): +def gdal_version() -> str: fcn = RFContext.active().lookup("buildInfo") return fcn()["GDAL"] -def build_info(): +def build_info() -> Dict[str, str]: fcn = RFContext.active().lookup("buildInfo") return fcn() diff --git a/pyrasterframes/src/main/python/pyrasterframes/version.py b/pyrasterframes/src/main/python/pyrasterframes/version.py index 7253bac59..86c68f9f5 100644 --- a/pyrasterframes/src/main/python/pyrasterframes/version.py +++ b/pyrasterframes/src/main/python/pyrasterframes/version.py @@ -20,4 +20,4 @@ # # Translating Java version from version.sbt to PEP440 norms -__version__ = '0.9.0.dev0' +__version__: str = '0.9.0.dev0' diff --git a/pyrasterframes/src/main/python/setup.py b/pyrasterframes/src/main/python/setup.py index 70f4b2dcc..876a31b0c 100644 --- a/pyrasterframes/src/main/python/setup.py +++ b/pyrasterframes/src/main/python/setup.py @@ -57,7 +57,6 @@ class PweaveDocs(distutils.cmd.Command): ('quick=', 'q', 'Check to see if the source file is newer than existing output before building. Defaults to `False`.') ] - def initialize_options(self): """Set default values for options.""" # Each user option must be listed here with their default value. @@ -149,6 +148,7 @@ def dest_file(self, src_file): pytest = 'pytest>=4.0.0,<5.0.0' pypandoc = 'pypandoc' boto3 = 'boto3' +deprecation = 'deprecation' setup( name='pyrasterframes', @@ -188,7 +188,8 @@ def dest_file(self, src_file): pweave, fiona, rasterio, - folium + folium, + deprecation ], tests_require=[ pytest, @@ -218,7 +219,7 @@ def dest_file(self, src_file): 'License :: OSI Approved :: Apache Software License', 'Natural Language :: English', 'Operating System :: Unix', - 'Programming Language :: Python', + 'Programming Language :: Python :: 3', 'Topic :: Software Development :: Libraries', 'Topic :: Scientific/Engineering :: GIS', 'Topic :: Multimedia :: Graphics :: Graphics Conversion', diff --git a/pyrasterframes/src/main/python/tests/PyRasterFramesTests.py b/pyrasterframes/src/main/python/tests/PyRasterFramesTests.py index 0dc36a8e7..d828f85c6 100644 --- a/pyrasterframes/src/main/python/tests/PyRasterFramesTests.py +++ b/pyrasterframes/src/main/python/tests/PyRasterFramesTests.py @@ -326,9 +326,7 @@ def test_division(self): self.assertTrue(np.array_equal(r2, np.array([[1,1], [1, 1]], dtype=r2.dtype))) def test_matmul(self): - # if sys.version >= '3.5': # per https://docs.python.org/3.7/library/operator.html#operator.matmul new in 3.5 - # r1 = self.t1 @ self.t2 - r1 = self.t1.__matmul__(self.t2) + r1 = self.t1 @ self.t2 # The behavior of np.matmul with masked arrays is not well documented # it seems to treat the 2nd arg as if not a MaskedArray diff --git a/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py b/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py index adcccd7a6..95d74e7b4 100644 --- a/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py +++ b/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py @@ -26,6 +26,7 @@ import numpy as np from numpy.testing import assert_equal +from deprecation import fail_if_not_removed from unittest import skip from . import TestEnvironment @@ -133,6 +134,12 @@ def test_aggregations(self): self.assertEqual(row['rf_agg_no_data_cells(tile)'], 1000) self.assertEqual(row['rf_agg_stats(tile)'].data_cells, row['rf_agg_data_cells(tile)']) + @fail_if_not_removed + def test_add_scalar(self): + # Trivial test to trigger the deprecation failure at the right time. + result: Row = self.rf.select(rf_local_add_double('tile', 99.9), rf_local_add_int('tile', 42)).first() + self.assertTrue(True) + def test_sql(self): self.rf.createOrReplaceTempView("rf_test_sql") diff --git a/pyrasterframes/src/main/python/tests/__init__.py b/pyrasterframes/src/main/python/tests/__init__.py index b09b5f6f3..4121a3dc4 100644 --- a/pyrasterframes/src/main/python/tests/__init__.py +++ b/pyrasterframes/src/main/python/tests/__init__.py @@ -18,21 +18,17 @@ # SPDX-License-Identifier: Apache-2.0 # -import glob import os import unittest from pyrasterframes.utils import create_rf_spark_session -import sys -if sys.version_info[0] > 2: - import builtins -else: - import __builtin__ as builtins +import builtins app_name = 'pyrasterframes test suite' + def resource_dir(): def pdir(curr): return os.path.dirname(curr) From 2c06e7887448da040f0dc637cbd5f4db7848d936 Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Thu, 9 Jan 2020 10:40:36 -0500 Subject: [PATCH 10/94] Add map display to raster-read docs with tile and extent Signed-off-by: Jason T. Brown --- .../src/main/python/docs/raster-read.pymd | 80 +++++++++++++++++-- 1 file changed, 75 insertions(+), 5 deletions(-) diff --git a/pyrasterframes/src/main/python/docs/raster-read.pymd b/pyrasterframes/src/main/python/docs/raster-read.pymd index cfefb603c..aed272b4f 100644 --- a/pyrasterframes/src/main/python/docs/raster-read.pymd +++ b/pyrasterframes/src/main/python/docs/raster-read.pymd @@ -39,13 +39,83 @@ rf.select( ) ``` -You can also see that the single raster has been broken out into many arbitrary non-overlapping regions. Doing so takes advantage of parallel in-memory reads from the cloud hosted data source and allows Spark to work on manageable amounts of data per task. The following code fragment shows us how many subtiles were created from a single source image. - -```python, count_by_uri -counts = rf.groupby(rf.proj_raster_path).count() -counts +You can also see that the single raster has been broken out into many rows containing arbitrary non-overlapping regions. Doing so takes advantage of parallel in-memory reads from the cloud hosted data source and allows Spark to work on manageable amounts of data per row. +The map below shows downsampled imagery with the bounds of the individual tiles. + +@@@ note + +The image contains visible "seams" between the tile extents due to reprojection and downsampling used to create the image. +The native imagery in the DataFrame does not contain any gaps in the source raster's coverage. + +@@@ + +```python, folium_map_of_tile_extents, echo=False +from pyrasterframes.rf_types import Extent +import folium +import pyproj +from functools import partial +from shapely.ops import transform as shtransform +from shapely.geometry import box +import geopandas +import numpy + +wm_crs = 'EPSG:3857' +crs84 = 'urn:ogc:def:crs:OGC:1.3:CRS84' + +# Generate overview image +wm_extent = rf.agg( + rf_agg_reprojected_extent(rf_extent('proj_raster'), rf_crs('proj_raster'), wm_crs) + ).first()[0] +aoi = Extent.from_row(wm_extent) + +aspect = aoi.width / aoi.height +ov_size = 1024 +ov = rf.agg( + rf_agg_overview_raster('proj_raster', int(ov_size * aspect), ov_size, aoi) + ).first()[0] + +# Reproject the web mercator extent to WGS84 +project = partial( + pyproj.transform, + pyproj.Proj(wm_crs), + pyproj.Proj(crs84) + ) +crs84_extent = shtransform(project, box(*wm_extent)) + +# Individual tile WGS84 extents in a dataframe +tile_extents_df = rf.select( + st_reproject( + rf_geometry('proj_raster'), + rf_crs('proj_raster'), + rf_mk_crs('epsg:4326') + ).alias('geometry') +).toPandas() + +ntiles = numpy.nanquantile(ov.cells, [0.03, 0.97]) + +# use `filled` because folium doesn't know how to maskedArray +a = numpy.clip(ov.cells.filled(0), ntiles[0], ntiles[1]) + +m = folium.Map([crs84_extent.centroid.y, crs84_extent.centroid.x], + zoom_start=9) \ + .add_child( + folium.raster_layers.ImageOverlay( + a, + [[crs84_extent.bounds[1], crs84_extent.bounds[0]], + [crs84_extent.bounds[3], crs84_extent.bounds[2]]], + name='rf.proj_raster.tile' + ) + ) \ + .add_child(folium.GeoJson( + geopandas.GeoDataFrame(tile_extents_df, crs=crs84), + name='rf.proj_raster.extent', + style_function=lambda _: {'fillOpacity':0} + )) \ + .add_child(folium.LayerControl(collapsed=False)) +m ``` + Let's select a single _tile_ and view it. The _tile_ preview image as well as the string representation provide some basic information about the _tile_: its dimensions as numbers of columns and rows and the cell type, or data type of all the cells in the _tile_. For more about cell types, refer to @ref:[this discussion](nodata-handling.md#cell-types). ```python, show_tile_sample From 5f083753cef6c4ea62e00efe90ea5c650e0157b3 Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Tue, 14 Jan 2020 16:46:03 -0500 Subject: [PATCH 11/94] Add rf_local_min, rf_local_max, and rf_local_clip functions Signed-off-by: Jason T. Brown --- .../expressions/localops/Clip.scala | 74 +++++++++++++++++++ .../expressions/localops/Max.scala | 54 ++++++++++++++ .../expressions/localops/Min.scala | 54 ++++++++++++++ .../rasterframes/expressions/package.scala | 3 + .../functions/LocalFunctions.scala | 24 ++++++ .../functions/TileFunctionsSpec.scala | 44 +++++++++++ docs/src/main/paradox/reference.md | 24 ++++++ docs/src/main/paradox/release-notes.md | 1 + .../main/python/docs/supervised-learning.pymd | 5 +- .../python/pyrasterframes/rasterfunctions.py | 23 ++++++ .../main/python/tests/RasterFunctionsTests.py | 32 +++++++- 11 files changed, 334 insertions(+), 4 deletions(-) create mode 100644 core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Clip.scala create mode 100644 core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Max.scala create mode 100644 core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Min.scala diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Clip.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Clip.scala new file mode 100644 index 000000000..77ede42b3 --- /dev/null +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Clip.scala @@ -0,0 +1,74 @@ +package org.locationtech.rasterframes.expressions.localops + +import org.apache.spark.sql.Column +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionDescription, TernaryExpression} +import org.apache.spark.sql.functions.lit +import org.apache.spark.sql.rf.TileUDT +import org.apache.spark.sql.types.DataType +import org.locationtech.rasterframes.encoders.CatalystSerializer._ +import org.locationtech.rasterframes.expressions.DynamicExtractors._ +import org.locationtech.rasterframes.expressions.row + +@ExpressionDescription( + usage = "_FUNC_(tile, min, max) - Return the tile with its values clipped to a range defined by min and max," + + " doing so cellwise if min or max are tile type", + arguments = """ + Arguments: + * tile - the tile to operate on + * min - scalar or tile setting the minimum value for each cell + * max - scalar or tile setting the maximum value for each cell""" +) +case class Clip(left: Expression, middle: Expression, right: Expression) + extends TernaryExpression with CodegenFallback with Serializable { + override def dataType: DataType = left.dataType + + override def children: Seq[Expression] = Seq(left, middle, right) + + override val nodeName = "rf_clip" + + override def checkInputDataTypes(): TypeCheckResult = { + if (!tileExtractor.isDefinedAt(left.dataType)) { + TypeCheckFailure(s"Input type '${left.dataType}' does not conform to a Tile type") + } else if (!tileExtractor.isDefinedAt(middle.dataType) && !numberArgExtractor.isDefinedAt(middle.dataType)) { + TypeCheckFailure(s"Input type '${middle.dataType}' does not conform to a Tile or numeric type") + } else if (!tileExtractor.isDefinedAt(right.dataType) && !numberArgExtractor.isDefinedAt(right.dataType)) { + TypeCheckFailure(s"Input type '${right.dataType}' does not conform to a Tile or numeric type") + } + else TypeCheckSuccess + } + + override protected def nullSafeEval(input1: Any, input2: Any, input3: Any): Any = { + implicit val tileSer = TileUDT.tileSerializer + val (targetTile, targetCtx) = tileExtractor(left.dataType)(row(input1)) + val minVal = tileOrNumberExtractor(middle.dataType)(input2) + val maxVal = tileOrNumberExtractor(right.dataType)(input3) + + val result = (minVal, maxVal) match { + case (mn: TileArg, mx: TileArg) ⇒ targetTile.localMin(mx.tile).localMax(mn.tile) + case (mn: TileArg, mx: IntegerArg) ⇒ targetTile.localMin(mx.value).localMax(mn.tile) + case (mn: TileArg, mx: DoubleArg) ⇒ targetTile.localMin(mx.value).localMax(mn.tile) + case (mn: IntegerArg, mx: TileArg) ⇒ targetTile.localMin(mx.tile).localMax(mn.value) + case (mn: IntegerArg, mx: IntegerArg) ⇒ targetTile.localMin(mx.value).localMax(mn.value) + case (mn: IntegerArg, mx: DoubleArg) ⇒ targetTile.localMin(mx.value).localMax(mn.value) + case (mn: DoubleArg, mx: TileArg) ⇒ targetTile.localMin(mx.tile).localMax(mn.value) + case (mn: DoubleArg, mx: IntegerArg) ⇒ targetTile.localMin(mx.value).localMax(mn.value) + case (mn: DoubleArg, mx: DoubleArg) ⇒ targetTile.localMin(mx.value).localMax(mn.value) + } + + targetCtx match { + case Some(ctx) => ctx.toProjectRasterTile(result).toInternalRow + case None => result.toInternalRow + } + } + +} +object Clip { + def apply(tile: Column, min: Column, max: Column): Column = new Column(Clip(tile.expr, min.expr, max.expr)) + def apply[N: Numeric](tile: Column, min: N, max: Column): Column = new Column(Clip(tile.expr, lit(min).expr, max.expr)) + def apply[N: Numeric](tile: Column, min: Column, max: N): Column = new Column(Clip(tile.expr, min.expr, lit(max).expr)) + def apply[N: Numeric](tile: Column, min: N, max: N): Column = new Column(Clip(tile.expr, lit(min).expr, lit(max).expr)) + +} diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Max.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Max.scala new file mode 100644 index 000000000..ed92d329a --- /dev/null +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Max.scala @@ -0,0 +1,54 @@ +/* + * This software is licensed under the Apache 2 license, quoted below. + * + * Copyright 2019 Astraea, Inc. + * + * 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. + * + * SPDX-License-Identifier: Apache-2.0 + * + */ + +package org.locationtech.rasterframes.expressions.localops + +import geotrellis.raster.Tile +import org.apache.spark.sql.Column +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionDescription} +import org.apache.spark.sql.functions.lit +import org.locationtech.rasterframes.expressions.BinaryLocalRasterOp + +@ExpressionDescription( + usage = "_FUNC_(tile, rhs) - Performs cell-wise maximum two tiles or a tile and a scalar.", + arguments = """ + Arguments: + * tile - left-hand-side tile + * rhs - a tile or scalar value""", + examples = """ + Examples: + > SELECT _FUNC_(tile, 1.5); + ... + > SELECT _FUNC_(tile1, tile2); + ...""" +) +case class Max(left: Expression, right:Expression) extends BinaryLocalRasterOp with CodegenFallback { + + override val nodeName = "rf_local_max" + override protected def op(left: Tile, right: Tile): Tile = left.localMax(right) + override protected def op(left: Tile, right: Double): Tile = left.localMax(right) + override protected def op(left: Tile, right: Int): Tile = left.localMax(right) +} +object Max { + def apply(left: Column, right: Column): Column = new Column(Max(left.expr, right.expr)) + def apply[N: Numeric](tile: Column, value: N): Column = new Column(Max(tile.expr, lit(value).expr)) +} diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Min.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Min.scala new file mode 100644 index 000000000..769892709 --- /dev/null +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Min.scala @@ -0,0 +1,54 @@ +/* + * This software is licensed under the Apache 2 license, quoted below. + * + * Copyright 2019 Astraea, Inc. + * + * 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. + * + * SPDX-License-Identifier: Apache-2.0 + * + */ + +package org.locationtech.rasterframes.expressions.localops + +import geotrellis.raster.Tile +import org.apache.spark.sql.Column +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionDescription} +import org.apache.spark.sql.functions.lit +import org.locationtech.rasterframes.expressions.BinaryLocalRasterOp + +@ExpressionDescription( + usage = "_FUNC_(tile, rhs) - Performs cell-wise minimum two tiles or a tile and a scalar.", + arguments = """ + Arguments: + * tile - left-hand-side tile + * rhs - a tile or scalar value""", + examples = """ + Examples: + > SELECT _FUNC_(tile, 1.5); + ... + > SELECT _FUNC_(tile1, tile2); + ...""" +) +case class Min(left: Expression, right:Expression) extends BinaryLocalRasterOp with CodegenFallback { + + override val nodeName = "rf_local_min" + override protected def op(left: Tile, right: Tile): Tile = left.localMin(right) + override protected def op(left: Tile, right: Double): Tile = left.localMin(right) + override protected def op(left: Tile, right: Int): Tile = left.localMin(right) +} +object Min { + def apply(left: Column, right: Column): Column = new Column(Min(left.expr, right.expr)) + def apply[N: Numeric](tile: Column, value: N): Column = new Column(Min(tile.expr, lit(value).expr)) +} diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/package.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/package.scala index b5507ad8a..dd6bc9cae 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/package.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/package.scala @@ -90,6 +90,9 @@ package object expressions { registry.registerExpression[IsIn]("rf_local_is_in") registry.registerExpression[Undefined]("rf_local_no_data") registry.registerExpression[Defined]("rf_local_data") + registry.registerExpression[Min]("rf_local_min") + registry.registerExpression[Max]("rf_local_max") + registry.registerExpression[Clip]("rf_local_clip") registry.registerExpression[Sum]("rf_tile_sum") registry.registerExpression[Round]("rf_round") registry.registerExpression[Abs]("rf_abs") diff --git a/core/src/main/scala/org/locationtech/rasterframes/functions/LocalFunctions.scala b/core/src/main/scala/org/locationtech/rasterframes/functions/LocalFunctions.scala index 2f3dbb9fe..5bdba7ae6 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/functions/LocalFunctions.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/functions/LocalFunctions.scala @@ -56,6 +56,30 @@ trait LocalFunctions { /** Cellwise division of a tile by a scalar value. */ def rf_local_divide[T: Numeric](tileCol: Column, value: T): Column = Divide(tileCol, value) + /** Cellwise minimum between Tiles. */ + def rf_local_min(left: Column, right: Column): Column = Min(left, right) + + /** Cellwise minimum between Tiles. */ + def rf_local_min[T: Numeric](left: Column, right: T): Column = Min(left, right) + + /** Cellwise maximum between Tiles. */ + def rf_local_max(left: Column, right: Column): Column = Max(left, right) + + /** Cellwise maximum between Tiles. */ + def rf_local_max[T: Numeric](left: Column, right: T): Column = Max(left, right) + + /** Return the tile with its values clipped to a range defined by min and max. */ + def rf_local_clip(tile: Column, min: Column, max: Column) = Clip(tile, min, max) + + /** Return the tile with its values clipped to a range defined by min and max. */ + def rf_local_clip[T: Numeric](tile: Column, min: T, max: Column) = Clip(tile, min, max) + + /** Return the tile with its values clipped to a range defined by min and max. */ + def rf_local_clip[T: Numeric](tile: Column, min: Column, max: T) = Clip(tile, min, max) + + /** Return the tile with its values clipped to a range defined by min and max. */ + def rf_local_clip[T: Numeric](tile: Column, min: T, max: T) = Clip(tile, min, max) + /** Perform an arbitrary GeoTrellis `LocalTileBinaryOp` between two Tile columns. */ def rf_local_algebra(op: LocalTileBinaryOp, left: Column, right: Column): TypedColumn[Any, Tile] = withTypedAlias(opName(op), left, right)(udf[Tile, Tile, Tile](op.apply).apply(left, right)) diff --git a/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala index 9e4b4c7fb..2bb454885 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala @@ -210,6 +210,50 @@ class TileFunctionsSpec extends TestEnvironment with RasterMatchers { } } + describe("tile min max and clip") { + it("should support SQL API"){ + checkDocs("rf_local_min") + checkDocs("rf_local_max") + checkDocs("rf_local_clip") + } + it("should evaluate rf_local_min") { + val df = Seq((randPRT, three)).toDF("tile", "three") + val result1 = df.select(rf_local_min($"tile", $"three") as "t") + .select(rf_tile_max($"t")) + .first() + result1 should be <= 3.0 + } + it("should evaluate rf_local_min with scalar") { + val df = Seq(randPRT).toDF("tile") + val result1 = df.select(rf_local_min($"tile", 3) as "t") + .select(rf_tile_max($"t")) + .first() + result1 should be <= 3.0 + } + it("should evaluate rf_local_max") { + val df = Seq((randPRT, three)).toDF("tile", "three") + val result1 = df.select(rf_local_max($"tile", $"three") as "t") + .select(rf_tile_min($"t")) + .first() + result1 should be >= 3.0 + } + it("should evaluate rf_local_max with scalar") { + val df = Seq(randPRT).toDF("tile") + val result1 = df.select(rf_local_max($"tile", 3) as "t") + .select(rf_tile_min($"t")) + .first() + result1 should be >= 3.0 + } + it("should evaluate rf_local_clip"){ + val df = Seq((randPRT, two, six)).toDF("t", "two", "six") + val result = df.select(rf_local_clip($"t", $"two", $"six") as "t") + .select(rf_tile_min($"t") as "min", rf_tile_max($"t") as "max") + .first() + result(0) should be (2) + result(1) should be (6) + } + } + describe("raster metadata") { it("should get the TileDimensions of a Tile") { val t = Seq(randPRT).toDF("tile").select(rf_dimensions($"tile")).first() diff --git a/docs/src/main/paradox/reference.md b/docs/src/main/paradox/reference.md index 5819edbbf..1358f806a 100644 --- a/docs/src/main/paradox/reference.md +++ b/docs/src/main/paradox/reference.md @@ -451,6 +451,30 @@ Extract value from specified bits of the cells' underlying binary data. Working A common use case for this function is covered by @ref:[`rf_mask_by_bits`](reference.md#rf-mask-by-bits). + +### rf_local_min + + Tile rf_local_min(Tile tile, Tile max) + Tile rf_local_min(Tile tile, Numeric max) + +Performs cell-wise minimum two tiles or a tile and a scalar. + +### rf_local_max + + Tile rf_local_max(Tile tile, Tile max) + Tile rf_local_max(Tile tile, Numeric max) + +Performs cell-wise maximum two tiles or a tile and a scalar. + +### rf_local_clip + + Tile rf_local_clip(Tile tile, Tile min, Tile max) + Tile rf_local_clip(Tile tile, Numeric min, Tile max) + Tile rf_local_clip(Tile tile, Tile min, Numeric max) + Tile rf_local_clip(Tile tile, Numeric min, Numeric max) + +Return the tile with its values clipped to a range defined by min and max, inclusive. + ### rf_round Tile rf_round(Tile tile) diff --git a/docs/src/main/paradox/release-notes.md b/docs/src/main/paradox/release-notes.md index 2e4b8d25c..c094b3655 100644 --- a/docs/src/main/paradox/release-notes.md +++ b/docs/src/main/paradox/release-notes.md @@ -15,6 +15,7 @@ * In `rf_ipython`, improved rendering of dataframe binary contents with PNG preamble. * Throw an `IllegalArgumentException` when attempting to apply a mask to a `Tile` whose `CellType` has no NoData defined. ([#409](https://github.com/locationtech/rasterframes/issues/384)) * Add `rf_agg_approx-quantiles` function to compute cell quantiles across an entire column. +* Add `rf_local_min`, `rf_local_max`, and `rf_local_clip` functions. ### 0.8.4 diff --git a/pyrasterframes/src/main/python/docs/supervised-learning.pymd b/pyrasterframes/src/main/python/docs/supervised-learning.pymd index 81a81f634..6304432ca 100644 --- a/pyrasterframes/src/main/python/docs/supervised-learning.pymd +++ b/pyrasterframes/src/main/python/docs/supervised-learning.pymd @@ -206,10 +206,9 @@ Take a look at a sample of the resulting output and the corresponding area's red Recall the label coding: 1 is forest (purple), 2 is cropland (green) and 3 is developed areas(yellow). ```python, display_rgb +scaling_quantiles = retiled.agg(rf_agg_approx_quantiles( sample = retiled \ - .select('prediction', 'red', 'grn', 'blu') \ - .sort(-rf_tile_sum(rf_local_equal('prediction', lit(3.0)))) \ - .first() + .select('prediction', 'red', 'grn', 'blu') sample_rgb = np.concatenate([sample['red'].cells[:, :, None], sample['grn'].cells[ :, :, None], diff --git a/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py b/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py index 1f569b775..07f8781d7 100644 --- a/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py +++ b/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py @@ -605,6 +605,29 @@ def rf_local_unequal(left_tile_col, right_tile_col): return _apply_column_function('rf_local_unequal', left_tile_col, right_tile_col) +def rf_local_min(tile_col, min): + """Performs cell-wise minimum two tiles or a tile and a scalar.""" + if isinstance(min, (int, float)): + min = lit(min) + return _apply_column_function('rf_local_min', tile_col, min) + + +def rf_local_max(tile_col, max): + """Performs cell-wise maximum two tiles or a tile and a scalar.""" + if isinstance(max, (int, float)): + max = lit(max) + return _apply_column_function('rf_local_max', tile_col, max) + + +def rf_local_clip(tile_col, min, max): + """Performs cell-wise maximum two tiles or a tile and a scalar.""" + if isinstance(min, (int, float)): + min = lit(min) + if isinstance(max, (int, float)): + max = lit(max) + return _apply_column_function('rf_local_clip', tile_col, min, max) + + def rf_round(tile_col): """Round cell values to the nearest integer without changing the cell type""" return _apply_column_function('rf_round', tile_col) diff --git a/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py b/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py index a6d19fb2c..e72156a11 100644 --- a/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py +++ b/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py @@ -31,7 +31,6 @@ import numpy as np from numpy.testing import assert_equal, assert_allclose -from unittest import skip from . import TestEnvironment @@ -505,6 +504,37 @@ def test_rf_local_is_in(self): "Tile value {} should contain two 1s as: [[1, 0, 1],[0, 0, 0]]" .format(result['in_list'].cells)) + def test_local_min_max_clip(self): + tile = Tile(np.random.randint(-20, 20, (10, 10)), CellType.int8()) + min_tile = Tile(np.random.randint(-20, 0, (10, 10)), CellType.int8()) + max_tile = Tile(np.random.randint(0, 20, (10, 10)), CellType.int8()) + + df = self.spark.createDataFrame([Row(t=tile, mn=min_tile, mx=max_tile)]) + assert_equal( + df.select(rf_local_min('t', 'mn')).first()[0].cells, + np.clip(tile.cells, None, min_tile.cells) + ) + + assert_equal( + df.select(rf_local_min('t', -5)).first()[0].cells, + np.clip(tile.cells, None, -5) + ) + + assert_equal( + df.select(rf_local_max('t', 'mx')).first()[0].cells, + np.clip(tile.cells, max_tile.cells, None) + ) + + assert_equal( + df.select(rf_local_max('t', 5)).first()[0].cells, + np.clip(tile.cells, 5, None) + ) + + assert_equal( + df.select(rf_local_clip('t', 'mn', 'mx')).first()[0].cells, + np.clip(tile.cells, min_tile.cells, max_tile.cells) + ) + def test_rf_agg_overview_raster(self): width = 500 height = 400 From 6f2540fab0f80a905d78a9baeae007b63f9d3c67 Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Wed, 15 Jan 2020 16:54:52 -0500 Subject: [PATCH 12/94] Add rf_where and rf_standardize functions Signed-off-by: Jason T. Brown --- .../expressions/localops/Where.scala | 99 +++++++++++++++++ .../rasterframes/expressions/package.scala | 2 + .../transformers/Standardize.scala | 105 ++++++++++++++++++ .../functions/LocalFunctions.scala | 18 +++ .../locationtech/rasterframes/package.scala | 7 ++ .../functions/TileFunctionsSpec.scala | 62 +++++++++++ docs/src/main/paradox/reference.md | 28 ++++- docs/src/main/paradox/release-notes.md | 5 +- .../python/pyrasterframes/rasterfunctions.py | 20 ++++ .../main/python/tests/RasterFunctionsTests.py | 23 ++++ 10 files changed, 366 insertions(+), 3 deletions(-) create mode 100644 core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Where.scala create mode 100644 core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/Standardize.scala diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Where.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Where.scala new file mode 100644 index 000000000..bdc13568d --- /dev/null +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Where.scala @@ -0,0 +1,99 @@ +package org.locationtech.rasterframes.expressions.localops + +import com.typesafe.scalalogging.Logger +import geotrellis.raster.Tile +import org.apache.spark.sql.Column +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionDescription, TernaryExpression} +import org.apache.spark.sql.rf.TileUDT +import org.apache.spark.sql.types.DataType +import org.locationtech.rasterframes._ +import org.locationtech.rasterframes.encoders.CatalystSerializer._ +import org.locationtech.rasterframes.expressions.DynamicExtractors._ +import org.locationtech.rasterframes.expressions.row +import org.slf4j.LoggerFactory + +@ExpressionDescription( + usage = "_FUNC_(tile, min, max) - Return a tile with cell values chosen from `x` or `y` depending on `condition`. Operates cell-wise in a similar fashion to Spark SQL `when` and `otherwise`.", + arguments = """ + Arguments: + * condition - the tile of values to evaluate as true + * x - tile with cell values to return if condition is true + * y - tile with cell values to return if condition is false""" +) +case class Where(left: Expression, middle: Expression, right: Expression) + extends TernaryExpression with CodegenFallback with Serializable { + + @transient protected lazy val logger = Logger(LoggerFactory.getLogger(getClass.getName)) + + override def dataType: DataType = middle.dataType + + override def children: Seq[Expression] = Seq(left, middle, right) + + override val nodeName = "rf_where" + + override def checkInputDataTypes(): TypeCheckResult = { + if (!tileExtractor.isDefinedAt(left.dataType)) { + TypeCheckFailure(s"Input type '${left.dataType}' does not conform to a Tile type") + } else if (!tileExtractor.isDefinedAt(middle.dataType)) { + TypeCheckFailure(s"Input type '${middle.dataType}' does not conform to a Tile type") + } else if (!tileExtractor.isDefinedAt(right.dataType)) { + TypeCheckFailure(s"Input type '${right.dataType}' does not conform to a Tile type") + } + else TypeCheckSuccess + } + + override protected def nullSafeEval(input1: Any, input2: Any, input3: Any): Any = { + implicit val tileSer = TileUDT.tileSerializer + val (conditionTile, conditionCtx) = tileExtractor(left.dataType)(row(input1)) + val (xTile, xCtx) = tileExtractor(middle.dataType)(row(input2)) + val (yTile, yCtx) = tileExtractor(right.dataType)(row(input3)) + + if (xCtx.isEmpty && yCtx.isDefined) + logger.warn( + s"Middle parameter '${middle}' provided an extent and CRS, but the right parameter " + + s"'${right}' didn't have any. Because the middle defines output type, the right-hand context will be lost.") + + if(xCtx.isDefined && yCtx.isDefined && xCtx != yCtx) + logger.warn(s"Both '${middle}' and '${right}' provided an extent and CRS, but they are different. The former will be used.") + + val result = op(conditionTile, xTile, yTile) + + xCtx match { + case Some(ctx) => ctx.toProjectRasterTile(result).toInternalRow + case None => result.toInternalRow + } + } + + def op(condition: Tile, x: Tile, y: Tile): Tile = { + import spire.syntax.cfor.cfor + require(condition.dimensions == x.dimensions) + require(x.dimensions == y.dimensions) + + val returnTile = x.mutable + + def getSet(c: Int, r: Int): Unit = { + (returnTile.cellType.isFloatingPoint, y.cellType.isFloatingPoint) match { + case (true, true) ⇒ returnTile.setDouble(c, r, y.getDouble(c, r)) + case (true, false) ⇒ returnTile.setDouble(c, r, y.get(c, r)) + case (false, true) ⇒ returnTile.set(c, r, y.getDouble(c, r).toInt) + case (false, false) ⇒ returnTile.set(c, r, y.get(c, r)) + } + } + + cfor(0)(_ < x.rows, _ + 1) { r ⇒ + cfor(0)(_ < x.cols, _ + 1) { c ⇒ + if(!isCellTrue(condition, c, r)) getSet(c, r) + } + } + + returnTile + } + +} +object Where { + def apply(condition: Column, x: Column, y: Column): Column = new Column(Where(condition.expr, x.expr, y.expr)) + +} diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/package.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/package.scala index dd6bc9cae..33deaa80c 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/package.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/package.scala @@ -93,6 +93,8 @@ package object expressions { registry.registerExpression[Min]("rf_local_min") registry.registerExpression[Max]("rf_local_max") registry.registerExpression[Clip]("rf_local_clip") + registry.registerExpression[Where]("rf_where") + registry.registerExpression[Standardize]("rf_standardize") registry.registerExpression[Sum]("rf_tile_sum") registry.registerExpression[Round]("rf_round") registry.registerExpression[Abs]("rf_abs") diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/Standardize.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/Standardize.scala new file mode 100644 index 000000000..5fcf7cbc9 --- /dev/null +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/Standardize.scala @@ -0,0 +1,105 @@ +/* + * This software is licensed under the Apache 2 license, quoted below. + * + * Copyright 2019 Astraea, Inc. + * + * 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. + * + * SPDX-License-Identifier: Apache-2.0 + * + */ + +package org.locationtech.rasterframes.expressions.transformers + +import geotrellis.raster.{FloatConstantNoDataCellType, Tile} +import org.apache.spark.sql.Column +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionDescription, TernaryExpression} +import org.apache.spark.sql.functions.lit +import org.apache.spark.sql.rf.TileUDT +import org.apache.spark.sql.types.DataType +import org.locationtech.rasterframes.encoders.CatalystSerializer._ +import org.locationtech.rasterframes.expressions.DynamicExtractors._ +import org.locationtech.rasterframes.expressions._ +import org.locationtech.rasterframes.expressions.tilestats.TileStats + +@ExpressionDescription( + usage = "_FUNC_(tile, mean, stddev) - Standardize cell values such that the mean is zero and the standard deviation is one. If specified, the `mean` and `stddev` are applied to all tiles in the column. If not specified, each tile will be standardized according to the statistics of its cell values; this can result in inconsistent values across rows in a tile column.", + arguments = """ + Arguments: + * tile - tile column to extract values + * mean - value to mean-center the cell values around + * stddev - standard deviation to apply in standardization + """, + examples = """ + Examples: + > SELECT _FUNC_(tile, lit(4.0), lit(2.2)) + ...""" +) +case class Standardize(child1: Expression, child2: Expression, child3: Expression) extends TernaryExpression with CodegenFallback with Serializable { + override val nodeName: String = "rf_standardize" + + override def children: Seq[Expression] = Seq(child1, child2, child3) + + override def dataType: DataType = child1.dataType + + override def checkInputDataTypes(): TypeCheckResult = + if(!tileExtractor.isDefinedAt(child1.dataType)) { + TypeCheckFailure(s"Input type '${child1.dataType}' does not conform to a raster type.") + } else if (!doubleArgExtractor.isDefinedAt(child2.dataType)) { + TypeCheckFailure(s"Input type '${child2.dataType}' isn't floating point type.") + } else if (!doubleArgExtractor.isDefinedAt(child3.dataType)) { + TypeCheckFailure(s"Input type '${child3.dataType}' isn't floating point type." ) + } else TypeCheckSuccess + + + override protected def nullSafeEval(input1: Any, input2: Any, input3: Any): Any = { + implicit val tileSer = TileUDT.tileSerializer + val (childTile, childCtx) = tileExtractor(child1.dataType)(row(input1)) + + val mean = doubleArgExtractor(child2.dataType)(input2).value + + val stdDev = doubleArgExtractor(child2.dataType)(input3).value + + childCtx match { + case Some(ctx) => ctx.toProjectRasterTile(op(childTile, mean, stdDev)).toInternalRow + case None => op(childTile, mean, stdDev).toInternalRow + } + } + + protected def op(tile: Tile, mean: Double, stdDev: Double): Tile = + tile.convert(FloatConstantNoDataCellType) + .localSubtract(mean) + .localDivide(stdDev) + +} +object Standardize { + def apply(tile: Column, mean: Column, stdDev: Column): Column = + new Column(Standardize(tile.expr, mean.expr, stdDev.expr)) + + def apply(tile: Column, mean: Double, stdDev: Double): Column = + new Column(Standardize(tile.expr, lit(mean).expr, lit(stdDev).expr)) + + def apply(tile: Column): Column = { + import org.apache.spark.sql.functions.sqrt + val stats = TileStats(tile) + val mean = stats.getField("mean").expr + val stdDev = sqrt(stats.getField("variance")).expr + + new Column(Standardize(tile.expr, mean, stdDev)) + } +} + + diff --git a/core/src/main/scala/org/locationtech/rasterframes/functions/LocalFunctions.scala b/core/src/main/scala/org/locationtech/rasterframes/functions/LocalFunctions.scala index 5bdba7ae6..05c65200e 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/functions/LocalFunctions.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/functions/LocalFunctions.scala @@ -80,6 +80,24 @@ trait LocalFunctions { /** Return the tile with its values clipped to a range defined by min and max. */ def rf_local_clip[T: Numeric](tile: Column, min: T, max: T) = Clip(tile, min, max) + /** Return a tile with cell values chosen from `x` or `y` depending on `condition`. + Operates cell-wise in a similar fashion to Spark SQL `when` and `otherwise`. */ + def rf_where(condition: Column, x: Column, y: Column): Column = Where(condition, x, y) + + /** Standardize cell values such that the mean is zero and the standard deviation is one. + * The `mean` and `stddev` are applied to all tiles in the column. + */ + def rf_standardize(tile: Column, mean: Column, stddev: Column): Column = Standardize(tile, mean, stddev) + + /** Standardize cell values such that the mean is zero and the standard deviation is one. + * The `mean` and `stddev` are applied to all tiles in the column. + */ + def rf_standardize(tile: Column, mean: Double, stddev: Double): Column = Standardize(tile, mean, stddev) + + /** Standardize cell values such that the mean is zero and the standard deviation is one. + * Each tile will be standardized according to the statistics of its cell values; this can result in inconsistent values across rows in a tile column. */ + def rf_standardize(tile: Column): Column = Standardize(tile) + /** Perform an arbitrary GeoTrellis `LocalTileBinaryOp` between two Tile columns. */ def rf_local_algebra(op: LocalTileBinaryOp, left: Column, right: Column): TypedColumn[Any, Tile] = withTypedAlias(opName(op), left, right)(udf[Tile, Tile, Tile](op.apply).apply(left, right)) diff --git a/core/src/main/scala/org/locationtech/rasterframes/package.scala b/core/src/main/scala/org/locationtech/rasterframes/package.scala index b1958d36b..8db3e36b5 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/package.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/package.scala @@ -140,4 +140,11 @@ package object rasterframes extends StandardColumns def isCellTrue(v: Double): Boolean = isData(v) & v != 0.0 /** Test if a cell value evaluates to true: it is not NoData and it is non-zero */ def isCellTrue(v: Int): Boolean = isData(v) & v != 0 + + /** Test if a Tile's cell value evaluates to true at a given position. Truth defined by not NoData and non-zero */ + def isCellTrue(t: Tile, col: Int, row: Int): Boolean = + if (t.cellType.isFloatingPoint) isCellTrue(t.getDouble(col, row)) + else isCellTrue(t.get(col, row)) + + } diff --git a/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala index 2bb454885..21f20771d 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala @@ -254,6 +254,68 @@ class TileFunctionsSpec extends TestEnvironment with RasterMatchers { } } + describe("conditional cell values"){ + + it("should support SQL API") { + checkDocs("rf_where") + } + + it("should evaluate rf_where"){ + val df = Seq((randPRT, one, six)).toDF("t", "one", "six") + val result = df.select( + rf_for_all( + rf_local_equal( + rf_where(rf_local_greater($"t", 0), $"one", $"six") as "result", + rf_local_add( + rf_local_multiply(rf_local_greater($"t", 0), $"one"), + rf_local_multiply(rf_local_less_equal($"t", 0), $"six") + ) as "expected" + ) + ) + ) + .first() + + result should be (true) + + } + } + + describe("standardize and normalize") { + + it("should be accssible in SQL API"){ + checkDocs("rf_standardize") +// checkDocs("rf_normalize") + } + + it("should evaluate rf_standardize") { + import org.apache.spark.sql.functions.sqrt + + val df = Seq(randPRT, six, one).toDF("tile") + val stats = df.agg(rf_agg_stats($"tile").alias("stat")).select($"stat.mean", sqrt($"stat.variance")) + .first() + val result = df.select(rf_standardize($"tile", stats.getAs[Double](0), stats.getAs[Double](1)) as "z") + .agg(rf_agg_stats($"z") as "zstats") + .select($"zstats.mean", $"zstats.variance") + .first() + + result.getAs[Double](0) should be (0.0 +- 0.00001) + result.getAs[Double](1) should be (1.0 +- 0.00001) + } + + it("should evaluate rf_standardize with tile -level stats") { + + val df = Seq(randPRT).toDF("tile") + val result = df.select(rf_standardize($"tile") as "z") + .select(rf_tile_stats($"z") as "zstat") + .select($"zstat.mean", $"zstat.variance") + .first() + + result.getAs[Double](0) should be (0.0 +- 0.02) + result.getAs[Double](1) should be (1.0 +- 0.00001) + } + + } + describe("raster metadata") { it("should get the TileDimensions of a Tile") { val t = Seq(randPRT).toDF("tile").select(rf_dimensions($"tile")).first() diff --git a/docs/src/main/paradox/reference.md b/docs/src/main/paradox/reference.md index 1358f806a..a481de789 100644 --- a/docs/src/main/paradox/reference.md +++ b/docs/src/main/paradox/reference.md @@ -475,6 +475,30 @@ Performs cell-wise maximum two tiles or a tile and a scalar. Return the tile with its values clipped to a range defined by min and max, inclusive. +### rf_where + + Tile rf_where(Tile condition, Tile x, Tile y) + +Return a tile with cell values chosen from `x` or `y` depending on `condition`. +Operates cell-wise in a similar fashion to Spark SQL `when` and `otherwise`. + +### rf_rescale + + Tile rf_rescale(Tile tile) + Tile rf_rescale(Tile tile, Double min, Double max) + +Rescale cell values such that the minimum is zero and the maximum is one. Other values will be linearly interpolated into the range. +If specified, the `min` parameter will become the zero value and the `max` parameter will become 1. See @ref:[`rf_agg_stats`](reference.md#rf_agg_stats). +Values outside the range will be clipped to 0 or 1. +If `min` and `max` are not specified, the __tile-wise__ minimum and maximum are used; this can result in inconsistent values across rows in a tile column. + +### rf_standardize + + rf_standardize(Tile tile) + rf_standardize(Tile tile, Double mean, Double stddev) + +Standardize cell values such that the mean is zero and the standard deviation is one. If specified, the `mean` and `stddev` are applied to all tiles in the column. See @ref:[`rf_agg_stats`](reference.md#rf_agg_stats). If not specified, each tile will be standardized according to the statistics of its cell values; this can result in inconsistent values across rows in a tile column. + ### rf_round Tile rf_round(Tile tile) @@ -632,7 +656,7 @@ Aggregates over the `tile` and return the mean of cell values, ignoring NoData. Long rf_agg_data_cells(Tile tile) -_SQL_: @ref:[`rf_agg_stats`](reference.md#rf-agg-stats)`(tile).dataCells` +_SQL_: @ref:[`rf_agg_stats`](reference.md#rf-agg-stats)`(tile).data_cells` Aggregates over the `tile` and return the count of data cells. Equivalent to @ref:[`rf_agg_stats`](reference.md#rf-agg-stats)`.dataCells`. @@ -640,7 +664,7 @@ Aggregates over the `tile` and return the count of data cells. Equivalent to @re Long rf_agg_no_data_cells(Tile tile) -_SQL_: @ref:[`rf_agg_stats`](reference.md#rf-agg-stats)`(tile).dataCells` +_SQL_: @ref:[`rf_agg_stats`](reference.md#rf-agg-stats)`(tile).no_data_cells` Aggregates over the `tile` and return the count of NoData cells. Equivalent to @ref:[`rf_agg_stats`](reference.md#rf-agg-stats)`.noDataCells`. C.F. @ref:[`rf_no_data_cells`](reference.md#rf-no-data-cells) a row-wise count of no data cells. diff --git a/docs/src/main/paradox/release-notes.md b/docs/src/main/paradox/release-notes.md index c094b3655..d0d946c59 100644 --- a/docs/src/main/paradox/release-notes.md +++ b/docs/src/main/paradox/release-notes.md @@ -15,7 +15,10 @@ * In `rf_ipython`, improved rendering of dataframe binary contents with PNG preamble. * Throw an `IllegalArgumentException` when attempting to apply a mask to a `Tile` whose `CellType` has no NoData defined. ([#409](https://github.com/locationtech/rasterframes/issues/384)) * Add `rf_agg_approx-quantiles` function to compute cell quantiles across an entire column. -* Add `rf_local_min`, `rf_local_max`, and `rf_local_clip` functions. +* Add functions for changing cell values based on either conditions or to achieve a distribution of values. ([#449](https://github.com/locationtech/rasterframes/pull/449)) + * Add `rf_local_min`, `rf_local_max`, and `rf_local_clip` functions. + * Add cell value scaling functions `rf_rescale` and `rf_standardize`. + * Add `rf_where` function, similar in spirit to numpy's `where`, or a cell-wise version of Spark SQL's `when` and `otherwise`. ### 0.8.4 diff --git a/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py b/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py index 07f8781d7..14f8c787e 100644 --- a/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py +++ b/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py @@ -628,6 +628,26 @@ def rf_local_clip(tile_col, min, max): return _apply_column_function('rf_local_clip', tile_col, min, max) +def rf_where(condition, x, y): + """Return a tile with cell values chosen from `x` or `y` depending on `condition`. + Operates cell-wise in a similar fashion to Spark SQL `when` and `otherwise`.""" + return _apply_column_function('rf_where', condition, x, y) + + +def rf_standardize(tile, mean, stddev): + """ + Standardize cell values such that the mean is zero and the standard deviation is one. + If specified, the `mean` and `stddev` are applied to all tiles in the column. + If not specified, each tile will be standardized according to the statistics of its cell values; + this can result in inconsistent values across rows in a tile column. + """ + if isinstance(mean, (int, float)): + mean = lit(mean) + if isinstance(stddev, (int, float)): + stddev = lit(stddev) + return _apply_column_function('rf_standardize', tile, mean, stddev) + + def rf_round(tile_col): """Round cell values to the nearest integer without changing the cell type""" return _apply_column_function('rf_round', tile_col) diff --git a/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py b/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py index e72156a11..fd10a9eac 100644 --- a/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py +++ b/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py @@ -535,6 +535,29 @@ def test_local_min_max_clip(self): np.clip(tile.cells, min_tile.cells, max_tile.cells) ) + def test_rf_where(self): + cond = Tile(np.random.binomial(1, 0.35, (10, 10)), CellType.uint8()) + x = Tile(np.random.randint(-20, 10, (10, 10)), CellType.int8()) + y = Tile(np.random.randint(0, 30, (10, 10)), CellType.int8()) + + df = self.spark.createDataFrame([Row(cond=cond, x=x, y=y)]) + result = df.select(rf_where('cond', 'x', 'y')).first()[0].cells + assert_equal(result, np.where(cond.cells, x.cells, y.cells)) + + def test_rf_standardize(self): + from pyspark.sql.functions import sqrt as F_sqrt + stats = self.prdf.select(rf_agg_stats('proj_raster').alias('stat')) \ + .select('stat.mean', F_sqrt('stat.variance').alias('sttdev')) \ + .first() + + result = self.prdf.select(rf_standardize('proj_raster', stats[0], stats[1]).alias('z')) \ + .select(rf_agg_stats('z').alias('z_stat')) \ + .select('z_stat.mean', 'z_stat.variance') \ + .first() + + self.assertAlmostEqual(result[0], 0.0) + self.assertAlmostEqual(result[1], 1.0) + def test_rf_agg_overview_raster(self): width = 500 height = 400 From 6554d0b21ace848a85b8faac3f5ae9d8c890e2dd Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Fri, 17 Jan 2020 12:18:13 -0500 Subject: [PATCH 13/94] Bumped dev version. --- pyrasterframes/src/main/python/pyrasterframes/version.py | 2 +- version.sbt | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pyrasterframes/src/main/python/pyrasterframes/version.py b/pyrasterframes/src/main/python/pyrasterframes/version.py index 850a9ab18..4d92531e7 100644 --- a/pyrasterframes/src/main/python/pyrasterframes/version.py +++ b/pyrasterframes/src/main/python/pyrasterframes/version.py @@ -20,4 +20,4 @@ # # Translating Java version from version.sbt to PEP440 norms -__version__ = '0.8.5' +__version__ = '0.8.6.dev0' diff --git a/version.sbt b/version.sbt index 891babf7b..2561a50df 100644 --- a/version.sbt +++ b/version.sbt @@ -1 +1 @@ -version in ThisBuild := "0.8.5" +version in ThisBuild := "0.8.6-SNAPSHOT" From 942ab5039213f01191aa8379c678262ab8303c32 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Fri, 17 Jan 2020 13:37:12 -0500 Subject: [PATCH 14/94] Test origanization/cleanup. --- .../rasterframes/RasterFramesStatsSpec.scala | 77 ----- .../rasterframes/RasterFunctionsSpec.scala | 313 +----------------- .../locationtech/rasterframes/TestData.scala | 4 +- .../functions/ArithmeticFunctionsSpec.scala | 302 +++++++++++++++++ .../StatFunctionsSpec.scala} | 249 ++++++++++++-- .../functions/TileFunctionsSpec.scala | 168 +--------- 6 files changed, 547 insertions(+), 566 deletions(-) delete mode 100644 core/src/test/scala/org/locationtech/rasterframes/RasterFramesStatsSpec.scala create mode 100644 core/src/test/scala/org/locationtech/rasterframes/functions/ArithmeticFunctionsSpec.scala rename core/src/test/scala/org/locationtech/rasterframes/{TileStatsSpec.scala => functions/StatFunctionsSpec.scala} (62%) diff --git a/core/src/test/scala/org/locationtech/rasterframes/RasterFramesStatsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/RasterFramesStatsSpec.scala deleted file mode 100644 index 11e5d9589..000000000 --- a/core/src/test/scala/org/locationtech/rasterframes/RasterFramesStatsSpec.scala +++ /dev/null @@ -1,77 +0,0 @@ -/* - * This software is licensed under the Apache 2 license, quoted below. - * - * Copyright 2018 Astraea, Inc. - * - * 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. - * - * SPDX-License-Identifier: Apache-2.0 - * - */ - -package org.locationtech.rasterframes - -import org.apache.spark.sql.functions.{col, explode} - -class RasterFramesStatsSpec extends TestEnvironment with TestData { - - import spark.implicits._ - - val df = TestData.sampleGeoTiff - .toDF() - .withColumn("tilePlus2", rf_local_add(col("tile"), 2)) - - - describe("Tile quantiles through built-in functions") { - - it("should compute approx percentiles for a single tile col") { - // Use "explode" - val result = df - .select(rf_explode_tiles($"tile")) - .stat - .approxQuantile("tile", Array(0.10, 0.50, 0.90), 0.00001) - - result.length should be(3) - - // computing externally with numpy we arrive at 7963, 10068, 12160 for these quantiles - result should contain inOrderOnly(7963.0, 10068.0, 12160.0) - - // Use "to_array" and built-in explode - val result2 = df - .select(explode(rf_tile_to_array_double($"tile")) as "tile") - .stat - .approxQuantile("tile", Array(0.10, 0.50, 0.90), 0.00001) - - result2.length should be(3) - - // computing externally with numpy we arrive at 7963, 10068, 12160 for these quantiles - result2 should contain inOrderOnly(7963.0, 10068.0, 12160.0) - - } - } - - describe("Tile quantiles through custom aggregate") { - it("should compute approx percentiles for a single tile col") { - val result = df - .select(rf_agg_approx_quantiles($"tile", Seq(0.1, 0.5, 0.9))) - .first() - - result.length should be(3) - - // computing externally with numpy we arrive at 7963, 10068, 12160 for these quantiles - result should contain inOrderOnly(7963.0, 10068.0, 12160.0) - } - - } -} - diff --git a/core/src/test/scala/org/locationtech/rasterframes/RasterFunctionsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/RasterFunctionsSpec.scala index f297fdce3..4dcff9034 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/RasterFunctionsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/RasterFunctionsSpec.scala @@ -24,160 +24,13 @@ package org.locationtech.rasterframes import geotrellis.raster._ import geotrellis.raster.testkit.RasterMatchers import org.apache.spark.sql.functions._ -import org.locationtech.rasterframes.expressions.accessors.ExtractTile import org.locationtech.rasterframes.tiles.ProjectedRasterTile class RasterFunctionsSpec extends TestEnvironment with RasterMatchers { import TestData._ import spark.implicits._ - describe("arithmetic tile operations") { - it("should local_add") { - val df = Seq((one, two)).toDF("one", "two") - - val maybeThree = df.select(rf_local_add($"one", $"two")).as[ProjectedRasterTile] - assertEqual(maybeThree.first(), three) - - assertEqual(df.selectExpr("rf_local_add(one, two)").as[ProjectedRasterTile].first(), three) - - val maybeThreeTile = df.select(rf_local_add(ExtractTile($"one"), ExtractTile($"two"))).as[Tile] - assertEqual(maybeThreeTile.first(), three.toArrayTile()) - checkDocs("rf_local_add") - } - - it("should rf_local_subtract") { - val df = Seq((three, two)).toDF("three", "two") - val maybeOne = df.select(rf_local_subtract($"three", $"two")).as[ProjectedRasterTile] - assertEqual(maybeOne.first(), one) - - assertEqual(df.selectExpr("rf_local_subtract(three, two)").as[ProjectedRasterTile].first(), one) - - val maybeOneTile = - df.select(rf_local_subtract(ExtractTile($"three"), ExtractTile($"two"))).as[Tile] - assertEqual(maybeOneTile.first(), one.toArrayTile()) - checkDocs("rf_local_subtract") - } - - it("should rf_local_multiply") { - val df = Seq((three, two)).toDF("three", "two") - - val maybeSix = df.select(rf_local_multiply($"three", $"two")).as[ProjectedRasterTile] - assertEqual(maybeSix.first(), six) - - assertEqual(df.selectExpr("rf_local_multiply(three, two)").as[ProjectedRasterTile].first(), six) - - val maybeSixTile = - df.select(rf_local_multiply(ExtractTile($"three"), ExtractTile($"two"))).as[Tile] - assertEqual(maybeSixTile.first(), six.toArrayTile()) - checkDocs("rf_local_multiply") - } - - it("should rf_local_divide") { - val df = Seq((six, two)).toDF("six", "two") - val maybeThree = df.select(rf_local_divide($"six", $"two")).as[ProjectedRasterTile] - assertEqual(maybeThree.first(), three) - - assertEqual(df.selectExpr("rf_local_divide(six, two)").as[ProjectedRasterTile].first(), three) - - assertEqual( - df.selectExpr("rf_local_multiply(rf_local_divide(six, 2.0), two)") - .as[ProjectedRasterTile] - .first(), - six) - - val maybeThreeTile = - df.select(rf_local_divide(ExtractTile($"six"), ExtractTile($"two"))).as[Tile] - assertEqual(maybeThreeTile.first(), three.toArrayTile()) - checkDocs("rf_local_divide") - } - } - - describe("scalar tile operations") { - it("should rf_local_add") { - val df = Seq(one).toDF("one") - val maybeThree = df.select(rf_local_add($"one", 2)).as[ProjectedRasterTile] - assertEqual(maybeThree.first(), three) - - val maybeThreeD = df.select(rf_local_add($"one", 2.1)).as[ProjectedRasterTile] - assertEqual(maybeThreeD.first(), three.convert(DoubleConstantNoDataCellType).localAdd(0.1)) - - val maybeThreeTile = df.select(rf_local_add(ExtractTile($"one"), 2)).as[Tile] - assertEqual(maybeThreeTile.first(), three.toArrayTile()) - } - - it("should rf_local_subtract") { - val df = Seq(three).toDF("three") - - val maybeOne = df.select(rf_local_subtract($"three", 2)).as[ProjectedRasterTile] - assertEqual(maybeOne.first(), one) - - val maybeOneD = df.select(rf_local_subtract($"three", 2.0)).as[ProjectedRasterTile] - assertEqual(maybeOneD.first(), one) - - val maybeOneTile = df.select(rf_local_subtract(ExtractTile($"three"), 2)).as[Tile] - assertEqual(maybeOneTile.first(), one.toArrayTile()) - } - - it("should rf_local_multiply") { - val df = Seq(three).toDF("three") - - val maybeSix = df.select(rf_local_multiply($"three", 2)).as[ProjectedRasterTile] - assertEqual(maybeSix.first(), six) - - val maybeSixD = df.select(rf_local_multiply($"three", 2.0)).as[ProjectedRasterTile] - assertEqual(maybeSixD.first(), six) - - val maybeSixTile = df.select(rf_local_multiply(ExtractTile($"three"), 2)).as[Tile] - assertEqual(maybeSixTile.first(), six.toArrayTile()) - } - - it("should rf_local_divide") { - val df = Seq(six).toDF("six") - - val maybeThree = df.select(rf_local_divide($"six", 2)).as[ProjectedRasterTile] - assertEqual(maybeThree.first(), three) - - val maybeThreeD = df.select(rf_local_divide($"six", 2.0)).as[ProjectedRasterTile] - assertEqual(maybeThreeD.first(), three) - - val maybeThreeTile = df.select(rf_local_divide(ExtractTile($"six"), 2)).as[Tile] - assertEqual(maybeThreeTile.first(), three.toArrayTile()) - } - } - - describe("analytical transformations") { - - it("should return local data and nodata") { - checkDocs("rf_local_data") - checkDocs("rf_local_no_data") - - val df = Seq(randNDPRT) - .toDF("t") - .withColumn("ld", rf_local_data($"t")) - .withColumn("lnd", rf_local_no_data($"t")) - - val ndResult = df.select($"lnd").as[Tile].first() - ndResult should be(randNDPRT.localUndefined()) - - val dResult = df.select($"ld").as[Tile].first() - dResult should be(randNDPRT.localDefined()) - } - - it("should compute rf_normalized_difference") { - val df = Seq((three, two)).toDF("three", "two") - - df.select(rf_tile_to_array_double(rf_normalized_difference($"three", $"two"))) - .first() - .forall(_ == 0.2) shouldBe true - - df.selectExpr("rf_tile_to_array_double(rf_normalized_difference(three, two))") - .as[Array[Double]] - .first() - .forall(_ == 0.2) shouldBe true - - checkDocs("rf_normalized_difference") - } - + describe("Misc raster functions") { it("should render ascii art") { val df = Seq[Tile](ProjectedRasterTile(TestData.l8Labels)).toDF("tile") val r1 = df.select(rf_render_ascii($"tile")) @@ -194,125 +47,6 @@ class RasterFunctionsSpec extends TestEnvironment with RasterMatchers { checkDocs("rf_render_matrix") } - it("should round tile cell values") { - - val three_plus = TestData.projectedRasterTile(cols, rows, 3.12, extent, crs, DoubleConstantNoDataCellType) - val three_less = TestData.projectedRasterTile(cols, rows, 2.92, extent, crs, DoubleConstantNoDataCellType) - val three_double = TestData.projectedRasterTile(cols, rows, 3.0, extent, crs, DoubleConstantNoDataCellType) - - val df = Seq((three_plus, three_less, three)).toDF("three_plus", "three_less", "three") - - assertEqual(df.select(rf_round($"three")).as[ProjectedRasterTile].first(), three) - assertEqual(df.select(rf_round($"three_plus")).as[ProjectedRasterTile].first(), three_double) - assertEqual(df.select(rf_round($"three_less")).as[ProjectedRasterTile].first(), three_double) - - assertEqual(df.selectExpr("rf_round(three)").as[ProjectedRasterTile].first(), three) - assertEqual(df.selectExpr("rf_round(three_plus)").as[ProjectedRasterTile].first(), three_double) - assertEqual(df.selectExpr("rf_round(three_less)").as[ProjectedRasterTile].first(), three_double) - - checkDocs("rf_round") - } - - it("should abs cell values") { - val minus = one.mapTile(t => t.convert(IntConstantNoDataCellType) * -1) - val df = Seq((minus, one)).toDF("minus", "one") - - assertEqual(df.select(rf_abs($"minus").as[ProjectedRasterTile]).first(), one) - - checkDocs("rf_abs") - } - - it("should take logarithms positive cell values") { - // rf_log10 1000 == 3 - val thousand = TestData.projectedRasterTile(cols, rows, 1000, extent, crs, ShortConstantNoDataCellType) - val threesDouble = TestData.projectedRasterTile(cols, rows, 3.0, extent, crs, DoubleConstantNoDataCellType) - val zerosDouble = TestData.projectedRasterTile(cols, rows, 0.0, extent, crs, DoubleConstantNoDataCellType) - - val df1 = Seq(thousand).toDF("tile") - assertEqual(df1.select(rf_log10($"tile")).as[ProjectedRasterTile].first(), threesDouble) - - // ln random tile == rf_log10 random tile / rf_log10(e); random tile square to ensure all positive cell values - val df2 = Seq(randPositiveDoubleTile).toDF("tile") - val log10e = math.log10(math.E) - assertEqual( - df2.select(rf_log($"tile")).as[ProjectedRasterTile].first(), - df2.select(rf_log10($"tile")).as[ProjectedRasterTile].first() / log10e) - - lazy val maybeZeros = df2 - .selectExpr(s"rf_local_subtract(rf_log(tile), rf_local_divide(rf_log10(tile), ${log10e}))") - .as[ProjectedRasterTile] - .first() - assertEqual(maybeZeros, zerosDouble) - - // rf_log1p for zeros should be ln(1) - val ln1 = math.log1p(0.0) - val df3 = Seq(zero).toDF("tile") - val maybeLn1 = df3.selectExpr(s"rf_log1p(tile)").as[ProjectedRasterTile].first() - assert(maybeLn1.toArrayDouble().forall(_ == ln1)) - - checkDocs("rf_log") - checkDocs("rf_log2") - checkDocs("rf_log10") - checkDocs("rf_log1p") - } - - it("should take logarithms with non-positive cell values") { - val ni_float = TestData.projectedRasterTile(cols, rows, Double.NegativeInfinity, extent, crs, DoubleConstantNoDataCellType) - val zero_float = TestData.projectedRasterTile(cols, rows, 0.0, extent, crs, DoubleConstantNoDataCellType) - - // tile zeros ==> -Infinity - val df_0 = Seq(zero).toDF("tile") - assertEqual(df_0.select(rf_log($"tile")).as[ProjectedRasterTile].first(), ni_float) - assertEqual(df_0.select(rf_log10($"tile")).as[ProjectedRasterTile].first(), ni_float) - assertEqual(df_0.select(rf_log2($"tile")).as[ProjectedRasterTile].first(), ni_float) - // rf_log1p of zeros should be 0. - assertEqual(df_0.select(rf_log1p($"tile")).as[ProjectedRasterTile].first(), zero_float) - - // tile negative values ==> NaN - assert(df_0.selectExpr("rf_log(rf_local_subtract(tile, 42))").as[ProjectedRasterTile].first().isNoDataTile) - assert(df_0.selectExpr("rf_log2(rf_local_subtract(tile, 42))").as[ProjectedRasterTile].first().isNoDataTile) - assert(df_0.select(rf_log1p(rf_local_subtract($"tile", 42))).as[ProjectedRasterTile].first().isNoDataTile) - assert(df_0.select(rf_log10(rf_local_subtract($"tile", lit(0.01)))).as[ProjectedRasterTile].first().isNoDataTile) - - } - - it("should take exponential") { - val df = Seq(six).toDF("tile") - - // rf_exp inverses rf_log - assertEqual( - df.select(rf_exp(rf_log($"tile"))).as[ProjectedRasterTile].first(), - six - ) - - // base 2 - assertEqual(df.select(rf_exp2(rf_log2($"tile"))).as[ProjectedRasterTile].first(), six) - - // base 10 - assertEqual(df.select(rf_exp10(rf_log10($"tile"))).as[ProjectedRasterTile].first(), six) - - // plus/minus 1 - assertEqual(df.select(rf_expm1(rf_log1p($"tile"))).as[ProjectedRasterTile].first(), six) - - // SQL - assertEqual(df.selectExpr("rf_exp(rf_log(tile))").as[ProjectedRasterTile].first(), six) - - // SQL base 10 - assertEqual(df.selectExpr("rf_exp10(rf_log10(tile))").as[ProjectedRasterTile].first(), six) - - // SQL base 2 - assertEqual(df.selectExpr("rf_exp2(rf_log2(tile))").as[ProjectedRasterTile].first(), six) - - // SQL rf_expm1 - assertEqual(df.selectExpr("rf_expm1(rf_log1p(tile))").as[ProjectedRasterTile].first(), six) - - checkDocs("rf_exp") - checkDocs("rf_exp10") - checkDocs("rf_exp2") - checkDocs("rf_expm1") - - } - it("should resample") { def lowRes = { def base = ArrayTile(Array(1, 2, 3, 4), 2, 2) @@ -353,50 +87,5 @@ class RasterFunctionsSpec extends TestEnvironment with RasterMatchers { checkDocs("rf_resample") } - - it("should interpret cell values with a specified cell type") { - checkDocs("rf_interpret_cell_type_as") - val df = Seq(randNDPRT).toDF("t") - .withColumn("tile", rf_interpret_cell_type_as($"t", "int8raw")) - val resultTile = df.select("tile").as[Tile].first() - - resultTile.cellType should be(CellType.fromName("int8raw")) - // should have same number of values that are -2 the old ND - val countOldNd = df.select( - rf_tile_sum(rf_local_equal($"tile", ct.noDataValue)), - rf_no_data_cells($"t") - ).first() - countOldNd._1 should be(countOldNd._2) - - // should not have no data any more (raw type) - val countNewNd = df.select(rf_no_data_cells($"tile")).first() - countNewNd should be(0L) - } - - it("should check values is_in") { - checkDocs("rf_local_is_in") - - // tile is 3 by 3 with values, 1 to 9 - val rf = Seq(byteArrayTile).toDF("t") - .withColumn("one", lit(1)) - .withColumn("five", lit(5)) - .withColumn("ten", lit(10)) - .withColumn("in_expect_2", rf_local_is_in($"t", array($"one", $"five"))) - .withColumn("in_expect_1", rf_local_is_in($"t", array($"ten", $"five"))) - .withColumn("in_expect_1a", rf_local_is_in($"t", Array(10, 5))) - .withColumn("in_expect_0", rf_local_is_in($"t", array($"ten"))) - - val e2Result = rf.select(rf_tile_sum($"in_expect_2")).as[Double].first() - e2Result should be(2.0) - - val e1Result = rf.select(rf_tile_sum($"in_expect_1")).as[Double].first() - e1Result should be(1.0) - - val e1aResult = rf.select(rf_tile_sum($"in_expect_1a")).as[Double].first() - e1aResult should be(1.0) - - val e0Result = rf.select($"in_expect_0").as[Tile].first() - e0Result.toArray() should contain only (0) - } } } diff --git a/core/src/test/scala/org/locationtech/rasterframes/TestData.scala b/core/src/test/scala/org/locationtech/rasterframes/TestData.scala index 6ce750e7d..8d3460cef 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/TestData.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/TestData.scala @@ -180,7 +180,9 @@ trait TestData { TestData.randomTile(cols, rows, UByteConstantNoDataCellType) )).map(ProjectedRasterTile(_, extent, crs)) :+ null - def lazyPRT = RasterRef(RFRasterSource(TestData.l8samplePath), 0, None, None).tile + def rasterRef = RasterRef(RFRasterSource(TestData.l8samplePath), 0, None, None) + def lazyPRT = rasterRef.tile + object GeomData { val fact = new GeometryFactory() diff --git a/core/src/test/scala/org/locationtech/rasterframes/functions/ArithmeticFunctionsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/functions/ArithmeticFunctionsSpec.scala new file mode 100644 index 000000000..0c698c22e --- /dev/null +++ b/core/src/test/scala/org/locationtech/rasterframes/functions/ArithmeticFunctionsSpec.scala @@ -0,0 +1,302 @@ +/* + * This software is licensed under the Apache 2 license, quoted below. + * + * Copyright 2020 Astraea, Inc. + * + * 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. + * + * SPDX-License-Identifier: Apache-2.0 + * + */ + +package org.locationtech.rasterframes.functions + +import geotrellis.raster.testkit.RasterMatchers +import org.locationtech.rasterframes.TestEnvironment +import geotrellis.raster._ +import geotrellis.raster.testkit.RasterMatchers +import org.apache.spark.sql.functions._ +import org.locationtech.rasterframes.expressions.accessors.ExtractTile +import org.locationtech.rasterframes.tiles.ProjectedRasterTile +import org.locationtech.rasterframes._ +class ArithmeticFunctionsSpec extends TestEnvironment with RasterMatchers { + + import TestData._ + import spark.implicits._ + + describe("arithmetic tile operations") { + it("should local_add") { + val df = Seq((one, two)).toDF("one", "two") + + val maybeThree = df.select(rf_local_add($"one", $"two")).as[ProjectedRasterTile] + assertEqual(maybeThree.first(), three) + + assertEqual(df.selectExpr("rf_local_add(one, two)").as[ProjectedRasterTile].first(), three) + + val maybeThreeTile = df.select(rf_local_add(ExtractTile($"one"), ExtractTile($"two"))).as[Tile] + assertEqual(maybeThreeTile.first(), three.toArrayTile()) + checkDocs("rf_local_add") + } + + it("should rf_local_subtract") { + val df = Seq((three, two)).toDF("three", "two") + val maybeOne = df.select(rf_local_subtract($"three", $"two")).as[ProjectedRasterTile] + assertEqual(maybeOne.first(), one) + + assertEqual(df.selectExpr("rf_local_subtract(three, two)").as[ProjectedRasterTile].first(), one) + + val maybeOneTile = + df.select(rf_local_subtract(ExtractTile($"three"), ExtractTile($"two"))).as[Tile] + assertEqual(maybeOneTile.first(), one.toArrayTile()) + checkDocs("rf_local_subtract") + } + + it("should rf_local_multiply") { + val df = Seq((three, two)).toDF("three", "two") + + val maybeSix = df.select(rf_local_multiply($"three", $"two")).as[ProjectedRasterTile] + assertEqual(maybeSix.first(), six) + + assertEqual(df.selectExpr("rf_local_multiply(three, two)").as[ProjectedRasterTile].first(), six) + + val maybeSixTile = + df.select(rf_local_multiply(ExtractTile($"three"), ExtractTile($"two"))).as[Tile] + assertEqual(maybeSixTile.first(), six.toArrayTile()) + checkDocs("rf_local_multiply") + } + + it("should rf_local_divide") { + val df = Seq((six, two)).toDF("six", "two") + val maybeThree = df.select(rf_local_divide($"six", $"two")).as[ProjectedRasterTile] + assertEqual(maybeThree.first(), three) + + assertEqual(df.selectExpr("rf_local_divide(six, two)").as[ProjectedRasterTile].first(), three) + + assertEqual( + df.selectExpr("rf_local_multiply(rf_local_divide(six, 2.0), two)") + .as[ProjectedRasterTile] + .first(), + six) + + val maybeThreeTile = + df.select(rf_local_divide(ExtractTile($"six"), ExtractTile($"two"))).as[Tile] + assertEqual(maybeThreeTile.first(), three.toArrayTile()) + checkDocs("rf_local_divide") + } + } + + describe("scalar tile operations") { + it("should rf_local_add") { + val df = Seq(one).toDF("one") + val maybeThree = df.select(rf_local_add($"one", 2)).as[ProjectedRasterTile] + assertEqual(maybeThree.first(), three) + + val maybeThreeD = df.select(rf_local_add($"one", 2.1)).as[ProjectedRasterTile] + assertEqual(maybeThreeD.first(), three.convert(DoubleConstantNoDataCellType).localAdd(0.1)) + + val maybeThreeTile = df.select(rf_local_add(ExtractTile($"one"), 2)).as[Tile] + assertEqual(maybeThreeTile.first(), three.toArrayTile()) + } + + it("should rf_local_subtract") { + val df = Seq(three).toDF("three") + + val maybeOne = df.select(rf_local_subtract($"three", 2)).as[ProjectedRasterTile] + assertEqual(maybeOne.first(), one) + + val maybeOneD = df.select(rf_local_subtract($"three", 2.0)).as[ProjectedRasterTile] + assertEqual(maybeOneD.first(), one) + + val maybeOneTile = df.select(rf_local_subtract(ExtractTile($"three"), 2)).as[Tile] + assertEqual(maybeOneTile.first(), one.toArrayTile()) + } + + it("should rf_local_multiply") { + val df = Seq(three).toDF("three") + + val maybeSix = df.select(rf_local_multiply($"three", 2)).as[ProjectedRasterTile] + assertEqual(maybeSix.first(), six) + + val maybeSixD = df.select(rf_local_multiply($"three", 2.0)).as[ProjectedRasterTile] + assertEqual(maybeSixD.first(), six) + + val maybeSixTile = df.select(rf_local_multiply(ExtractTile($"three"), 2)).as[Tile] + assertEqual(maybeSixTile.first(), six.toArrayTile()) + } + + it("should rf_local_divide") { + val df = Seq(six).toDF("six") + + val maybeThree = df.select(rf_local_divide($"six", 2)).as[ProjectedRasterTile] + assertEqual(maybeThree.first(), three) + + val maybeThreeD = df.select(rf_local_divide($"six", 2.0)).as[ProjectedRasterTile] + assertEqual(maybeThreeD.first(), three) + + val maybeThreeTile = df.select(rf_local_divide(ExtractTile($"six"), 2)).as[Tile] + assertEqual(maybeThreeTile.first(), three.toArrayTile()) + } + } + + describe("analytical transformations") { + + it("should return local data and nodata") { + checkDocs("rf_local_data") + checkDocs("rf_local_no_data") + + val df = Seq(randNDPRT) + .toDF("t") + .withColumn("ld", rf_local_data($"t")) + .withColumn("lnd", rf_local_no_data($"t")) + + val ndResult = df.select($"lnd").as[Tile].first() + ndResult should be(randNDPRT.localUndefined()) + + val dResult = df.select($"ld").as[Tile].first() + dResult should be(randNDPRT.localDefined()) + } + + it("should compute rf_normalized_difference") { + val df = Seq((three, two)).toDF("three", "two") + + df.select(rf_tile_to_array_double(rf_normalized_difference($"three", $"two"))) + .first() + .forall(_ == 0.2) shouldBe true + + df.selectExpr("rf_tile_to_array_double(rf_normalized_difference(three, two))") + .as[Array[Double]] + .first() + .forall(_ == 0.2) shouldBe true + + checkDocs("rf_normalized_difference") + } + it("should round tile cell values") { + val three_plus = TestData.projectedRasterTile(cols, rows, 3.12, extent, crs, DoubleConstantNoDataCellType) + val three_less = TestData.projectedRasterTile(cols, rows, 2.92, extent, crs, DoubleConstantNoDataCellType) + val three_double = TestData.projectedRasterTile(cols, rows, 3.0, extent, crs, DoubleConstantNoDataCellType) + + val df = Seq((three_plus, three_less, three)).toDF("three_plus", "three_less", "three") + + assertEqual(df.select(rf_round($"three")).as[ProjectedRasterTile].first(), three) + assertEqual(df.select(rf_round($"three_plus")).as[ProjectedRasterTile].first(), three_double) + assertEqual(df.select(rf_round($"three_less")).as[ProjectedRasterTile].first(), three_double) + + assertEqual(df.selectExpr("rf_round(three)").as[ProjectedRasterTile].first(), three) + assertEqual(df.selectExpr("rf_round(three_plus)").as[ProjectedRasterTile].first(), three_double) + assertEqual(df.selectExpr("rf_round(three_less)").as[ProjectedRasterTile].first(), three_double) + + checkDocs("rf_round") + } + + it("should abs cell values") { + val minus = one.mapTile(t => t.convert(IntConstantNoDataCellType) * -1) + val df = Seq((minus, one)).toDF("minus", "one") + + assertEqual(df.select(rf_abs($"minus").as[ProjectedRasterTile]).first(), one) + + checkDocs("rf_abs") + } + + it("should take logarithms positive cell values") { + // rf_log10 1000 == 3 + val thousand = TestData.projectedRasterTile(cols, rows, 1000, extent, crs, ShortConstantNoDataCellType) + val threesDouble = TestData.projectedRasterTile(cols, rows, 3.0, extent, crs, DoubleConstantNoDataCellType) + val zerosDouble = TestData.projectedRasterTile(cols, rows, 0.0, extent, crs, DoubleConstantNoDataCellType) + + val df1 = Seq(thousand).toDF("tile") + assertEqual(df1.select(rf_log10($"tile")).as[ProjectedRasterTile].first(), threesDouble) + + // ln random tile == rf_log10 random tile / rf_log10(e); random tile square to ensure all positive cell values + val df2 = Seq(randPositiveDoubleTile).toDF("tile") + val log10e = math.log10(math.E) + assertEqual( + df2.select(rf_log($"tile")).as[ProjectedRasterTile].first(), + df2.select(rf_log10($"tile")).as[ProjectedRasterTile].first() / log10e) + + lazy val maybeZeros = df2 + .selectExpr(s"rf_local_subtract(rf_log(tile), rf_local_divide(rf_log10(tile), ${log10e}))") + .as[ProjectedRasterTile] + .first() + assertEqual(maybeZeros, zerosDouble) + + // rf_log1p for zeros should be ln(1) + val ln1 = math.log1p(0.0) + val df3 = Seq(zero).toDF("tile") + val maybeLn1 = df3.selectExpr(s"rf_log1p(tile)").as[ProjectedRasterTile].first() + assert(maybeLn1.toArrayDouble().forall(_ == ln1)) + + checkDocs("rf_log") + checkDocs("rf_log2") + checkDocs("rf_log10") + checkDocs("rf_log1p") + } + + it("should take logarithms with non-positive cell values") { + val ni_float = TestData.projectedRasterTile(cols, rows, Double.NegativeInfinity, extent, crs, DoubleConstantNoDataCellType) + val zero_float = TestData.projectedRasterTile(cols, rows, 0.0, extent, crs, DoubleConstantNoDataCellType) + + // tile zeros ==> -Infinity + val df_0 = Seq(zero).toDF("tile") + assertEqual(df_0.select(rf_log($"tile")).as[ProjectedRasterTile].first(), ni_float) + assertEqual(df_0.select(rf_log10($"tile")).as[ProjectedRasterTile].first(), ni_float) + assertEqual(df_0.select(rf_log2($"tile")).as[ProjectedRasterTile].first(), ni_float) + // rf_log1p of zeros should be 0. + assertEqual(df_0.select(rf_log1p($"tile")).as[ProjectedRasterTile].first(), zero_float) + + // tile negative values ==> NaN + assert(df_0.selectExpr("rf_log(rf_local_subtract(tile, 42))").as[ProjectedRasterTile].first().isNoDataTile) + assert(df_0.selectExpr("rf_log2(rf_local_subtract(tile, 42))").as[ProjectedRasterTile].first().isNoDataTile) + assert(df_0.select(rf_log1p(rf_local_subtract($"tile", 42))).as[ProjectedRasterTile].first().isNoDataTile) + assert(df_0.select(rf_log10(rf_local_subtract($"tile", lit(0.01)))).as[ProjectedRasterTile].first().isNoDataTile) + + } + + it("should take exponential") { + val df = Seq(six).toDF("tile") + + // rf_exp inverses rf_log + assertEqual( + df.select(rf_exp(rf_log($"tile"))).as[ProjectedRasterTile].first(), + six + ) + + // base 2 + assertEqual(df.select(rf_exp2(rf_log2($"tile"))).as[ProjectedRasterTile].first(), six) + + // base 10 + assertEqual(df.select(rf_exp10(rf_log10($"tile"))).as[ProjectedRasterTile].first(), six) + + // plus/minus 1 + assertEqual(df.select(rf_expm1(rf_log1p($"tile"))).as[ProjectedRasterTile].first(), six) + + // SQL + assertEqual(df.selectExpr("rf_exp(rf_log(tile))").as[ProjectedRasterTile].first(), six) + + // SQL base 10 + assertEqual(df.selectExpr("rf_exp10(rf_log10(tile))").as[ProjectedRasterTile].first(), six) + + // SQL base 2 + assertEqual(df.selectExpr("rf_exp2(rf_log2(tile))").as[ProjectedRasterTile].first(), six) + + // SQL rf_expm1 + assertEqual(df.selectExpr("rf_expm1(rf_log1p(tile))").as[ProjectedRasterTile].first(), six) + + checkDocs("rf_exp") + checkDocs("rf_exp10") + checkDocs("rf_exp2") + checkDocs("rf_expm1") + + } + + } +} \ No newline at end of file diff --git a/core/src/test/scala/org/locationtech/rasterframes/TileStatsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/functions/StatFunctionsSpec.scala similarity index 62% rename from core/src/test/scala/org/locationtech/rasterframes/TileStatsSpec.scala rename to core/src/test/scala/org/locationtech/rasterframes/functions/StatFunctionsSpec.scala index 049eb5b83..9ccdfa62e 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/TileStatsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/functions/StatFunctionsSpec.scala @@ -1,7 +1,7 @@ /* * This software is licensed under the Apache 2 license, quoted below. * - * Copyright 2017 Astraea, Inc. + * Copyright 2020 Astraea, Inc. * * 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 @@ -19,27 +19,235 @@ * */ -package org.locationtech.rasterframes +package org.locationtech.rasterframes.functions import geotrellis.raster._ -import geotrellis.raster.mapalgebra.local.{Add, Max, Min} +import geotrellis.raster.mapalgebra.local._ import geotrellis.spark._ import org.apache.spark.sql.Column import org.apache.spark.sql.functions._ -import org.locationtech.rasterframes.TestData.randomTile -import org.locationtech.rasterframes.stats.CellHistogram -import org.locationtech.rasterframes.util.DataBiasedOp.BiasedAdd +import org.locationtech.rasterframes.TestData._ +import org.locationtech.rasterframes._ +import org.locationtech.rasterframes.stats._ +import org.locationtech.rasterframes.util.DataBiasedOp._ + +class StatFunctionsSpec extends TestEnvironment with TestData { -/** - * Test rig associated with computing statistics and other descriptive - * information over tiles. - * - * @since 9/18/17 - */ -class TileStatsSpec extends TestEnvironment with TestData { - import TestData.injectND import spark.implicits._ + val df = TestData.sampleGeoTiff + .toDF() + .withColumn("tilePlus2", rf_local_add(col("tile"), 2)) + + + describe("Tile quantiles through built-in functions") { + + it("should compute approx percentiles for a single tile col") { + // Use "explode" + val result = df + .select(rf_explode_tiles($"tile")) + .stat + .approxQuantile("tile", Array(0.10, 0.50, 0.90), 0.00001) + + result.length should be(3) + + // computing externally with numpy we arrive at 7963, 10068, 12160 for these quantiles + result should contain inOrderOnly(7963.0, 10068.0, 12160.0) + + // Use "to_array" and built-in explode + val result2 = df + .select(explode(rf_tile_to_array_double($"tile")) as "tile") + .stat + .approxQuantile("tile", Array(0.10, 0.50, 0.90), 0.00001) + + result2.length should be(3) + + // computing externally with numpy we arrive at 7963, 10068, 12160 for these quantiles + result2 should contain inOrderOnly(7963.0, 10068.0, 12160.0) + } + } + + describe("Tile quantiles through custom aggregate") { + it("should compute approx percentiles for a single tile col") { + val result = df + .select(rf_agg_approx_quantiles($"tile", Seq(0.1, 0.5, 0.9))) + .first() + + result.length should be(3) + + // computing externally with numpy we arrive at 7963, 10068, 12160 for these quantiles + result should contain inOrderOnly(7963.0, 10068.0, 12160.0) + } + } + + describe("per-tile stats") { + it("should compute data cell counts") { + val df = Seq(TestData.injectND(numND)(two)).toDF("two") + df.select(rf_data_cells($"two")).first() shouldBe (cols * rows - numND).toLong + + val df2 = randNDTilesWithNull.toDF("tile") + df2 + .select(rf_data_cells($"tile") as "cells") + .agg(sum("cells")) + .as[Long] + .first() should be(expectedRandData) + + checkDocs("rf_data_cells") + } + it("should compute no-data cell counts") { + val df = Seq(TestData.injectND(numND)(two)).toDF("two") + df.select(rf_no_data_cells($"two")).first() should be(numND) + + val df2 = randNDTilesWithNull.toDF("tile") + df2 + .select(rf_no_data_cells($"tile") as "cells") + .agg(sum("cells")) + .as[Long] + .first() should be(expectedRandNoData) + + checkDocs("rf_no_data_cells") + } + + it("should properly count data and nodata cells on constant tiles") { + val rf = Seq(randPRT).toDF("tile") + + val df = rf + .withColumn("make", rf_make_constant_tile(99, 3, 4, ByteConstantNoDataCellType)) + .withColumn("make2", rf_with_no_data($"make", 99)) + + val counts = df + .select( + rf_no_data_cells($"make").alias("nodata1"), + rf_data_cells($"make").alias("data1"), + rf_no_data_cells($"make2").alias("nodata2"), + rf_data_cells($"make2").alias("data2") + ) + .as[(Long, Long, Long, Long)] + .first() + + counts should be((0L, 12L, 12L, 0L)) + } + + it("should detect no-data tiles") { + val df = Seq(nd).toDF("nd") + df.select(rf_is_no_data_tile($"nd")).first() should be(true) + val df2 = Seq(two).toDF("not_nd") + df2.select(rf_is_no_data_tile($"not_nd")).first() should be(false) + checkDocs("rf_is_no_data_tile") + } + + it("should evaluate exists and for_all") { + val df0 = Seq(zero).toDF("tile") + df0.select(rf_exists($"tile")).first() should be(false) + df0.select(rf_for_all($"tile")).first() should be(false) + + Seq(one).toDF("tile").select(rf_exists($"tile")).first() should be(true) + Seq(one).toDF("tile").select(rf_for_all($"tile")).first() should be(true) + + val dfNd = Seq(TestData.injectND(1)(one)).toDF("tile") + dfNd.select(rf_exists($"tile")).first() should be(true) + dfNd.select(rf_for_all($"tile")).first() should be(false) + + checkDocs("rf_exists") + checkDocs("rf_for_all") + } + + it("should check values is_in") { + checkDocs("rf_local_is_in") + + // tile is 3 by 3 with values, 1 to 9 + val rf = Seq(byteArrayTile).toDF("t") + .withColumn("one", lit(1)) + .withColumn("five", lit(5)) + .withColumn("ten", lit(10)) + .withColumn("in_expect_2", rf_local_is_in($"t", array($"one", $"five"))) + .withColumn("in_expect_1", rf_local_is_in($"t", array($"ten", $"five"))) + .withColumn("in_expect_1a", rf_local_is_in($"t", Array(10, 5))) + .withColumn("in_expect_0", rf_local_is_in($"t", array($"ten"))) + + val e2Result = rf.select(rf_tile_sum($"in_expect_2")).as[Double].first() + e2Result should be(2.0) + + val e1Result = rf.select(rf_tile_sum($"in_expect_1")).as[Double].first() + e1Result should be(1.0) + + val e1aResult = rf.select(rf_tile_sum($"in_expect_1a")).as[Double].first() + e1aResult should be(1.0) + + val e0Result = rf.select($"in_expect_0").as[Tile].first() + e0Result.toArray() should contain only (0) + } + it("should find the minimum cell value") { + val min = randNDPRT.toArray().filter(c => isData(c)).min.toDouble + val df = Seq(randNDPRT).toDF("rand") + df.select(rf_tile_min($"rand")).first() should be(min) + df.selectExpr("rf_tile_min(rand)").as[Double].first() should be(min) + checkDocs("rf_tile_min") + } + + it("should find the maximum cell value") { + val max = randNDPRT.toArray().filter(c => isData(c)).max.toDouble + val df = Seq(randNDPRT).toDF("rand") + df.select(rf_tile_max($"rand")).first() should be(max) + df.selectExpr("rf_tile_max(rand)").as[Double].first() should be(max) + checkDocs("rf_tile_max") + } + it("should compute the tile mean cell value") { + val values = randNDPRT.toArray().filter(c => isData(c)) + val mean = values.sum.toDouble / values.length + val df = Seq(randNDPRT).toDF("rand") + df.select(rf_tile_mean($"rand")).first() should be(mean) + df.selectExpr("rf_tile_mean(rand)").as[Double].first() should be(mean) + checkDocs("rf_tile_mean") + } + + it("should compute the tile summary statistics") { + val values = randNDPRT.toArray().filter(c => isData(c)) + val mean = values.sum.toDouble / values.length + val df = Seq(randNDPRT).toDF("rand") + val stats = df.select(rf_tile_stats($"rand")).first() + stats.mean should be(mean +- 0.00001) + + val stats2 = df + .selectExpr("rf_tile_stats(rand) as stats") + .select($"stats".as[CellStatistics]) + .first() + stats2 should be(stats) + + df.select(rf_tile_stats($"rand") as "stats") + .select($"stats.mean") + .as[Double] + .first() should be(mean +- 0.00001) + df.selectExpr("rf_tile_stats(rand) as stats") + .select($"stats.no_data_cells") + .as[Long] + .first() should be <= (cols * rows - numND).toLong + + val df2 = randNDTilesWithNull.toDF("tile") + df2 + .select(rf_tile_stats($"tile")("data_cells") as "cells") + .agg(sum("cells")) + .as[Long] + .first() should be(expectedRandData) + + checkDocs("rf_tile_stats") + } + + it("should compute the tile histogram") { + val df = Seq(randNDPRT).toDF("rand") + val h1 = df.select(rf_tile_histogram($"rand")).first() + + val h2 = df + .selectExpr("rf_tile_histogram(rand) as hist") + .select($"hist".as[CellHistogram]) + .first() + + h1 should be(h2) + + checkDocs("rf_tile_histogram") + } + } + describe("computing statistics over tiles") { //import org.apache.spark.sql.execution.debug._ it("should report dimensions") { @@ -60,9 +268,9 @@ class TileStatsSpec extends TestEnvironment with TestData { df.repartition(4).createOrReplaceTempView("tmp") assert( - sql("select dims.* from (select rf_dimensions(tile2) as dims from tmp)") - .as[(Int, Int)] - .first() === (3, 3)) + sql("select dims.* from (select rf_dimensions(tile2) as dims from tmp)") + .as[(Int, Int)] + .first() === (3, 3)) } it("should report cell type") { @@ -278,14 +486,14 @@ class TileStatsSpec extends TestEnvironment with TestData { val countArray = dsNd.select(rf_agg_local_data_cells($"tiles")).first().toArray() val expectedCount = (completeTile.localDefined().toArray zip incompleteTile.localDefined().toArray()).toSeq.map( - pr => pr._1 * 20 + pr._2) + pr => pr._1 * 20 + pr._2) assert(countArray === expectedCount) val countNodataArray = dsNd.select(rf_agg_local_no_data_cells($"tiles")).first().toArray assert(countNodataArray === incompleteTile.localUndefined().toArray) -// val meanTile = dsNd.select(rf_agg_local_mean($"tiles")).first() -// assert(meanTile.toArray() === completeTile.toArray()) + // val meanTile = dsNd.select(rf_agg_local_mean($"tiles")).first() + // assert(meanTile.toArray() === completeTile.toArray()) val maxTile = dsNd.select(rf_agg_local_max($"tiles")).first() assert(maxTile.toArray() === completeTile.toArray()) @@ -401,3 +609,4 @@ class TileStatsSpec extends TestEnvironment with TestData { } } } + diff --git a/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala index 0afa83e7b..d0ce1c65b 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala @@ -23,13 +23,12 @@ package org.locationtech.rasterframes.functions import java.io.ByteArrayInputStream import geotrellis.proj4.CRS -import geotrellis.raster.testkit.RasterMatchers import geotrellis.raster._ +import geotrellis.raster.testkit.RasterMatchers import javax.imageio.ImageIO import org.apache.spark.sql.Encoders import org.apache.spark.sql.functions.sum import org.locationtech.rasterframes._ -import org.locationtech.rasterframes.stats._ import org.locationtech.rasterframes.tiles.ProjectedRasterTile import org.locationtech.rasterframes.util.ColorRampNames @@ -89,6 +88,7 @@ class TileFunctionsSpec extends TestEnvironment with RasterMatchers { checkDocs("rf_convert_cell_type") } + it("should change NoData value") { val df = Seq((TestData.injectND(7)(three), TestData.injectND(12)(two))).toDF("three", "two") @@ -107,21 +107,19 @@ class TileFunctionsSpec extends TestEnvironment with RasterMatchers { // Should maintain original cell type. ndCT.select(rf_cell_type($"two")).first().withDefaultNoData() should be(ct.withDefaultNoData()) } + it("should interpret cell values with a specified cell type") { checkDocs("rf_interpret_cell_type_as") - val df = Seq(randNDPRT) - .toDF("t") + val df = Seq(randNDPRT).toDF("t") .withColumn("tile", rf_interpret_cell_type_as($"t", "int8raw")) val resultTile = df.select("tile").as[Tile].first() resultTile.cellType should be(CellType.fromName("int8raw")) // should have same number of values that are -2 the old ND - val countOldNd = df - .select( - rf_tile_sum(rf_local_equal($"tile", ct.noDataValue)), - rf_no_data_cells($"t") - ) - .first() + val countOldNd = df.select( + rf_tile_sum(rf_local_equal($"tile", ct.noDataValue)), + rf_no_data_cells($"t") + ).first() countOldNd._1 should be(countOldNd._2) // should not have no data any more (raw type) @@ -239,158 +237,16 @@ class TileFunctionsSpec extends TestEnvironment with RasterMatchers { } it("should get the CRS of a RasterRef") { - fail() - //val e = Seq(Tuple1(TestData.rasterRef)).toDF("ref").select(rf_crs($"ref")).first() - //e should be(rasterRef.crs) + val e = Seq(Tuple1(TestData.rasterRef)).toDF("ref").select(rf_crs($"ref")).first() + e should be(rasterRef.crs) } it("should get the Extent of a RasterRef") { - fail() - //val e = Seq(Tuple1(rasterRef)).toDF("ref").select(rf_extent($"ref")).first() - //e should be(rasterRef.extent) + val e = Seq(Tuple1(rasterRef)).toDF("ref").select(rf_extent($"ref")).first() + e should be(rasterRef.extent) } } - describe("per-tile stats") { - it("should compute data cell counts") { - val df = Seq(TestData.injectND(numND)(two)).toDF("two") - df.select(rf_data_cells($"two")).first() shouldBe (cols * rows - numND).toLong - - val df2 = randNDTilesWithNull.toDF("tile") - df2 - .select(rf_data_cells($"tile") as "cells") - .agg(sum("cells")) - .as[Long] - .first() should be(expectedRandData) - - checkDocs("rf_data_cells") - } - it("should compute no-data cell counts") { - val df = Seq(TestData.injectND(numND)(two)).toDF("two") - df.select(rf_no_data_cells($"two")).first() should be(numND) - - val df2 = randNDTilesWithNull.toDF("tile") - df2 - .select(rf_no_data_cells($"tile") as "cells") - .agg(sum("cells")) - .as[Long] - .first() should be(expectedRandNoData) - - checkDocs("rf_no_data_cells") - } - - it("should properly count data and nodata cells on constant tiles") { - val rf = Seq(randPRT).toDF("tile") - - val df = rf - .withColumn("make", rf_make_constant_tile(99, 3, 4, ByteConstantNoDataCellType)) - .withColumn("make2", rf_with_no_data($"make", 99)) - - val counts = df - .select( - rf_no_data_cells($"make").alias("nodata1"), - rf_data_cells($"make").alias("data1"), - rf_no_data_cells($"make2").alias("nodata2"), - rf_data_cells($"make2").alias("data2") - ) - .as[(Long, Long, Long, Long)] - .first() - - counts should be((0l, 12l, 12l, 0l)) - } - - it("should detect no-data tiles") { - val df = Seq(nd).toDF("nd") - df.select(rf_is_no_data_tile($"nd")).first() should be(true) - val df2 = Seq(two).toDF("not_nd") - df2.select(rf_is_no_data_tile($"not_nd")).first() should be(false) - checkDocs("rf_is_no_data_tile") - } - - it("should evaluate exists and for_all") { - val df0 = Seq(zero).toDF("tile") - df0.select(rf_exists($"tile")).first() should be(false) - df0.select(rf_for_all($"tile")).first() should be(false) - - Seq(one).toDF("tile").select(rf_exists($"tile")).first() should be(true) - Seq(one).toDF("tile").select(rf_for_all($"tile")).first() should be(true) - val dfNd = Seq(TestData.injectND(1)(one)).toDF("tile") - dfNd.select(rf_exists($"tile")).first() should be(true) - dfNd.select(rf_for_all($"tile")).first() should be(false) - - checkDocs("rf_exists") - checkDocs("rf_for_all") - } - it("should find the minimum cell value") { - val min = randNDPRT.toArray().filter(c => isData(c)).min.toDouble - val df = Seq(randNDPRT).toDF("rand") - df.select(rf_tile_min($"rand")).first() should be(min) - df.selectExpr("rf_tile_min(rand)").as[Double].first() should be(min) - checkDocs("rf_tile_min") - } - - it("should find the maximum cell value") { - val max = randNDPRT.toArray().filter(c => isData(c)).max.toDouble - val df = Seq(randNDPRT).toDF("rand") - df.select(rf_tile_max($"rand")).first() should be(max) - df.selectExpr("rf_tile_max(rand)").as[Double].first() should be(max) - checkDocs("rf_tile_max") - } - it("should compute the tile mean cell value") { - val values = randNDPRT.toArray().filter(c => isData(c)) - val mean = values.sum.toDouble / values.length - val df = Seq(randNDPRT).toDF("rand") - df.select(rf_tile_mean($"rand")).first() should be(mean) - df.selectExpr("rf_tile_mean(rand)").as[Double].first() should be(mean) - checkDocs("rf_tile_mean") - } - - it("should compute the tile summary statistics") { - val values = randNDPRT.toArray().filter(c => isData(c)) - val mean = values.sum.toDouble / values.length - val df = Seq(randNDPRT).toDF("rand") - val stats = df.select(rf_tile_stats($"rand")).first() - stats.mean should be(mean +- 0.00001) - - val stats2 = df - .selectExpr("rf_tile_stats(rand) as stats") - .select($"stats".as[CellStatistics]) - .first() - stats2 should be(stats) - - df.select(rf_tile_stats($"rand") as "stats") - .select($"stats.mean") - .as[Double] - .first() should be(mean +- 0.00001) - df.selectExpr("rf_tile_stats(rand) as stats") - .select($"stats.no_data_cells") - .as[Long] - .first() should be <= (cols * rows - numND).toLong - - val df2 = randNDTilesWithNull.toDF("tile") - df2 - .select(rf_tile_stats($"tile")("data_cells") as "cells") - .agg(sum("cells")) - .as[Long] - .first() should be(expectedRandData) - - checkDocs("rf_tile_stats") - } - - it("should compute the tile histogram") { - val df = Seq(randNDPRT).toDF("rand") - val h1 = df.select(rf_tile_histogram($"rand")).first() - - val h2 = df - .selectExpr("rf_tile_histogram(rand) as hist") - .select($"hist".as[CellHistogram]) - .first() - - h1 should be(h2) - - checkDocs("rf_tile_histogram") - } - } describe("conversion operations") { it("should convert tile into array") { From d567689c9ce65a529237ff33ed877ac214ac2104 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Fri, 17 Jan 2020 14:32:50 -0500 Subject: [PATCH 15/94] Fixed regressions. --- .../rasterframes/functions/AggregateFunctionsSpec.scala | 8 +++++--- ...hmeticFunctionsSpec.scala => LocalFunctionsSpec.scala} | 2 +- .../rasterframes/functions/TileFunctionsSpec.scala | 6 ++++-- project/build.properties | 2 +- 4 files changed, 11 insertions(+), 7 deletions(-) rename core/src/test/scala/org/locationtech/rasterframes/functions/{ArithmeticFunctionsSpec.scala => LocalFunctionsSpec.scala} (99%) diff --git a/core/src/test/scala/org/locationtech/rasterframes/functions/AggregateFunctionsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/functions/AggregateFunctionsSpec.scala index 8295cc112..ae9175446 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/functions/AggregateFunctionsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/functions/AggregateFunctionsSpec.scala @@ -114,9 +114,10 @@ class AggregateFunctionsSpec extends TestEnvironment with RasterMatchers { .toDF("tile") .withColumn("id", monotonically_increasing_id()) - df.select(rf_agg_local_mean($"tile")).first() should be(three.toArrayTile()) + val expected = three.toArrayTile().convert(DoubleConstantNoDataCellType) + df.select(rf_agg_local_mean($"tile")).first() should be(expected) - df.selectExpr("rf_agg_local_mean(tile)").as[Tile].first() should be(three.toArrayTile()) + df.selectExpr("rf_agg_local_mean(tile)").as[Tile].first() should be(expected) noException should be thrownBy { df.groupBy($"id") @@ -139,7 +140,8 @@ class AggregateFunctionsSpec extends TestEnvironment with RasterMatchers { val t2 = df.selectExpr("rf_agg_local_no_data_cells(tile) as cnt").select($"cnt".as[Tile]).first() t1 should be(t2) val t3 = df.select(rf_local_add(rf_agg_local_data_cells($"tile"), rf_agg_local_no_data_cells($"tile"))).as[Tile].first() - t3 should be(three.toArrayTile()) + val expected = three.toArrayTile().convert(IntConstantNoDataCellType) + t3 should be(expected) checkDocs("rf_agg_local_no_data_cells") } } diff --git a/core/src/test/scala/org/locationtech/rasterframes/functions/ArithmeticFunctionsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/functions/LocalFunctionsSpec.scala similarity index 99% rename from core/src/test/scala/org/locationtech/rasterframes/functions/ArithmeticFunctionsSpec.scala rename to core/src/test/scala/org/locationtech/rasterframes/functions/LocalFunctionsSpec.scala index 0c698c22e..cb11b06d6 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/functions/ArithmeticFunctionsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/functions/LocalFunctionsSpec.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.functions._ import org.locationtech.rasterframes.expressions.accessors.ExtractTile import org.locationtech.rasterframes.tiles.ProjectedRasterTile import org.locationtech.rasterframes._ -class ArithmeticFunctionsSpec extends TestEnvironment with RasterMatchers { +class LocalFunctionsSpec extends TestEnvironment with RasterMatchers { import TestData._ import spark.implicits._ diff --git a/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala index d0ce1c65b..2518bf604 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala @@ -29,6 +29,7 @@ import javax.imageio.ImageIO import org.apache.spark.sql.Encoders import org.apache.spark.sql.functions.sum import org.locationtech.rasterframes._ +import org.locationtech.rasterframes.ref.RasterRef import org.locationtech.rasterframes.tiles.ProjectedRasterTile import org.locationtech.rasterframes.util.ColorRampNames @@ -235,14 +236,15 @@ class TileFunctionsSpec extends TestEnvironment with RasterMatchers { g should be(extent.toPolygon()) checkDocs("rf_geometry") } + implicit val enc = Encoders.tuple(Encoders.scalaInt, RasterRef.rrEncoder) it("should get the CRS of a RasterRef") { - val e = Seq(Tuple1(TestData.rasterRef)).toDF("ref").select(rf_crs($"ref")).first() + val e = Seq((1, TestData.rasterRef)).toDF("index", "ref").select(rf_crs($"ref")).first() e should be(rasterRef.crs) } it("should get the Extent of a RasterRef") { - val e = Seq(Tuple1(rasterRef)).toDF("ref").select(rf_extent($"ref")).first() + val e = Seq((1, rasterRef)).toDF("index", "ref").select(rf_extent($"ref")).first() e should be(rasterRef.extent) } } diff --git a/project/build.properties b/project/build.properties index 5a9ed9251..a82bb05e1 100644 --- a/project/build.properties +++ b/project/build.properties @@ -1 +1 @@ -sbt.version=1.3.4 +sbt.version=1.3.7 From a9e2a4c1a76279fe78fee57dc49421dc8a592f0d Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Sat, 18 Jan 2020 12:47:09 -0500 Subject: [PATCH 16/94] Upgraded to GeoTrellis 3.2. Regressions. --- .../locationtech/rasterframes/TestEnvironment.scala | 7 +++++-- docs/src/main/paradox/release-notes.md | 5 +++-- project/RFDependenciesPlugin.scala | 2 +- pyrasterframes/src/main/python/docs/sjoin.pymd | 11 ----------- .../src/main/python/pyrasterframes/rf_ipython.py | 5 +++-- 5 files changed, 12 insertions(+), 18 deletions(-) delete mode 100644 pyrasterframes/src/main/python/docs/sjoin.pymd diff --git a/core/src/test/scala/org/locationtech/rasterframes/TestEnvironment.scala b/core/src/test/scala/org/locationtech/rasterframes/TestEnvironment.scala index b89dffe8a..012b4ac91 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/TestEnvironment.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/TestEnvironment.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.functions.col import org.apache.spark.sql.types.StructType import org.apache.spark.{SparkConf, SparkContext} import org.locationtech.jts.geom.Geometry +import org.locationtech.rasterframes.ref.RasterRef import org.locationtech.rasterframes.tiles.ProjectedRasterTile import org.locationtech.rasterframes.util._ import org.scalactic.Tolerance @@ -131,6 +132,8 @@ trait TestEnvironment extends FunSpec docs shouldNot include("N/A") } - implicit def pairEnc: Encoder[(ProjectedRasterTile, ProjectedRasterTile)] = Encoders.tuple(ProjectedRasterTile.prtEncoder, ProjectedRasterTile.prtEncoder) - implicit def tripEnc: Encoder[(ProjectedRasterTile, ProjectedRasterTile, ProjectedRasterTile)] = Encoders.tuple(ProjectedRasterTile.prtEncoder, ProjectedRasterTile.prtEncoder, ProjectedRasterTile.prtEncoder) + implicit def prt2Enc: Encoder[(ProjectedRasterTile, ProjectedRasterTile)] = Encoders.tuple(ProjectedRasterTile.prtEncoder, ProjectedRasterTile.prtEncoder) + implicit def prt3Enc: Encoder[(ProjectedRasterTile, ProjectedRasterTile, ProjectedRasterTile)] = Encoders.tuple(ProjectedRasterTile.prtEncoder, ProjectedRasterTile.prtEncoder, ProjectedRasterTile.prtEncoder) + implicit def rr2Enc: Encoder[(RasterRef, RasterRef)] = Encoders.tuple(RasterRef.rrEncoder, RasterRef.rrEncoder) + implicit def rr3Enc: Encoder[(RasterRef, RasterRef, RasterRef)] = Encoders.tuple(RasterRef.rrEncoder, RasterRef.rrEncoder, RasterRef.rrEncoder) } \ No newline at end of file diff --git a/docs/src/main/paradox/release-notes.md b/docs/src/main/paradox/release-notes.md index 0897b604c..ece67d3ef 100644 --- a/docs/src/main/paradox/release-notes.md +++ b/docs/src/main/paradox/release-notes.md @@ -4,7 +4,7 @@ ### 0.9.0 -* Upgraded to GeoTrellis 3.1.0. This includes a number of _breaking_ changes enumerated as a part of the [PR's](https://github.com/locationtech/rasterframes/pull/398) change log. These include: +* Upgraded to GeoTrellis 3.2.0. This includes a number of _breaking_ changes enumerated as a part of the [PR's](https://github.com/locationtech/rasterframes/pull/398) change log. These include: - Add `Int` type parameter to `Grid` - Add `Int` type parameter to `CellGrid` - Add `Int` type parameter to `GridBounds`... or `TileBounds` @@ -22,7 +22,8 @@ - Revisit use of `Tile` equality since [it's more strict](https://github.com/locationtech/geotrellis/pull/2991) - Update `reference.conf` to use `geotrellis.raster.gdal` namespace. - Replace all uses of `TileDimensions` with `geotrellis.raster.Dimensions[Int]`. -* Formally abandon support for Python 2. Python 2 is dead. Long live Python 2. +* Formally abandoned support for Python 2. Python 2 is dead. Long live Python 2. +* Introduction of type hints in Python API. ## 0.8.x diff --git a/project/RFDependenciesPlugin.scala b/project/RFDependenciesPlugin.scala index 0fe60b956..d4432daae 100644 --- a/project/RFDependenciesPlugin.scala +++ b/project/RFDependenciesPlugin.scala @@ -72,7 +72,7 @@ object RFDependenciesPlugin extends AutoPlugin { }, // NB: Make sure to update the Spark version in pyrasterframes/python/setup.py rfSparkVersion := "2.4.4", - rfGeoTrellisVersion := "3.1.0", + rfGeoTrellisVersion := "3.2.0", rfGeoMesaVersion := "2.2.1" ) } diff --git a/pyrasterframes/src/main/python/docs/sjoin.pymd b/pyrasterframes/src/main/python/docs/sjoin.pymd deleted file mode 100644 index bfff6210b..000000000 --- a/pyrasterframes/src/main/python/docs/sjoin.pymd +++ /dev/null @@ -1,11 +0,0 @@ -# Spatial Join - -```python -import geopandas -url = "http://d2ad6b4ur7yvpq.cloudfront.net/naturalearth-3.3.0/ne_110m_land.geojson" -df = geopandas.read_file(url) -df2 = geopandas.read_file(url) - -geopandas.sjoin(df, df2) -``` - diff --git a/pyrasterframes/src/main/python/pyrasterframes/rf_ipython.py b/pyrasterframes/src/main/python/pyrasterframes/rf_ipython.py index d2b4640fb..4af614770 100644 --- a/pyrasterframes/src/main/python/pyrasterframes/rf_ipython.py +++ b/pyrasterframes/src/main/python/pyrasterframes/rf_ipython.py @@ -21,7 +21,7 @@ import pyrasterframes.rf_types from pyrasterframes.rf_types import Tile from shapely.geometry.base import BaseGeometry -import matplotlib.axes.Axes +from matplotlib.axes import Axes import numpy as np from pandas import DataFrame from typing import Optional, Tuple @@ -30,12 +30,13 @@ def plot_tile(tile: Tile, normalize: bool = True, lower_percentile: float = 1., upper_percentile: float = 99., - axis: Optional[matplotlib.axis.Axes] = None, **imshow_args): + axis: Optional[Axes] = None, **imshow_args): """ Display an image of the tile Parameters ---------- + tile: item to plot normalize: if True, will normalize the data between using lower_percentile and upper_percentile as bounds lower_percentile: between 0 and 100 inclusive. From 95af9296385990264d9ec16ab06e1d95d872e75a Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Mon, 20 Jan 2020 12:50:48 -0500 Subject: [PATCH 17/94] Fix #452. Upgrade rasterframes-notebook container version of rtree. Signed-off-by: Jason T. Brown --- rf-notebook/src/main/docker/requirements-nb.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/rf-notebook/src/main/docker/requirements-nb.txt b/rf-notebook/src/main/docker/requirements-nb.txt index edd82bd96..e82e45453 100644 --- a/rf-notebook/src/main/docker/requirements-nb.txt +++ b/rf-notebook/src/main/docker/requirements-nb.txt @@ -8,3 +8,4 @@ folium>=0.10.1,<0.11 geopandas>=0.6.2,<0.7 descartes>=1.1.0,<1.2 pyarrow +rtree>=0.9.2 From b97d3df614d5907cd00a04ad52b77fdb693d0c59 Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Mon, 20 Jan 2020 14:19:15 -0500 Subject: [PATCH 18/94] Add Rescale function to Scala, Python and SQL APIs Signed-off-by: Jason T. Brown --- .../rasterframes/expressions/package.scala | 1 + .../expressions/transformers/Rescale.scala | 113 ++++++++++++++++++ .../transformers/Standardize.scala | 2 +- .../functions/LocalFunctions.scala | 19 +++ .../functions/TileFunctionsSpec.scala | 45 ++++++- docs/src/main/paradox/reference.md | 2 +- .../python/pyrasterframes/rasterfunctions.py | 25 +++- .../main/python/tests/RasterFunctionsTests.py | 43 +++++++ 8 files changed, 240 insertions(+), 10 deletions(-) create mode 100644 core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/Rescale.scala diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/package.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/package.scala index 33deaa80c..b990e7208 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/package.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/package.scala @@ -95,6 +95,7 @@ package object expressions { registry.registerExpression[Clip]("rf_local_clip") registry.registerExpression[Where]("rf_where") registry.registerExpression[Standardize]("rf_standardize") + registry.registerExpression[Rescale]("rf_rescale") registry.registerExpression[Sum]("rf_tile_sum") registry.registerExpression[Round]("rf_round") registry.registerExpression[Abs]("rf_abs") diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/Rescale.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/Rescale.scala new file mode 100644 index 000000000..4f733323d --- /dev/null +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/Rescale.scala @@ -0,0 +1,113 @@ +/* + * This software is licensed under the Apache 2 license, quoted below. + * + * Copyright 2019 Astraea, Inc. + * + * 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. + * + * SPDX-License-Identifier: Apache-2.0 + * + */ + +package org.locationtech.rasterframes.expressions.transformers + +import geotrellis.raster.{FloatConstantNoDataCellType, Tile} +import org.apache.spark.sql.Column +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionDescription, TernaryExpression} +import org.apache.spark.sql.functions.lit +import org.apache.spark.sql.rf.TileUDT +import org.apache.spark.sql.types.DataType +import org.locationtech.rasterframes.encoders.CatalystSerializer._ +import org.locationtech.rasterframes.expressions.DynamicExtractors._ +import org.locationtech.rasterframes.expressions._ +import org.locationtech.rasterframes.expressions.localops.Clip +import org.locationtech.rasterframes.expressions.tilestats.TileStats + +@ExpressionDescription( + usage = "_FUNC_(tile, min, max) - Rescale cell values such that the minimum is zero and the maximum is one. Other values will be linearly interpolated into the range. If specified, the `min` parameter will become the zero value and the `max` parameter will become 1. Values outside the range will be clipped to 0 or 1. If `min` and `max` are not specified, the tile-wise minimum and maximum are used; this can result in inconsistent values across rows in a tile column.", + arguments = """ + Arguments: + * tile - tile column to extract values + * min - cell value that will become 0; values below this are clipped to 0 + * max - cell value that will become 1; values above this are clipped to 1 + """, + examples = """ + Examples: + > SELECT _FUNC_(tile, lit(-2.2), lit(2.2)) + ...""" +) +case class Rescale(child1: Expression, child2: Expression, child3: Expression) extends TernaryExpression with CodegenFallback with Serializable { + override val nodeName: String = "rf_rescale" + + override def children: Seq[Expression] = Seq(child1, child2, child3) + + override def dataType: DataType = child1.dataType + + override def checkInputDataTypes(): TypeCheckResult = + if(!tileExtractor.isDefinedAt(child1.dataType)) { + TypeCheckFailure(s"Input type '${child1.dataType}' does not conform to a raster type.") + } else if (!doubleArgExtractor.isDefinedAt(child2.dataType)) { + TypeCheckFailure(s"Input type '${child2.dataType}' isn't numeric type.") + } else if (!doubleArgExtractor.isDefinedAt(child3.dataType)) { + TypeCheckFailure(s"Input type '${child3.dataType}' isn't numeric type." ) + } else TypeCheckSuccess + + + override protected def nullSafeEval(input1: Any, input2: Any, input3: Any): Any = { + implicit val tileSer = TileUDT.tileSerializer + val (childTile, childCtx) = tileExtractor(child1.dataType)(row(input1)) + + val min = doubleArgExtractor(child2.dataType)(input2).value + + val max = doubleArgExtractor(child3.dataType)(input3).value + + val result = op(childTile, min, max) + + childCtx match { + case Some(ctx) => ctx.toProjectRasterTile(result).toInternalRow + case None => result.toInternalRow + } + } + + protected def op(tile: Tile, min: Double, max: Double): Tile = { + // convert tile to float if not + // clip to min and max + // "normalize" linearlly rescale to 0,1 range + tile.convert(FloatConstantNoDataCellType) + .localMin(max) // See Clip + .localMax(min) + .normalize(min, max, 0.0, 1.0) + } + +} + +object Rescale { + def apply(tile: Column, min: Column, max: Column): Column = + new Column(Rescale(tile.expr, min.expr, max.expr)) + + def apply(tile: Column, min: Double, max: Double): Column = + new Column(Rescale(tile.expr, lit(min).expr, lit(max).expr)) + + def apply(tile: Column): Column = { + val stats = TileStats(tile) + val min = stats.getField("min").expr + val max = stats.getField("max").expr + + new Column(Rescale(tile.expr, min, max)) + } +} + + diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/Standardize.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/Standardize.scala index 5fcf7cbc9..e1d1aaa87 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/Standardize.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/Standardize.scala @@ -71,7 +71,7 @@ case class Standardize(child1: Expression, child2: Expression, child3: Expressio val mean = doubleArgExtractor(child2.dataType)(input2).value - val stdDev = doubleArgExtractor(child2.dataType)(input3).value + val stdDev = doubleArgExtractor(child3.dataType)(input3).value childCtx match { case Some(ctx) => ctx.toProjectRasterTile(op(childTile, mean, stdDev)).toInternalRow diff --git a/core/src/main/scala/org/locationtech/rasterframes/functions/LocalFunctions.scala b/core/src/main/scala/org/locationtech/rasterframes/functions/LocalFunctions.scala index 05c65200e..16961377f 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/functions/LocalFunctions.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/functions/LocalFunctions.scala @@ -98,6 +98,25 @@ trait LocalFunctions { * Each tile will be standardized according to the statistics of its cell values; this can result in inconsistent values across rows in a tile column. */ def rf_standardize(tile: Column): Column = Standardize(tile) + + /** Rescale cell values such that the minimum is zero and the maximum is one. Other values will be linearly interpolated into the range. + * Cells with the tile-wise minimum value will become the zero value and those at the tile-wise maximum value will become 1. + * This can result in inconsistent values across rows in a tile column. + */ + def rf_rescale(tile: Column): Column = Rescale(tile) + + /** Rescale cell values such that the minimum is zero and the maximum is one. Other values will be linearly interpolated into the range. + * The `min` parameter will become the zero value and the `max` parameter will become 1. + * Values outside the range will be clipped to 0 or 1. + */ + def rf_rescale(tile: Column, min: Column, max: Column): Column = Rescale(tile, min, max) + + /** Rescale cell values such that the minimum is zero and the maximum is one. Other values will be linearly interpolated into the range. + * The `min` parameter will become the zero value and the `max` parameter will become 1. + * Values outside the range will be clipped to 0 or 1. + */ + def rf_rescale(tile: Column, min: Double, max: Double): Column = Rescale(tile, min, max) + /** Perform an arbitrary GeoTrellis `LocalTileBinaryOp` between two Tile columns. */ def rf_local_algebra(op: LocalTileBinaryOp, left: Column, right: Column): TypedColumn[Any, Tile] = withTypedAlias(opName(op), left, right)(udf[Tile, Tile, Tile](op.apply).apply(left, right)) diff --git a/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala index 21f20771d..ac3a0aa5d 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala @@ -280,11 +280,11 @@ class TileFunctionsSpec extends TestEnvironment with RasterMatchers { } } - describe("standardize and normalize") { + describe("standardize and rescale") { it("should be accssible in SQL API"){ checkDocs("rf_standardize") -// checkDocs("rf_normalize") + checkDocs("rf_rescale") } it("should evaluate rf_standardize") { @@ -302,18 +302,51 @@ class TileFunctionsSpec extends TestEnvironment with RasterMatchers { result.getAs[Double](1) should be (1.0 +- 0.00001) } - it("should evaluate rf_standardize with tile -level stats") { - - val df = Seq(randPRT).toDF("tile") + it("should evaluate rf_standardize with tile-level stats") { + // this tile should already be Z distributed. + val df = Seq(randDoubleTile).toDF("tile") val result = df.select(rf_standardize($"tile") as "z") .select(rf_tile_stats($"z") as "zstat") .select($"zstat.mean", $"zstat.variance") .first() - result.getAs[Double](0) should be (0.0 +- 0.02) + result.getAs[Double](0) should be (0.0 +- 0.00001) result.getAs[Double](1) should be (1.0 +- 0.00001) } + it("should evaluate rf_rescale") { + import org.apache.spark.sql.functions.{min, max} + val df = Seq(randPRT, six, one).toDF("tile") + val stats = df.agg(rf_agg_stats($"tile").alias("stat")).select($"stat.min", $"stat.max") + .first() + + val result = df.select( + rf_rescale($"tile", stats.getDouble(0), stats.getDouble(1)).alias("t") + ) + .agg( + max(rf_tile_min($"t")), + min(rf_tile_max($"t")), + rf_agg_stats($"t").getField("min"), + rf_agg_stats($"t").getField("max")) + .first() + + result.getDouble(0) should be > (0.0) + result.getDouble(1) should be < (1.0) + result.getDouble(2) should be (0.0 +- 1e-8) + result.getDouble(3) should be (1.0 +- 1e-8) + + } + + it("should evaluate rf_rescale with tile-level stats") { + val df = Seq(randDoubleTile).toDF("tile") + val result = df.select(rf_rescale($"tile") as "t") + .select(rf_tile_stats($"t") as "tstat") + .select($"tstat.min", $"tstat.max") + .first() + result.getAs[Double](0) should be (0.0 +- 1e-8) + result.getAs[Double](1) should be (1.0 +- 1e-8) + } + } describe("raster metadata") { diff --git a/docs/src/main/paradox/reference.md b/docs/src/main/paradox/reference.md index a481de789..8f7cd4855 100644 --- a/docs/src/main/paradox/reference.md +++ b/docs/src/main/paradox/reference.md @@ -578,7 +578,7 @@ Computes the sum of cells in each row of column `tile`, ignoring NoData values. Computes the mean of cells in each row of column `tile`, ignoring NoData values. -### rf_tile_min +### rf_tile_min Double rf_tile_min(Tile tile) diff --git a/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py b/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py index 14f8c787e..495100f10 100644 --- a/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py +++ b/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py @@ -634,7 +634,7 @@ def rf_where(condition, x, y): return _apply_column_function('rf_where', condition, x, y) -def rf_standardize(tile, mean, stddev): +def rf_standardize(tile, mean=None, stddev=None): """ Standardize cell values such that the mean is zero and the standard deviation is one. If specified, the `mean` and `stddev` are applied to all tiles in the column. @@ -645,7 +645,28 @@ def rf_standardize(tile, mean, stddev): mean = lit(mean) if isinstance(stddev, (int, float)): stddev = lit(stddev) - return _apply_column_function('rf_standardize', tile, mean, stddev) + if mean is None and stddev is None: + return _apply_column_function('rf_standardize', tile) + if mean is not None and stddev is not None: + return _apply_column_function('rf_standardize', tile, mean, stddev) + raise ValueError('Either `mean` or `stddev` should both be specified or omitted in call to rf_standardize.') + + +def rf_rescale(tile, min=None, max=None): + """ + Rescale cell values such that the minimum is zero and the maximum is one. Other values will be linearly interpolated into the range. + If specified, the `min` parameter will become the zero value and the `max` parameter will become 1. See @ref:[`rf_agg_stats`](reference.md#rf_agg_stats). + Values outside the range will be clipped to 0 or 1. + If `min` and `max` are not specified, the __tile-wise__ minimum and maximum are used; this can result in inconsistent values across rows in a tile column. + """ + if isinstance(min, (int, float)): + min = lit(min) + if isinstance(max, (int, float)): + max = lit(max) + if min is None and max is None: + return _apply_column_function('rf_rescale', tile) + if min is not None and max is not None: + return _apply_column_function('rf_rescale', tile, min, max) def rf_round(tile_col): diff --git a/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py b/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py index fd10a9eac..f54799ec2 100644 --- a/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py +++ b/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py @@ -558,6 +558,49 @@ def test_rf_standardize(self): self.assertAlmostEqual(result[0], 0.0) self.assertAlmostEqual(result[1], 1.0) + def test_rf_standardize_per_tile(self): + + # 10k samples so should be pretty stable + x = Tile(np.random.randint(-20, 0, (100, 100)), CellType.int8()) + df = self.spark.createDataFrame([Row(x=x)]) + + result = df.select(rf_standardize('x').alias('z')) \ + .select(rf_agg_stats('z').alias('z_stat')) \ + .select('z_stat.mean', 'z_stat.variance') \ + .first() + + self.assertAlmostEqual(result[0], 0.0) + self.assertAlmostEqual(result[1], 1.0) + + def test_rf_rescale(self): + + x1 = Tile(np.random.randint(-20, 42, (10, 10)), CellType.int8()) + x2 = Tile(np.random.randint(20, 242, (10, 10)), CellType.int8()) + df = self.spark.createDataFrame([Row(x=x1), Row(x=x2)]) + result = df.select(rf_rescale('x').alias('x_prime')) \ + .agg(rf_agg_stats('x_prime').alias('stat')) \ + .select('stat.min', 'stat.max') \ + .first() + + self.assertEqual(result[0], 0) + self.assertEqual(result[1], 1) + + def test_rf_rescale_per_tile(self): + from pyspark.sql.functions import min as F_min + from pyspark.sql.functions import max as F_max + + x1 = Tile(np.random.randint(-20, 42, (10, 10)), CellType.int8()) + x2 = Tile(np.random.randint(20, 242, (10, 10)), CellType.int8()) + df = self.spark.createDataFrame([Row(x=x1), Row(x=x2)]) + result = df.select(rf_rescale('x').alias('x_prime')) \ + .agg( + F_max(rf_tile_min('x_prime')), + F_min(rf_tile_max('x_prime')) + ).first() + + self.assertGreater(result[0], 0) + self.assertLess(result[1], 1) + def test_rf_agg_overview_raster(self): width = 500 height = 400 From 9a827ad6a419a78531184a1e61883a7a5394da85 Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Mon, 20 Jan 2020 17:02:28 -0500 Subject: [PATCH 19/94] rf_rescale tests and refinements Signed-off-by: Jason T. Brown --- .../expressions/transformers/Rescale.scala | 5 +-- .../functions/TileFunctionsSpec.scala | 8 ++-- .../python/pyrasterframes/rasterfunctions.py | 5 ++- .../main/python/tests/RasterFunctionsTests.py | 38 ++++++++++--------- 4 files changed, 30 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/Rescale.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/Rescale.scala index 4f733323d..4bde81305 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/Rescale.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/Rescale.scala @@ -33,7 +33,6 @@ import org.apache.spark.sql.types.DataType import org.locationtech.rasterframes.encoders.CatalystSerializer._ import org.locationtech.rasterframes.expressions.DynamicExtractors._ import org.locationtech.rasterframes.expressions._ -import org.locationtech.rasterframes.expressions.localops.Clip import org.locationtech.rasterframes.expressions.tilestats.TileStats @ExpressionDescription( @@ -60,9 +59,9 @@ case class Rescale(child1: Expression, child2: Expression, child3: Expression) e if(!tileExtractor.isDefinedAt(child1.dataType)) { TypeCheckFailure(s"Input type '${child1.dataType}' does not conform to a raster type.") } else if (!doubleArgExtractor.isDefinedAt(child2.dataType)) { - TypeCheckFailure(s"Input type '${child2.dataType}' isn't numeric type.") + TypeCheckFailure(s"Input type '${child2.dataType}' isn't floating point type.") } else if (!doubleArgExtractor.isDefinedAt(child3.dataType)) { - TypeCheckFailure(s"Input type '${child3.dataType}' isn't numeric type." ) + TypeCheckFailure(s"Input type '${child3.dataType}' isn't floating point type." ) } else TypeCheckSuccess diff --git a/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala index ac3a0aa5d..c9b947f90 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala @@ -332,8 +332,8 @@ class TileFunctionsSpec extends TestEnvironment with RasterMatchers { result.getDouble(0) should be > (0.0) result.getDouble(1) should be < (1.0) - result.getDouble(2) should be (0.0 +- 1e-8) - result.getDouble(3) should be (1.0 +- 1e-8) + result.getDouble(2) should be (0.0 +- 1e-7) + result.getDouble(3) should be (1.0 +- 1e-7) } @@ -343,8 +343,8 @@ class TileFunctionsSpec extends TestEnvironment with RasterMatchers { .select(rf_tile_stats($"t") as "tstat") .select($"tstat.min", $"tstat.max") .first() - result.getAs[Double](0) should be (0.0 +- 1e-8) - result.getAs[Double](1) should be (1.0 +- 1e-8) + result.getAs[Double](0) should be (0.0 +- 1e-7) + result.getAs[Double](1) should be (1.0 +- 1e-7) } } diff --git a/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py b/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py index 495100f10..95f9b95a3 100644 --- a/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py +++ b/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py @@ -660,13 +660,14 @@ def rf_rescale(tile, min=None, max=None): If `min` and `max` are not specified, the __tile-wise__ minimum and maximum are used; this can result in inconsistent values across rows in a tile column. """ if isinstance(min, (int, float)): - min = lit(min) + min = lit(float(min)) if isinstance(max, (int, float)): - max = lit(max) + max = lit(float(max)) if min is None and max is None: return _apply_column_function('rf_rescale', tile) if min is not None and max is not None: return _apply_column_function('rf_rescale', tile, min, max) + raise ValueError('Either `min` or `max` should both be specified or omitted in call to rf_rescale.') def rf_round(tile_col): diff --git a/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py b/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py index f54799ec2..4b4212217 100644 --- a/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py +++ b/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py @@ -573,33 +573,37 @@ def test_rf_standardize_per_tile(self): self.assertAlmostEqual(result[1], 1.0) def test_rf_rescale(self): + from pyspark.sql.functions import min as F_min + from pyspark.sql.functions import max as F_max - x1 = Tile(np.random.randint(-20, 42, (10, 10)), CellType.int8()) - x2 = Tile(np.random.randint(20, 242, (10, 10)), CellType.int8()) + x1 = Tile(np.random.randint(-60, 12, (10, 10)), CellType.int8()) + x2 = Tile(np.random.randint(15, 122, (10, 10)), CellType.int8()) df = self.spark.createDataFrame([Row(x=x1), Row(x=x2)]) - result = df.select(rf_rescale('x').alias('x_prime')) \ - .agg(rf_agg_stats('x_prime').alias('stat')) \ - .select('stat.min', 'stat.max') \ - .first() + # Note there will be some clipping + rescaled = df.select(rf_rescale('x', -20, 50).alias('x_prime'), 'x') + result = rescaled \ + .agg( + F_max(rf_tile_min('x_prime')), + F_min(rf_tile_max('x_prime')) + ).first() - self.assertEqual(result[0], 0) - self.assertEqual(result[1], 1) + self.assertGreater(result[0], 0.0, f'Expected max tile_min to be > 0 (strictly); but it is ' + f'{rescaled.select("x", "x_prime", rf_tile_min("x_prime")).take(2)}') + self.assertLess(result[1], 1.0, f'Expected min tile_max to be < 1 (strictly); it is' + f'{rescaled.select(rf_tile_max("x_prime")).take(2)}') def test_rf_rescale_per_tile(self): - from pyspark.sql.functions import min as F_min - from pyspark.sql.functions import max as F_max - x1 = Tile(np.random.randint(-20, 42, (10, 10)), CellType.int8()) x2 = Tile(np.random.randint(20, 242, (10, 10)), CellType.int8()) df = self.spark.createDataFrame([Row(x=x1), Row(x=x2)]) result = df.select(rf_rescale('x').alias('x_prime')) \ - .agg( - F_max(rf_tile_min('x_prime')), - F_min(rf_tile_max('x_prime')) - ).first() + .agg(rf_agg_stats('x_prime').alias('stat')) \ + .select('stat.min', 'stat.max') \ + .first() + + self.assertEqual(result[0], 0.0) + self.assertEqual(result[1], 1.0) - self.assertGreater(result[0], 0) - self.assertLess(result[1], 1) def test_rf_agg_overview_raster(self): width = 500 From 4a07392b8a8cce9fec5474c9883eac29eed8fb5f Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Tue, 21 Jan 2020 14:37:55 -0500 Subject: [PATCH 20/94] Use rf_local_clip and other viz functions in supervised learning doc page Signed-off-by: Jason T. Brown --- .../main/python/docs/supervised-learning.pymd | 38 ++++++++----------- 1 file changed, 16 insertions(+), 22 deletions(-) diff --git a/pyrasterframes/src/main/python/docs/supervised-learning.pymd b/pyrasterframes/src/main/python/docs/supervised-learning.pymd index 6304432ca..ad749f829 100644 --- a/pyrasterframes/src/main/python/docs/supervised-learning.pymd +++ b/pyrasterframes/src/main/python/docs/supervised-learning.pymd @@ -202,26 +202,20 @@ retiled = scored \ retiled.printSchema() ``` -Take a look at a sample of the resulting output and the corresponding area's red-green-blue composite image. -Recall the label coding: 1 is forest (purple), 2 is cropland (green) and 3 is developed areas(yellow). - -```python, display_rgb -scaling_quantiles = retiled.agg(rf_agg_approx_quantiles( -sample = retiled \ - .select('prediction', 'red', 'grn', 'blu') - -sample_rgb = np.concatenate([sample['red'].cells[:, :, None], - sample['grn'].cells[ :, :, None], - sample['blu'].cells[ :, :, None]], axis=2) -# plot scaled RGB -scaling_quantiles = np.nanpercentile(sample_rgb, [3.00, 97.00], axis=(0,1)) -scaled = np.clip(sample_rgb, scaling_quantiles[0, :], scaling_quantiles[1, :]) -scaled -= scaling_quantiles[0, :] -scaled /= (scaling_quantiles[1, : ] - scaling_quantiles[0, :]) - -fig, (ax1, ax2) = plt.subplots(1, 2, figsize=(10, 5)) -ax1.imshow(scaled) - -# display prediction -ax2.imshow(sample['prediction'].cells) +Take a look at a sample of the resulting prediction and the corresponding area's red-green-blue composite image. Note that because each `prediction` tile is rendered independently, the colors may not have the same meaning across rows. + +```python +scaling_quantiles = retiled.agg( + rf_agg_approx_quantiles('red', [0.03, 0.97]).alias('red_q'), + rf_agg_approx_quantiles('grn', [0.03, 0.97]).alias('grn_q'), + rf_agg_approx_quantiles('blu', [0.03, 0.97]).alias('blu_q') + ).first() +clipped = retiled.select( + rf_render_png( + rf_local_clip('red', *scaling_quantiles['red_q']).alias('red'), + rf_local_clip('grn', *scaling_quantiles['grn_q']).alias('grn'), + rf_local_clip('blu', *scaling_quantiles['blu_q']).alias('blu') + ).alias('tci'), + rf_render_color_ramp_png('prediction', 'ClassificationBoldLandUse').alias('prediction') + ) ``` From b1bcb016f76dc211d7075da5f45d71a740f3e88e Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Wed, 22 Jan 2020 13:55:59 -0500 Subject: [PATCH 21/94] Python deprecation lib should be install_requires not setup_requires Signed-off-by: Jason T. Brown --- pyrasterframes/src/main/python/setup.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pyrasterframes/src/main/python/setup.py b/pyrasterframes/src/main/python/setup.py index 876a31b0c..26588565b 100644 --- a/pyrasterframes/src/main/python/setup.py +++ b/pyrasterframes/src/main/python/setup.py @@ -170,7 +170,8 @@ def dest_file(self, src_file): shapely, pyspark, numpy, - pandas + pandas, + deprecation, ], setup_requires=[ pytz, @@ -189,7 +190,6 @@ def dest_file(self, src_file): fiona, rasterio, folium, - deprecation ], tests_require=[ pytest, From e0393cfae318dd2b178cce7977a593ad8e588478 Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Wed, 29 Jan 2020 15:59:45 -0500 Subject: [PATCH 22/94] Refactor clip to clamp Signed-off-by: Jason T. Brown --- .../localops/{Clip.scala => Clamp.scala} | 16 +++++++-------- .../rasterframes/expressions/package.scala | 2 +- .../expressions/transformers/Rescale.scala | 8 ++++---- .../functions/LocalFunctions.scala | 20 +++++++++---------- .../functions/TileFunctionsSpec.scala | 8 ++++---- docs/src/main/paradox/reference.md | 14 ++++++------- .../main/python/docs/supervised-learning.pymd | 17 ++++++++-------- .../python/pyrasterframes/rasterfunctions.py | 8 ++++---- .../main/python/tests/RasterFunctionsTests.py | 4 ++-- 9 files changed, 49 insertions(+), 48 deletions(-) rename core/src/main/scala/org/locationtech/rasterframes/expressions/localops/{Clip.scala => Clamp.scala} (89%) diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Clip.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Clamp.scala similarity index 89% rename from core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Clip.scala rename to core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Clamp.scala index 77ede42b3..68b3ee516 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Clip.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Clamp.scala @@ -13,7 +13,7 @@ import org.locationtech.rasterframes.expressions.DynamicExtractors._ import org.locationtech.rasterframes.expressions.row @ExpressionDescription( - usage = "_FUNC_(tile, min, max) - Return the tile with its values clipped to a range defined by min and max," + + usage = "_FUNC_(tile, min, max) - Return the tile with its values limited to a range defined by min and max," + " doing so cellwise if min or max are tile type", arguments = """ Arguments: @@ -21,13 +21,13 @@ import org.locationtech.rasterframes.expressions.row * min - scalar or tile setting the minimum value for each cell * max - scalar or tile setting the maximum value for each cell""" ) -case class Clip(left: Expression, middle: Expression, right: Expression) +case class Clamp(left: Expression, middle: Expression, right: Expression) extends TernaryExpression with CodegenFallback with Serializable { override def dataType: DataType = left.dataType override def children: Seq[Expression] = Seq(left, middle, right) - override val nodeName = "rf_clip" + override val nodeName = "rf_local_clamp" override def checkInputDataTypes(): TypeCheckResult = { if (!tileExtractor.isDefinedAt(left.dataType)) { @@ -65,10 +65,10 @@ case class Clip(left: Expression, middle: Expression, right: Expression) } } -object Clip { - def apply(tile: Column, min: Column, max: Column): Column = new Column(Clip(tile.expr, min.expr, max.expr)) - def apply[N: Numeric](tile: Column, min: N, max: Column): Column = new Column(Clip(tile.expr, lit(min).expr, max.expr)) - def apply[N: Numeric](tile: Column, min: Column, max: N): Column = new Column(Clip(tile.expr, min.expr, lit(max).expr)) - def apply[N: Numeric](tile: Column, min: N, max: N): Column = new Column(Clip(tile.expr, lit(min).expr, lit(max).expr)) +object Clamp { + def apply(tile: Column, min: Column, max: Column): Column = new Column(Clamp(tile.expr, min.expr, max.expr)) + def apply[N: Numeric](tile: Column, min: N, max: Column): Column = new Column(Clamp(tile.expr, lit(min).expr, max.expr)) + def apply[N: Numeric](tile: Column, min: Column, max: N): Column = new Column(Clamp(tile.expr, min.expr, lit(max).expr)) + def apply[N: Numeric](tile: Column, min: N, max: N): Column = new Column(Clamp(tile.expr, lit(min).expr, lit(max).expr)) } diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/package.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/package.scala index b990e7208..901073797 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/package.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/package.scala @@ -92,7 +92,7 @@ package object expressions { registry.registerExpression[Defined]("rf_local_data") registry.registerExpression[Min]("rf_local_min") registry.registerExpression[Max]("rf_local_max") - registry.registerExpression[Clip]("rf_local_clip") + registry.registerExpression[Clamp]("rf_local_clamp") registry.registerExpression[Where]("rf_where") registry.registerExpression[Standardize]("rf_standardize") registry.registerExpression[Rescale]("rf_rescale") diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/Rescale.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/Rescale.scala index 4bde81305..9ceb3bdd0 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/Rescale.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/Rescale.scala @@ -36,12 +36,12 @@ import org.locationtech.rasterframes.expressions._ import org.locationtech.rasterframes.expressions.tilestats.TileStats @ExpressionDescription( - usage = "_FUNC_(tile, min, max) - Rescale cell values such that the minimum is zero and the maximum is one. Other values will be linearly interpolated into the range. If specified, the `min` parameter will become the zero value and the `max` parameter will become 1. Values outside the range will be clipped to 0 or 1. If `min` and `max` are not specified, the tile-wise minimum and maximum are used; this can result in inconsistent values across rows in a tile column.", + usage = "_FUNC_(tile, min, max) - Rescale cell values such that the minimum is zero and the maximum is one. Other values will be linearly interpolated into the range. If specified, the `min` parameter will become the zero value and the `max` parameter will become 1. Values outside the range will be set to 0 or 1. If `min` and `max` are not specified, the tile-wise minimum and maximum are used; this can result in inconsistent values across rows in a tile column.", arguments = """ Arguments: * tile - tile column to extract values - * min - cell value that will become 0; values below this are clipped to 0 - * max - cell value that will become 1; values above this are clipped to 1 + * min - cell value that will become 0; cells below this are set to 0 + * max - cell value that will become 1; cells above this are set to 1 """, examples = """ Examples: @@ -83,7 +83,7 @@ case class Rescale(child1: Expression, child2: Expression, child3: Expression) e protected def op(tile: Tile, min: Double, max: Double): Tile = { // convert tile to float if not - // clip to min and max + // clamp to min and max // "normalize" linearlly rescale to 0,1 range tile.convert(FloatConstantNoDataCellType) .localMin(max) // See Clip diff --git a/core/src/main/scala/org/locationtech/rasterframes/functions/LocalFunctions.scala b/core/src/main/scala/org/locationtech/rasterframes/functions/LocalFunctions.scala index 16961377f..714253ca7 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/functions/LocalFunctions.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/functions/LocalFunctions.scala @@ -68,17 +68,17 @@ trait LocalFunctions { /** Cellwise maximum between Tiles. */ def rf_local_max[T: Numeric](left: Column, right: T): Column = Max(left, right) - /** Return the tile with its values clipped to a range defined by min and max. */ - def rf_local_clip(tile: Column, min: Column, max: Column) = Clip(tile, min, max) + /** Return the tile with its values limited to a range defined by min and max. */ + def rf_local_clamp(tile: Column, min: Column, max: Column) = Clamp(tile, min, max) - /** Return the tile with its values clipped to a range defined by min and max. */ - def rf_local_clip[T: Numeric](tile: Column, min: T, max: Column) = Clip(tile, min, max) + /** Return the tile with its values limited to a range defined by min and max. */ + def rf_local_clamp[T: Numeric](tile: Column, min: T, max: Column) = Clamp(tile, min, max) - /** Return the tile with its values clipped to a range defined by min and max. */ - def rf_local_clip[T: Numeric](tile: Column, min: Column, max: T) = Clip(tile, min, max) + /** Return the tile with its values limited to a range defined by min and max. */ + def rf_local_clamp[T: Numeric](tile: Column, min: Column, max: T) = Clamp(tile, min, max) - /** Return the tile with its values clipped to a range defined by min and max. */ - def rf_local_clip[T: Numeric](tile: Column, min: T, max: T) = Clip(tile, min, max) + /** Return the tile with its values limited to a range defined by min and max. */ + def rf_local_clamp[T: Numeric](tile: Column, min: T, max: T) = Clamp(tile, min, max) /** Return a tile with cell values chosen from `x` or `y` depending on `condition`. Operates cell-wise in a similar fashion to Spark SQL `when` and `otherwise`. */ @@ -107,13 +107,13 @@ trait LocalFunctions { /** Rescale cell values such that the minimum is zero and the maximum is one. Other values will be linearly interpolated into the range. * The `min` parameter will become the zero value and the `max` parameter will become 1. - * Values outside the range will be clipped to 0 or 1. + * Values outside the range will be set to 0 or 1. */ def rf_rescale(tile: Column, min: Column, max: Column): Column = Rescale(tile, min, max) /** Rescale cell values such that the minimum is zero and the maximum is one. Other values will be linearly interpolated into the range. * The `min` parameter will become the zero value and the `max` parameter will become 1. - * Values outside the range will be clipped to 0 or 1. + * Values outside the range will be set to 0 or 1. */ def rf_rescale(tile: Column, min: Double, max: Double): Column = Rescale(tile, min, max) diff --git a/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala index c9b947f90..59c2cc337 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala @@ -210,11 +210,11 @@ class TileFunctionsSpec extends TestEnvironment with RasterMatchers { } } - describe("tile min max and clip") { + describe("tile min max and clamp") { it("should support SQL API"){ checkDocs("rf_local_min") checkDocs("rf_local_max") - checkDocs("rf_local_clip") + checkDocs("rf_local_clamp") } it("should evaluate rf_local_min") { val df = Seq((randPRT, three)).toDF("tile", "three") @@ -244,9 +244,9 @@ class TileFunctionsSpec extends TestEnvironment with RasterMatchers { .first() result1 should be >= 3.0 } - it("should evaluate rf_local_clip"){ + it("should evaluate rf_local_clamp"){ val df = Seq((randPRT, two, six)).toDF("t", "two", "six") - val result = df.select(rf_local_clip($"t", $"two", $"six") as "t") + val result = df.select(rf_local_clamp($"t", $"two", $"six") as "t") .select(rf_tile_min($"t") as "min", rf_tile_max($"t") as "max") .first() result(0) should be (2) diff --git a/docs/src/main/paradox/reference.md b/docs/src/main/paradox/reference.md index 8f7cd4855..7c5751f2d 100644 --- a/docs/src/main/paradox/reference.md +++ b/docs/src/main/paradox/reference.md @@ -466,14 +466,14 @@ Performs cell-wise minimum two tiles or a tile and a scalar. Performs cell-wise maximum two tiles or a tile and a scalar. -### rf_local_clip +### rf_local_clamp - Tile rf_local_clip(Tile tile, Tile min, Tile max) - Tile rf_local_clip(Tile tile, Numeric min, Tile max) - Tile rf_local_clip(Tile tile, Tile min, Numeric max) - Tile rf_local_clip(Tile tile, Numeric min, Numeric max) + Tile rf_local_clamp(Tile tile, Tile min, Tile max) + Tile rf_local_clamp(Tile tile, Numeric min, Tile max) + Tile rf_local_clamp(Tile tile, Tile min, Numeric max) + Tile rf_local_clamp(Tile tile, Numeric min, Numeric max) -Return the tile with its values clipped to a range defined by min and max, inclusive. +Return the tile with its values limited to a range defined by min and max, inclusive. ### rf_where @@ -489,7 +489,7 @@ Operates cell-wise in a similar fashion to Spark SQL `when` and `otherwise`. Rescale cell values such that the minimum is zero and the maximum is one. Other values will be linearly interpolated into the range. If specified, the `min` parameter will become the zero value and the `max` parameter will become 1. See @ref:[`rf_agg_stats`](reference.md#rf_agg_stats). -Values outside the range will be clipped to 0 or 1. +Values outside the range will be set to 0 or 1. If `min` and `max` are not specified, the __tile-wise__ minimum and maximum are used; this can result in inconsistent values across rows in a tile column. ### rf_standardize diff --git a/pyrasterframes/src/main/python/docs/supervised-learning.pymd b/pyrasterframes/src/main/python/docs/supervised-learning.pymd index ad749f829..4f0cfe0d0 100644 --- a/pyrasterframes/src/main/python/docs/supervised-learning.pymd +++ b/pyrasterframes/src/main/python/docs/supervised-learning.pymd @@ -210,12 +210,13 @@ scaling_quantiles = retiled.agg( rf_agg_approx_quantiles('grn', [0.03, 0.97]).alias('grn_q'), rf_agg_approx_quantiles('blu', [0.03, 0.97]).alias('blu_q') ).first() -clipped = retiled.select( - rf_render_png( - rf_local_clip('red', *scaling_quantiles['red_q']).alias('red'), - rf_local_clip('grn', *scaling_quantiles['grn_q']).alias('grn'), - rf_local_clip('blu', *scaling_quantiles['blu_q']).alias('blu') - ).alias('tci'), - rf_render_color_ramp_png('prediction', 'ClassificationBoldLandUse').alias('prediction') - ) + +retiled.select( + rf_render_png( + rf_local_clamp('red', *scaling_quantiles['red_q']).alias('red'), + rf_local_clamp('grn', *scaling_quantiles['grn_q']).alias('grn'), + rf_local_clamp('blu', *scaling_quantiles['blu_q']).alias('blu') + ).alias('tci'), + rf_render_color_ramp_png('prediction', 'ClassificationBoldLandUse').alias('prediction') + ) ``` diff --git a/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py b/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py index 95f9b95a3..c9b996f29 100644 --- a/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py +++ b/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py @@ -619,13 +619,13 @@ def rf_local_max(tile_col, max): return _apply_column_function('rf_local_max', tile_col, max) -def rf_local_clip(tile_col, min, max): - """Performs cell-wise maximum two tiles or a tile and a scalar.""" +def rf_local_clamp(tile_col, min, max): + """ Return the tile with its values limited to a range defined by min and max, inclusive. """ if isinstance(min, (int, float)): min = lit(min) if isinstance(max, (int, float)): max = lit(max) - return _apply_column_function('rf_local_clip', tile_col, min, max) + return _apply_column_function('rf_local_clamp', tile_col, min, max) def rf_where(condition, x, y): @@ -656,7 +656,7 @@ def rf_rescale(tile, min=None, max=None): """ Rescale cell values such that the minimum is zero and the maximum is one. Other values will be linearly interpolated into the range. If specified, the `min` parameter will become the zero value and the `max` parameter will become 1. See @ref:[`rf_agg_stats`](reference.md#rf_agg_stats). - Values outside the range will be clipped to 0 or 1. + Values outside the range will be set to 0 or 1. If `min` and `max` are not specified, the __tile-wise__ minimum and maximum are used; this can result in inconsistent values across rows in a tile column. """ if isinstance(min, (int, float)): diff --git a/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py b/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py index 4b4212217..176c50b8c 100644 --- a/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py +++ b/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py @@ -504,7 +504,7 @@ def test_rf_local_is_in(self): "Tile value {} should contain two 1s as: [[1, 0, 1],[0, 0, 0]]" .format(result['in_list'].cells)) - def test_local_min_max_clip(self): + def test_local_min_max_clamp(self): tile = Tile(np.random.randint(-20, 20, (10, 10)), CellType.int8()) min_tile = Tile(np.random.randint(-20, 0, (10, 10)), CellType.int8()) max_tile = Tile(np.random.randint(0, 20, (10, 10)), CellType.int8()) @@ -531,7 +531,7 @@ def test_local_min_max_clip(self): ) assert_equal( - df.select(rf_local_clip('t', 'mn', 'mx')).first()[0].cells, + df.select(rf_local_clamp('t', 'mn', 'mx')).first()[0].cells, np.clip(tile.cells, min_tile.cells, max_tile.cells) ) From a89703b96c680eb5701c01db7fb85833f9c54e1a Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Tue, 4 Feb 2020 15:31:49 -0500 Subject: [PATCH 23/94] =?UTF-8?q?Added=20code=20to=20catch=20GDAL=20Except?= =?UTF-8?q?ions=20and=20wrap=20with=20source=20URI.=E2=80=A8?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../locationtech/rasterframes/ref/GDALRasterSource.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/GDALRasterSource.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/GDALRasterSource.scala index 382844012..999f824ad 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/GDALRasterSource.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/GDALRasterSource.scala @@ -21,6 +21,7 @@ package org.locationtech.rasterframes.ref +import java.io.IOException import java.net.URI import com.azavea.gdal.GDALWarp @@ -68,7 +69,12 @@ case class GDALRasterSource(source: URI) extends RFRasterSource with URIRasterSo override def tags: Tags = Tags(metadata, List.empty) override protected def readBounds(bounds: Traversable[GridBounds[Int]], bands: Seq[Int]): Iterator[Raster[MultibandTile]] = - gdal.readBounds(bounds.map(_.toGridType[Long]), bands) + try { + gdal.readBounds(bounds.map(_.toGridType[Long]), bands) + } + catch { + case e: Exception => throw new IOException(s"Error reading '$source'", e) + } } object GDALRasterSource extends LazyLogging { From 124068345de4e8e47aaf1f0c60323bc22ebd2cce Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Tue, 4 Feb 2020 16:09:21 -0500 Subject: [PATCH 24/94] Added test to python build to throw an error if more than one assembly is found. --- project/PythonBuildPlugin.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/project/PythonBuildPlugin.scala b/project/PythonBuildPlugin.scala index 37404ddae..c05950c04 100644 --- a/project/PythonBuildPlugin.scala +++ b/project/PythonBuildPlugin.scala @@ -64,8 +64,14 @@ object PythonBuildPlugin extends AutoPlugin { val pyWhlImp = Def.task { val log = streams.value.log val buildDir = (Python / target).value + + val jars = (buildDir / "deps" / "jars" ** "*.jar").get() + if (jars.size > 1) { + throw new MessageOnlyException("Two assemblies found in the package. Run 'clean'.\n" + jars.mkString(", ")) + } + val retcode = pySetup.toTask(" build bdist_wheel").value - if(retcode != 0) throw new RuntimeException(s"'python setup.py' returned $retcode") + if(retcode != 0) throw new MessageOnlyException(s"'python setup.py' returned $retcode") val whls = (buildDir / "dist" ** "pyrasterframes*.whl").get() require(whls.length == 1, "Running setup.py should have produced a single .whl file. Try running `clean` first.") log.info(s"Python .whl file written to '${whls.head}'") From 63ae5f2d205e49712142f6c335bc37a93c9635ce Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Fri, 7 Feb 2020 15:34:07 -0500 Subject: [PATCH 25/94] Created JP2-specific RasterSource to provide global thread lock as a workaround to locationtech/geotrelli#3184. --- .../ref/DelegatingRasterSource.scala | 2 +- .../rasterframes/ref/GDALRasterSource.scala | 4 +- .../ref/InMemoryRasterSource.scala | 2 +- .../ref/JP2GDALRasterSource.scala | 48 +++++++++++++++++++ .../rasterframes/ref/RFRasterSource.scala | 8 +++- .../ref/RangeReaderRasterSource.scala | 2 +- .../scala/examples/RasterSourceExercise.scala | 47 ++++++++++++++++++ 7 files changed, 106 insertions(+), 7 deletions(-) create mode 100644 core/src/main/scala/org/locationtech/rasterframes/ref/JP2GDALRasterSource.scala create mode 100644 datasource/src/test/scala/examples/RasterSourceExercise.scala diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/DelegatingRasterSource.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/DelegatingRasterSource.scala index 9eb2633a6..1b845d6e4 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/DelegatingRasterSource.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/DelegatingRasterSource.scala @@ -74,7 +74,7 @@ abstract class DelegatingRasterSource(source: URI, delegateBuilder: () => GTRast override def bandCount: Int = info.bandCount override def tags: Tags = info.tags - override protected def readBounds(bounds: Traversable[GridBounds[Int]], bands: Seq[Int]): Iterator[Raster[MultibandTile]] = + override def readBounds(bounds: Traversable[GridBounds[Int]], bands: Seq[Int]): Iterator[Raster[MultibandTile]] = retryableRead(_.readBounds(bounds.map(_.toGridType[Long]), bands)) override def read(bounds: GridBounds[Int], bands: Seq[Int]): Raster[MultibandTile] = diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/GDALRasterSource.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/GDALRasterSource.scala index 999f824ad..47c7037f5 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/GDALRasterSource.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/GDALRasterSource.scala @@ -37,7 +37,7 @@ import org.locationtech.rasterframes.ref.RFRasterSource.URIRasterSource case class GDALRasterSource(source: URI) extends RFRasterSource with URIRasterSource { @transient - private lazy val gdal: VLMRasterSource = { + protected lazy val gdal: VLMRasterSource = { val cleaned = source.toASCIIString .replace("gdal+", "") .replace("gdal:/", "") @@ -68,7 +68,7 @@ case class GDALRasterSource(source: URI) extends RFRasterSource with URIRasterSo override def tags: Tags = Tags(metadata, List.empty) - override protected def readBounds(bounds: Traversable[GridBounds[Int]], bands: Seq[Int]): Iterator[Raster[MultibandTile]] = + override def readBounds(bounds: Traversable[GridBounds[Int]], bands: Seq[Int]): Iterator[Raster[MultibandTile]] = try { gdal.readBounds(bounds.map(_.toGridType[Long]), bands) } diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/InMemoryRasterSource.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/InMemoryRasterSource.scala index 4bb6b7d0b..fb53f3b63 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/InMemoryRasterSource.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/InMemoryRasterSource.scala @@ -41,7 +41,7 @@ case class InMemoryRasterSource(tile: Tile, extent: Extent, crs: CRS) extends RF override def tags: Tags = EMPTY_TAGS - override protected def readBounds(bounds: Traversable[GridBounds[Int]], bands: Seq[Int]): Iterator[Raster[MultibandTile]] = { + override def readBounds(bounds: Traversable[GridBounds[Int]], bands: Seq[Int]): Iterator[Raster[MultibandTile]] = { bounds .map(b => { val subext = rasterExtent.extentFor(b) diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/JP2GDALRasterSource.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/JP2GDALRasterSource.scala new file mode 100644 index 000000000..5a06f9104 --- /dev/null +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/JP2GDALRasterSource.scala @@ -0,0 +1,48 @@ +/* + * This software is licensed under the Apache 2 license, quoted below. + * + * Copyright 2019 Astraea, Inc. + * + * 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. + * + * SPDX-License-Identifier: Apache-2.0 + * + */ + +package org.locationtech.rasterframes.ref + +import java.net.URI + +import geotrellis.raster.{GridBounds, MultibandTile, Raster} + +/** + * Temporary fix for https://github.com/locationtech/geotrellis/issues/3184, providing thread locking over + * wrapped GeoTrellis RasterSource + */ +class JP2GDALRasterSource(source: URI) extends GDALRasterSource(source) { + + override protected def tiffInfo = JP2GDALRasterSource.synchronized { + SimpleRasterInfo(source.toASCIIString, _ => SimpleRasterInfo(gdal)) + } + + override def readBounds(bounds: Traversable[GridBounds[Int]], bands: Seq[Int]): Iterator[Raster[MultibandTile]] = + JP2GDALRasterSource.synchronized { + super.readBounds(bounds, bands) + } +} + +object JP2GDALRasterSource { + def apply(source: URI): JP2GDALRasterSource = new JP2GDALRasterSource(source) +} + + diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/RFRasterSource.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/RFRasterSource.scala index ec4053ec9..6096718df 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/RFRasterSource.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/RFRasterSource.scala @@ -66,7 +66,7 @@ abstract class RFRasterSource extends CellGrid[Int] with ProjectedRasterLike wit def readAll(dims: Dimensions[Int] = NOMINAL_TILE_DIMS, bands: Seq[Int] = SINGLEBAND): Seq[Raster[MultibandTile]] = layoutBounds(dims).map(read(_, bands)) - protected def readBounds(bounds: Traversable[GridBounds[Int]], bands: Seq[Int]): Iterator[Raster[MultibandTile]] + def readBounds(bounds: Traversable[GridBounds[Int]], bands: Seq[Int]): Iterator[Raster[MultibandTile]] def rasterExtent = RasterExtent(extent, cols, rows) @@ -109,7 +109,11 @@ object RFRasterSource extends LazyLogging { def apply(source: URI): RFRasterSource = rsCache.get( source.toASCIIString, _ => source match { - case IsGDAL() => GDALRasterSource(source) + case IsGDAL() => + if (source.getPath.toLowerCase().endsWith("jp2")) + JP2GDALRasterSource(source) + else + GDALRasterSource(source) case IsHadoopGeoTiff() => // TODO: How can we get the active hadoop configuration // TODO: without having to pass it through? diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/RangeReaderRasterSource.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/RangeReaderRasterSource.scala index aaf1ddad2..28854ee7d 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/RangeReaderRasterSource.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/RangeReaderRasterSource.scala @@ -55,7 +55,7 @@ trait RangeReaderRasterSource extends RFRasterSource with GeoTiffInfoSupport { override def tags: Tags = tiffInfo.tags - override protected def readBounds(bounds: Traversable[GridBounds[Int]], bands: Seq[Int]): Iterator[Raster[MultibandTile]] = { + override def readBounds(bounds: Traversable[GridBounds[Int]], bands: Seq[Int]): Iterator[Raster[MultibandTile]] = { val info = realInfo val geoTiffTile = GeoTiffReader.geoTiffMultibandTile(info) val intersectingBounds = bounds.flatMap(_.intersection(this.gridBounds)).toSeq diff --git a/datasource/src/test/scala/examples/RasterSourceExercise.scala b/datasource/src/test/scala/examples/RasterSourceExercise.scala new file mode 100644 index 000000000..4433337fd --- /dev/null +++ b/datasource/src/test/scala/examples/RasterSourceExercise.scala @@ -0,0 +1,47 @@ +/* + * This software is licensed under the Apache 2 license, quoted below. + * + * Copyright 2020 Astraea, Inc. + * + * 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. + * + * SPDX-License-Identifier: Apache-2.0 + * + */ + +package examples + +import java.net.URI + +import geotrellis.raster._ +import org.apache.spark.sql.SparkSession +import org.locationtech.rasterframes.ref.RFRasterSource + +object RasterSourceExercise extends App { + val path = "s3://sentinel-s2-l2a/tiles/22/L/EP/2019/5/31/0/R60m/B08.jp2" + + + implicit val spark = SparkSession.builder(). + master("local[*]").appName("Hit me").getOrCreate() + + spark.range(1000).rdd + .map(_ => path) + .flatMap(uri => { + val rs = RFRasterSource(URI.create(uri)) + val grid = GridBounds(0, 0, rs.cols - 1, rs.rows - 1) + val tileBounds = grid.split(256, 256).toSeq + rs.readBounds(tileBounds, Seq(0)) + }) + .foreach(_ => ()) + +} From 5be2a42577fcea6f1aa708d8dc1d410e24c34761 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Mon, 10 Feb 2020 11:40:03 -0500 Subject: [PATCH 26/94] Added global thread lock on JP2 GDAL file reading. See locationtech/geotrellis#3184 --- .sbtrc | 2 +- .../locationtech/rasterframes/ref/JP2GDALRasterSource.scala | 6 +++++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/.sbtrc b/.sbtrc index eacfdb79d..b253350e2 100644 --- a/.sbtrc +++ b/.sbtrc @@ -1 +1 @@ -alias openHere=eval "open .".! +alias openHere=eval scala.sys.process.Process(Seq("open", ".")).! diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/JP2GDALRasterSource.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/JP2GDALRasterSource.scala index 5a06f9104..15869d1bd 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/JP2GDALRasterSource.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/JP2GDALRasterSource.scala @@ -32,13 +32,17 @@ import geotrellis.raster.{GridBounds, MultibandTile, Raster} class JP2GDALRasterSource(source: URI) extends GDALRasterSource(source) { override protected def tiffInfo = JP2GDALRasterSource.synchronized { - SimpleRasterInfo(source.toASCIIString, _ => SimpleRasterInfo(gdal)) + SimpleRasterInfo(source.toASCIIString, _ => JP2GDALRasterSource.synchronized(SimpleRasterInfo(gdal))) } override def readBounds(bounds: Traversable[GridBounds[Int]], bands: Seq[Int]): Iterator[Raster[MultibandTile]] = JP2GDALRasterSource.synchronized { super.readBounds(bounds, bands) } + override def read(bounds: GridBounds[Int], bands: Seq[Int]): Raster[MultibandTile] = + JP2GDALRasterSource.synchronized { + readBounds(Seq(bounds), bands).next() + } } object JP2GDALRasterSource { From 452747b4a3fd782697fcf4e05dd203ec1e338abe Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Mon, 10 Feb 2020 14:49:04 -0500 Subject: [PATCH 27/94] Made JP2 GDAL thread lock configurable. --- core/src/main/resources/reference.conf | 1 + .../org/locationtech/rasterframes/ref/RFRasterSource.scala | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/resources/reference.conf b/core/src/main/resources/reference.conf index 5f5b06d0a..941825fc6 100644 --- a/core/src/main/resources/reference.conf +++ b/core/src/main/resources/reference.conf @@ -5,6 +5,7 @@ rasterframes { showable-max-cells = 20 max-truncate-row-element-length = 40 raster-source-cache-timeout = 120 seconds + jp2-gdal-thread-lock = false } geotrellis.raster.gdal { options { diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/RFRasterSource.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/RFRasterSource.scala index 6096718df..e3ac69c66 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/RFRasterSource.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/RFRasterSource.scala @@ -110,7 +110,7 @@ object RFRasterSource extends LazyLogging { rsCache.get( source.toASCIIString, _ => source match { case IsGDAL() => - if (source.getPath.toLowerCase().endsWith("jp2")) + if (rfConfig.getBoolean("jp2-gdal-thread-lock") && source.getPath.toLowerCase().endsWith("jp2")) JP2GDALRasterSource(source) else GDALRasterSource(source) From 5a9113a3a159ddd07c5256681a185f89031c8290 Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Tue, 11 Feb 2020 11:02:21 -0500 Subject: [PATCH 28/94] Add rf_sqrt square root Signed-off-by: Jason T. Brown --- .../expressions/localops/Exp.scala | 21 +++++++++++++++++++ .../rasterframes/expressions/package.scala | 1 + .../functions/LocalFunctions.scala | 3 +++ .../rasterframes/RasterFunctionsSpec.scala | 8 +++++++ docs/src/main/paradox/reference.md | 7 +++++++ .../python/pyrasterframes/rasterfunctions.py | 4 ++++ .../main/python/tests/RasterFunctionsTests.py | 1 + 7 files changed, 45 insertions(+) diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Exp.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Exp.scala index 6a8b3e2bd..4d0c1bc5a 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Exp.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Exp.scala @@ -112,3 +112,24 @@ case class ExpM1(child: Expression) extends UnaryLocalRasterOp with CodegenFallb object ExpM1{ def apply(tile: Column): Column = new Column(ExpM1(tile.expr)) } + +@ExpressionDescription( + usage = "_FUNC_(tile) - Perform cell-wise square root", + arguments = """ + Arguments: + * tile - input tile + """, + examples = + """ + Examples: + > SELECT _FUNC_(tile) + ... """ +) +case class Sqrt(child: Expression) extends UnaryLocalRasterOp with CodegenFallback { + override val nodeName: String = "rf_sqrt" + override protected def op(tile: Tile): Tile = fpTile(tile).localPowValue(0.5) + override def dataType: DataType = child.dataType +} +object Sqrt { + def apply(tile: Column): Column = new Column(Sqrt(tile.expr)) +} diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/package.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/package.scala index 901073797..2617edc0a 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/package.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/package.scala @@ -107,6 +107,7 @@ package object expressions { registry.registerExpression[Exp10]("rf_exp10") registry.registerExpression[Exp2]("rf_exp2") registry.registerExpression[ExpM1]("rf_expm1") + registry.registerExpression[Sqrt]("rf_sqrt") registry.registerExpression[Resample]("rf_resample") registry.registerExpression[TileToArrayDouble]("rf_tile_to_array_double") registry.registerExpression[TileToArrayInt]("rf_tile_to_array_int") diff --git a/core/src/main/scala/org/locationtech/rasterframes/functions/LocalFunctions.scala b/core/src/main/scala/org/locationtech/rasterframes/functions/LocalFunctions.scala index 714253ca7..1388a82fb 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/functions/LocalFunctions.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/functions/LocalFunctions.scala @@ -299,6 +299,9 @@ trait LocalFunctions { /** Exponential of cell values, less one*/ def rf_expm1(tileCol: Column): Column = ExpM1(tileCol) + /** Square root of cell values */ + def rf_sqrt(tileCol: Column): Column = Sqrt(tileCol) + /** Return the incoming tile untouched. */ def rf_identity(tileCol: Column): Column = Identity(tileCol) } diff --git a/core/src/test/scala/org/locationtech/rasterframes/RasterFunctionsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/RasterFunctionsSpec.scala index f297fdce3..0832c2705 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/RasterFunctionsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/RasterFunctionsSpec.scala @@ -313,6 +313,14 @@ class RasterFunctionsSpec extends TestEnvironment with RasterMatchers { } + it("should take square root") { + val df = Seq(three).toDF("tile") + assertEqual( + df.select(rf_sqrt(rf_local_multiply($"tile", $"tile"))).as[ProjectedRasterTile].first(), + three + ) + } + it("should resample") { def lowRes = { def base = ArrayTile(Array(1, 2, 3, 4), 2, 2) diff --git a/docs/src/main/paradox/reference.md b/docs/src/main/paradox/reference.md index 7c5751f2d..9ae88c0b7 100644 --- a/docs/src/main/paradox/reference.md +++ b/docs/src/main/paradox/reference.md @@ -559,6 +559,13 @@ Performs cell-wise logarithm with base 2. Performs natural logarithm of cell values plus one. Inverse of @ref:[`rf_expm1`](reference.md#rf-expm1). + +### rf_sqrt + + Tile rf_sqrt(Tile tile) + +Perform cell-wise square root. + ## Tile Statistics The following functions compute a statistical summary per row of a `tile` column. The statistics are computed across the cells of a single `tile`, within each DataFrame Row. diff --git a/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py b/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py index c9b996f29..91f0bcc0b 100644 --- a/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py +++ b/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py @@ -720,6 +720,10 @@ def rf_expm1(tile_col): return _apply_column_function('rf_expm1', tile_col) +def rf_sqrt(tile_col): + """Performs cell-wise square root""" + return _apply_column_function('rf_sqrt', tile_col) + def rf_identity(tile_col): """Pass tile through unchanged""" return _apply_column_function('rf_identity', tile_col) diff --git a/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py b/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py index 176c50b8c..ed61d7ce2 100644 --- a/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py +++ b/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py @@ -101,6 +101,7 @@ def test_general(self): .withColumn('log', rf_log('tile')) \ .withColumn('exp', rf_exp('tile')) \ .withColumn('expm1', rf_expm1('tile')) \ + .withColumn('sqrt', rf_sqrt('tile')) \ .withColumn('round', rf_round('tile')) \ .withColumn('abs', rf_abs('tile')) From 4f60dfeb0ff022338bec103577ea4176dccaa297 Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Tue, 11 Feb 2020 12:30:06 -0500 Subject: [PATCH 29/94] Add failing test for issue 458; failing to render null values in tile column Signed-off-by: Jason T. Brown --- .../src/main/python/tests/IpythonTests.py | 61 +++++++++++++++++++ 1 file changed, 61 insertions(+) create mode 100644 pyrasterframes/src/main/python/tests/IpythonTests.py diff --git a/pyrasterframes/src/main/python/tests/IpythonTests.py b/pyrasterframes/src/main/python/tests/IpythonTests.py new file mode 100644 index 000000000..f221ee7df --- /dev/null +++ b/pyrasterframes/src/main/python/tests/IpythonTests.py @@ -0,0 +1,61 @@ +# +# This software is licensed under the Apache 2 license, quoted below. +# +# Copyright 2019 Astraea, Inc. +# +# 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. +# +# SPDX-License-Identifier: Apache-2.0 +# + +from unittest import skip + + +import pyrasterframes +import pyrasterframes.rf_ipython +from pyrasterframes.rasterfunctions import * +from pyrasterframes.rf_types import * + +from IPython.display import display_markdown +from IPython.display import display_html + +import numpy as np + +from py4j.protocol import Py4JJavaError +from . import TestEnvironment + +class IpythonTests(TestEnvironment): + + def setUp(self): + self.create_layer() + + def test_all_nodata_tile(self): + # https://github.com/locationtech/rasterframes/issues/458 + + from pyspark.sql.types import StructType, StructField + + from pyspark.sql import Row + df = self.spark.createDataFrame([ + Row( + tile=Tile(np.array([[np.nan, np.nan, np.nan], [np.nan, np.nan, np.nan]], dtype='float64'), + CellType.float64()) + ), + Row(tile=None) + ], schema=StructType([StructField('tile', TileUDT(), True)])) + + try: + pyrasterframes.rf_ipython.spark_df_to_html(df) + except Py4JJavaError: + self.fail("test_all_nodata_tile failed with Py4JJavaError") + except: + self.fail("um") From 15ba0bef700ed0ea0210ad1f1e0d699e7cf08f6d Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Tue, 11 Feb 2020 12:55:41 -0500 Subject: [PATCH 30/94] Skip failing test for 458 Signed-off-by: Jason T. Brown --- pyrasterframes/src/main/python/tests/IpythonTests.py | 1 + 1 file changed, 1 insertion(+) diff --git a/pyrasterframes/src/main/python/tests/IpythonTests.py b/pyrasterframes/src/main/python/tests/IpythonTests.py index f221ee7df..677dabdbd 100644 --- a/pyrasterframes/src/main/python/tests/IpythonTests.py +++ b/pyrasterframes/src/main/python/tests/IpythonTests.py @@ -39,6 +39,7 @@ class IpythonTests(TestEnvironment): def setUp(self): self.create_layer() + @skip("Pending fix for issue #458") def test_all_nodata_tile(self): # https://github.com/locationtech/rasterframes/issues/458 From 81d19da5d5c185cb26cbdd61b327657136fe2068 Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Thu, 3 Oct 2019 12:08:51 -0400 Subject: [PATCH 31/94] Failing unit test for st_geometry extents bug Signed-off-by: Jason T. Brown --- .../expressions/transformers/ExtentToGeometry.scala | 2 +- .../src/main/python/pyrasterframes/rasterfunctions.py | 5 +++-- .../src/main/python/tests/RasterFunctionsTests.py | 9 +++++++++ 3 files changed, 13 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/ExtentToGeometry.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/ExtentToGeometry.scala index 9d2d12d2f..61325c5b3 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/ExtentToGeometry.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/ExtentToGeometry.scala @@ -51,7 +51,7 @@ case class ExtentToGeometry(child: Expression) extends UnaryExpression with Code child.dataType match { case dt if dt == envSchema || dt == extSchema ⇒ TypeCheckSuccess case o ⇒ TypeCheckFailure( - s"Expected bounding box of form '${envSchema}' but received '${o.simpleString}'." + s"Expected bounding box of form '${envSchema}' or '${extSchema}' but received '${o.simpleString}'." ) } } diff --git a/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py b/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py index c9b996f29..4706d6c49 100644 --- a/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py +++ b/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py @@ -763,9 +763,10 @@ def rf_proj_raster(tile, extent, crs): """ return _apply_column_function('rf_proj_raster', tile, extent, crs) -def st_geometry(geom_col): + +def st_geometry(extent_col): """Convert the given extent/bbox to a polygon""" - return _apply_column_function('st_geometry', geom_col) + return _apply_column_function('st_geometry', extent_col) def rf_geometry(proj_raster_col): diff --git a/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py b/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py index 176c50b8c..03898e871 100644 --- a/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py +++ b/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py @@ -106,6 +106,15 @@ def test_general(self): df.first() + def test_st_geometry_from_struct(self): + from pyspark.sql import Row + from pyspark.sql.functions import struct + df = self.spark.createDataFrame([Row(xmin=0, ymin=1, xmax=2, ymax=3)]) + df.select(st_geometry(struct(df.xmin, df.ymin, df.xmax, df.ymax)).alias('geom')) + + actual_bounds = df.first()['geom'].bounds + self.assertEqual(actual_bounds, (1, 2, 3, 4)) + def test_agg_mean(self): mean = self.rf.agg(rf_agg_mean('tile')).first()['rf_agg_mean(tile)'] self.assertTrue(self.rounded_compare(mean, 10160)) From 2811c8beea07f80a5815e3f100fa9561d5aa1f1f Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Wed, 12 Feb 2020 15:28:18 -0500 Subject: [PATCH 32/94] Added support for Extents defined with bigint in Python. --- .../expressions/DynamicExtractors.scala | 6 ++- .../transformers/ExtentToGeometry.scala | 22 +++----- .../rasterframes/model/LongExtent.scala | 51 +++++++++++++++++++ .../main/python/tests/RasterFunctionsTests.py | 6 +-- 4 files changed, 66 insertions(+), 19 deletions(-) create mode 100644 core/src/main/scala/org/locationtech/rasterframes/model/LongExtent.scala diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala index 91cd8f037..0115bd837 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String import org.locationtech.jts.geom.{Envelope, Point} import org.locationtech.rasterframes.encoders.CatalystSerializer._ -import org.locationtech.rasterframes.model.{LazyCRS, TileContext} +import org.locationtech.rasterframes.model.{LazyCRS, LongExtent, TileContext} import org.locationtech.rasterframes.ref.{ProjectedRasterLike, RasterRef, RasterSource} import org.locationtech.rasterframes.tiles.ProjectedRasterTile @@ -110,6 +110,8 @@ object DynamicExtractors { (input: Any) => input.asInstanceOf[InternalRow].to[Extent] case t if t.conformsTo[Envelope] => (input: Any) => Extent(input.asInstanceOf[InternalRow].to[Envelope]) + case t if t.conformsTo[LongExtent] => + (input: Any) => input.asInstanceOf[InternalRow].to[LongExtent].toExtent } val fromPRL = projectedRasterLikeExtractor.andThen(_.andThen(_.extent)) @@ -122,6 +124,8 @@ object DynamicExtractors { (input: Any) => JTSTypes.GeometryTypeInstance.deserialize(input).getEnvelopeInternal case t if t.conformsTo[Extent] => (input: Any) => input.asInstanceOf[InternalRow].to[Extent].jtsEnvelope + case t if t.conformsTo[LongExtent] => + (input: Any) => input.asInstanceOf[InternalRow].to[LongExtent].toExtent.jtsEnvelope case t if t.conformsTo[Envelope] => (input: Any) => input.asInstanceOf[InternalRow].to[Envelope] } diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/ExtentToGeometry.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/ExtentToGeometry.scala index 61325c5b3..4ba52558b 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/ExtentToGeometry.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/ExtentToGeometry.scala @@ -22,7 +22,7 @@ package org.locationtech.rasterframes.expressions.transformers import org.locationtech.rasterframes.encoders.CatalystSerializer._ -import org.locationtech.rasterframes.expressions.row +import org.locationtech.rasterframes.expressions.{DynamicExtractors, row} import org.locationtech.jts.geom.{Envelope, Geometry} import geotrellis.vector.Extent import org.apache.spark.sql.catalyst.analysis.TypeCheckResult @@ -44,27 +44,19 @@ case class ExtentToGeometry(child: Expression) extends UnaryExpression with Code override def dataType: DataType = JTSTypes.GeometryTypeInstance - private val envSchema = schemaOf[Envelope] - private val extSchema = schemaOf[Extent] - override def checkInputDataTypes(): TypeCheckResult = { - child.dataType match { - case dt if dt == envSchema || dt == extSchema ⇒ TypeCheckSuccess - case o ⇒ TypeCheckFailure( - s"Expected bounding box of form '${envSchema}' or '${extSchema}' but received '${o.simpleString}'." + if (!DynamicExtractors.extentExtractor.isDefinedAt(child.dataType)) { + TypeCheckFailure( + s"Expected bounding box of form '${schemaOf[Envelope]}' or '${schemaOf[Extent]}' " + + s"but received '${child.dataType.simpleString}'." ) } + else TypeCheckSuccess } override protected def nullSafeEval(input: Any): Any = { val r = row(input) - val extent = if(child.dataType == envSchema) { - val env = r.to[Envelope] - Extent(env) - } - else { - r.to[Extent] - } + val extent = DynamicExtractors.extentExtractor(child.dataType)(r) val geom = extent.jtsGeom JTSTypes.GeometryTypeInstance.serialize(geom) } diff --git a/core/src/main/scala/org/locationtech/rasterframes/model/LongExtent.scala b/core/src/main/scala/org/locationtech/rasterframes/model/LongExtent.scala new file mode 100644 index 000000000..f18ea88af --- /dev/null +++ b/core/src/main/scala/org/locationtech/rasterframes/model/LongExtent.scala @@ -0,0 +1,51 @@ +/* + * This software is licensed under the Apache 2 license, quoted below. + * + * Copyright 2020 Astraea, Inc. + * + * 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. + * + * SPDX-License-Identifier: Apache-2.0 + * + */ + +package org.locationtech.rasterframes.model + +import geotrellis.vector.Extent +import org.apache.spark.sql.types.{LongType, StructField, StructType} +import org.locationtech.rasterframes.encoders.CatalystSerializer +import org.locationtech.rasterframes.encoders.CatalystSerializer.CatalystIO + +case class LongExtent(xmin: Long, ymin: Long, xmax: Long, ymax: Long) { + def toExtent: Extent = Extent(xmin.toDouble, ymin.toDouble, xmax.toDouble, ymax.toDouble) +} + +object LongExtent { + implicit val bigIntExtentSerializer: CatalystSerializer[LongExtent] = new CatalystSerializer[LongExtent] { + override val schema: StructType = StructType(Seq( + StructField("xmin", LongType, false), + StructField("ymin", LongType, false), + StructField("xmax", LongType, false), + StructField("ymax", LongType, false) + )) + override def to[R](t: LongExtent, io: CatalystIO[R]): R = io.create( + t.xmin, t.ymin, t.xmax, t.ymax + ) + override def from[R](row: R, io: CatalystIO[R]): LongExtent = LongExtent( + io.getLong(row, 0), + io.getLong(row, 1), + io.getLong(row, 2), + io.getLong(row, 3) + ) + } +} diff --git a/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py b/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py index 03898e871..7e53b4690 100644 --- a/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py +++ b/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py @@ -110,10 +110,10 @@ def test_st_geometry_from_struct(self): from pyspark.sql import Row from pyspark.sql.functions import struct df = self.spark.createDataFrame([Row(xmin=0, ymin=1, xmax=2, ymax=3)]) - df.select(st_geometry(struct(df.xmin, df.ymin, df.xmax, df.ymax)).alias('geom')) + df2 = df.select(st_geometry(struct(df.xmin, df.ymin, df.xmax, df.ymax)).alias('geom')) - actual_bounds = df.first()['geom'].bounds - self.assertEqual(actual_bounds, (1, 2, 3, 4)) + actual_bounds = df2.first()['geom'].bounds + self.assertEqual((0.0, 1.0, 2.0, 3.0), actual_bounds) def test_agg_mean(self): mean = self.rf.agg(rf_agg_mean('tile')).first()['rf_agg_mean(tile)'] From 587b6e30a7507561edc49a908a79353321536794 Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Wed, 12 Feb 2020 14:40:03 -0500 Subject: [PATCH 33/94] Add failing unit test for issue 462 Signed-off-by: Jason T. Brown (cherry picked from commit 7d328257fa9352125fa8fa3367182f05e6670319) --- .../main/python/tests/PyRasterFramesTests.py | 39 +++++++++++++++++++ 1 file changed, 39 insertions(+) diff --git a/pyrasterframes/src/main/python/tests/PyRasterFramesTests.py b/pyrasterframes/src/main/python/tests/PyRasterFramesTests.py index 0dc36a8e7..2ce89a952 100644 --- a/pyrasterframes/src/main/python/tests/PyRasterFramesTests.py +++ b/pyrasterframes/src/main/python/tests/PyRasterFramesTests.py @@ -434,6 +434,45 @@ def test_raster_join(self): with self.assertRaises(AssertionError): self.rf.raster_join(rf_prime, join_exprs=self.rf.extent) + def test_raster_join_with_null_left_head(self): + # https://github.com/locationtech/rasterframes/issues/462 + + from py4j.protocol import Py4JJavaError + + ones = np.ones((10, 10), dtype='uint8') + e = Extent(0.0, 0.0, 40.0, 40.0) + c = 'EPSG:32611' + + left = self.spark.createDataFrame( + [ + Row(i=1, t=Tile(ones, CellType.uint8()), e=e, c=c), + Row(i=1, t=None, e=e, c=c) + ] + ) + + right = self.spark.createDataFrame( + [ + Row(i=1, r=Tile(ones, CellType.uint8()), e=e, c=c), + ]) + + try: + joined = left.raster_join(right, + join_exprs=left.i == right.i, + left_extent=left.e, right_extent=right.e, + left_crs=left.c, right_crs=right.c) + + self.assertEqual(joined.count(), 2) + + collected = joined.select(rf_dimensions('r').cols.alias('cols'), + rf_dimensions('r').rows.alias('rows')) \ + .collect() + for r in collected: + self.assertEqual(r.rows, 10) + self.assertEqual(r.cols, 10) + + except Py4JJavaError as e: + self.fail('test_raster_join_with_null_left_head failed with Py4JJavaError:' + e) + def suite(): function_tests = unittest.TestSuite() From 4b475c2e10aeffdc03209a61edd98803a94aa657 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Thu, 13 Feb 2020 09:37:21 -0500 Subject: [PATCH 34/94] Fixed misnamed field in Envelope serializer. --- .../rasterframes/encoders/StandardSerializers.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/locationtech/rasterframes/encoders/StandardSerializers.scala b/core/src/main/scala/org/locationtech/rasterframes/encoders/StandardSerializers.scala index bcb7f856a..f1e56f4fb 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/encoders/StandardSerializers.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/encoders/StandardSerializers.scala @@ -49,7 +49,7 @@ trait StandardSerializers { )) override protected def to[R](t: Envelope, io: CatalystIO[R]): R = io.create( - t.getMinX, t.getMaxX, t.getMinY, t.getMaxX + t.getMinX, t.getMaxX, t.getMinY, t.getMaxY ) override protected def from[R](t: R, io: CatalystIO[R]): Envelope = new Envelope( From f07074c7ffe98ad31e1317867287676973d48f34 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Thu, 13 Feb 2020 09:49:26 -0500 Subject: [PATCH 35/94] Added ability to specify Extent with arbitrarily ordered fields in Python. --- .../expressions/DynamicExtractors.scala | 36 ++++++++ .../expressions/DynamicExtractorsSpec.scala | 92 +++++++++++++++++++ 2 files changed, 128 insertions(+) create mode 100644 core/src/test/scala/org/locationtech/rasterframes/expressions/DynamicExtractorsSpec.scala diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala index 91cd8f037..efe8efc9d 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala @@ -102,6 +102,41 @@ object DynamicExtractors { (v: Any) => v.asInstanceOf[InternalRow].to[CRS] } + /** This is necessary because extents created from Python Rows will reorder field names. */ + object ExtentLike { + + def rightShape(struct: StructType) = + struct.size == 4 && { + val n = struct.fieldNames.map(_.toLowerCase).toSet + n == Set("xmin", "ymin", "xmax", "ymax")|| n == Set("minx", "miny", "maxx", "maxy") + } && struct.fields.map(_.dataType).toSet == Set(DoubleType) + + + def unapply(dt: DataType): Option[Any => Extent] = dt match { + case dt: StructType if rightShape(dt) => + Some((input: Any) => { + val row = input.asInstanceOf[InternalRow] + + def maybeValue(name: String): Option[Double] = { + dt.indexWhere(_.name.toLowerCase == name) match { + case idx if idx >= 0 => Some(row.getDouble(idx)) + case _ => None + } + } + + def value(n1: String, n2: String): Double = + maybeValue(n1).orElse(maybeValue(n2)).getOrElse(throw new IllegalArgumentException(s"Missing field $n1 or $n2")) + + val xmin = value("xmin", "minx") + val ymin = value("ymin", "miny") + val xmax = value("xmax", "maxx") + val ymax = value("ymax", "maxy") + Extent(xmin, ymin, xmax, ymax) + }) + case _ => None + } + } + lazy val extentExtractor: PartialFunction[DataType, Any ⇒ Extent] = { val base: PartialFunction[DataType, Any ⇒ Extent]= { case t if org.apache.spark.sql.rf.WithTypeConformity(t).conformsTo(JTSTypes.GeometryTypeInstance) => @@ -110,6 +145,7 @@ object DynamicExtractors { (input: Any) => input.asInstanceOf[InternalRow].to[Extent] case t if t.conformsTo[Envelope] => (input: Any) => Extent(input.asInstanceOf[InternalRow].to[Envelope]) + case ExtentLike(e) => e } val fromPRL = projectedRasterLikeExtractor.andThen(_.andThen(_.extent)) diff --git a/core/src/test/scala/org/locationtech/rasterframes/expressions/DynamicExtractorsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/expressions/DynamicExtractorsSpec.scala new file mode 100644 index 000000000..46a4af078 --- /dev/null +++ b/core/src/test/scala/org/locationtech/rasterframes/expressions/DynamicExtractorsSpec.scala @@ -0,0 +1,92 @@ +/* + * This software is licensed under the Apache 2 license, quoted below. + * + * Copyright 2020 Astraea, Inc. + * + * 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. + * + * SPDX-License-Identifier: Apache-2.0 + * + */ + +package org.locationtech.rasterframes.expressions + +import geotrellis.vector.Extent +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.{Encoder, Encoders} +import org.locationtech.jts.geom.Envelope +import org.locationtech.rasterframes.TestEnvironment +import org.locationtech.rasterframes.encoders.CatalystSerializer._ +import org.locationtech.rasterframes.expressions.DynamicExtractors._ +import org.locationtech.rasterframes.expressions.DynamicExtractorsSpec.{SnowflakeExtent1, SnowflakeExtent2} +import org.scalatest.Inspectors + +class DynamicExtractorsSpec extends TestEnvironment with Inspectors { + describe("Extent extraction") { + val expected = Extent(1, 2, 3, 4) + it("should handle normal Extent") { + extentExtractor.isDefinedAt(schemaOf[Extent]) should be(true) + + val row = expected.toInternalRow + extentExtractor(schemaOf[Extent])(row) should be (expected) + } + it("should handle Envelope") { + extentExtractor.isDefinedAt(schemaOf[Envelope]) should be(true) + + val e = expected.jtsEnvelope + + val row = e.toInternalRow + extentExtractor(schemaOf[Envelope])(row) should be (expected) + } + + it("should handle artisanally constructed Extents") { + // Tests the case where PySpark will reorder manually constructed fields. + // See https://stackoverflow.com/questions/35343525/how-do-i-order-fields-of-my-row-objects-in-spark-python/35343885#35343885 + + import spark.implicits._ + withClue("case 1"){ + val special = SnowflakeExtent1(expected.xmax, expected.ymin, expected.xmin, expected.ymax) + val df = Seq(Tuple1(special)).toDF("extent") + val encodedType = df.schema.fields(0).dataType + val encodedRow = SnowflakeExtent1.enc.toRow(special) + extentExtractor.isDefinedAt(encodedType) should be(true) + extentExtractor(encodedType)(encodedRow) should be(expected) + } + + withClue("case 2") { + val special = SnowflakeExtent2(expected.xmax, expected.ymin, expected.xmin, expected.ymax) + val df = Seq(Tuple1(special)).toDF("extent") + val encodedType = df.schema.fields(0).dataType + val encodedRow = SnowflakeExtent2.enc.toRow(special) + extentExtractor.isDefinedAt(encodedType) should be(true) + extentExtractor(encodedType)(encodedRow) should be(expected) + } + } + } + +} + +object DynamicExtractorsSpec { + case class SnowflakeExtent1(xmax: Double, ymin: Double, xmin: Double, ymax: Double) + + object SnowflakeExtent1 { + implicit val enc: ExpressionEncoder[SnowflakeExtent1] = Encoders.product[SnowflakeExtent1].asInstanceOf[ExpressionEncoder[SnowflakeExtent1]] + } + + case class SnowflakeExtent2(xmax: Double, ymin: Double, xmin: Double, ymax: Double) + + object SnowflakeExtent2 { + implicit val enc: ExpressionEncoder[SnowflakeExtent2] = Encoders.product[SnowflakeExtent2].asInstanceOf[ExpressionEncoder[SnowflakeExtent2]] + } + +} From 6d7d58c18c3132bc94b463944c80ffdc1be42295 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Thu, 13 Feb 2020 10:27:36 -0500 Subject: [PATCH 36/94] Additional fixes to type handling of CRS and Extent from Python. --- .../expressions/DynamicExtractors.scala | 15 ++++++++----- .../expressions/accessors/GetCRS.scala | 13 +++++------ .../rasterframes/extensions/RasterJoin.scala | 22 ++++++++++++++----- .../expressions/DynamicExtractorsSpec.scala | 2 +- .../main/python/pyrasterframes/__init__.py | 2 +- .../main/python/pyrasterframes/rf_types.py | 16 +++++++++----- .../main/python/tests/PyRasterFramesTests.py | 2 +- 7 files changed, 47 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala index efe8efc9d..d41e9e428 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala @@ -96,10 +96,15 @@ object DynamicExtractors { } lazy val crsExtractor: PartialFunction[DataType, Any => CRS] = { - case _: StringType => - (v: Any) => LazyCRS(v.asInstanceOf[UTF8String].toString) - case t if t.conformsTo[CRS] => - (v: Any) => v.asInstanceOf[InternalRow].to[CRS] + val base: PartialFunction[DataType, Any ⇒ CRS] = { + case _: StringType => + (v: Any) => LazyCRS(v.asInstanceOf[UTF8String].toString) + case t if t.conformsTo[CRS] => + (v: Any) => v.asInstanceOf[InternalRow].to[CRS] + } + + val fromPRL = projectedRasterLikeExtractor.andThen(_.andThen(_.crs)) + fromPRL orElse base } /** This is necessary because extents created from Python Rows will reorder field names. */ @@ -138,7 +143,7 @@ object DynamicExtractors { } lazy val extentExtractor: PartialFunction[DataType, Any ⇒ Extent] = { - val base: PartialFunction[DataType, Any ⇒ Extent]= { + val base: PartialFunction[DataType, Any ⇒ Extent] = { case t if org.apache.spark.sql.rf.WithTypeConformity(t).conformsTo(JTSTypes.GeometryTypeInstance) => (input: Any) => Extent(JTSTypes.GeometryTypeInstance.deserialize(input).getEnvelopeInternal) case t if t.conformsTo[Extent] => diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetCRS.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetCRS.scala index ae166a51d..66bf4bf66 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetCRS.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetCRS.scala @@ -27,12 +27,12 @@ import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback -import org.apache.spark.sql.types.{DataType, StringType} +import org.apache.spark.sql.types.DataType import org.apache.spark.sql.{Column, TypedColumn} import org.apache.spark.unsafe.types.UTF8String import org.locationtech.rasterframes.encoders.CatalystSerializer._ import org.locationtech.rasterframes.encoders.StandardEncoders.crsEncoder -import org.locationtech.rasterframes.expressions.DynamicExtractors.projectedRasterLikeExtractor +import org.locationtech.rasterframes.expressions.DynamicExtractors._ import org.locationtech.rasterframes.model.LazyCRS /** @@ -52,9 +52,8 @@ case class GetCRS(child: Expression) extends UnaryExpression with CodegenFallbac override def nodeName: String = "rf_crs" override def checkInputDataTypes(): TypeCheckResult = { - if (child.dataType != StringType && !projectedRasterLikeExtractor.isDefinedAt(child.dataType)) { - TypeCheckFailure(s"Input type '${child.dataType}' does not conform to `String` or `ProjectedRasterLike`.") - } + if (!crsExtractor.isDefinedAt(child.dataType) ) + TypeCheckFailure(s"Input type '${child.dataType}' does not conform to a CRS or something with one.") else TypeCheckSuccess } @@ -62,8 +61,8 @@ case class GetCRS(child: Expression) extends UnaryExpression with CodegenFallbac input match { case s: UTF8String => LazyCRS(s.toString).toInternalRow case row: InternalRow ⇒ - val prl = projectedRasterLikeExtractor(child.dataType)(row) - prl.crs.toInternalRow + val crs = crsExtractor(child.dataType)(row) + crs.toInternalRow case o ⇒ throw new IllegalArgumentException(s"Unsupported input type: $o") } } diff --git a/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala b/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala index 6bd66bab4..f1f82cc95 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala @@ -22,9 +22,10 @@ package org.locationtech.rasterframes.extensions import org.apache.spark.sql._ import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.DataType import org.locationtech.rasterframes._ import org.locationtech.rasterframes.encoders.serialized_literal -import org.locationtech.rasterframes.expressions.SpatialRelation +import org.locationtech.rasterframes.expressions.{DynamicExtractors, SpatialRelation} import org.locationtech.rasterframes.expressions.accessors.ExtractTile import org.locationtech.rasterframes.functions.reproject_and_merge import org.locationtech.rasterframes.util._ @@ -58,10 +59,19 @@ object RasterJoin { apply(left, right, joinExpr, leftExtent, leftCRS, rightExtent, rightCRS) } + private def checkType[T](col: Column, description: String, extractor: PartialFunction[DataType, Any => T]): Unit = { + require(extractor.isDefinedAt(col.expr.dataType), s"Expected column ${col} to be of type $description, but was ${col.expr.dataType}.") + } + def apply(left: DataFrame, right: DataFrame, joinExprs: Column, leftExtent: Column, leftCRS: Column, rightExtent: Column, rightCRS: Column): DataFrame = { // Convert resolved column into a symbolic one. def unresolved(c: Column): Column = col(c.columnName) + checkType(leftExtent, "Extent", DynamicExtractors.extentExtractor) + checkType(leftCRS, "CRS", DynamicExtractors.crsExtractor) + checkType(rightExtent, "Extent", DynamicExtractors.extentExtractor) + checkType(rightCRS, "CRS", DynamicExtractors.crsExtractor) + // Unique id for temporary columns val id = Random.alphanumeric.take(5).mkString("_", "", "_") @@ -80,7 +90,7 @@ object RasterJoin { // On the LHS we just want the first thing (subsequent ones should be identical. val leftAggCols = left.columns.map(s => first(left(s), true) as s) // On the RHS we collect result as a list. - val rightAggCtx = Seq(collect_list(rightExtent) as rightExtent2, collect_list(rightCRS) as rightCRS2) + val rightAggCtx = Seq(collect_list(rightExtent) as rightExtent2, collect_list(rf_crs(rightCRS)) as rightCRS2) val rightAggTiles = right.tileColumns.map(c => collect_list(ExtractTile(c)) as c.columnName) val rightAggOther = right.notTileColumns .filter(n => n.columnName != rightExtent.columnName && n.columnName != rightCRS.columnName) @@ -94,9 +104,11 @@ object RasterJoin { .map(t => rf_dimensions(unresolved(t))) .getOrElse(serialized_literal(NOMINAL_TILE_DIMS)) - val reprojCols = rightAggTiles.map(t => reproject_and_merge( - col(leftExtent2), col(leftCRS2), col(t.columnName), col(rightExtent2), col(rightCRS2), destDims - ) as t.columnName) + val reprojCols = rightAggTiles.map(t => { + reproject_and_merge( + col(leftExtent2), col(leftCRS2), col(t.columnName), col(rightExtent2), col(rightCRS2), destDims + ) as t.columnName + }) val finalCols = leftAggCols.map(unresolved) ++ reprojCols ++ rightAggOther.map(unresolved) diff --git a/core/src/test/scala/org/locationtech/rasterframes/expressions/DynamicExtractorsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/expressions/DynamicExtractorsSpec.scala index 46a4af078..4aae0a119 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/expressions/DynamicExtractorsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/expressions/DynamicExtractorsSpec.scala @@ -22,8 +22,8 @@ package org.locationtech.rasterframes.expressions import geotrellis.vector.Extent +import org.apache.spark.sql.Encoders import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.{Encoder, Encoders} import org.locationtech.jts.geom.Envelope import org.locationtech.rasterframes.TestEnvironment import org.locationtech.rasterframes.encoders.CatalystSerializer._ diff --git a/pyrasterframes/src/main/python/pyrasterframes/__init__.py b/pyrasterframes/src/main/python/pyrasterframes/__init__.py index 5f89508b1..00539d99c 100644 --- a/pyrasterframes/src/main/python/pyrasterframes/__init__.py +++ b/pyrasterframes/src/main/python/pyrasterframes/__init__.py @@ -83,7 +83,7 @@ def _raster_join(df, other, left_extent=None, left_crs=None, right_extent=None, else: jdf = ctx._jrfctx.rasterJoin(df._jdf, other._jdf) - return RasterFrameLayer(jdf, ctx._spark_session) + return DataFrame(jdf, ctx._spark_session) def _layer_reader(df_reader, format_key, path, **options): diff --git a/pyrasterframes/src/main/python/pyrasterframes/rf_types.py b/pyrasterframes/src/main/python/pyrasterframes/rf_types.py index fe9194b22..7fbd3b83a 100644 --- a/pyrasterframes/src/main/python/pyrasterframes/rf_types.py +++ b/pyrasterframes/src/main/python/pyrasterframes/rf_types.py @@ -38,7 +38,7 @@ class here provides the PyRasterFrames entry point. import numpy as np -__all__ = ['RasterFrameLayer', 'Tile', 'TileUDT', 'CellType', 'Extent', 'RasterSourceUDT', 'TileExploder', 'NoDataFilter'] +__all__ = ['RasterFrameLayer', 'Tile', 'TileUDT', 'CellType', 'Extent', 'CRS', 'RasterSourceUDT', 'TileExploder', 'NoDataFilter'] class cached_property(object): @@ -221,16 +221,22 @@ def __str__(self): return self.__jvm__.toString() class CRS(object): - def __init__(self, proj4_str): - self.proj4_str = proj4_str + # NB: The name `crsProj4` has to match what's used in StandardSerializers.crsSerializers + def __init__(self, crsProj4): + self.crsProj4 = crsProj4 @cached_property def __jvm__(self): comp = RFContext.active().companion_of("org.locationtech.rasterframes.model.LazyCRS") - return comp.apply(self.proj4_str) + return comp.apply(self.crsProj4) def __str__(self): - return self.proj4_str + return self.crsProj4 + + @property + def proj4_str(self): + """Alias for `crsProj4`""" + return self.crsProj4 class CellType(object): diff --git a/pyrasterframes/src/main/python/tests/PyRasterFramesTests.py b/pyrasterframes/src/main/python/tests/PyRasterFramesTests.py index 2ce89a952..42d73d75b 100644 --- a/pyrasterframes/src/main/python/tests/PyRasterFramesTests.py +++ b/pyrasterframes/src/main/python/tests/PyRasterFramesTests.py @@ -441,7 +441,7 @@ def test_raster_join_with_null_left_head(self): ones = np.ones((10, 10), dtype='uint8') e = Extent(0.0, 0.0, 40.0, 40.0) - c = 'EPSG:32611' + c = CRS('EPSG:32611') left = self.spark.createDataFrame( [ From e9fae38aedc3a311ae82c9fa6454e1d6ed329ec5 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Thu, 13 Feb 2020 14:53:47 -0500 Subject: [PATCH 37/94] reproject_and_merge returns null if LHS dimension, extent or crs are null. --- .../rasterframes/extensions/RasterJoin.scala | 10 +++---- .../rasterframes/functions/package.scala | 28 +++++++++++-------- 2 files changed, 20 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala b/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala index f1f82cc95..4a5cdc427 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala @@ -24,9 +24,8 @@ import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.DataType import org.locationtech.rasterframes._ -import org.locationtech.rasterframes.encoders.serialized_literal -import org.locationtech.rasterframes.expressions.{DynamicExtractors, SpatialRelation} import org.locationtech.rasterframes.expressions.accessors.ExtractTile +import org.locationtech.rasterframes.expressions.{DynamicExtractors, SpatialRelation} import org.locationtech.rasterframes.functions.reproject_and_merge import org.locationtech.rasterframes.util._ @@ -99,10 +98,9 @@ object RasterJoin { // After the aggregation we take all the tiles we've collected and resample + merge // into LHS extent/CRS. - // Use a representative tile from the left for the tile dimensions - val destDims = left.tileColumns.headOption - .map(t => rf_dimensions(unresolved(t))) - .getOrElse(serialized_literal(NOMINAL_TILE_DIMS)) + // Use a representative tile from the left for the tile dimensions. + // Assumes all LHS tiles in a row are of the same size. + val destDims = rf_dimensions(coalesce(left.tileColumns.map(unresolved): _*)) val reprojCols = rightAggTiles.map(t => { reproject_and_merge( diff --git a/core/src/main/scala/org/locationtech/rasterframes/functions/package.scala b/core/src/main/scala/org/locationtech/rasterframes/functions/package.scala index 0326046f3..903315882 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/functions/package.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/functions/package.scala @@ -109,18 +109,22 @@ package object functions { val rightExtents = rightExtentEnc.map(_.to[Extent]) val rightCRSs = rightCRSEnc.map(_.to[CRS]) - val cellType = tiles.map(_.cellType).reduceOption(_ union _).getOrElse(tiles.head.cellType) - - // TODO: how to allow control over... expression? - val projOpts = Reproject.Options.DEFAULT - val dest: Tile = ArrayTile.empty(cellType, leftDims.cols, leftDims.rows) - //is there a GT function to do all this? - tiles.zip(rightExtents).zip(rightCRSs).map { - case ((tile, extent), crs) => - tile.reproject(extent, crs, leftCRS, projOpts) - }.foldLeft(dest)((d, t) => - d.merge(leftExtent, t.extent, t.tile, projOpts.method) - ) + if (leftExtent == null || leftDims == null || leftCRS == null) null + else { + + val cellType = tiles.map(_.cellType).reduceOption(_ union _).getOrElse(tiles.head.cellType) + + // TODO: how to allow control over... expression? + val projOpts = Reproject.Options.DEFAULT + val dest: Tile = ArrayTile.empty(cellType, leftDims.cols, leftDims.rows) + //is there a GT function to do all this? + tiles.zip(rightExtents).zip(rightCRSs).map { + case ((tile, extent), crs) => + tile.reproject(extent, crs, leftCRS, projOpts) + }.foldLeft(dest)((d, t) => + d.merge(leftExtent, t.extent, t.tile, projOpts.method) + ) + } } } From 8222ab501fa9ef3b5bb75aa2b65a0a7e00e6c21c Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Thu, 13 Feb 2020 14:56:09 -0500 Subject: [PATCH 38/94] Typo. --- .../org/locationtech/rasterframes/extensions/RasterJoin.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala b/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala index 4a5cdc427..79c3b2fe8 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala @@ -112,7 +112,7 @@ object RasterJoin { // Here's the meat: left - // 1. Add a unique ID to each LHS row for subequent grouping. + // 1. Add a unique ID to each LHS row for subsequent grouping. .withColumn(id, monotonically_increasing_id()) // 2. Perform the left-outer join .join(right, joinExprs, joinType = "left") From 95d60f539320d6017b7b957edb7b78104599e08c Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Thu, 13 Feb 2020 17:00:47 -0500 Subject: [PATCH 39/94] Updated raster_join test to filter NA rows. --- pyrasterframes/src/main/python/tests/PyRasterFramesTests.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pyrasterframes/src/main/python/tests/PyRasterFramesTests.py b/pyrasterframes/src/main/python/tests/PyRasterFramesTests.py index 42d73d75b..6e62fd70e 100644 --- a/pyrasterframes/src/main/python/tests/PyRasterFramesTests.py +++ b/pyrasterframes/src/main/python/tests/PyRasterFramesTests.py @@ -465,10 +465,11 @@ def test_raster_join_with_null_left_head(self): collected = joined.select(rf_dimensions('r').cols.alias('cols'), rf_dimensions('r').rows.alias('rows')) \ + .dropna() \ .collect() for r in collected: - self.assertEqual(r.rows, 10) - self.assertEqual(r.cols, 10) + self.assertEqual(10, r.rows) + self.assertEqual(10, r.cols) except Py4JJavaError as e: self.fail('test_raster_join_with_null_left_head failed with Py4JJavaError:' + e) From 2a28dedd732befae2a0e1eb7f34ef3eec8150864 Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Fri, 14 Feb 2020 10:17:50 -0500 Subject: [PATCH 40/94] Expand raster_join test case with null value in left hand tile column Signed-off-by: Jason T. Brown --- .../main/python/tests/PyRasterFramesTests.py | 31 +++++++++++++++++-- 1 file changed, 28 insertions(+), 3 deletions(-) diff --git a/pyrasterframes/src/main/python/tests/PyRasterFramesTests.py b/pyrasterframes/src/main/python/tests/PyRasterFramesTests.py index 6e62fd70e..862413049 100644 --- a/pyrasterframes/src/main/python/tests/PyRasterFramesTests.py +++ b/pyrasterframes/src/main/python/tests/PyRasterFramesTests.py @@ -440,13 +440,14 @@ def test_raster_join_with_null_left_head(self): from py4j.protocol import Py4JJavaError ones = np.ones((10, 10), dtype='uint8') + t = Tile(ones, CellType.uint8()) e = Extent(0.0, 0.0, 40.0, 40.0) c = CRS('EPSG:32611') left = self.spark.createDataFrame( [ - Row(i=1, t=Tile(ones, CellType.uint8()), e=e, c=c), - Row(i=1, t=None, e=e, c=c) + Row(i=1, j='a', t=t, u=t, e=e, c=c), + Row(i=1, j='b', t=None, u=t, e=e, c=c) ] ) @@ -462,15 +463,39 @@ def test_raster_join_with_null_left_head(self): left_crs=left.c, right_crs=right.c) self.assertEqual(joined.count(), 2) + # In the case where the head column is null it will be passed thru + self.assertTrue(joined.select(isnull('t')).filter(col('j') == 'b').first()[0]) + # The right hand side tile should get dimensions from col `u` however collected = joined.select(rf_dimensions('r').cols.alias('cols'), rf_dimensions('r').rows.alias('rows')) \ - .dropna() \ .collect() + for r in collected: self.assertEqual(10, r.rows) self.assertEqual(10, r.cols) + # If there is no non-null tile on the LHS then the RHS is ill defined + joined_no_left_tile = left.drop('u') \ + .raster_join(right, + join_exprs=left.i == right.i, + left_extent=left.e, right_extent=right.e, + left_crs=left.c, right_crs=right.c) + self.assertEqual(joined_no_left_tile.count(), 2) + + # Tile col from Left side passed thru as null + self.assertTrue( + joined_no_left_tile.select(isnull('t')) \ + .filter(col('j') == 'b') \ + .first()[0] + ) + # Because no non-null tile col on Left side, the right side is null too + self.assertTrue( + joined_no_left_tile.select(isnull('r')) \ + .filter(col('j') == 'b') \ + .first()[0] + ) + except Py4JJavaError as e: self.fail('test_raster_join_with_null_left_head failed with Py4JJavaError:' + e) From 6b725a456b54adbe91595453a2e73c125257e973 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Denis=20Gigu=C3=A8re?= Date: Fri, 14 Feb 2020 07:12:19 -0500 Subject: [PATCH 41/94] Cherry picked: de8464b5688eff2f6e4f5997bf2c1d7699fef39a MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Fix #467: missing geotrellis-s3-spark dependency Signed-off-by: Jean-Denis Giguère Author: Jean-Denis Giguère --- build.sbt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/build.sbt b/build.sbt index 4fcb29806..75471b706 100644 --- a/build.sbt +++ b/build.sbt @@ -61,6 +61,8 @@ lazy val core = project spark("sql").value % Provided, geotrellis("spark").value, geotrellis("raster").value, + geotrellis("gdal").value, + geotrellis("s3-spark").value, geotrellis("s3").value, geotrellis("spark-testkit").value % Test excludeAll ( ExclusionRule(organization = "org.scalastic"), From 4dd8ab7bb511783b9f294c7629279c84840b6a46 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Fri, 14 Feb 2020 12:52:07 -0500 Subject: [PATCH 42/94] Regressions and GT 3.x updates. --- .../rasterframes/expressions/DynamicExtractors.scala | 8 ++++---- .../rasterframes/expressions/accessors/GetCRS.scala | 2 -- .../expressions/transformers/ExtentToGeometry.scala | 2 +- .../rasterframes/extensions/RasterJoin.scala | 8 +++++++- .../rasterframes/extensions/ReprojectToLayer.scala | 2 ++ .../expressions/DynamicExtractorsSpec.scala | 8 ++++++++ .../src/main/python/tests/PyRasterFramesTests.py | 11 ++++++++--- 7 files changed, 30 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala index 2d9f05072..398becd95 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala @@ -32,10 +32,8 @@ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String import org.locationtech.jts.geom.{Envelope, Point} import org.locationtech.rasterframes.encoders.CatalystSerializer._ -import org.locationtech.rasterframes.model.{LazyCRS, TileContext} -import org.locationtech.rasterframes.ref.{ProjectedRasterLike, RasterRef, RFRasterSource} import org.locationtech.rasterframes.model.{LazyCRS, LongExtent, TileContext} -import org.locationtech.rasterframes.ref.{ProjectedRasterLike, RasterRef, RasterSource} +import org.locationtech.rasterframes.ref.{ProjectedRasterLike, RFRasterSource, RasterRef} import org.locationtech.rasterframes.tiles.ProjectedRasterTile private[rasterframes] @@ -132,12 +130,14 @@ object DynamicExtractors { } def value(n1: String, n2: String): Double = - maybeValue(n1).orElse(maybeValue(n2)).getOrElse(throw new IllegalArgumentException(s"Missing field $n1 or $n2")) + maybeValue(n1).orElse(maybeValue(n2)) + .getOrElse(throw new IllegalArgumentException(s"Missing field $n1 or $n2")) val xmin = value("xmin", "minx") val ymin = value("ymin", "miny") val xmax = value("xmax", "maxx") val ymax = value("ymax", "maxy") + println(Extent(xmin, ymin, xmax, ymax)) Extent(xmin, ymin, xmax, ymax) }) case _ => None diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetCRS.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetCRS.scala index 52be775a0..68784b2c2 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetCRS.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/accessors/GetCRS.scala @@ -32,8 +32,6 @@ import org.apache.spark.sql.{Column, TypedColumn} import org.apache.spark.unsafe.types.UTF8String import org.locationtech.rasterframes.encoders.CatalystSerializer._ import org.locationtech.rasterframes.encoders.StandardEncoders.crsSparkEncoder -import org.locationtech.rasterframes.expressions.DynamicExtractors.projectedRasterLikeExtractor -import org.locationtech.rasterframes.encoders.StandardEncoders.crsEncoder import org.locationtech.rasterframes.expressions.DynamicExtractors._ import org.locationtech.rasterframes.model.LazyCRS diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/ExtentToGeometry.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/ExtentToGeometry.scala index 4ba52558b..37b8a3c6c 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/ExtentToGeometry.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/ExtentToGeometry.scala @@ -57,7 +57,7 @@ case class ExtentToGeometry(child: Expression) extends UnaryExpression with Code override protected def nullSafeEval(input: Any): Any = { val r = row(input) val extent = DynamicExtractors.extentExtractor(child.dataType)(r) - val geom = extent.jtsGeom + val geom = extent.toPolygon() JTSTypes.GeometryTypeInstance.serialize(geom) } } diff --git a/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala b/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala index 79c3b2fe8..15ab4eb3f 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala @@ -23,7 +23,9 @@ package org.locationtech.rasterframes.extensions import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.DataType +import org.locationtech.rasterframes import org.locationtech.rasterframes._ +import org.locationtech.rasterframes.encoders.serialized_literal import org.locationtech.rasterframes.expressions.accessors.ExtractTile import org.locationtech.rasterframes.expressions.{DynamicExtractors, SpatialRelation} import org.locationtech.rasterframes.functions.reproject_and_merge @@ -100,7 +102,11 @@ object RasterJoin { // into LHS extent/CRS. // Use a representative tile from the left for the tile dimensions. // Assumes all LHS tiles in a row are of the same size. - val destDims = rf_dimensions(coalesce(left.tileColumns.map(unresolved): _*)) + val destDims = + if (left.tileColumns.nonEmpty) + rf_dimensions(coalesce(left.tileColumns.map(unresolved): _*)) + else + serialized_literal(NOMINAL_TILE_DIMS) val reprojCols = rightAggTiles.map(t => { reproject_and_merge( diff --git a/core/src/main/scala/org/locationtech/rasterframes/extensions/ReprojectToLayer.scala b/core/src/main/scala/org/locationtech/rasterframes/extensions/ReprojectToLayer.scala index e46c87fc3..d7be09f0d 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/extensions/ReprojectToLayer.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/extensions/ReprojectToLayer.scala @@ -36,6 +36,8 @@ object ReprojectToLayer { val gb = tlm.tileBounds val crs = tlm.crs + require(tlm.tileLayout.tileDimensions == NOMINAL_TILE_DIMS, "Non-256^2 layouts are not yet supported.") + import df.sparkSession.implicits._ implicit val enc = Encoders.tuple(spatialKeyEncoder, extentEncoder, crsSparkEncoder) diff --git a/core/src/test/scala/org/locationtech/rasterframes/expressions/DynamicExtractorsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/expressions/DynamicExtractorsSpec.scala index 4aae0a119..e8076e66e 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/expressions/DynamicExtractorsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/expressions/DynamicExtractorsSpec.scala @@ -29,6 +29,7 @@ import org.locationtech.rasterframes.TestEnvironment import org.locationtech.rasterframes.encoders.CatalystSerializer._ import org.locationtech.rasterframes.expressions.DynamicExtractors._ import org.locationtech.rasterframes.expressions.DynamicExtractorsSpec.{SnowflakeExtent1, SnowflakeExtent2} +import org.locationtech.rasterframes.model.LongExtent import org.scalatest.Inspectors class DynamicExtractorsSpec extends TestEnvironment with Inspectors { @@ -49,6 +50,13 @@ class DynamicExtractorsSpec extends TestEnvironment with Inspectors { extentExtractor(schemaOf[Envelope])(row) should be (expected) } + it("should handle LongExtent") { + extentExtractor.isDefinedAt(schemaOf[LongExtent]) should be(true) + val expected2 = LongExtent(1L, 2L, 3L, 4L) + val row = expected2.toInternalRow + extentExtractor(schemaOf[LongExtent])(row) should be (expected) + } + it("should handle artisanally constructed Extents") { // Tests the case where PySpark will reorder manually constructed fields. // See https://stackoverflow.com/questions/35343525/how-do-i-order-fields-of-my-row-objects-in-spark-python/35343885#35343885 diff --git a/pyrasterframes/src/main/python/tests/PyRasterFramesTests.py b/pyrasterframes/src/main/python/tests/PyRasterFramesTests.py index 3a2f987ce..eb18fc877 100644 --- a/pyrasterframes/src/main/python/tests/PyRasterFramesTests.py +++ b/pyrasterframes/src/main/python/tests/PyRasterFramesTests.py @@ -442,22 +442,27 @@ def test_raster_join_with_null_left_head(self): e = Extent(0.0, 0.0, 40.0, 40.0) c = CRS('EPSG:32611') + # Note: there's a bug in Spark 2.x whereby the serialization of Extent + # reorders the fields, causing deserialization errors in the JVM side. + # So we end up manually forcing ordering with the use of `struct`. + # See https://stackoverflow.com/questions/35343525/how-do-i-order-fields-of-my-row-objects-in-spark-python/35343885#35343885 left = self.spark.createDataFrame( [ Row(i=1, j='a', t=t, u=t, e=e, c=c), Row(i=1, j='b', t=None, u=t, e=e, c=c) ] - ) + ).withColumn('e2', struct('e.xmin', 'e.ymin', 'e.xmax', 'e.ymax')) + right = self.spark.createDataFrame( [ Row(i=1, r=Tile(ones, CellType.uint8()), e=e, c=c), - ]) + ]).withColumn('e2', struct('e.xmin', 'e.ymin', 'e.xmax', 'e.ymax')) try: joined = left.raster_join(right, join_exprs=left.i == right.i, - left_extent=left.e, right_extent=right.e, + left_extent=left.e2, right_extent=right.e2, left_crs=left.c, right_crs=right.c) self.assertEqual(joined.count(), 2) From 675a0db2864b8cf7e0a39d43ca74b2618d8c9b9d Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Fri, 14 Feb 2020 13:37:22 -0500 Subject: [PATCH 43/94] Fix for propogating TileLayerMetadata tile dimensions to layer creation. --- .../rasterframes/extensions/DataFrameMethods.scala | 6 +++--- .../rasterframes/extensions/RasterJoin.scala | 14 +++++++------- .../rasterframes/extensions/ReprojectToLayer.scala | 4 +--- .../datasource/geotrellis/GeoTrellisRelation.scala | 7 ++----- .../locationtech/rasterframes/py/PyRFContext.scala | 6 +++--- 5 files changed, 16 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/locationtech/rasterframes/extensions/DataFrameMethods.scala b/core/src/main/scala/org/locationtech/rasterframes/extensions/DataFrameMethods.scala index c40c628dc..b6dfa12db 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/extensions/DataFrameMethods.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/extensions/DataFrameMethods.scala @@ -167,7 +167,7 @@ trait DataFrameMethods[DF <: DataFrame] extends MethodExtensions[DF] with Metada * @param right Right side of the join. * @return joined dataframe */ - def rasterJoin(right: DataFrame): DataFrame = RasterJoin(self, right) + def rasterJoin(right: DataFrame): DataFrame = RasterJoin(self, right, None) /** * Performs a jeft join on the dataframe `right` to this one, reprojecting and merging tiles as necessary. @@ -186,7 +186,7 @@ trait DataFrameMethods[DF <: DataFrame] extends MethodExtensions[DF] with Metada * @return joined dataframe */ def rasterJoin(right: DataFrame, leftExtent: Column, leftCRS: Column, rightExtent: Column, rightCRS: Column): DataFrame = - RasterJoin(self, right, leftExtent, leftCRS, rightExtent, rightCRS) + RasterJoin(self, right, leftExtent, leftCRS, rightExtent, rightCRS, None) /** * Performs a jeft join on the dataframe `right` to this one, reprojecting and merging tiles as necessary. @@ -203,7 +203,7 @@ trait DataFrameMethods[DF <: DataFrame] extends MethodExtensions[DF] with Metada * @return joined dataframe */ def rasterJoin(right: DataFrame, joinExpr: Column, leftExtent: Column, leftCRS: Column, rightExtent: Column, rightCRS: Column): DataFrame = - RasterJoin(self, right, joinExpr, leftExtent, leftCRS, rightExtent, rightCRS) + RasterJoin(self, right, joinExpr, leftExtent, leftCRS, rightExtent, rightCRS, None) /** Layout contents of RasterFrame to a layer. Assumes CRS and extent columns exist. */ diff --git a/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala b/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala index 15ab4eb3f..0ec993edc 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala @@ -20,10 +20,10 @@ */ package org.locationtech.rasterframes.extensions +import geotrellis.raster.Dimensions import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.DataType -import org.locationtech.rasterframes import org.locationtech.rasterframes._ import org.locationtech.rasterframes.encoders.serialized_literal import org.locationtech.rasterframes.expressions.accessors.ExtractTile @@ -36,7 +36,7 @@ import scala.util.Random object RasterJoin { /** Perform a raster join on dataframes that each have proj_raster columns, or crs and extent explicitly included. */ - def apply(left: DataFrame, right: DataFrame): DataFrame = { + def apply(left: DataFrame, right: DataFrame, fallbackDimensions: Option[Dimensions[Int]]): DataFrame = { def usePRT(d: DataFrame) = d.projRasterColumns.headOption .map(p => (rf_crs(p), rf_extent(p))) @@ -50,21 +50,21 @@ object RasterJoin { val (ldf, lcrs, lextent) = usePRT(left) val (rdf, rcrs, rextent) = usePRT(right) - apply(ldf, rdf, lextent, lcrs, rextent, rcrs) + apply(ldf, rdf, lextent, lcrs, rextent, rcrs, fallbackDimensions) } - def apply(left: DataFrame, right: DataFrame, leftExtent: Column, leftCRS: Column, rightExtent: Column, rightCRS: Column): DataFrame = { + def apply(left: DataFrame, right: DataFrame, leftExtent: Column, leftCRS: Column, rightExtent: Column, rightCRS: Column, fallbackDimensions: Option[Dimensions[Int]]): DataFrame = { val leftGeom = st_geometry(leftExtent) val rightGeomReproj = st_reproject(st_geometry(rightExtent), rightCRS, leftCRS) val joinExpr = new Column(SpatialRelation.Intersects(leftGeom.expr, rightGeomReproj.expr)) - apply(left, right, joinExpr, leftExtent, leftCRS, rightExtent, rightCRS) + apply(left, right, joinExpr, leftExtent, leftCRS, rightExtent, rightCRS, fallbackDimensions) } private def checkType[T](col: Column, description: String, extractor: PartialFunction[DataType, Any => T]): Unit = { require(extractor.isDefinedAt(col.expr.dataType), s"Expected column ${col} to be of type $description, but was ${col.expr.dataType}.") } - def apply(left: DataFrame, right: DataFrame, joinExprs: Column, leftExtent: Column, leftCRS: Column, rightExtent: Column, rightCRS: Column): DataFrame = { + def apply(left: DataFrame, right: DataFrame, joinExprs: Column, leftExtent: Column, leftCRS: Column, rightExtent: Column, rightCRS: Column, fallbackDimensions: Option[Dimensions[Int]]): DataFrame = { // Convert resolved column into a symbolic one. def unresolved(c: Column): Column = col(c.columnName) @@ -106,7 +106,7 @@ object RasterJoin { if (left.tileColumns.nonEmpty) rf_dimensions(coalesce(left.tileColumns.map(unresolved): _*)) else - serialized_literal(NOMINAL_TILE_DIMS) + serialized_literal(fallbackDimensions.getOrElse(NOMINAL_TILE_DIMS)) val reprojCols = rightAggTiles.map(t => { reproject_and_merge( diff --git a/core/src/main/scala/org/locationtech/rasterframes/extensions/ReprojectToLayer.scala b/core/src/main/scala/org/locationtech/rasterframes/extensions/ReprojectToLayer.scala index d7be09f0d..816e99085 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/extensions/ReprojectToLayer.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/extensions/ReprojectToLayer.scala @@ -36,8 +36,6 @@ object ReprojectToLayer { val gb = tlm.tileBounds val crs = tlm.crs - require(tlm.tileLayout.tileDimensions == NOMINAL_TILE_DIMS, "Non-256^2 layouts are not yet supported.") - import df.sparkSession.implicits._ implicit val enc = Encoders.tuple(spatialKeyEncoder, extentEncoder, crsSparkEncoder) @@ -50,7 +48,7 @@ object ReprojectToLayer { // Create effectively a target RasterFrame, but with no tiles. val dest = gridItems.toSeq.toDF(SPATIAL_KEY_COLUMN.columnName, EXTENT_COLUMN.columnName, CRS_COLUMN.columnName) - val joined = RasterJoin(broadcast(dest), df) + val joined = RasterJoin(broadcast(dest), df, Some(tlm.tileLayout.tileDimensions)) joined.asLayer(SPATIAL_KEY_COLUMN, tlm) } diff --git a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisRelation.scala b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisRelation.scala index ec4f5035c..5562ffe72 100644 --- a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisRelation.scala +++ b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisRelation.scala @@ -72,17 +72,12 @@ case class GeoTrellisRelation(sqlContext: SQLContext, @transient protected lazy val logger = Logger(LoggerFactory.getLogger(getClass.getName)) - implicit val sc = sqlContext.sparkContext - /** Create new relation with the give filter added. */ def withFilter(value: Filter): GeoTrellisRelation = copy(filters = filters :+ value) /** Check to see if relation already exists in this. */ def hasFilter(filter: Filter): Boolean = filters.contains(filter) - @transient - private implicit val spark = sqlContext.sparkSession - @transient private lazy val attributes = AttributeStore(uri) @@ -128,6 +123,7 @@ case class GeoTrellisRelation(sqlContext: SQLContext, * in the metadata anywhere. This is potentially an expensive hack, which needs further quantifying of impact. * Another option is to force the user to specify the number of bands. */ private lazy val peekBandCount = { + implicit val sc = sqlContext.sparkContext tileClass match { case t if t =:= typeOf[MultibandTile] ⇒ val reader = keyType match { @@ -230,6 +226,7 @@ case class GeoTrellisRelation(sqlContext: SQLContext, logger.trace(s"Required columns: ${requiredColumns.mkString(", ")}") logger.trace(s"Filters: $filters") + implicit val sc = sqlContext.sparkContext val reader = LayerReader(uri) val columnIndexes = requiredColumns.map(schema.fieldIndex) diff --git a/pyrasterframes/src/main/scala/org/locationtech/rasterframes/py/PyRFContext.scala b/pyrasterframes/src/main/scala/org/locationtech/rasterframes/py/PyRFContext.scala index dcfa3db06..1e9385a74 100644 --- a/pyrasterframes/src/main/scala/org/locationtech/rasterframes/py/PyRFContext.scala +++ b/pyrasterframes/src/main/scala/org/locationtech/rasterframes/py/PyRFContext.scala @@ -109,19 +109,19 @@ class PyRFContext(implicit sparkSession: SparkSession) extends RasterFunctions /** * Left spatial join managing reprojection and merging of `other` */ - def rasterJoin(df: DataFrame, other: DataFrame): DataFrame = RasterJoin(df, other) + def rasterJoin(df: DataFrame, other: DataFrame): DataFrame = RasterJoin(df, other, None) /** * Left spatial join managing reprojection and merging of `other`; uses extent and CRS columns to determine if rows intersect */ def rasterJoin(df: DataFrame, other: DataFrame, leftExtent: Column, leftCRS: Column, rightExtent: Column, rightCRS: Column): DataFrame = - RasterJoin(df, other, leftExtent, leftCRS, rightExtent, rightCRS) + RasterJoin(df, other, leftExtent, leftCRS, rightExtent, rightCRS, None) /** * Left spatial join managing reprojection and merging of `other`; uses joinExprs to conduct initial join then extent and CRS columns to determine if rows intersect */ def rasterJoin(df: DataFrame, other: DataFrame, joinExprs: Column, leftExtent: Column, leftCRS: Column, rightExtent: Column, rightCRS: Column): DataFrame = - RasterJoin(df, other, joinExprs, leftExtent, leftCRS, rightExtent, rightCRS) + RasterJoin(df, other, joinExprs, leftExtent, leftCRS, rightExtent, rightCRS, None) /** From 97760bb669a9d0946cdfe47f9694491c812381a2 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Tue, 18 Feb 2020 10:17:07 -0500 Subject: [PATCH 44/94] Regressions. --- build.sbt | 10 ++++++++-- .../extensions/DataFrameMethods.scala | 6 +++--- .../rasterframes/extensions/RasterJoin.scala | 18 ++++++++++++------ .../extensions/ReprojectToLayer.scala | 5 ++++- .../rasterframes/TileStatsSpec.scala | 1 - .../rasterframes/py/PyRFContext.scala | 6 +++--- 6 files changed, 30 insertions(+), 16 deletions(-) diff --git a/build.sbt b/build.sbt index 75471b706..8e5fa05eb 100644 --- a/build.sbt +++ b/build.sbt @@ -61,8 +61,6 @@ lazy val core = project spark("sql").value % Provided, geotrellis("spark").value, geotrellis("raster").value, - geotrellis("gdal").value, - geotrellis("s3-spark").value, geotrellis("s3").value, geotrellis("spark-testkit").value % Test excludeAll ( ExclusionRule(organization = "org.scalastic"), @@ -71,6 +69,14 @@ lazy val core = project scaffeine, scalatest ), + libraryDependencies ++= { + val gv = rfGeoTrellisVersion.value + if (gv.startsWith("3")) Seq[ModuleID]( + geotrellis("gdal").value, + geotrellis("s3-spark").value + ) + else Seq.empty[ModuleID] + }, buildInfoKeys ++= Seq[BuildInfoKey]( moduleName, version, scalaVersion, sbtVersion, rfGeoTrellisVersion, rfGeoMesaVersion, rfSparkVersion ), diff --git a/core/src/main/scala/org/locationtech/rasterframes/extensions/DataFrameMethods.scala b/core/src/main/scala/org/locationtech/rasterframes/extensions/DataFrameMethods.scala index 9a57b9dd8..c0efeb1a0 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/extensions/DataFrameMethods.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/extensions/DataFrameMethods.scala @@ -168,7 +168,7 @@ trait DataFrameMethods[DF <: DataFrame] extends MethodExtensions[DF] with Metada * @param right Right side of the join. * @return joined dataframe */ - def rasterJoin(right: DataFrame): DataFrame = RasterJoin(self, right) + def rasterJoin(right: DataFrame): DataFrame = RasterJoin(self, right, None) /** * Performs a jeft join on the dataframe `right` to this one, reprojecting and merging tiles as necessary. @@ -187,7 +187,7 @@ trait DataFrameMethods[DF <: DataFrame] extends MethodExtensions[DF] with Metada * @return joined dataframe */ def rasterJoin(right: DataFrame, leftExtent: Column, leftCRS: Column, rightExtent: Column, rightCRS: Column): DataFrame = - RasterJoin(self, right, leftExtent, leftCRS, rightExtent, rightCRS) + RasterJoin(self, right, leftExtent, leftCRS, rightExtent, rightCRS, None) /** * Performs a jeft join on the dataframe `right` to this one, reprojecting and merging tiles as necessary. @@ -204,7 +204,7 @@ trait DataFrameMethods[DF <: DataFrame] extends MethodExtensions[DF] with Metada * @return joined dataframe */ def rasterJoin(right: DataFrame, joinExpr: Column, leftExtent: Column, leftCRS: Column, rightExtent: Column, rightCRS: Column): DataFrame = - RasterJoin(self, right, joinExpr, leftExtent, leftCRS, rightExtent, rightCRS) + RasterJoin(self, right, joinExpr, leftExtent, leftCRS, rightExtent, rightCRS, None) /** Layout contents of RasterFrame to a layer. Assumes CRS and extent columns exist. */ diff --git a/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala b/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala index 79c3b2fe8..9214aebc7 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala @@ -24,9 +24,11 @@ import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.DataType import org.locationtech.rasterframes._ +import org.locationtech.rasterframes.encoders.serialized_literal import org.locationtech.rasterframes.expressions.accessors.ExtractTile import org.locationtech.rasterframes.expressions.{DynamicExtractors, SpatialRelation} import org.locationtech.rasterframes.functions.reproject_and_merge +import org.locationtech.rasterframes.model.TileDimensions import org.locationtech.rasterframes.util._ import scala.util.Random @@ -34,7 +36,7 @@ import scala.util.Random object RasterJoin { /** Perform a raster join on dataframes that each have proj_raster columns, or crs and extent explicitly included. */ - def apply(left: DataFrame, right: DataFrame): DataFrame = { + def apply(left: DataFrame, right: DataFrame, fallbackDimensions: Option[TileDimensions]): DataFrame = { def usePRT(d: DataFrame) = d.projRasterColumns.headOption .map(p => (rf_crs(p), rf_extent(p))) @@ -48,21 +50,21 @@ object RasterJoin { val (ldf, lcrs, lextent) = usePRT(left) val (rdf, rcrs, rextent) = usePRT(right) - apply(ldf, rdf, lextent, lcrs, rextent, rcrs) + apply(ldf, rdf, lextent, lcrs, rextent, rcrs, fallbackDimensions) } - def apply(left: DataFrame, right: DataFrame, leftExtent: Column, leftCRS: Column, rightExtent: Column, rightCRS: Column): DataFrame = { + def apply(left: DataFrame, right: DataFrame, leftExtent: Column, leftCRS: Column, rightExtent: Column, rightCRS: Column, fallbackDimensions: Option[TileDimensions]): DataFrame = { val leftGeom = st_geometry(leftExtent) val rightGeomReproj = st_reproject(st_geometry(rightExtent), rightCRS, leftCRS) val joinExpr = new Column(SpatialRelation.Intersects(leftGeom.expr, rightGeomReproj.expr)) - apply(left, right, joinExpr, leftExtent, leftCRS, rightExtent, rightCRS) + apply(left, right, joinExpr, leftExtent, leftCRS, rightExtent, rightCRS, fallbackDimensions) } private def checkType[T](col: Column, description: String, extractor: PartialFunction[DataType, Any => T]): Unit = { require(extractor.isDefinedAt(col.expr.dataType), s"Expected column ${col} to be of type $description, but was ${col.expr.dataType}.") } - def apply(left: DataFrame, right: DataFrame, joinExprs: Column, leftExtent: Column, leftCRS: Column, rightExtent: Column, rightCRS: Column): DataFrame = { + def apply(left: DataFrame, right: DataFrame, joinExprs: Column, leftExtent: Column, leftCRS: Column, rightExtent: Column, rightCRS: Column, fallbackDimensions: Option[TileDimensions]): DataFrame = { // Convert resolved column into a symbolic one. def unresolved(c: Column): Column = col(c.columnName) @@ -100,7 +102,11 @@ object RasterJoin { // into LHS extent/CRS. // Use a representative tile from the left for the tile dimensions. // Assumes all LHS tiles in a row are of the same size. - val destDims = rf_dimensions(coalesce(left.tileColumns.map(unresolved): _*)) + val destDims = + if (left.tileColumns.nonEmpty) + rf_dimensions(coalesce(left.tileColumns.map(unresolved): _*)) + else + serialized_literal(fallbackDimensions.getOrElse(NOMINAL_TILE_DIMS)) val reprojCols = rightAggTiles.map(t => { reproject_and_merge( diff --git a/core/src/main/scala/org/locationtech/rasterframes/extensions/ReprojectToLayer.scala b/core/src/main/scala/org/locationtech/rasterframes/extensions/ReprojectToLayer.scala index 6c80e9d0d..19a048df3 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/extensions/ReprojectToLayer.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/extensions/ReprojectToLayer.scala @@ -25,6 +25,7 @@ import geotrellis.spark.{SpatialKey, TileLayerMetadata} import org.apache.spark.sql._ import org.apache.spark.sql.functions.broadcast import org.locationtech.rasterframes._ +import org.locationtech.rasterframes.model.TileDimensions import org.locationtech.rasterframes.util._ /** Algorithm for projecting an arbitrary RasterFrame into a layer with consistent CRS and gridding. */ @@ -47,7 +48,9 @@ object ReprojectToLayer { // Create effectively a target RasterFrame, but with no tiles. val dest = gridItems.toSeq.toDF(SPATIAL_KEY_COLUMN.columnName, EXTENT_COLUMN.columnName, CRS_COLUMN.columnName) - val joined = RasterJoin(broadcast(dest), df) + val dims = TileDimensions(tlm.tileLayout.tileDimensions) + + val joined = RasterJoin(broadcast(dest), df, Some(dims)) joined.asLayer(SPATIAL_KEY_COLUMN, tlm) } diff --git a/core/src/test/scala/org/locationtech/rasterframes/TileStatsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/TileStatsSpec.scala index 45a5d612a..80405cdbb 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/TileStatsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/TileStatsSpec.scala @@ -103,7 +103,6 @@ class TileStatsSpec extends TestEnvironment with TestData { withClue("max") { val max = ds.agg(rf_agg_local_max($"tiles")) - max.printSchema() val expected = Max(byteArrayTile, byteConstantTile) write(max) assert(max.as[Tile].first() === expected) diff --git a/pyrasterframes/src/main/scala/org/locationtech/rasterframes/py/PyRFContext.scala b/pyrasterframes/src/main/scala/org/locationtech/rasterframes/py/PyRFContext.scala index 6401ba551..9215e5338 100644 --- a/pyrasterframes/src/main/scala/org/locationtech/rasterframes/py/PyRFContext.scala +++ b/pyrasterframes/src/main/scala/org/locationtech/rasterframes/py/PyRFContext.scala @@ -109,19 +109,19 @@ class PyRFContext(implicit sparkSession: SparkSession) extends RasterFunctions /** * Left spatial join managing reprojection and merging of `other` */ - def rasterJoin(df: DataFrame, other: DataFrame): DataFrame = RasterJoin(df, other) + def rasterJoin(df: DataFrame, other: DataFrame): DataFrame = RasterJoin(df, other, None) /** * Left spatial join managing reprojection and merging of `other`; uses extent and CRS columns to determine if rows intersect */ def rasterJoin(df: DataFrame, other: DataFrame, leftExtent: Column, leftCRS: Column, rightExtent: Column, rightCRS: Column): DataFrame = - RasterJoin(df, other, leftExtent, leftCRS, rightExtent, rightCRS) + RasterJoin(df, other, leftExtent, leftCRS, rightExtent, rightCRS, None) /** * Left spatial join managing reprojection and merging of `other`; uses joinExprs to conduct initial join then extent and CRS columns to determine if rows intersect */ def rasterJoin(df: DataFrame, other: DataFrame, joinExprs: Column, leftExtent: Column, leftCRS: Column, rightExtent: Column, rightCRS: Column): DataFrame = - RasterJoin(df, other, joinExprs, leftExtent, leftCRS, rightExtent, rightCRS) + RasterJoin(df, other, joinExprs, leftExtent, leftCRS, rightExtent, rightCRS, None) /** From df26131e3c5572bee2f1b6228b138485e9b0d42d Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Tue, 18 Feb 2020 12:36:18 -0500 Subject: [PATCH 45/94] Disabled column type checking in RasterJoin. --- .../locationtech/rasterframes/extensions/RasterJoin.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala b/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala index 0ec993edc..2ffa80b6f 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala @@ -68,10 +68,10 @@ object RasterJoin { // Convert resolved column into a symbolic one. def unresolved(c: Column): Column = col(c.columnName) - checkType(leftExtent, "Extent", DynamicExtractors.extentExtractor) - checkType(leftCRS, "CRS", DynamicExtractors.crsExtractor) - checkType(rightExtent, "Extent", DynamicExtractors.extentExtractor) - checkType(rightCRS, "CRS", DynamicExtractors.crsExtractor) +// checkType(leftExtent, "Extent", DynamicExtractors.extentExtractor) +// checkType(leftCRS, "CRS", DynamicExtractors.crsExtractor) +// checkType(rightExtent, "Extent", DynamicExtractors.extentExtractor) +// checkType(rightCRS, "CRS", DynamicExtractors.crsExtractor) // Unique id for temporary columns val id = Random.alphanumeric.take(5).mkString("_", "", "_") From 3fd54334ba18e53be05d2a1b6812f99b25dc7ee1 Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Wed, 19 Feb 2020 10:59:37 -0500 Subject: [PATCH 46/94] Fix for pyspark AttributeError on sparksession _conf member Signed-off-by: Jason T. Brown --- pyrasterframes/src/main/python/pyrasterframes/__init__.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pyrasterframes/src/main/python/pyrasterframes/__init__.py b/pyrasterframes/src/main/python/pyrasterframes/__init__.py index 00539d99c..f930d328c 100644 --- a/pyrasterframes/src/main/python/pyrasterframes/__init__.py +++ b/pyrasterframes/src/main/python/pyrasterframes/__init__.py @@ -83,7 +83,7 @@ def _raster_join(df, other, left_extent=None, left_crs=None, right_extent=None, else: jdf = ctx._jrfctx.rasterJoin(df._jdf, other._jdf) - return DataFrame(jdf, ctx._spark_session) + return DataFrame(jdf, ctx._spark_session._wrapped) def _layer_reader(df_reader, format_key, path, **options): From 40282c6454c26464e2e63adf2b8e922e1015f671 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Fri, 14 Feb 2020 13:37:22 -0500 Subject: [PATCH 47/94] Fix for propogating TileLayerMetadata tile dimensions to layer creation. Cherry pick from 675a0db286 Signed-off-by: Jason T. Brown --- .../extensions/DataFrameMethods.scala | 6 +++--- .../rasterframes/extensions/RasterJoin.scala | 19 +++++++++++++------ .../extensions/ReprojectToLayer.scala | 4 +++- .../rasterframes/RasterLayerSpec.scala | 6 ++++-- .../geotrellis/GeoTrellisRelation.scala | 7 ++----- .../rasterframes/py/PyRFContext.scala | 6 +++--- 6 files changed, 28 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/locationtech/rasterframes/extensions/DataFrameMethods.scala b/core/src/main/scala/org/locationtech/rasterframes/extensions/DataFrameMethods.scala index 9a57b9dd8..c0efeb1a0 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/extensions/DataFrameMethods.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/extensions/DataFrameMethods.scala @@ -168,7 +168,7 @@ trait DataFrameMethods[DF <: DataFrame] extends MethodExtensions[DF] with Metada * @param right Right side of the join. * @return joined dataframe */ - def rasterJoin(right: DataFrame): DataFrame = RasterJoin(self, right) + def rasterJoin(right: DataFrame): DataFrame = RasterJoin(self, right, None) /** * Performs a jeft join on the dataframe `right` to this one, reprojecting and merging tiles as necessary. @@ -187,7 +187,7 @@ trait DataFrameMethods[DF <: DataFrame] extends MethodExtensions[DF] with Metada * @return joined dataframe */ def rasterJoin(right: DataFrame, leftExtent: Column, leftCRS: Column, rightExtent: Column, rightCRS: Column): DataFrame = - RasterJoin(self, right, leftExtent, leftCRS, rightExtent, rightCRS) + RasterJoin(self, right, leftExtent, leftCRS, rightExtent, rightCRS, None) /** * Performs a jeft join on the dataframe `right` to this one, reprojecting and merging tiles as necessary. @@ -204,7 +204,7 @@ trait DataFrameMethods[DF <: DataFrame] extends MethodExtensions[DF] with Metada * @return joined dataframe */ def rasterJoin(right: DataFrame, joinExpr: Column, leftExtent: Column, leftCRS: Column, rightExtent: Column, rightCRS: Column): DataFrame = - RasterJoin(self, right, joinExpr, leftExtent, leftCRS, rightExtent, rightCRS) + RasterJoin(self, right, joinExpr, leftExtent, leftCRS, rightExtent, rightCRS, None) /** Layout contents of RasterFrame to a layer. Assumes CRS and extent columns exist. */ diff --git a/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala b/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala index 79c3b2fe8..c75b22fe3 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala @@ -20,13 +20,16 @@ */ package org.locationtech.rasterframes.extensions + import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.DataType import org.locationtech.rasterframes._ +import org.locationtech.rasterframes.encoders.serialized_literal import org.locationtech.rasterframes.expressions.accessors.ExtractTile import org.locationtech.rasterframes.expressions.{DynamicExtractors, SpatialRelation} import org.locationtech.rasterframes.functions.reproject_and_merge +import org.locationtech.rasterframes.model.TileDimensions import org.locationtech.rasterframes.util._ import scala.util.Random @@ -34,7 +37,7 @@ import scala.util.Random object RasterJoin { /** Perform a raster join on dataframes that each have proj_raster columns, or crs and extent explicitly included. */ - def apply(left: DataFrame, right: DataFrame): DataFrame = { + def apply(left: DataFrame, right: DataFrame, fallbackDimensions: Option[TileDimensions]): DataFrame = { def usePRT(d: DataFrame) = d.projRasterColumns.headOption .map(p => (rf_crs(p), rf_extent(p))) @@ -48,21 +51,21 @@ object RasterJoin { val (ldf, lcrs, lextent) = usePRT(left) val (rdf, rcrs, rextent) = usePRT(right) - apply(ldf, rdf, lextent, lcrs, rextent, rcrs) + apply(ldf, rdf, lextent, lcrs, rextent, rcrs, fallbackDimensions) } - def apply(left: DataFrame, right: DataFrame, leftExtent: Column, leftCRS: Column, rightExtent: Column, rightCRS: Column): DataFrame = { + def apply(left: DataFrame, right: DataFrame, leftExtent: Column, leftCRS: Column, rightExtent: Column, rightCRS: Column, fallbackDimensions: Option[TileDimensions]): DataFrame = { val leftGeom = st_geometry(leftExtent) val rightGeomReproj = st_reproject(st_geometry(rightExtent), rightCRS, leftCRS) val joinExpr = new Column(SpatialRelation.Intersects(leftGeom.expr, rightGeomReproj.expr)) - apply(left, right, joinExpr, leftExtent, leftCRS, rightExtent, rightCRS) + apply(left, right, joinExpr, leftExtent, leftCRS, rightExtent, rightCRS, fallbackDimensions) } private def checkType[T](col: Column, description: String, extractor: PartialFunction[DataType, Any => T]): Unit = { require(extractor.isDefinedAt(col.expr.dataType), s"Expected column ${col} to be of type $description, but was ${col.expr.dataType}.") } - def apply(left: DataFrame, right: DataFrame, joinExprs: Column, leftExtent: Column, leftCRS: Column, rightExtent: Column, rightCRS: Column): DataFrame = { + def apply(left: DataFrame, right: DataFrame, joinExprs: Column, leftExtent: Column, leftCRS: Column, rightExtent: Column, rightCRS: Column, fallbackDimensions: Option[TileDimensions]): DataFrame = { // Convert resolved column into a symbolic one. def unresolved(c: Column): Column = col(c.columnName) @@ -100,7 +103,11 @@ object RasterJoin { // into LHS extent/CRS. // Use a representative tile from the left for the tile dimensions. // Assumes all LHS tiles in a row are of the same size. - val destDims = rf_dimensions(coalesce(left.tileColumns.map(unresolved): _*)) + val destDims = + if (left.tileColumns.nonEmpty) + rf_dimensions(coalesce(left.tileColumns.map(unresolved): _*)) + else + serialized_literal(fallbackDimensions.getOrElse(NOMINAL_TILE_DIMS)) val reprojCols = rightAggTiles.map(t => { reproject_and_merge( diff --git a/core/src/main/scala/org/locationtech/rasterframes/extensions/ReprojectToLayer.scala b/core/src/main/scala/org/locationtech/rasterframes/extensions/ReprojectToLayer.scala index 6c80e9d0d..189a3e0af 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/extensions/ReprojectToLayer.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/extensions/ReprojectToLayer.scala @@ -25,6 +25,7 @@ import geotrellis.spark.{SpatialKey, TileLayerMetadata} import org.apache.spark.sql._ import org.apache.spark.sql.functions.broadcast import org.locationtech.rasterframes._ +import org.locationtech.rasterframes.model.TileDimensions import org.locationtech.rasterframes.util._ /** Algorithm for projecting an arbitrary RasterFrame into a layer with consistent CRS and gridding. */ @@ -47,7 +48,8 @@ object ReprojectToLayer { // Create effectively a target RasterFrame, but with no tiles. val dest = gridItems.toSeq.toDF(SPATIAL_KEY_COLUMN.columnName, EXTENT_COLUMN.columnName, CRS_COLUMN.columnName) - val joined = RasterJoin(broadcast(dest), df) + val joined = RasterJoin(broadcast(dest), df, + Some(TileDimensions(tlm.tileLayout.tileDimensions))) joined.asLayer(SPATIAL_KEY_COLUMN, tlm) } diff --git a/core/src/test/scala/org/locationtech/rasterframes/RasterLayerSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/RasterLayerSpec.scala index 221e882eb..838afc898 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/RasterLayerSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/RasterLayerSpec.scala @@ -241,7 +241,9 @@ class RasterLayerSpec extends TestEnvironment with MetadataKeys def project(r: Raster[MultibandTile]): Seq[ProjectedRasterTile] = r.tile.bands.map(b => ProjectedRasterTile(b, r.extent, srcCrs)) - val rasters = src.readAll(bands = Seq(0, 1, 2)).map(project).map(p => (p(0), p(1), p(2))) + val rasters = src.readAll(bands = Seq(0, 1, 2)) + .map(project) + .map(p => (p(0), p(1), p(2))) val df = rasters.toDF("red", "green", "blue") @@ -251,7 +253,7 @@ class RasterLayerSpec extends TestEnvironment with MetadataKeys val layout = LayoutDefinition(extent, TileLayout(2, 2, 32, 32)) val tlm = new TileLayerMetadata[SpatialKey]( - UByteConstantNoDataCellType, + UByteConstantNoDataCellType, layout, extent, crs, diff --git a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisRelation.scala b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisRelation.scala index 49a7a0af0..8bb31aaae 100644 --- a/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisRelation.scala +++ b/datasource/src/main/scala/org/locationtech/rasterframes/datasource/geotrellis/GeoTrellisRelation.scala @@ -70,17 +70,12 @@ case class GeoTrellisRelation(sqlContext: SQLContext, @transient protected lazy val logger = Logger(LoggerFactory.getLogger(getClass.getName)) - implicit val sc = sqlContext.sparkContext - /** Create new relation with the give filter added. */ def withFilter(value: Filter): GeoTrellisRelation = copy(filters = filters :+ value) /** Check to see if relation already exists in this. */ def hasFilter(filter: Filter): Boolean = filters.contains(filter) - @transient - private implicit val spark = sqlContext.sparkSession - @transient private lazy val attributes = AttributeStore(uri) @@ -126,6 +121,7 @@ case class GeoTrellisRelation(sqlContext: SQLContext, * in the metadata anywhere. This is potentially an expensive hack, which needs further quantifying of impact. * Another option is to force the user to specify the number of bands. */ private lazy val peekBandCount = { + implicit val sc = sqlContext.sparkContext tileClass match { case t if t =:= typeOf[MultibandTile] ⇒ val reader = keyType match { @@ -228,6 +224,7 @@ case class GeoTrellisRelation(sqlContext: SQLContext, logger.trace(s"Required columns: ${requiredColumns.mkString(", ")}") logger.trace(s"Filters: $filters") + implicit val sc = sqlContext.sparkContext val reader = LayerReader(uri) val columnIndexes = requiredColumns.map(schema.fieldIndex) diff --git a/pyrasterframes/src/main/scala/org/locationtech/rasterframes/py/PyRFContext.scala b/pyrasterframes/src/main/scala/org/locationtech/rasterframes/py/PyRFContext.scala index 6401ba551..9215e5338 100644 --- a/pyrasterframes/src/main/scala/org/locationtech/rasterframes/py/PyRFContext.scala +++ b/pyrasterframes/src/main/scala/org/locationtech/rasterframes/py/PyRFContext.scala @@ -109,19 +109,19 @@ class PyRFContext(implicit sparkSession: SparkSession) extends RasterFunctions /** * Left spatial join managing reprojection and merging of `other` */ - def rasterJoin(df: DataFrame, other: DataFrame): DataFrame = RasterJoin(df, other) + def rasterJoin(df: DataFrame, other: DataFrame): DataFrame = RasterJoin(df, other, None) /** * Left spatial join managing reprojection and merging of `other`; uses extent and CRS columns to determine if rows intersect */ def rasterJoin(df: DataFrame, other: DataFrame, leftExtent: Column, leftCRS: Column, rightExtent: Column, rightCRS: Column): DataFrame = - RasterJoin(df, other, leftExtent, leftCRS, rightExtent, rightCRS) + RasterJoin(df, other, leftExtent, leftCRS, rightExtent, rightCRS, None) /** * Left spatial join managing reprojection and merging of `other`; uses joinExprs to conduct initial join then extent and CRS columns to determine if rows intersect */ def rasterJoin(df: DataFrame, other: DataFrame, joinExprs: Column, leftExtent: Column, leftCRS: Column, rightExtent: Column, rightCRS: Column): DataFrame = - RasterJoin(df, other, joinExprs, leftExtent, leftCRS, rightExtent, rightCRS) + RasterJoin(df, other, joinExprs, leftExtent, leftCRS, rightExtent, rightCRS, None) /** From eff852bfcabe4592069dd2c5b2ee5c7758545c95 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Tue, 18 Feb 2020 12:36:18 -0500 Subject: [PATCH 48/94] Disabled column type checking in RasterJoin. Signed-off-by: Jason T. Brown --- .../locationtech/rasterframes/extensions/RasterJoin.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala b/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala index c75b22fe3..a2c8e4a04 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala @@ -69,10 +69,10 @@ object RasterJoin { // Convert resolved column into a symbolic one. def unresolved(c: Column): Column = col(c.columnName) - checkType(leftExtent, "Extent", DynamicExtractors.extentExtractor) - checkType(leftCRS, "CRS", DynamicExtractors.crsExtractor) - checkType(rightExtent, "Extent", DynamicExtractors.extentExtractor) - checkType(rightCRS, "CRS", DynamicExtractors.crsExtractor) +// checkType(leftExtent, "Extent", DynamicExtractors.extentExtractor) +// checkType(leftCRS, "CRS", DynamicExtractors.crsExtractor) +// checkType(rightExtent, "Extent", DynamicExtractors.extentExtractor) +// checkType(rightCRS, "CRS", DynamicExtractors.crsExtractor) // Unique id for temporary columns val id = Random.alphanumeric.take(5).mkString("_", "", "_") From 8c96bc68aea435e9694b5057ab7bb6a7e3b08f93 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Denis=20Gigu=C3=A8re?= Date: Wed, 19 Feb 2020 16:57:01 -0500 Subject: [PATCH 49/94] rf_tile_to_array_int returns Int, not double. Fix #473 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Jean-Denis Giguère (cherry picked from commit 380e40b885d621b5b53331e9a135f50575140f80) --- .../locationtech/rasterframes/functions/TileFunctions.scala | 4 ++-- .../rasterframes/functions/TileFunctionsSpec.scala | 4 ++++ 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/locationtech/rasterframes/functions/TileFunctions.scala b/core/src/main/scala/org/locationtech/rasterframes/functions/TileFunctions.scala index 44b7e1127..588e596f0 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/functions/TileFunctions.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/functions/TileFunctions.scala @@ -48,8 +48,8 @@ trait TileFunctions { TileToArrayDouble(col) /** Flattens Tile into an integer array. */ - def rf_tile_to_array_int(col: Column): TypedColumn[Any, Array[Double]] = - TileToArrayDouble(col) + def rf_tile_to_array_int(col: Column): TypedColumn[Any, Array[Int]] = + TileToArrayInt(col) /** Convert array in `arrayCol` into a Tile of dimensions `cols` and `rows`*/ def rf_array_to_tile(arrayCol: Column, cols: Int, rows: Int): TypedColumn[Any, Tile] = withTypedAlias("rf_array_to_tile")( diff --git a/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala index 59c2cc337..a7e9d09f8 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala @@ -543,6 +543,10 @@ class TileFunctionsSpec extends TestEnvironment with RasterMatchers { val arrayDF = df.select(rf_tile_to_array_double($"tile").as[Array[Double]]) arrayDF.first().sum should be(110.0 +- 0.0001) + val arrayDFInt = df.select(rf_tile_to_array_int($"tile")) + val arrayDFIntDType = arrayDFInt.dtypes + arrayDFIntDType(0)._2 should be("ArrayType(IntegerType,false)") + checkDocs("rf_tile_to_array_int") checkDocs("rf_tile_to_array_double") } From 32a7b80a8eb3bbe599c29f8d211a3f5eecc0e443 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Denis=20Gigu=C3=A8re?= Date: Wed, 19 Feb 2020 16:57:01 -0500 Subject: [PATCH 50/94] rf_tile_to_array_int returns Int, not double. Fix #473 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Jean-Denis Giguère (cherry picked from commit 380e40b885d621b5b53331e9a135f50575140f80) --- .../locationtech/rasterframes/functions/TileFunctions.scala | 4 ++-- .../rasterframes/functions/TileFunctionsSpec.scala | 4 ++++ 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/locationtech/rasterframes/functions/TileFunctions.scala b/core/src/main/scala/org/locationtech/rasterframes/functions/TileFunctions.scala index 44b7e1127..588e596f0 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/functions/TileFunctions.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/functions/TileFunctions.scala @@ -48,8 +48,8 @@ trait TileFunctions { TileToArrayDouble(col) /** Flattens Tile into an integer array. */ - def rf_tile_to_array_int(col: Column): TypedColumn[Any, Array[Double]] = - TileToArrayDouble(col) + def rf_tile_to_array_int(col: Column): TypedColumn[Any, Array[Int]] = + TileToArrayInt(col) /** Convert array in `arrayCol` into a Tile of dimensions `cols` and `rows`*/ def rf_array_to_tile(arrayCol: Column, cols: Int, rows: Int): TypedColumn[Any, Tile] = withTypedAlias("rf_array_to_tile")( diff --git a/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala index 6f44b1ee3..7ac72dad9 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala @@ -402,6 +402,10 @@ class TileFunctionsSpec extends TestEnvironment with RasterMatchers { val arrayDF = df.select(rf_tile_to_array_double($"tile").as[Array[Double]]) arrayDF.first().sum should be(110.0 +- 0.0001) + val arrayDFInt = df.select(rf_tile_to_array_int($"tile")) + val arrayDFIntDType = arrayDFInt.dtypes + arrayDFIntDType(0)._2 should be("ArrayType(IntegerType,false)") + checkDocs("rf_tile_to_array_int") checkDocs("rf_tile_to_array_double") } From dafc51f63a2c243d4a048922ea20aeb963f2cb4e Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Thu, 20 Feb 2020 11:07:09 -0500 Subject: [PATCH 51/94] Fixed merge issue. --- .../org/locationtech/rasterframes/extensions/RasterJoin.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala b/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala index a7abf1f48..f2513cb03 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/extensions/RasterJoin.scala @@ -26,10 +26,9 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.DataType import org.locationtech.rasterframes._ import org.locationtech.rasterframes.encoders.serialized_literal +import org.locationtech.rasterframes.expressions.SpatialRelation import org.locationtech.rasterframes.expressions.accessors.ExtractTile -import org.locationtech.rasterframes.expressions.{DynamicExtractors, SpatialRelation} import org.locationtech.rasterframes.functions.reproject_and_merge -import org.locationtech.rasterframes.model.TileDimensions import org.locationtech.rasterframes.util._ import scala.util.Random From 9d0ce628f526ff3b9a1a5eae54409a3225d1eb09 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Thu, 20 Feb 2020 11:36:28 -0500 Subject: [PATCH 52/94] Updated allocation of features to correct version in release notes. Updated CircleCI config to gather test results. --- .circleci/config.yml | 10 ++++++++++ docs/src/main/paradox/release-notes.md | 8 +++----- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 6f81f1122..e085851d1 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -42,6 +42,7 @@ jobs: - run: ulimit -c unlimited -S - run: cat /dev/null | sbt -batch core/test datasource/test experimental/test pyrasterframes/test + - run: command: | mkdir -p /tmp/core_dumps @@ -51,6 +52,15 @@ jobs: - store_artifacts: path: /tmp/core_dumps + - store_test_results: + path: core/target/test-reports + + - store_test_results: + path: datasource/target/test-reports + + - store_test_results: + path: experimental/target/test-reports + - run: *unsetenv - save_cache: <<: *save_cache diff --git a/docs/src/main/paradox/release-notes.md b/docs/src/main/paradox/release-notes.md index 679f619c5..16f9dd1ee 100644 --- a/docs/src/main/paradox/release-notes.md +++ b/docs/src/main/paradox/release-notes.md @@ -24,16 +24,14 @@ - Replace all uses of `TileDimensions` with `geotrellis.raster.Dimensions[Int]`. * Formally abandoned support for Python 2. Python 2 is dead. Long live Python 2. * Introduction of type hints in Python API. - -## 0.8.x - -### 0.8.6 - * Add functions for changing cell values based on either conditions or to achieve a distribution of values. ([#449](https://github.com/locationtech/rasterframes/pull/449)) * Add `rf_local_min`, `rf_local_max`, and `rf_local_clip` functions. * Add cell value scaling functions `rf_rescale` and `rf_standardize`. * Add `rf_where` function, similar in spirit to numpy's `where`, or a cell-wise version of Spark SQL's `when` and `otherwise`. + +## 0.8.x + ### 0.8.5 * Added `rf_z2_index` for constructing a Z2 index on types with bounds. From 085e83454d74f0efd7f1fc5aa0978aa12b40337c Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Thu, 20 Feb 2020 14:58:42 -0500 Subject: [PATCH 53/94] Fixed duplicate/overriding Python functions. --- .../python/pyrasterframes/rasterfunctions.py | 31 ------------------- .../main/python/tests/RasterFunctionsTests.py | 2 +- 2 files changed, 1 insertion(+), 32 deletions(-) diff --git a/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py b/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py index 8bb3e95d6..daa4de1d6 100644 --- a/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py +++ b/pyrasterframes/src/main/python/pyrasterframes/rasterfunctions.py @@ -26,7 +26,6 @@ from pyspark.sql.column import Column, _to_java_column from pyspark.sql.functions import lit from .rf_context import RFContext -from .rf_types import CellType from .version import __version__ from deprecation import deprecated @@ -664,36 +663,6 @@ def rf_round(tile_col: Column_type) -> Column: return _apply_column_function('rf_round', tile_col) -def rf_local_less(left_tile_col, right_tile_col): - """Cellwise less than comparison between two tiles""" - return _apply_column_function('rf_local_less', left_tile_col, right_tile_col) - - -def rf_local_less_equal(left_tile_col, right_tile_col): - """Cellwise less than or equal to comparison between two tiles""" - return _apply_column_function('rf_local_less_equal', left_tile_col, right_tile_col) - - -def rf_local_greater(left_tile_col, right_tile_col): - """Cellwise greater than comparison between two tiles""" - return _apply_column_function('rf_local_greater', left_tile_col, right_tile_col) - - -def rf_local_greater_equal(left_tile_col, right_tile_col): - """Cellwise greater than or equal to comparison between two tiles""" - return _apply_column_function('rf_local_greater_equal', left_tile_col, right_tile_col) - - -def rf_local_equal(left_tile_col, right_tile_col): - """Cellwise equality comparison between two tiles""" - return _apply_column_function('rf_local_equal', left_tile_col, right_tile_col) - - -def rf_local_unequal(left_tile_col, right_tile_col): - """Cellwise inequality comparison between two tiles""" - return _apply_column_function('rf_local_unequal', left_tile_col, right_tile_col) - - def rf_local_min(tile_col, min): """Performs cell-wise minimum two tiles or a tile and a scalar.""" if isinstance(min, (int, float)): diff --git a/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py b/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py index 57a8ac3ce..c51541fca 100644 --- a/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py +++ b/pyrasterframes/src/main/python/tests/RasterFunctionsTests.py @@ -249,7 +249,7 @@ def test_mask_by_value(self): rf1 = self.rf.select(self.rf.tile, rf_local_multiply( rf_convert_cell_type( - rf_local_greater_int(self.rf.tile, 25000), + rf_local_greater(self.rf.tile, 25000), "uint8"), lit(mask_value)).alias('mask')) rf2 = rf1.select(rf1.tile, rf_mask_by_value(rf1.tile, rf1.mask, lit(mask_value), False).alias('masked')) From e6864df4144c6ce27a449c42144c4fa68c267b2e Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Fri, 21 Feb 2020 10:33:54 -0500 Subject: [PATCH 54/94] Merge pull request #469 from jdenisgiguere/fix/466.1 Unit tests: write a rasteframes to geotrellis with pyrasterframes (cherry picked from commit faeba8754a9b9ef1cce8359758b9af2bdc703f63) --- build.sbt | 3 +- core/src/main/resources/reference.conf | 2 +- .../rasterframes/TileUDTSpec.scala | 18 ++--- project/RFAssemblyPlugin.scala | 1 - project/RFDependenciesPlugin.scala | 1 + .../src/main/python/tests/GeotrellisTests.py | 66 ++++++++++++++++++ .../src/main/python/tests/__init__.py | 4 ++ .../test/resources/L8-B4_3_2-Elkton-VA.tiff | Bin 0 -> 189138 bytes 8 files changed, 84 insertions(+), 11 deletions(-) create mode 100644 pyrasterframes/src/main/python/tests/GeotrellisTests.py create mode 100644 pyrasterframes/src/test/resources/L8-B4_3_2-Elkton-VA.tiff diff --git a/build.sbt b/build.sbt index 383f27f25..8f1582e35 100644 --- a/build.sbt +++ b/build.sbt @@ -67,7 +67,8 @@ lazy val core = project ExclusionRule(organization = "org.scalatest") ), scaffeine, - scalatest + scalatest, + `scala-logging` ), libraryDependencies ++= { val gv = rfGeoTrellisVersion.value diff --git a/core/src/main/resources/reference.conf b/core/src/main/resources/reference.conf index 941825fc6..fc76eb5a6 100644 --- a/core/src/main/resources/reference.conf +++ b/core/src/main/resources/reference.conf @@ -1,7 +1,7 @@ rasterframes { nominal-tile-size = 256 prefer-gdal = true - showable-tiles = true + showable-tiles = false showable-max-cells = 20 max-truncate-row-element-length = 40 raster-source-cache-timeout = 120 seconds diff --git a/core/src/test/scala/org/locationtech/rasterframes/TileUDTSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/TileUDTSpec.scala index 62ddeeb70..122bc3398 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/TileUDTSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/TileUDTSpec.scala @@ -96,16 +96,18 @@ class TileUDTSpec extends TestEnvironment with TestData with Inspectors { it("should provide a pretty-print tile") { import spark.implicits._ - forEveryConfig { tile => - val stringified = Seq(tile).toDF("tile").select($"tile".cast(StringType)).as[String].first() - stringified should be(ShowableTile.show(tile)) - if(!tile.cellType.isInstanceOf[NoNoData]) { - val withNd = tile.mutable - withNd.update(0, raster.NODATA) - ShowableTile.show(withNd) should include("--") + if (rfConfig.getBoolean("showable-tiles")) + forEveryConfig { tile => + val stringified = Seq(tile).toDF("tile").select($"tile".cast(StringType)).as[String].first() + stringified should be(ShowableTile.show(tile)) + + if(!tile.cellType.isInstanceOf[NoNoData]) { + val withNd = tile.mutable + withNd.update(0, raster.NODATA) + ShowableTile.show(withNd) should include("--") + } } - } } } } diff --git a/project/RFAssemblyPlugin.scala b/project/RFAssemblyPlugin.scala index 3a39bc917..906a4727a 100644 --- a/project/RFAssemblyPlugin.scala +++ b/project/RFAssemblyPlugin.scala @@ -56,7 +56,6 @@ object RFAssemblyPlugin extends AutoPlugin { "org.apache.http", "com.google.guava", "com.google.common", - "com.typesafe.scalalogging", "com.typesafe.config", "com.fasterxml.jackson", "io.netty" diff --git a/project/RFDependenciesPlugin.scala b/project/RFDependenciesPlugin.scala index d4432daae..e64d46d22 100644 --- a/project/RFDependenciesPlugin.scala +++ b/project/RFDependenciesPlugin.scala @@ -46,6 +46,7 @@ object RFDependenciesPlugin extends AutoPlugin { val `slf4j-api` = "org.slf4j" % "slf4j-api" % "1.7.25" val scaffeine = "com.github.blemale" %% "scaffeine" % "3.1.0" val `spray-json` = "io.spray" %% "spray-json" % "1.3.4" + val `scala-logging` = "com.typesafe.scala-logging" %% "scala-logging" % "3.8.0" } import autoImport._ diff --git a/pyrasterframes/src/main/python/tests/GeotrellisTests.py b/pyrasterframes/src/main/python/tests/GeotrellisTests.py new file mode 100644 index 000000000..ccfa31082 --- /dev/null +++ b/pyrasterframes/src/main/python/tests/GeotrellisTests.py @@ -0,0 +1,66 @@ +# +# This software is licensed under the Apache 2 license, quoted below. +# +# Copyright 2019 Astraea, Inc. +# +# 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. +# +# SPDX-License-Identifier: Apache-2.0 +import os +import shutil +import tempfile +import pathlib +from . import TestEnvironment + + +class GeotrellisTests(TestEnvironment): + + def test_write_geotrellis_layer(self): + rf = self.spark.read.geotiff(self.img_uri) + rf_count = rf.count() + self.assertTrue(rf_count > 0) + + layer = "gt_layer" + zoom = 0 + + dest = tempfile.mkdtemp() + dest_uri = pathlib.Path(dest).as_uri() + rf.write.option("layer", layer).option("zoom", zoom).geotrellis(dest_uri) + + rf_gt = self.spark.read.format("geotrellis").option("layer", layer).option("zoom", zoom).load(dest_uri) + rf_gt_count = rf_gt.count() + self.assertTrue(rf_gt_count > 0) + + rf_gt.show(1) + + shutil.rmtree(dest) + + def test_write_geotrellis_multiband_layer(self): + rf = self.spark.read.geotiff(self.img_rgb_uri) + rf_count = rf.count() + self.assertTrue(rf_count > 0) + + layer = "gt_multiband_layer" + zoom = 0 + + dest = tempfile.mkdtemp() + dest_uri = pathlib.Path(dest).as_uri() + rf.write.option("layer", layer).option("zoom", zoom).geotrellis(dest_uri) + + rf_gt = self.spark.read.format("geotrellis").option("layer", layer).option("zoom", zoom).load(dest_uri) + rf_gt_count = rf_gt.count() + self.assertTrue(rf_gt_count > 0) + + rf_gt.show(1) + + shutil.rmtree(dest) diff --git a/pyrasterframes/src/main/python/tests/__init__.py b/pyrasterframes/src/main/python/tests/__init__.py index 844580622..330857c14 100644 --- a/pyrasterframes/src/main/python/tests/__init__.py +++ b/pyrasterframes/src/main/python/tests/__init__.py @@ -75,6 +75,10 @@ def setUpClass(cls): cls.img_uri = 'file://' + cls.img_path + cls.img_rgb_path = os.path.join(cls.resource_dir, 'L8-B4_3_2-Elkton-VA.tiff') + + cls.img_rgb_uri = 'file://' + cls.img_rgb_path + @classmethod def l8band_uri(cls, band_index): return 'file://' + os.path.join(cls.resource_dir, 'L8-B{}-Elkton-VA.tiff'.format(band_index)) diff --git a/pyrasterframes/src/test/resources/L8-B4_3_2-Elkton-VA.tiff b/pyrasterframes/src/test/resources/L8-B4_3_2-Elkton-VA.tiff new file mode 100644 index 0000000000000000000000000000000000000000..c351f58871f83b14788402a35a8d05dafae252ed GIT binary patch literal 189138 zcmb5Wb&wR-7dE^|aQA7sa?ZUyv$(rka1ZWo!Ciy9ySs;2NCZ4uRm#`;h$J zs`sz2>Vu-`nw{O9>AClu$4-|mn?5RDR8&;5sHhmRqGI4|C;r9w-}x$>$HaLo{C^z( zV*T&&ug{C~|C}d}5j!gM?_*I>ssH!**Y8R5zw@{_PmA-f&pC^K-v9Hx)o>mYkA3}9 z9p_)auPuH-k|BxyJ@16`!9x>8#lhnvalURutSE)^?Ks~sB4(7p`CFWS{gdq9zeC0U zzrXckr@;%Dt)iktk*KJ}4Wgogeo;}YXGcY4-VqhG@qAQN?$@|3sbWMG&JiQ(*Qzn1 zO5;Tx9v>sB(%KkNr;o&ls`)TR)TMYaqZ)=|M%^qGGpc3tm{AW0$BgQ*C}!02Ju#zt z{1p?|_Uqp&U*_rr_X`yr8skm+R5uG1{ral^zy6HY`C{DZYl}rq8qo7*{wl@(?{mNY ztf(kl*Vy>9zW)8+|1o247?B2Fi4kwZh`3Q{qT=AgjS)4TqY{1nzs=N72F0h^Ke4b{W zm{pY2EH?`%iLZ-ZbP`J`4ga8W@IHPn-%u;gF85L;?kWaQGd`_4a%z)b4dUfI zL0ePEG*Lft8naOZ{41|=ujzu6#{EG**59}_^fNuqU8-}@T`wIa<=buxU7W~2skcyC z|A0PDv)v9lBjxvg*ZrxQ--P0DCw~fg{Maq4veS8IuCX{rYs1dU}q|OMg1Q z;L2xruj4uO{KM)ppKw;o_1YD$^f30EVj`_R=}ebn^(uFs+(5USxn(mzkJd6xeNy`>y}OFBmNy<)m0 z{p^O+8=cH8D=+Kf&IHjzCw6L!7Wzx5oM^5;gi4FHI*wBh=Z@0>uQQ3$R&>#Eor>an z9nYyDy6K41QVh_Uuzn`$R{l11j5esa9M2>Uw%PZ#eth}E=} z&#U(w73|D1KilQ(y~RjA$5rT~9UD9l%V-Pl7rAMmsjSLz4O>@Epdx0Y+QP}~Ak~m- z+kE~Tb&J>4i7i}LzS)kiw$4_70&wf9AqphwOjF;3rg#*1k>wR=}A#cmyrPhn+fwjV>+ z@)D_x)Y?x(F4yuqVa-qVW3b@`{y3a}=ik=XXr|u|*R6;cLUs5X`GOwt0r!r2p|?3g z4%Y)@Z9Zd~>(aJO(BHeNy3-ivCF($2M+jGcc6!Qns0Ld_G5p@NGQaNTtP(3wC3=X` z`k>oD<)&CznR6+vSfzLKS+$sx+Dz_Ck%yW%%|tt$(McofVhu!#$~uFSP?W(@itqH( z&_>_U1KbWO97u5KcBHx)CFO&4>|`7DD=CbKbOe^hcUSU#M!KKYcbDheV6=s-f&zJav8qaUItSZD4xu5)- zI`BHNgdDzxt8s{rs9l`YB-TkxZ8Jcer~SOu{arVuuI?tCk}A6cbaHats`|Z-=ho40 z^&PJW9i~Ozb-kN5dy8~4>g3JBahrEbPoWf8lUMNxKBcN$*w04SD3!Zep3zU-7fRDm zZ=PO7XHjQ<)wM%S{j;i1sJnk#H41&_-%#yB9Z_3qI5S0i{lSSP59&wGVX<33bS{ZY z`j&G{?9%@@mr+&TIB&%feFN3zh>nXob6bCM?&3NB_jpYl)i<5vVx!*eHbrH7;SpEwb@K~Hl($%}fi8%y2S)x5lFGR`))Vg_KR>n-tNC7pFem+zfxE4wq8wBytK5HQu~eQ5DoX|&}ACokD=?- z$)8F0X}-UgVsZ~JwH`ppy+*1UeR1x{4|<(5Odi#X98Yf4Wt;_~mrjH_(@aO<*a+38 zrf7^hQ(d&uX`Js+XI_R1qQVq(T47&obi#7Gu8P|Jm|iN!tTN-mD}#~2y2zE__n=&` zB(ULk(Pc8O&Uhok(M*jp&(Cxw(}FD9GY`w0F3VkUoycfE89kzh{%rb||G+UHfADY6 zcf8b(;{N>7FGa(7o}9`z%uLyfkC}PYE|?O&D6^XLW{S>en+8QxbDm+c=!<;8CZ%fT zqFqYYOtxUB8pzdbCDoMY+X`|CT`~_u2P$u3>4RO{EJkCXZn9rF5cuP#OJ=>ze+i{uRnp_(;ojX%Ferl zhrZuHo#J#hu^6vo@p=)H_HmS&#-B|^*@~N(;r=?*(rCAu+@|+oZ8z2j+*RtH&hKuQ z_w;1qa{$m7iaaq9b}Prw9&-vXqre z>L=XZ#I`4G>0q%*ZQI)><`rtlAEva;Za3J?cB{Q%I@(fpt4U%j*hxIx>@{)BQ*+nE zGUv=&Q;^D;!Dggd!>_p%>d9#;C4Qxev{Y=RZk&Z~m~!^8ketj6bXV%~6z$H`|LE23 zKAoRJKqGH-dbfdoty{UnbRrt=j?!7FhTj?$dyw}~52n8A8MiV&;b+d|(tb@!$F9i3 zz1i`$p`O&lb$g?acsJ15KX@7|c~9t(|54Qree0i5<3fY{x2k%mmH$xv7Fy*;>mAN~ zu}-tA(T|e5iR5`rZf^MySn-lLq3;00oYvQ!>tert?VJW~`P=zJoYj||{o;@R9dF|~ zhwu~9%Ln=&cakbeL%bzGQ@Q*I9i;5uCRCeKPEI*eH}cZ!oak{^^bVTp@6+dLy{bXBjY?n)c{J3uVe{QmTkmf?Nfsa)i_ zZ*Z+Ld(o;E#c&N!%^>u$@&DqMI=Y=RP|Vip{4{Dca9}L?htBRV0n#vPoS9)(2Zhpi zj*c?ZY!_QH@*qeQE)_l*9v}Ib{%*!JS-NCOk|l1|cUe@H6PagaT$eFsru*UyB{#|Z z%Q_J!@OS7CANHo}t907SLJ2vkSWmz4FfovBb4{_4HgX2v0)s8^XV5oXUYrK{Ju7~q z=DdNL+u}iNRFz1OPgbO@rYq1(MRSbyn#{o&N@SBo{-H^>V7LMIvq{2&+M5;O5o#Xy zvB{{3DG{s`SM+Bt25j2K?DA@>vvgFv)$h5Fn4$BS6EY7iFjC~v_4u_jO1#l6f#MqJ zTW+*0O*5UJ{qZ`3yHhOF)tw(iMg5m274a9RzSyB}dkIu$T9tlzxJGzFraO^5ky`$F zpsJ&OZe z-g~a<(*n1oNXYv8t*0fl!pp@w&`-bP?G%V9 z+#EIVSKe*5t0BCHBbeyu0c&?^)?4UJw z5UnUDUlq-0D)$3!>5QLn9?SGEcdQ(THGCQTI{|@US>K8%yt22EsM8B zkD(^sGd-Q+<2@cgg&9E?sDKya>zuMHjWwZ0m<>F3dZh2HzAbusq`yoS;~;Zxp-9yEY<0@+5W zi&qV6sk*<3V(>bD5mx(DKZ>5x5dUxekW#CxJlb@W-}6Y*l*8dwk#^KPco8lsV(hHIq+#D@mQykMtu7Q#fgVWeqb`bd8J^jcR)Q|g%G@RKy6fXAE7q7Z1$?btu z`|2uAQ}AWqNe^8mmh;#zphM0}zo<^?oI`DS?rf7gbhH5HvWhA>bNFiddllmo16I6!Rs4q&$xCKWpN+P(d*<|)n zR@2q|N)^pm(~lCFp{5UcX1dA1tIcsU##RZ3A_oezDuAZzd13!*1}OG|f>PC>PR zMv~HT_ZPhP;=nD5sg*lHC!=-l0@R!uZddKlZKtw+sgJpPbt}38<)H-cM|FD6F*q~$ z*;T>3pCvfaGfL@AMi)K`rR9#!>%RwLZ|HXc3+sbj7VsQ5m&!@&L-YOEdP`_MKGTv; z5m_I#_a*wyY-g|N42|WE*ap_~575Xz&L=1>U!0iow0`G&gsu|hJ{LDpVZPw_5|1AO zo;itU{&#$b=U>t}+@kWe&f$$#ZBbh`=_SPIyj!q$=II|Or=MF-pmcr`y@s->gVfzr z6?v&6|0c_DK2y}0Eu>!Qrd4I>zS~|7p&3pW)Rs10Ej^J2gX4Uncz!;*1Z8Zio<&h! z13iR_dI$7M@WdkY3nlS;qK-8MH@`re{j*qm2fQcx4=}{-s3&8bMtIHJoZIq|UhmwO z&-GmAmVBirIFIB%dZhCd$MN1}u(SNK6ts>%y$@$MJ+|r0Aiel7=Yl-I6i3^CCQ0MxJq`VOLWIAu*k7j6a+&=evP<3AKU03rc zlOK;X51_ube$YwZW|T zI@$*RvHCT1*?+Azh5q(4=q=6~th%A@0{Kkm@sg@6R0j2HGUXCIsUX+)DyYV^-W?1q zUCSLIFX=Te@tD${+ZeS)>aD1%m}Q>(_1zs)Kddi&g?f|43`;CvPwxqr=`e zz!u+%MO=#~%1wNh^Qh^3hDZ8&!D1@;8E765`|z(^kJ^}(Ha*2P{mep2X!@JOsDNWl z9Uzu{W;vfXm(2hpZ97}ej%vqQib{N{ zxVK1+YUmo<|@vV~#2q)~;#tu#8^;eZ}ZFlL#l+FK+Hq%aA&%bG*xQNx;UHqe8K%JhZ zdee7)F8v#Hvz>YsH* zIhdbgci*Cp=7k>u0;w&(_SSjgGL7a2sIkYn9DZgD6Gh|f@AhDZV(H^XYM}BgycD*<7=!(Q}>1TwtJ7L2>%S+_J^ESg;_{h+o(hk-5q@zu9UkmHETAfbRF39jJ|Y zhe!7CCQ}4md!ET5FVJ%{LM1gT%}#y;?d1|*v$4Y!D1m)ud+4dAm7M~HmEE2sVg5GD zymPWWm2zC{rgfnc{v)*pRpNpAJ#^R4rn^JmJEe=b9pz6tIUJMBKqFDY>XNRNv-oD667wcapjIJbwh<7>$nE82I*qn@8mZ zo*#kldf~0X=bm3!9>;g!hFs+?w8YFc%XHW@Hx2a@;Q1PIC$3&MDrmC=C-hqr!*+zy z(Kxuz(}Kg{DMC;KRMGbuy9PM9s*dH(00&>`{Dd`7)3>@QpZ4Nl2W@wL1Uu{twXL1b z?_?IObxNSPd{A2ALkEuGT*G^;>h_fJsHK;e7gHr~4{FO*uN?2EHQpxv1D?wizD1ej zAH15=q93i{AE2TA%-@R5z+KUzH@%~8z<-ZYK72+w%xeC}Oty1P8q2m1IMM|Z8(vXL z8*NwOZzub}F0`ji4?D~j*Y`QDDQtGwdp4Q5V$TLWQ5C=EEN&B>o(j7|bXtml^}N?Z z+_~DLh3+&RO?jc*euCn(07#~YpA~52i(eLs)(9xHii`WXzzdW=6T2y)AD^DUQhHtio$xJvrblq zKA%CnQUjfz!6@%Iuf=F+>8a&T{mgj^b!8o{@BuhShk$#Zf(`uwZ9fiF;$zM%)Rw=T zed6c;9UtO*kLrk9K;GA3Z=k9`2fb53DH;5#v=>V3SJu1|PFWV{2~{XQ--Tkig%0^X zwALT}y6`gEdbw3Yn&~c-n)CiTQ<>luprfZw#EWuzv$u z%}wtO{IacHgmzJKzYm?DnqEWQ2iiy-`6KX{mA!R7X9xOf3}>Ebh;?#A^w))*n|P;3 zobvLBP7f7jj&AF2lIfsNeuMR&+DV5;Qb0-msJA-(fYKK_YvfDRrLkaq?a^EBLPam8 zw^JX{9)9AlG>@m)#6|>%?YWE@GgQj>Z4fJ*BDykqYdJep-{M&oD!4grFNv<4)5P=F zYs<;}^ZFJQ6>H!g-j?e*p{d1l(3x)X4I5?WtB5IX7VFMX`4aobwBmTOF!wcYsHa^Y zZVMKaJ~B?nhq_XQwiy-Fr1{`SanyG5ZDr`HPk_fi+AQG-RM}<>H&>1E8ae< z7v3FoqJj43@F2QrYTDR=8@A!Jc2dwGvd~@$qM}cjl0o@MJ9{_yA@Ves7rq)<0aXVT z)Sn8?=40ripH;sLd18t_63QcT>7Gt@*+$QHr()$)^OY)2)BJVnd+H)@>54oGd~!9< z1)FL@aaAq8%)?|sUd#W;=BUyi`KFCAFP^!kePyu4|JkNs``o=4U z-t@cQ3>fc~sKGCJrn|vE3-pWkBm+lYTz6CDi z1pg1c0J`o3`B=Mf19RyUp@&cgBrsM&Ztjc}d-Y<`f_fS49n%$PsGC#0(|hfr@1?qGg8VqH(k(~iO_dRbgA=`l`nO(-m z`7u+8Q`)oUl`fB3`k16GXcO7y!Tul>Z8r;SYILjZ+}phjKd7v`5y)g1Pzs|*&xeoP z7HlX4w4aP-P#5t#O$SO@hu>Sr&j?4dvfl#iCJl6z#M}_BZW@jU-hCb2ydmADs?K)w zh(oTg8&VhlHtI57qdyT9@-F)FX`MUd`*+lU&_X{hcGGpZa=Vt$sK`TC?w^1$cte#2x zJwt2gg_}?p0gJ4LURl*SifTF9c_s#d{rw@PYI2r|$>2V_#cEWWp>UHAhgzZ^T@CHP zc|RwgJf}+mA+Od;-IKBaopY0@BE)#ViAcIjWfCgjIC77cPExr>Z+Aw(fgS6HRSw$c zO#xduBY;7-uWsmM<~_#6MDP7U4W z1(et9+}>6R>O}8|Tnv{8QiWFqRaIXolRfa>f1-NmbaBl!UCnk51^{WygqCs@yML>k zPYLZttxdGL`%E^1D>Om5aC}0@X0$`^K4N7=OZ=_I9ulMBi+$d4f@K%l-#ZYJB?#sc{9}vD3eB4-?|;C0g7Xd+y*snkT?fs0p6O{4DI1?V$7FH)=EHLO!hX$91bxzLLes%3nGKggf> zDZiJ!`2ojJo%jjw1!B3tzlly%i!1qq^d5>5S>RBd_0v%@9soz+0bLQBIh7fvDYzdr z;yyv~@F8}BQNdJmJs2B4OV{o1L2+eGMeECoJjVRy#l_Wo<~CE~Xt`5SPKQF$RwUB< zLZ9Jfd_`ZW^l()gKl2DU|2*CxG6D~j@)}SF`W79zE*){Nz*DZ^{tlO69(?M4x|)+z zz`u0`_DEYfm7apb-lit-(K_=9dg#S4m*|O~*hILefS$-5MVK4$7EzK1!7tv5$~jqf z<8|CZ9HNX|O^l}$+*wS(?p`g6qh`LLX=aK|X?@$(cH_s-N%)3#fG`_DuiggTFRdA8CW~4~v+Q;^=&H~pV^J%-_s9BM8tX;V9y%d%@=;DB zKEkt}1D(7OmqL9=z=NS;<>2rAWALyR`iZzIax4?!U0!y}tAbPn`z427;iUjCXzriH zGuz-;idT7i^&v_LM?Sx99?B2Lc~$5V^!WnVL6xDdj8)@xE+m2~!HY?!%26*p+$^<) z^-aEPhsacX#q89z>}lIrB{qF*W7O^jeAz#wEAu2j70^*8sEd!F0DL{}fhw4ve76i3 zN0c`aoT!O61b$XRaH3OC;2#11|0IsWyJ;X#f!)_qEw~#`kf%8dr&T5RA>VT911(g5 zgECMy(`_^7cr$d$||d7Af9ub_qSj8>6w*T}cJ zrh89DQD^4_QWK5gn2*t^oGoIrej8c_M<5d(8K+mnD@RrewQGdVi}X}~UBmfRjD!m| z0VqX?eELsXBojm7>EJ$qEMc-B62n?g=p`3sT~pMdKML9_Zrw}3}~ zRX=qf${nV?bSv&98dF9SAFA~OBut>Zn?d?Gp9+TPZJZ*!Ma43k z>_uBDyd&H~r!)O+4)k1?Z;Ji!K-cP~aF845Pn;>pq3fBAHZS)4?*XazKqZBBdQ&sJ z*-wezQdjTcc!3Lb#Rg~9S<^dc5ssDqS^7MxJm0c`nt|$3h4%#~!aL=3I18=7tXFU+ zQHDNq11(IRptDTRs5*WdtvRRL9Jn#3+eyyWU-52f?&9@QYk>UkDD-HkA|E-vTu=2m zoBSR6|qdtiTec(MFU zi+Lk3Kvk{{RrhCT6t%EIBEU8WxQ>_y3^NWs;A`sUztdsf(3$Y^a(@qA!Fpi^O_Y$Q7ye zG4&l-ZEPUA_$CB};WV}N3KR4y?~|@Yt-Y(d6#al+Rs~+?JM6V8-cZ#EyLA&7cS2mf z-{}MVg4FQJOYk?SiZRSnx+^|$SKcd1aSxsjcdrEx70Xe7nu~?#`?I~j)Ib`8q}@r1 zC+Z+Ql2?p@8`uSETQnEcG0g$f(%!UjgMOx`tzz3#XVb(^0RR1LMtddU$CdC>LQM_C z30&JiOrmzES3|gvxuf22Ih#;6Q|A8cXK6?nLqP2KQ?-LbN?_H z#J5O3Wa7p^BN@2|&a?0k{~lgpW4|99mXh8g?Bcwh)$OP}>Q51_2k$jA*N5sG;lk*@ zDQQ}$xBpx<3pIiVJtnjgN@{a3g^K!5C%5X4Ix|o-(EG(a{elMuDzYF_$fOCr33oQN zg0vB1UkBSGy>w=D$(ia7wX<>6805i;Y-6sA(&HwK>!95}qz~PMt-DaEnL1qAo zD5Z(#wyr7aC%l{}d za1&sUZ{hjoL5(WtFW1BAg?|~J;~RL9+4+*F$0Jd5O2SDw2PgVxFyE271Ko1}QKjgR zGg~g#jhw`CiY@~5au9k;6f}X#PD&gPI=SULaP2y%A6J}0^1beaTCq>(LLMwG4fgT_ z&-ZsLp+=PkHe0SM;(K?&c}#-mxX8C0)L|zhvN2hpGVOp{eIAwSIcnDmr1Jg{o3yK! zAop~HMw&sUHu$__uA?d*;Q{g!Ju*YnkBUx}K7i&wF8p;L@sj;_W)F>Bi>Q=e|!!KfRHxR3k;9$$ZXmL_m9 zxd-m&6?lJDIk}tyY&KD*gDzcD{lv{oTXllJGvn}TOCc}%9w}2oQtBVtEONkG{{}Ux z2cHr1v0ui@d|(g>WELLFe~Q#x%?$RY=+2ZGepv*{XpW$MFv+HgjEu}v3!t=269YIf zv-m}rqWeKzA8axLN8P1MUT=KYB;eaB^w!M^6{54#PS%9WGYL9#d}pmFq?2PWFEV|h zme)e%j7i_oMDVY^)X#gY8_`Jbq^|KV<8Dulz14W+0@!_3c(1*YtQhXM2O3}JcL(-( z;LieP>jmE=245E~`3KG{Mq#&35PiUT9l4Hv;%||}TW#X#A|`%#EVR56yjT@BM@=7e zkmTq%n#=Jf@tk_HN7KzwYmnMKV=u}@NEdVy4A8LPvn%W%QZL_l1`1FbTvgvVXJ!$PURCXhldj*w7*QjI!Ew;xj$uH$c3p%}3D&TJcq}om!f0 zbjC)Ia!C=a3Gajx)D$U^;<^ZzhbEKCb^%toVC$(loYubAn~>i6gbx0Wdhrs}lLMxPo@q9lWO&>_(>{mY@kE@Z-oQWccs>`B(L9fv$&FB83X2Q$ zJ$~jUuz>z}=0KB;irPH(ib)$@8~)7O!X}bawt*kNQjCT!njgQtF#7mY=x-a4zZp+6 z{E~VY?ehnM85IG;iqD(ee(Du)>{B556!@g_=wGBo-OJ)l(-*0P>f$e&0j$#1m@D?~nlh1{i)jw9ql)0ruu;zZ>%K&7l$90VnE3Va_JD zQW;!r3sjItenA3fK7C^+25;@=Nc`xvd@+a-yopYgae3C(Dxaxpl0a*k$EAeBC3yw* zcn&_}dsK|Gig7fa9Z??M**`Kr)=z3xfaBV4v6hzeD&53(3u>ENL7T|$^sTL9bDD{E zVQ@~D!+vfl{zO*O)mN+=JQXK7Vn<`YWCX8yj9O6zDa2gJP)$c=`v>pz0>$yRs4{fS zJ)!C$KQ;f`>ujg}RB&hO2my5yg zO?g<$o5N|~)fO=S9ml$H@EeCXJAr^Ix?|w;7WVS!_Eg5Z47^bYxbZU--L+s+`%JL*KJPq;u-Ed2@QA_l@ zE#Tb;aFotQu@PU3SuSg{QeQJC z*k{HDKSy#QJ)1ck(ZBH`dsn`P58DbE$B>yT>Lb5#QH-E;<|eppPFqJ0F|C6VemA`a zi87bo@dGd~C%CF+GU&T$9MlZ|9W;t~>Eoz0+|VY+JL$}0uydcAV*X@Zg44p^=}M`w z1{yL0`J^%RfPC5lc^;$zoEz1-1rL&c(GuX97jW?Fr~`0N3#pW5vKfr@@;TE@uBKwB zGY|P2TU)m@4@`d9leeKVti{jCh1b6t%I7{fp?UpL`X)6H_h=;_AwM`8Tn~~&d(l~R zJCoLC1+R+^^3ZzlhT?u&S_XXb9(+8dzd|v=2tr!>7ylU)ux9?BstvN)kDwXe@f(5X z9(Tv78&DUY$)-RrIi%73{nJp?Z-S3#q@c-kH}DQnRsQh4hk|#+D}Xw4-AhaZD3SO9 z9#c+uwPCEcPLzQKH0O>;4lGADW-jt70@_6c6>OBB1l=)K9}e#}RJp`takVR;=Z=Qbl0Jx1lUV_~ zb_&i0hox((SWlMZpEwsP@iM*-Vdeevkw7aHqX&ukVS)Py_bqxrT!D zvNH6y`Z6}hGV|0d9&Vc>!`7XHo#jVya50gYwweU7$ z|Ne$lUTN^$#;QG~!B;=&EMhMdo64dgXXOL_YA~!-_;eC;6*w=2xuocfX`?A3KUYWI zH6EPkMexm+qE;LgvyjzEE`Ns4ohXms$ z>Ci3j1W?36Z?C!vuj(ZB;#wrQ3S;%oMjq@f^5Ku6RuPn?Z?3KBm*B46Wc5$5EvkP}1V$uWO4yVNaaQy|7L|b$r&W)@6 zo9Qp3xI55Jao%WSiA4I864!O82`d8Hd)=)=F0UNsCAr*s1Qn;oJrn+~?0N?|&gOV}fq zIa=L>UcEw2NA@NsP*$bzC3`FUA#%&6i0q9dq&{|9ut~nhnjEOsb5`>g=EV*nhf_qy z##GC1Dj9fpJH3w2)WOO}IAG#9#V zHvbo$j$6o-+{ZN1({NR;dA(^SJ%y+IC#p&-S`B~1BE6qc=7c(wRGdM@cfzmg?HF$t^xYd!>CdBe@Ci&DOS z=j`5izu>wFe?O|nSA3Ee-FYD@XIg(SxJDefMF}xU6dQH#0VWg6&L=Le84sir&dpZ^8P!naF5EWR2Pob5M9>YBJaTs&qup~ zE;eJzY(27$)uH`1*U4DZR{LkL44V94t|;q)W8T4 zd$u+9>^eP6yhqZzr|8JV_>sQ>b!MYKo>FpaQ4tKh1f0IooEN!>99-AS3%_gy=0pbQ z-<&I0Ww+q$>_df#1NHR_9OqwNCjxEz}p0`d*I5-$L)b;0;%;Xf8B|v3x*P z=2z%7snC`3VAd#_e+8@5+|B(_=B7Mue`qyF;Z*jge88*m%ugz#Ibf%#L!8Gh)*X

BVcSjH`|^4VsIKU8nvi)sNFP+DEq%&u=cnZQYcS#2mcY+Rb$$3Cx=skgS59@{3rS?N!1I!L^8DtpY2h& zlShzGt4C*foE~RlhGzw*BSkZ&^Qx*mwAPEIOVSEa6`Xy#{=k{c9eATB&3h!z8=B{E zL#CKH$}^u$3q8nGvYDU&WJM1uW0KjossWOyne-aWKi1HPObXjl-NbvDD0_ovJ=EWu zWsbtTS^D2uhF4@nU7SbsdQ2wWM=hF(>8fPh0I8t^KqGUJJzI|)P9~&NN+M-i z5Y=N7YGy8fG%8O%Oo@8P(Iuou6!D`_RW|xR0PRf0PRu~Vk&%0&I-$0_Q`JJ{{8;)X zl92h4Oe>-u=)IUzj!;Tb0=m-;Zz`1kC>4+Enos(SIc<~U^%t6pC(bD7uA3AxyMg^G^ZEjD+?$khTG_OhWbeh&icy zs4ClmH2xMpBExe}tkGwoxGvF0p`6VBx3;X*2Gb3P^?d<@6Z52*c_v)ryf~hbMR+zR zQQ7zy{{e3|rWpfIs4nXN31EP_NbJv}J>bZT%v>GValuY`5x(LK;2r#8*EW;3{s<37>Y9G z^(vDu_yE^+eWWyY@q(ZQ{HBZijY$+-3J;(b_F#CU`k9N`gK8v_FYWv^sDuxp=R73B zr@RrmM*|Zp$RSrT+cWMBnH8C?4dmLn>U&cyI4(ATCB?<$-W498;v0=1BK;O!+DFDr%Ty{;kR-JKQ>RmxoY!n)tzhOy)QC(ZFWo!?ltSwOHOF(>mH_= zeTWKE(jJ9=``F}FJ8&<7c}imcu2L1uN7GC%L>4Tblstu%+RIf<3I1d<1tvOnhB6rg z%tPf5H~sgujqwLjL!RO8P(%qFDHSxa%xaXrAg)2P6=1xHq4Q+KzW9v(wScyY&+snK z`6Kia`qpoyyF)!Fq0`VkcQdB;UOM?zN4-3hL`3O4s=G;Rz9#PW83)q@)i|AB1G(-c z-dohDQpj~rp$*;yOtF=M_jHdpBRMk{RrDbc<~t-dPXjfbfv$MWOG*7{lNW==PzApN zIMIFhwzJ_uG@%#NA2n?_eojsrh#A{t)S6;IB`%A3-h;XvR`^^@5gvC(>pV!;pT*;u zQH7S!Jbx!p(=vYuDq9=>DSA^ns0YjGXYW3y(KmyaE#$t?yK0$+$hVio4AdH=Tn@q| zsbg-)Vf?N68^^tH|7Xz@?f_k-5N7~ZnT2%LHe?1m`SXzzSnYp=ht(4^5lMNiw+eGr zU%VIk8Q56?Fs4|TNq>Wx{mS@iTlcnPl1>Zcldp%4V`8X^(?`_O!@T(L>yyAqKTTzk z2)#-LQ6<2G(W#eG6P1U@nK3|w+ry_QGiJSu%8|623k208uOjH(nC4z3vhXMlKqU>^@0i}f_GnY?5c^iO<+SHi++y#Wti#N&wJP>U4 zAo^@)DB3@oE&4n!G=1fH>I1hlIkc+0P!gV+FKR2&+Ur#hBWw}u&5@V`nyZTNP`4;% znif-8J1D3fq=}@Cj0%>8Ba!S-_!9+<^#n86ZbR05hb;~*H5?oT7COwsp-*l__s?(A z1&M+lk(JSJ!n>pUWaz7RA%`2=`2W_cPPN)OR%E&Gn_YiDMts*B1QDis6S(M22(sZtYVmlc80Ek3eXX%{fyuuPYFJSo9o#oVQ@%2 z<##qEcd-}YMt$Zcb`0j2x`x|;J508lpfil+nPMd62f|s5dAsu}oq2C^K~s{}!#gSu zmH1EYf;q|)z`#djbyT6hWF^jPKFE`BO!mrIJjmpRPBhlWu&07);g%BGvuWxd*M0an z&`}hR1wY-2NwlUa6Y^}a)e|^zZDd)bwI8dj^cbroF&{)a;uU=nH}p506EmoB;7AVD zztDW|23&yh?qanaiEGxob$|Dh`UA6vmqZc$Qz))@sFSyyp?1NzMY#qzoy?bcCGbuXw@$&dz2 zf~m}~KOeikC}uJ0V{i=}8wMF8zd;Yb z%u|6~uW@{&!Ipqs9@H_+f5(5^$53{&I#a=kn&K74Ld7T#&e9z-K36eirg0yYcK&l{ z3xD|o(SyqP`@xSG>U{;yib=P3bT;$_6MOlQy(z4#K?(8bq1zb#)nG8A=XxDF*Fl;t zx^OQ3S^Nd`QW~C7L9QSens8lk%t(R=WqUokuX1eN5x^8gI!HnJ#d z@yIdYmd(IdQ}t5Zsb(!|&I#0<#kix*O=^W~%`x7J9R6U;m!(5Lje%syWIl{E{&bF| z6GEq0Kp&w)4y6ZBot~-PoX#AFPEyT;bY&A|*XtimdD9o}VJ+;PJjjeCfJ#%B9|0Ao zGu^|{(Jj*JNQr1peAo3L{N7$sCn0>8XjQn(;}qP}Rdd0LJ@s;DYs1uh4a30(bQ@FEJA_ z%%njw=RT5M<-xe45*@#gKTGh zVBpxY9_9tIsgl&j-3Mkvn5Zp|$}nDjuRnR^aR-kja3!Ci!H zGSCouxK-3)aN#9tDkk3_Kx3Tl%#t;bD6T1DAvqZXDVH^VUS3Zv#T$;prSu3}*Dm2L z!Jol*@POV!V?PhvlGw|RzShKRM#nMtksj-zgXqnlI0xqS4j>KQ0vu;QQZdVshe=0+ za6hF8)x|wM@=|Ra<54xTo==T|QWD{j5UNdMxS7eRJ@vueLcox`dhn-a1IY}ev3?mi zyzymrsJjzXY1~!eg~-8Ep{4?*(QSDjYG*=dVrR@GF^v{O$r%p~=LFQqIp(=Y#bJ{h z4*qohAmTuU=#Tqc%|y!m4>=ikHMxXqSjXR`KVnwMrFJ~dwMsy*KBQ9cA%BG~kBPZ^ zyemlKPLqB0BX19?%CDII`UUd>^-xuwB88Y6Q+v~)99}^(b{D2^+UU{r8}2ytj=G8} zz)pveWM#;?Ah^ za6{}eWj90b`V8jS1Ma|ScPHjDmWN2(QR9)HY^O7M!(|n^=;nq0aMvA#RWlsCxFWDh zcQDzGI7+TAnn2fmggj3yULwBbN^GFaD^4VC0Lx`|Yht$dD^}SE9F+(eu$Rs&V3@7A zFT@PYGOWdM5i)f@>b%f(PwVCWMx7J)9KfVKTjXHV;m%`7JR^%<6zJtI*_ns)B^BF* zO&R1r%2`*pHudc`wGs+jcPe4$+G3Q=?zL@kC3Bk-z;+?C7S;AHHx`K~r>TrPr!))7 zL?=wYKmGdf_hA)%H7Fir35SB-L96g{dTnBdf5W7|Ftefd%r&u9D^!MR@-3Jo!ry#&~y6=h+hv}UO0y!2mH{UYyhBC}9-hgx+D zPEd7IN9{yEU7`#Wm~Sbt8|@9dC|oGAi#rEH!|%a0&YF;y2>0Ab;P!Y;m8iX|3T{w)dyie*|?BZH# z7e3`F_z4F>3!&rXa>~j=z^K{PP+b?P+6~AoT|}}g@bmC8+!N+9A4VE*k|_m5ye8-t z^rWR$glnUtCFBH{zVA-6aG!_1tmRAOrN_%uCWG1F52Jd#!K+N`D4v%Awc?c*2p& zMZz$SxxwfCLUa;y1YLDzn(7tR`H+eq2%oE}{{Y_BH(=|XIfrbE6wnj32Krt>+)u19 zx5DgUGT!cORaJrc=c5kyLEfSzvpNB1%ELX}j_9>Wgce4!{f-jc-v1STMnpWMgLMj)~o;=L&iq*D;2K^p0n?tVY`d;i&s>v4TzXXf1JeCq`AQ@KkUDPU*H<8OUV_E#Fb z7=uY>OQtRb@8J#X%v{h++@?=RFTO)*KTVdiU!mY^t{&O{It}15TB9(mAPfbMVW)Oh z=%UUi=O&EX4ONk;HkwqT!upxrnESt({@-hPnDN2#F6mWU)Pz8CoPFnlKa0E6`9gF` zBXF~YVLyF>u0uCO63M!Nek+dM5R_i@Z02SgINPrz;bgei0%^e+Tn=e;gQn#vA6NdN4_+I4jS=U?>~`AdbsG1uA+(7YA1OUu>d!WdR1 zbpc;Yf1QLZyLz@@CaT1FJ*ppYva0Be^0|P$3@k zMVO20`wxVw;stvbYK^Ao8amJI{u7`pFZ}7PiCncUp)Q;Y(N<^ihkrY{93%0vHNxeQ zEL0LL=y}o$kNSFp)6Vj}G;UZKag)^-*#rHp2_ki%vGuDM7f36%kSH>XekNeRFUmYd zZ#9?nkBJJOjX6riQyWwk91ZK`N!#bHHP@<}dN4?DKKquMj907%r%6KRy!i}->IR%e zMzuY7OO6LW{|&#uKY<*wjocVWDa*-`fp;iK{|Tg_&isi_nQ9RjOO;p~%q-`~(ae?E z)gY^x&aDTd2JLuvJx&&T&R;&qqtNN>6 zAsi9v8}p7xtHuU*GIw9~zvn*RLDtGOagWrn>Gla{F3)dm^Sw2bDeN}b#de&M^HIHC zMmfIG$Oa$127T{Jy2>BqK3#-orY{wyD=KJ*WV9|!PchVu*1--|8ZfBmRx7Fscbx2l z&oKo`!p-zlHH?Kg^iu?9Q|FyP`p_bg0`1FvIYPA1hC^bXu48936XFQ#X_gi})GDhD z&bihk_-xe0%`AA;JEPc{#3?+)uH~+R-Fyi@^7VBt)$$gzc2cKPxE#v8By@2Jm_{oY zmE{eY*4%jrAHoBocWAz}*- zf=<-@cOe_Ke{pd8o**qYsCau7R5{Ij%2Zj+Fgm(b=5RRZc%j{3FLhy#1eqZ}S>MQY zC~`;Pth@`Zy9qz$XnNzdd}Ss7o-Ypg_k-w);8Ic@9tLgs5B|pdY%L;tHdNT7Z_LfcH7_GUC(QowKkl>*)&W!e%~^7RgbgC>ucSp)pAS7!-E$I!?g#P9CH1I{2b%){+D()7szR0Zh8*7w`*dcL7{cabP zWAz<-x4LGxcKV}t+U(%t)SaD%p`_}z4uo2f-dbKvA>FR2HCHCEKbrNx5@SF-a_Jwf zJZPWBhpLmQxQ?uw)ADubSCxdMsoUgfR0d1ApnT?4^b*O;&V1Tu(Dq~mqnJ+O=u7{% zI1A$Y>zEmEbu|M~S>S&h>Vre@9ciogm}~lpUjkYj1a*%m_J||C75LzW`BIxt(O0B5 zlkk9TIPX8vM!!YOqp&uxgQB^)pk$adc%9s6t6`3l^3s&_zV6 zq4Y4x$(ZdWHldIG8@_2_up8V{E2e^kJQ1H!sdTancv-!&_C6<(S2FZiJmJh;D&jFs z_d;#nmOFcdGdXlu7SrpkqI!h=53~G!`;IwNG|~Tiop1C+4gSe2!X((z_>TMM2csRx z>PVR4*Yb04s<=vS-e35g@6iJ*JBQJRuT(Yi!4H-Umb3bjX7e}BlFpEv<|KHSrFh^N~%<$x!@tnd<_#<+n6H%La!UAa*`kW0-e%ZqXbjZVxI1? zoD?OkfL!*U!^kEl^C$^>f1}UeLT=|;Vc@L%0|!$^rp0fJ_FVCVMpHWr&E6DLPl?o8{1@d>>{J4)U#pJNrCjG}8|~)sT4|qC(~Y26jnY2pnv-I>s%Id&kP9&XT=SSoz`B(FGWHt?Ft>@LK3M8MH2u z-tflECNt=butjh6Cabt|?Pe&HI>jtWj^7*uYTcgX!>{Z^ zb}*14xVj3EZ}1*%XIa$a<-{(38><&prJz`dRzCsRrvu4^xoHjXZKX5W;3Kbs8EO?7 z$enrC4udVMfm>`8xMdxrU#)Mo#OHF1?>XC_*{UK|&=FMzZF>?bEyf#b)DG~6pP7t$ z7{lmWP8BJ0qSOw~tQol1Q<>@Lmh;gwr{n1$Ge=DTM;fXfbE@2+HW~?3 z2{LC|q2S%d>{C-t40IF%-lr2R(ndPr$J|%>)G}2zbVS`z%R-%XG&5#n6e($VF8a~u z52ljr3$~@JxgG2d()*1u2Q+mV$WoNd8azO*TJC^ht-_T*Ei{1Kp$2q!zxkWe-JJuM zoS<%VzSn~%Dx;F=M4|xNs*bq!66?%?C+6=Wmp?gu-8mSvPh|G)z|Gp6#Epf{?BH5> zn@|4IXexfdsWgaeff}IGiKtOR+o*bW>9Vk~Gtql=K?PkBf652aQTEcS3=L)w>-bBP zp=POSJOF!7V{{dJEbd23D+&dCK%IA!B&w2&8+h=LN7=`j-viu92uqpg~ zI`^VARE@A7S_4&QykPlcA)OozYoGsr9SJkLjYOkK_}5DC)6*!0t395&6Bv&!?h2p4 zd=d*cknX7u62T$ zGKSuAFPec{)>7G5p3-^TA7G1Xdx?x=RwsV$C95WGK~r{Nrs#wx`fTWx9A|55h1v)% zVu#0tr#siZd0`JdYLquNWT?k>HFZgM0|UP0W{NCzk9onUI@W2l-p$1}++aIYZWIER zYHgo&l{lhOI|2I63T_i7w4_c>nDh?z;ovjtv#i53%9%rs=u~+qn3t zB71?e*qwpyxxM}s72;}gJC{>?j+054M45OYQ={{2!bBV!{Lz{&SCa$tkj$ts+|483 zax*F{)b}q46(dAtm6jnv0$zJFfmlYz^2~ban`!z*41N9@@izF1w8QeK zqpr!3MlSg?itAg}Jn^spG}F^R{^3@4VUol1BU#Xct@hNM%2o$a)5xOc^Xzw4n{ge^ zS65^@G*thRPnAn0;AyI%UdZ;;ldSYwS7cK4jP#PEVCgFf|d*YR78|t8* zqSBuXSNjbLqMxY@<2XGtzzbI*-8W9nQ`JI8$>)y`n-CgDl%4sLKiaDYcsr64pUpmjR@K7u9ZnP%X;s9u57m$r_;AE#U2d%&hmJq#DSgwFM zN`}VOCAapxL7KUm3obWborMLCr^zp5$}SDN+RAB(r?rQ^C;Qt4ou6bv`&%b{s6Nk6K9#;#3qfs5aGEb0s9}7BPlU#@k z%;lr-^4t(7tvBkpJ&v>fJCgjiIJe!|9|QXdGzUz0h0r4voqpu*7OO zhIWfzf@A46dvl)@Rn_Qq!>X;hQ)W@F`9-8rDPf6Ps+QoPvn z7V@0^z4O49-b1gCcw{H^YLlFkPFbPkq$T_kGI4fvvbutK%){w7#Eo@wyPMorq61S# zLv>ewatu815%*Sb8Bb+C@fJ+CtVjf+vdOggbjNe{&$OG6npIw}BDr{%Iw{}jJzzOHTjL2Qfk)_Y#J!s8-=Ek$X`s$ktx!&a-NO z6I>8Mdw}DjMcAhw^5j-PO+8XAwF@(~zH~m4|8rNJ4P>$U%j*8w=(GpJwxrfS(W&}$ zZ?zZ|?m8nKXUi(mtw-wb(4-B~!_Y`)QDdw))z#S|pXoPlRkH1-Iz^?D0wgcDD#lbDVesZOCF%B=Tu8qP#-_8pzlJzV7LL`wE4 zgz#!@1TCqCQ=Aw z(Sll$33Yd6l?cQ`QCn8R1o=^>O@kx2ZZ1_-nZaX}=483U`{R*xotadY-KgpIiVXH3 zJCQzWyKZ%LMIUp16zkPVos?9Ked-X^W2M?|F2Q|Sn=a)Ck{tT0?_o6BsJbAf^o430 zJ<<(Mx}u;l6G(8)N6y=Gs%3lsL$e7d>`v>oNEoz5W>h+pgo&4SG-%BF;1*F51#cb^ zNB+Pt)LFIs8OdeJ87LIWiHFY*@8EZHx%-`kB)N}v2Y6B5kM1;{sM7ip8t%9Hv~@*2 z)IY0}_6=u@bz5$+3kO!>h#`Si#OE&sE7P->)TXOwSmudf5e;xCj4&6frOYE=k)nx# zcvebTmb}WQVh-r~GqfTlgZZsNC;<|Y40(~X>T*mdeK=v$hC0ATt_OSh zRsW0gi==n_GM||vN`Qx~QOZ>yhqFGKPNAx8Z{)lce#OS?q80PUJ3=Yw#2` z<7aD#tccFLg)Zg(?38y_IuqRyUi--5@a#xo`-$ zBk&eY{@cJ|%aR{CiR#hc=2VSUU;5ZwxL4{*M}OjX&a7j=D)ZW@t-9>a_(YCwP+keR zoCP(>vfF}w*n~&>(;Nhfo7gyv%KEyMO*?i3>?n)=6D(pedYJeqv1gDfc@Cw=QRcpX zR6@I$dnYn3YI$sw{8>+Pd!TK;qB2_D;C1s+vBx?|$pX`y*WG{?r8*9d<+$6w>X5sk{@riwS1ZVWyGw?A z9RA0Y;%AtqIia&`3aDtV5C@n&KU*cpL%2vMn~~Z0j8jqcSEHOXoEbKn+pB6g=+F*2 zqE*JXOix9v?DDDFZR8LcRbqOZMs#{*=xYX>ANje4l~olRPe zaFR>Qi^qYw)&;Qz^6i`ihpgRMSdk^d-3BVRRvS5-G_4_!xG z=trLScZIGU_X6MxF2alYuT;mt8sqB=!3I}_Z@ZX&0>^V}%_?$*pHCvTI&aTdiz zNBej6ubtlQhxe_zb0k>D`V|b}3|Harz!7Tc?|+fF7AcC@;Ilzz6zpCnDLw%8l^mVX}r@2#ya zvB~9_Xyl)2HY87=i`kMht}+bi8?RdAV9XEVgi7c9@3oj*rE@vgO!9+tJ(M#SRZk}U z>LdYF_TnO0?TC{!TvxTV8@rRN)@nOU=mh(m+gLQvzuJA#^ez`ucoxDaSRRSVY6*J# zGL z)o%$-`y}9mzrK8vYRbG4^n_K+KGohp{!#hgFTNR3)AA?$yv|H1?ydPCD9ND>B>B zwqp-=Q7yd0wuBR@6oGt*Cm+E zy4rq{VJqk(+!d)&N33C1O=h-6J5d{Uc(%MueQqcB;20W1l9ghQz*IQJf?+>C>g--4 zXSe$*(oQ5JS*i@FDVEF+pS^%tvbo#~0-sjdpq56qQiBU<~)5W~}PmaQfS6or0B5s4$t_T>&--2Cg!C$B<%hBprqqbZG*Bt_q z(@I*w(|DGD39Kbmej>XJUW*Ofhqcv6*!?8#M4pg`P7#z6En!av!sYd_3gLf$Xx?N3 zY5_`k;imr zBL&raK+v-~a&<0&6&EvW%d$ARw!`%gC!Mvs{uPDT06Vd)W2bg;+VNyp0HfIMRuJ{c zt0`jD)Z=W$d0)`E215Qwg+ikwy{<^5v)ft?w0F9Pc`8#lSz(QzI$PkCONC3tJc*u= zts))8((rZnom)K8Joms{z4P2El&+v#IA`4X*cEXxGM7EX8DMOI*FI?vaa+07D-u=Mp6cEWKP6%6uka!7fYUqN$ervA45yQa)N*H-erx~Y^;Ac7 zOAnS#4R$RYafRH0cqx-RXF^-0w4>=rvN*$03;*N13Kb@mE+yyA4*W0Y`2=p0ecBx7 z{_nib;LblL_XHE8Mr#*%7;21m^e(>3I2muE_~_^Z?z-M7l-N<6QCso4~kj!WX|z+~aif z=yJzeDfv8~I&H#}!qd=Hyw`$FbKi@xus{D=DgC3^dsq?$bsc7sio|MAm$$SU%_y-=7IJE@rZXQ%>P5F9LP|Y;V2(jwD zyn)B$3*MQ{u%fT5(KsVogJ5^CKbymp(o@mcXI2w~2XUZuB15vYN@;D^x%AsmB0CJ5 zR#U%5X+MXw|K0(?R=(?0zXVJWcS#R;6l{s2=2)=1Y>TqxCOd}41=$iOE5SQ#k^cnS zqtja%+%HDR{;<`Z;e}6I73E6f5@`9Hpd`<~pwX6k^?>B$#&#hxPn)=Zh$|p?$*C}P z(7XN%U-QmvhRbOH+LHAu%IZw#oEQy3Z?tXA!3E2~w{4eS0@+AIIuXcjeHM+33m^s? z;JGqjIPXes{5@+13_?EZSLTtaL$@KWOAPMB?rN~;#+NI zByZoQ>zH7aqOR0u6I*Ruo(J_kXR~+N%N^-Q>f`n>+D`c$^=oK&dBltw7F9fIKW`Ja4j4+O-YpNpkK*mXdWuUBAbzo-t_S3sI0E(m5w~s zWt;|H`>3H&3uDH5eZ7fMBkiqr=5U(mW>K4BZe+U|r?Yp6*io&c6M(JcagHg^Ipt)G zP7&Q9x=pT_oK_rdsPf4oA7|!=Ke^ebCLI z^VqfBC9uN(xF2vqKX*3?8?8%AT-?fDXWk~GcQ*L8trqcI57xDCSlm@Bf?dEECz&P4 z6?qXzEh@A7uPP_t66VLBn4k}%Gr8=mZ>AuxZ!cZX;lMK#Qzg(DR%EN){LltbB5;qI z|A|`NQeGj!pG-u2@2g}p@ZOYapz+C?gI+7GbwG?pNzR6-;1clz{{69{wj9AJc1r#V zA9ox^=BivzDo71nGd=Kt?~>E3cl@n)*+g(n_X)PcFEa<<$B*oxn*>hdpjru$x8IT6 zp^j);DvQQwEK7m$S}5XH__r**scKdEoop$?XpzMgx{pNxO zpEKjcE8hw3G>`8Y7I{!=HPcy0tMJ>$QDV&3H*x$`LFt-YWx+|>19WaT2_6Q!3rV0- zBdIMpKxX2B=-xA$$}94v(I4lm43XeKBFaVa2vnghYPv(gkyM_(<~OP~b5;ynAy$Lf z%prYfA_~vseA+YMF>mAGZiZsK4<6F_xVB?q){c_O*iPMK;^<7}86CU?f87B6d_&pB z$btU9SJ0AZQJF8|!9$(0z6Fb(B<9Hq%$v*60Zyl?bO_$!JAW~5fr3sXiDQ`enk2FK z&TT7!9!Ae!MUQj-M8|jrl}=T%Wwuz8)ekm4B-?UR>!C2lLs75B>T0A|e-iP)b$7}0 zpvNoZ8GNh}@`tYq$*h!zGvc+emMr2T))>^4OJxwpSp)e}A9gm0Y3$KEAcpFlc0#b5 zj81kDB9Zm?P%pIS#Zbi7vm5Y}W`y&HkA+)ASB**&btWdAGs9gGb)6)}f9?Oh_CZH< zL@f+>)H(|fA3smN4Qj^O7yyL!btj|D;F@C*AlxTBr1iDT+TfAS7`(<7(c>RzSDHnSBI*+tyw!{uBsKu~`)JXOVIUpbz& z)c*X`3dg^L(|-i{(q)x8yB@XX|OwE z2+8)Bz){mOgY4w{tmg^sBZiXR)*k$@6WsA?I@>yIxk?+V&hDp_p*k?U6+*pWexHz8 zJ(`ZJ90n23oW5T{~fn}0n+Nm;J9BX&+3Y7i&(0I&O3D7N$e$Z2{qw}ZtUD~ zM&r`mrl*kln2=PHto-fs4Dfpufdk4A$6kQgwD?|F(Elbsy51J3t!`vH3kQ&KwjhTO^2Ra3pO%eaoy z-TfsjBjqDcqNF}xH}?94t45ARZ*VR+bHleI)x&9{TDo7{N|A+VU+TGQ)NEbd>lQ8) z$rnA^?e3L{3{*?(3f^h`$nN5eVagqW<_$+5I@pZlOIKI-NZ|j`noptv+r;f$ZVc1l zNVk&iWta5gfmZBs%EY{iQqdVBHNwRsJwUy(I3kb@)zcMUL$fX@Wwp>maX&Bxf8Sze z1e}<@Jf;@QsT)bz6BMLMZ1mLxomfjf`DiU*8~9Ue4k@yC$q>sx4q+y$P#*U0T*BA0 z(N`Y#%?4jVuE=d)2hRHQq|OG>s?|qJ|B5_w7z}ngSLfOP*(@ne`ik)NW4`=fHE;aC zNvTe=m}~C&hVoj9w6`K65p&jT@c?GWlnY_NSBj&-y>#_|un+W{s2u8w+VBs~!8v4r zXBU-KEp{%XkQ11F#=xHB2(_ZN?BTD-1fr|O6@Lk<9aSr}xC36X%33bUkT{x+-{&Sh z&r&K>!lS@+!)Z#iR zb2(v)o|}zuvDJrB3Xx#&J?f6O#%Ip)7d-Fdc)dr@@Ettz8dCH(gCjM?;q|SZg599>p+aMY0OY2~NG8KTnJ+3Z0^ zdO3~c-V4-_ zvng|a3AZ->+w=Bq+>RZ|8wui5UuRw8guhOz{3HD_G*Pug!L?uJ)G6>R`*mzEKAVxB z`fGu@-17~G;~N~95F#B9*U)tLJ>J?2UOQP`*L0>BTSPf>eM5A-k4duHte02=$R}K8 z{voUAd%^mmC0_EcuYcnYN~?zAeHciMy>GQ3M|3j|g^@Tn;Sku?8M3>AOTPqhB}am% zcpf^Rbo4gmWE%fy?z%(NlSJIZznd?tyS}$@ks@d@>#;&yl!O_qLZ0V#V>s9MJG^q; zRa+cSD&;<btj}9j$$QJ= z#JIl5GIxd7B;t22Ijh1W@mUpfALW^ryH>8<;WgpCQL}LN{^0FXLv%cE59v#BUM=sw zw>5Q)w)%$A_bx6GS^`M@-^Nr(7e*(iBpu$x6Pcf2!+EN$4RTWZ&w}9$rBH2DCXlixl zh?DsHT(+3${}^G@d}rP8)iX`;(6^NTn#e!N%*SS;L2y1v{oP=PANxk3Q~KA}nf#^V zsH1EAw^5=#X~P}43U~NA*cr8}2+qL{{PZsH$OX~1Jt5m^dEo1QfQm*xkzIWkyv+0; zi_+;TsXIf+>Q5Y)#MPWYFEk$g|6ienVkg_{`m^EX0yU>OTK7fbvcDw%eoi0NS3D0c z25tL{&v!B^>qFK8{63eN%H2R_Tn2l{Wj=_q{Ud6Ga-fcP+2p&Q%?^h_ut&>hFhLhV zOAgC_jyRiu6Hl9aN>-XIM(_|X48kI$2P{Azr37m9m z(A4!owURbi1KuiE@G~iSo7pk?jjV*5@(Kx!lR+SVwg#!9>Zy0k8|3(%25w!a1zC#~ zLBmSpVY!X^bP+Qq{U1N;FE+$oHqMh5cM2?Gvdm;OBbnhGNXwtHBlAu*+i*R4gdv}VB=oHsj==hGhcuqnGcFYqj91y@OC?=!Q)j0{81Fq{sd1RCkp^keU> zRh(Q;t*xNzEk#jtn<&c$x_{`_gX#q*=|ED9a?{&n=98>PZf6hj9D3lZDrQtfxwoE) z{^|d}9x?W#c%NnL6gOmkqcHi67Rk&x^;S|1=h+pko+v<`fX`HAp4|uT@r7ACkxCG# z1&=SuRfwY3O&1cJj0r;|_V}K0DxD*jvLWd-e_OXm+3hdulYclB1nCtj=z?lE-r;U` z3%i+H)pg||yf1ap^_|y)nfK#^cO{~#3!@X2Jc_*}cj=S=(*d130RY2_fRP^>QBkPaqV4DZJclT_i>ab7eov6Ghb6F=F3F(HfJIU`$L=w z>OL9BJJ>pwEpofa%tC=;Dm6{GtnKibpP^-QIkEL z1hn>d?>obW2CsrJSi=HjvfR?rONg*J?3&L^<>m5wvFs4+$nQKfoRVrV z!uQ;GOrPVy4*$-Sl#aaRdfd79d?QK5xZ|rx$MS;H%0^+^gVzu!eGFLZM0QD-Y-ubl zED}Gy1v9zN6fg&rwHMvR2s68Q39h2VM=FxRUet+TK5f{@P#phPnAvR&F95E%p=dQCE ze7#&8ScbYGH=4}#u#@AdT?6oxrIOFdcRj{7@P%xA*oWG)y9klT_yxSBp-QDUvKRQ6 zTxN6xM`~$Q=d}cOO%K<2Y(y2$VX7zBJI%>2McuD4>^(`Q| z7G#6}NTOY99MhwW8+^}V%pQMeE7C1o%&lavaNnYK$geuU{Ty?n zyxw7-+uHlV%j7om!r|KBE&q^%omrjIjrD4Krqf(2^s}3+r>dx($%-NSrL)+l8t5LV zJ?q+eaM~}V!i<22+7POz_UT$7Pp#CqgZ)KPwTbTcn$_2L_v>ltJIdbqWxji)(i!aP zT`b-k^+ErNqPHA_BfgTXsBePm4@Xz_2lGyV%9EKSnGL87cTe+QJA> zGuv2enRjZF`M6$vp}WgseiL_t|G(;m|kY9UD>>$%JM(p1)?~!1rbcUn5kGhl4f262B&&51|?#lrwaC z?#5c|^7z0LvB$qD)Jb$?*Cgo~Fk*{DTbPT*u>XHhuPk;kPZXwc4b$SXH=Xrg*AYDZ-kZ_Z5O!!`;OeAgiO{9NhUU*WS+ zNHFSg_V+oG=V+u8yZgQOB91%Z1lb0I!cRbkYlp|WJ3)1Z@?U@OvN0okQm;cjsP=nh zH#A!9b#ePIcak%~dF`#I-X)CWaaOsR3q>3^Z=O+5Fc@J$O%35ycMi1ME8KB36BxdzE!V8xx7qpt+cgi6Rmu&Y{j^IxGm zN{Ih`4j#slq2%%eY34%OvUJor~hQc8!;_D|9> z+si$4=J}~ALvWwXq}r4w73w-ppOCEdp9bSIoa{mp3EInX+-V2cLm`cWusrYZ94-8c zqJuas!`InGt&Dafk8F=jr+lh#a2FWPAgcwZUS-wDUgnmK^p2#D${dqC`c62FIhbm8 z26y>AvO~wAm3R?KDrXJwQ)eo3!lI`9Bf71m$!{lqbF6 z25fa2_Fk2zXFdql{aI}>&#ArY7A}eLYDQo`yT$i_Vm-DN;SQc7b_b8L3G`dzgXIUO zS|VntB;>}X(}$^t1wpCX<6i6_is8!X&tA=UR!iO1?qyb|A8Q8Iv`mf)EX)J)N`V42fpY4|TTUFywTU7|GE4y33o9^sz-AEnx z1~X1g{j0MyoRQSNU!7Fs3-@rQ;WU2lB#s;dnQ0Q~8krWgDx4%TENZNi&pqT#jWmub z9Novh>J5whDc9MJoUg8mv33oV)|s5-u=X3+36%tX=LVZcHflC&@)=g(BwORuv->z_ zywl8?>)b*7Hfg<#Ovv-Wmq&-)=tEI$qrBLH?p7QeaVSO_tH-#ryOT4VR(_OWvnDuy zCA0B=>~LZ@jzYn1ppJw1l-j}eUZSd$Bs;4bo14xu85IHFNU z(39k%+esU$4?}$t-gzC^!q+X9`~Slhnlc|fp{vP9AJP~c<`YxZ*9qu5_IDWmF-%*T z>0yfT?+Ip2am&|?j^(9qIGoZ=VSDNNCC2Q&P5H6+7q)bdSY~;*iF%equYiD zgeQ0x!;QoByqtDp=cK)jr}46V%*pEvc1wkaxkJ5lZhgCvyIanXH|^_YU8{i(nuAI4 z86OPuTWvLpQ@cFa(o7nyyJj0D`hoi@4?Bc5gQZMX-RytJa{pW1#!cLdeQje=EWF_V zDy(*K&Yri8P+#uG(ZLn)R`ZP4^ie^QrXIkF*M%8>Lw5a65wiB<=+D7CMHgk3#)UTr z1syph=3X&@t(rY#Hr)&d+A%U)GO2=kF`nLxPCa9`rP=Z@2{rhEuX_^h|Gn-uABcy_ zCXZ>eUdgFZL4D+ujG&UT(LeWlY>IZP0D?qg7`peK8d%^A2 zNv6?e)*rH9@O@|||2~P+^cFjq`pQYcEVva$1e37&{WB`@)l5ADMYKjaX_xLfkD z8?j8NBe@Ez=v5|gpATYl&=514?FQ4p=TG9h|6;Ykqj7}&^?kwH8?dMBPrlD-{@<-2 zkgwULe#tliKT;4>;SQX56!*Nz>m2<1bJ?C&8>Dmw8_?Q<*j%*k$*uhC7+u=9%apMb zFLr!Al07NO_1`G^I@&dj{B&Fwtij;GnaHtBOr~EOXDS)m1@SIu`>p#feCEHL>b>e$ zXTDdHDyHlpd3Wzn!=?aV${I`t+G&S!$PIcrZrgs&JiHIn(GxyX3T~@~>foG^b9Egz z0y135%kJ!VFNPo4`yDs(%xKNt*YR)-f1(qpD^u!i!7l7r`HJwDP=nY+Tvi>!WB!~j zaT`c)CY~}ph)g1-L_H9artErqLOMZF(AU~{lV6f!FqoYw-%%5GGU?<(b5apK$q_on zA|wx&<~P26i<#?*RUU_m45 z1dp?0HxJw*srMw6y%&So5ZaU(WT`a)-h3|@P6HhG7x3mJ3h+OnJY#9F)jKx6VzX4y{HY!XD*~lg_I#>4Co?fH4 zMNxh)X|`U`y}cZ9#iNUd(LTGU-7`^zBZs`s+5V1s5Q)nEckItGzeU%MIvpY#3c zM?%92K2aE~WsN(^#xNBtVLI66(q;>BJU zcxgoNd9=sFUY`uN#v%`%!9O{dn*^qkSQ5~qPA0gvsZB&NlIs6(jWZ)>+#@CYo+XJy0RA>&=_1( z#f-k7E~Q~!N~6Zlpl;}FW@;R=bq!OXcEq{XMAqY;?1UQXp*ckK52Pd4x(B-vFUx`c z#pZGm9UNhu78}syuV*VnE}q{&@M`E=wtHNIV@>NH%;{Eutz?2b#0o`;)`3L$UQ5AN zl;KqE$(?+WZafG}(hQXNypJ?-Fo({3p0|B2)yCoO3NqtQ1ASWTOJ+W@2Kk<%-)rhS zWE{a=8v~QF3k>12rTpzd9_IN@8;Q8{%kf;CfTeYrAg06Kj%L&1NHLOqSaIqQ881oo zOlz!)vojgR*lQUt*c?Q99U7#xI7}}HA2~KV!Nhur=46uBBOR-}o+L_;XY!lzy~wXp z8AZe`T=w0qx~jA_Q~ixr@QFIA>oRGU#NXE!hu2EBLi|KE*bL*G)ZdvWj(s$IGL?D8TY~_v zHXDnabQhKR%1SCyMqEPW&?IG{l4F*gn8nkFGb)!@5LWjZSXlAMPVArDhq zT7M(zq|I;W(v93_0_w+8ukh)7ivlR@Z_8hlgL$eVe@k=jmP@{*)Tng+D)eUAQ0{%p z7J$Evc+|H{%+V$MMfv(paPk`Bqcw+=&J$vTPGx5jYxOgo*BYvO=%?hFmsaV2R6Q*8X$xR_MYHKU)pr@W-$6UV*eWuejE$S251OZeGw@n; zAr&y4%n06oS)?ZKWwCNdORVlBwt92!S)oN}KbDKj>L;C!yqo9R!%x&ojbQ`G6||Hq z;N8Xqf8*54Nv&>5LPaktxqL|yX`26kozBma{vDDxNd~iIQxYfkz{IVB{nzZ9?t`;z4H$4M*v7V^2P{$+lrY!G;(LL+ z`~y2d+)x9$^!4=Bnf!IR)2flrUqy5!@uL#&DPSd(cR(5w@T6Sg+eVz0Cb zo}958tnt7DhNR*PE`bs_R|R91T}IWTo_7#()I ziae^UbHqrAujgXmC>>BQkc6M90H=7qn(`@CW(QMTs0aJqI`R63%(tl`5NH{ihAO^2 zsTVi-%2qmu<2?0wL3+#c46fq!0W~%T7thy3$;`}0rKv?5xfc2SOX)^zzFt`54>TlO zHO360YVKeR5X)5@Ue}jW28!X9D9`LO0$g`9j=ZnC82a)2hC@BsoTWi->(L`sqN+Th zmr$rv96F+XRFYD>hQV#-z;QQWpGzU}%y*o6^1Uwv!?+wQ=&iNUSH^s3&4g{gXnpTz zD-LyO65nS!SabqW%eRAMhMO=Nng3J%6rf(^Gb>RE*}wp|b;?lujaQiilKYPvas1VS zD_EOK)mU5(uD7FChwN%e-A@~2r4`XV^>%XZQ+P|fO5qw&6QkZn ztZWrxzl&ZLc@eJf^~06%R`=lQc5=TDXN?RE9~bT5Bv(0Q+!~Q+=cc10Sy9BN_u4Q! z_i%5@7CM{Tm&Dj-_8=>%nrgo{9Pv!O2^1lX{}4TSR(k&ZRPV}xgY5mO8XSkJ$TFXa zVyY`ubd~;~&e}EI{pyfi*S#So>)Q5Z_0SGFv(Uh#bv}Rr{$S6yEcHRZ5NXIkeF#SB zIdtoKva<{>`HtOy+~i!2Xdg+ZgTta#;+kSJg zwQ7`1rCW2)9K?V9^);t?T>OIH^Jm#jore$lfqhR^zV6=y%VMj7aZ4@**ZKv#>lb$W zUI78Shkke%`vmuZA`W7M>KxFOJdNc!Aaw3pdJwP7CjP;Kgv`MHn; z)Vg3JA@KZuRG5QIM*sR3gB29Tkz~;e?cl_$&i&t%PUEWv>x>XLTb@C;*>zlKs?W#B5H+ zPu6?13LmUy>^{rKGqn!|ooFxO~31{~{G%$_D3VfZn+1a#{-ncnD?9Ko7xUHsQ7q-rclVqw4VvZ;U zt9#Pl0~SW|376z~TtRP8mise5bJI4yoA^)bGLqS9xH!RCla8wL6cs@b_2xmmlOIKl zI%)hXI-yzWB&w(x!54T@x3WF6Kgmb~tSWL&Aft7MS~rruz6xw|IV#vuu3a@chhofA z1L|i%>0>fvmkW2la(3d^(ZDOa!F*YNoq#kDF1UwLIMdvt@3YCvtW9#8M zo(DF7?`;hXM0xcZq+*o1p|ZMT-Jgu2^s5y)MVdL;?ek7HuLK^*X7*rqYOb(Hh4QGm zCmg!X3WTx+ne^CQ_Or}1xW zu&1%RqN~#ll}t1#0oB|p;WuOuZEzE+<2r$JK-IAeIEuZVW1Oz?CT_kx@)H}0FOf@D z{{LT_>SN%GFG*KE;gnSM>?O`;S<8NB=jItWZl}_B$s+j%HhrxV&<&lxoOLppz0SFV z&bx+_3Y2@Clh;k`6%YHIEbc)skKSjuz-cwtx#A?V=Q!`2y?9v4IqOkGPiON*ZI#^~ zjq^HS3$Z{A)&;~;{4y&*f)d#AI1%50xK4$!8p>BHz{M}dkvE6jym;(Ep3fON+nPc` zLQf{TY_=(ysc8E&JMa3ENM8=!?JvxWev)tD^4Vz>QBCZN>_uG!BE3w_(VbAtrn1A< zG5JM*7b?Yep@~%5mO7{PuNC+5^R| z#%lbY{mqOjsPlkQ^|QN%TBv{UDc@F2^%I=nhjGxiBbjoMRfkRG_0jR9VT=7`J=1AI z-ty1(Q6}7WY|S`D)_9RX8gx$u{ax98@W$7idE~ioC>`B5{vW8Z@7bUoUrwVlTq8CF zc9WX8DOitWvlM}9=tJ@_edOW&MS1prhkr~amio@Y3>-w?wG0eu80i`5*!33H#mw^b z#??XLGN~`YJtVSrCmW}cu8i(0q>8d3=&T&aHnG=eP1}n#XmJmltvLOAG68+`&4+dB z!luh=bn*?EQ<5_`Wx>68mH%4HUxd`7XwHZLCsYyM+$^c!Th`kT>%)r8Mt2P88>PpVK-Cd7xJ z*#@7IX38w#Z^t)(Ey0z^+ z*)qf=3;z(?Ix3Y{fd6Lo}%t9TP^J$-Nl;3OSc%$|4^qtnPxw-3plqk(RnQPY0sVoZ#AC{Yl+kn z(n!AUDA`4lwy zZz1dE1exu4ob(hsEnfNsPDk@B4xlAQW3;tb#V)YinlQ8F^ldzPi|h^75YnRl_j-Zn zZWB3&UC8CQ7HW?FCy&Udx1u+jNTx+^_7Gh`M?H}QfjFxNofe2KxO7kJM|GbZ_+v7S zJy9e;ft*_7PIs$1)7>ZTc6Mzv_ja>;db;}=_5Wj?!#XO5>66wM^kY7&7|y{8^m`Aq zBUUQg9!fgZ3(c-3RmpB-u_acw;beWyUVzb}9%;9|NL2e3hkrl&D1X&#d+k^6hVH|a z?XIhd&3MiFS?6VP`!h)s=XDz_&76gwmB^q|r+?VF(C zshk1MNcWR>1i^cB;MI!s-;*hY5Hvq>L%r_z(>zf6`g zyP*X5h6=Kqtf!8+7e@tlF>9+pEQwwR$mgFdYFQ=W?yplxeo=kQbD|_(?5wH<85+Of zm3z)!jD=*}1=uITE@7!v8~nDX+1_4)dvXkVxfZfA6INw16BHGoF+PHFyk;e7?HPFs z8O1-KzRBn`PNU9R0eatxROES~>1^~XZQT&xlGU&W9IO)vNd|u_s!(hCoZL*)MkqJE z?mqbI-~Y>mIgJJ*j$W%ccS^qsl zs7QZuo$SgJ;LNqaVZaAvUR?-H+C3f5u*F?jz^F{(Tvn9)H)K<)|3S{GD|R_Mf#bO6 z_1Ms1a#V7WA@)oymI>h{v%1CIVd1`!Zjq?y|L!ksoB z75+P?1@8SB?of361)TY4!qPd4*(RIQKrXNo+F{ZQ>+2>c7T7{&E<=y;r%^>dL$^>C zUhKZnhZ-`DEt9KpP2MJPZVVY*N99uUwyeV*lUTf&sn{n~Or?}Ru@mO4{L#7UZVnVh zQ9Lnlg>J1Qc$b5}Y>|~*RU$_{J()IJND7jm1GlvXyZlQ{MC;$mxyIzy(EZ=*MEp(5 zylLo(dpU{C=OQ;df}hLI>Lh9wc0dQ(;iH@bAJ-FyQ4EaL`d~L45Yssge`HsCA9S9L z0<*#SFQOyOtbWI5R6w;OeQvNkPKr@lxr!}CDZ%-FV54hcIEZF+sy8?*yOSh#hurdf zRwDU~S>v|I5`4)OZ^t~+f^I(+3?YU){|Wn5$D`-m9GLl^&!P+Zx`y!J1IaR7g6~NN zGq9biRWJ@Ouz{lNGf48UIL-I+d!A#1%z5&M$Fo&!v|L2O!xr}J2z={JLVKt&`P5-H zi>{IV?O7nGw{;q)E(+Us?pTpWmv$bIld#=>W_4qSZyBo;)vqO;Pj!1UdsJ&Vz2pjg zni`eZzHDb#{p|EkGF92`#6F-Us*Rlh7QCwNOVTWx7KGMw?LyShYtAQG#J=aeU>08Q z3Uay=d+VuMhn-Pu4le8V)07D&1f{TIiocoe98(dtwM4 zv^)Q0xg=vpaCYk$b4qF4L4BFsYEa)cgoc3(ABSCAdg)&6p5BgOlB1OT}DqGO*ZEu*P$(r1G|ZGu(P+_W7n)ZcnJ`{h&FnYqgBOQhpzpr*+>;1wCHN}R}l<8Vo+?iwfQ=*pnHIx5#vqwdl( z?B%If_?xz=CpgX?>*sE@|D)+F!=1RkwvSUNPLX7M?>#fgG2GqVtvD1fQrumNySqcN zVg-u3ySux)mcsix{Xg%QT)9q9LryZY_FDJ4g?!t3e9j-jdsK`!GE@-Nz%G1vhSH}* zTCGJz`iEZlf%L|KNO}X{ia&RsJ9EII*H|rah+SlUBh6=m@e_=lI41ozD(~z(Zi3@_ zl5V6q|L!F6TNB6;^dz^zV5`9TI|jb-MVvzIei62OOB4ae@z=${6<4Mdp-bMrQ2fa1 zFjykw)8IVL9WNLLd1wt%x|GR!K3gN9U4{?%jqTDYGTriX&fjDQUCe#UX(rCNyP}UKJ759(h*P*ycd%!ZRl1Fx#u~UxDM;!WEJ{*!Cs6D1lg_*h?5hKK zMS1oS_31+A@~`g%-$+S%R1F*7MWZ8nq6$VsaG3Z;4RE8SWWMJVhfvMzz?tv?dvBlb z`jqpSN;Td|g3}apgC4|hWD%K+MZ`ZEXCC?Wz0fTt3pN6QbaD6`jk>KSeBB&cS^Kop zQ*Vcda3@&PE2tzBnEeas*YRh_$cAw>3XTEPpDp@adni}SYH{D)VI~pnRW1H4OZ*AgBz~pUh+RekF&EdUz1Gh#?re0LT}ejkAzgK!)QO)}kUB8U%cQQ* zH7ygDRezYDvsE9jEXtki;C_*|5v8m-QhOA3}uSE=+6puj}Yz|w}YS3ONpnClnvD+ zcNEvaQ5rU}wp*r)+ zT9EuRfll--HNkFYxFcmDIFM35>NA7D4+4rTt^D3%r-O5?3mm%xbH}>WwN*TGa*o$D-iTH3m%z%G^eM zF-~U3zbL+c4@@|ZcZV%)4K$M9@w}-EgCwp0yM0X_QUgdAIw&rnIA|2GUmgY33&camkw80Ei^}4}^We8+6kAoLG*U6CEHiL0m; z8o*xOr?0S=plv*FpM}v~4|m^b%yGnw0+ zYd>JVe`M#we|0|&J9quJ-cCP5Q^HflR_I~F`xg>Tr-=PbOwQn-@s+AyfcwH@Q&?Xd=!+A0&~GSzVH|cH=sTYx*-9y zCibF0YI>4g=u(fO8&2rQ_o~>HRHU1ljdrY-b$L$jl@#1zW( z7#FRl5fSzo5R9?72{m_<;v=`!eTP;@^L~%15#1~5e9VaG6Imm@dfoxQMd+h{9gEY? z>Xm!V$^fRZ(p=0H@-MlBDVb$FYRDbDD6ilWJQTDit=UDyCj)&2nIH+7*OD_CDXK~U zJ+Cb}s6pV%5aJFw5)nJS~H1hr5;~H?k-p)_@9vtQO zkmFWfYz=&7-**mocvIJ4p_H~Cb7D-xdulvc zA;Z)LHy-~_Rqn?s?kBH5_uX?f(agk5ZNgm%kOzHCj&mP64%x>R`ObB`mR2peZT+dw zjZ`5J;J@&@t4MWv1@o`~Nn~%N<~`O8=eCH!Q*Il%e(}U!nTDC9quT`cx{9Fan)|>V zjt4|m_Yf*hvKF2C>aIG7J6~5fEp_Ijyh&GFOzkCK{$Di-9!_d^z8wK-w8(l47X1SC zO)0sUEYyXhe>Md-n~WF42(lv|Q@dOzJ^ROA_->#DXWHsdKJ^Coj2B;Ra$obW?{IO_ zlULn7epWXQ+o~bnXn$?^oIe4-ozdQYuS+PgU(Vm=KUTwB+auk?RlZ|svOjrJGeuao zr*nQKk{S2yiJ}=!;OFp)nFv;F;Qsn2?3#k8HW~+aGm#Htztaw1h#qW~^Rh!^(-+9f zhI$!jOlM~VF3>sDIk?Ss1n0Gy?v^yb`-SFD zi8vV%aPGh^*<+N!hoOY^RE(Cptdrt5+00rd8sQ_foI5l%?8=WQ1unn%b)Lr6?BPAwoC|yMP$gTbeCe&r340^S` zC?Z?o)m)J)wz%l#c5rigPyN<@x%^M^Ey_Nt@QH%|7DyQC7E%!>*r%-U9Im&xZc^(3 z-r5D|w42Hk_|N2)SD3u|$+)H`rpl1{i^xnT*^W$;7cf{t@}d4OT-GVDK{|+i=&VP> zyqXAWrj&Nkvc+w9$dCA5l3E+tuU#}pf$Hyo;oJg5VuN!QRpuJ;*09LREEQ}DlPzm7 z2lx1Vm`HPB1!RN^;{>x(TM7rOnl0!^Yw`-|+2H>S%P2Yh%>N$e2Frk@9|aAY?(ER3 zF>|j890VyJ6Ntm1>Y}RXLAd8l{|-|D9Zg z^yj2BmV(t;-r2`BpI5ABQpzhAXt}`6QUr4{iI0UF^U@ij4MMxRlr3;GSff+<{e!US z>d;H~1?S&{tMg1Y1&^#*Oga^*2DaAQiV&6Ib>wn#YVE9(P7fGv31m$slttbIuCa+~ zvUyY-mOtU;@|s+RKSV(NVC;sK(a22A&%-c2$EiVj6#JRR#vb~mv1S^4BOB|z?fD`D zZj$5OA3#VidsFE!V*K;)nFn}3p#aVAm9fgw|?PdJVx{FLO} z7xmH5dz-yMbiFmaN4AZt-aw|{2I>zogJzRY^guLKIY6>jtLFFwmULf;2_&fxy}9PA0^fPwpZ89=}!&CS9jd2{v)@px7^!n{>xm< z=i-i6VfyAWs=3~cxg$cG%?7-tHXrBL%Gz>1@3ovw|2qF__3d=Hu~op?pdDJ3dg2d| znd@AW_qiUn!e;r$K0$wS86{&gCZK6dJs<25fqSGA{2utrIE|M}414M`|M~7tfeG`- z9;;tMS$qw4L_WFQP62k6&wL@?%gow8R&r5I&xSIkg1(a+rIF4tCHdidq{2d>V>|-?!Ms7 z1s|K>wT0P|&#h-CQH9ksk=Xs)|A%zE`w`k!hmpL@kx$Nr$X#H(~1(>ug3g zI6KE=Y8l?7N!1}v)z0#Oo)1;v3jHQJp*W!}*(PO+Zm%*02 zXD$HmdT5SC`FqctgZg8eISwA{2$s zjY(oUAg;siPQsj93bpKR_#~;FRpdm(vYQT5bEXBB($$O(H0Svyo~H*}YXIvtzV?cV z>cw=h1=kd}A=wc1e90k*-`9?O0( zCk%%Sa5W`dh&eoNgtykq(Q$0p24-8?)?`!I9pvuflC;W+r!}O;PB)_2iJZYzxB^vZ z7nto1yk*vlu(g$qilEY#at`S$>4ED|^*nWr**q_5k?wFCmKk-)(rsb%KyUj*^k%C* zSN5V~$%Y3hsqr|JmV;lEP3~7+*_iw(JCf0pS2@;8ysvdKlrOk8%Bx6k6+Yp!)M;ZC z*hO;lkW)ZiVB=hhZ0-s8nY=^&lux$Mrcr5E;8Au+=4HFq)jOz4`?*8aobD>MHx515 zpDK-U*N(@8jtjObXgxxwwuNNFuIy+ouvy4SS9VgxxjzbwRLy#lWl6T z(*h>KVf!_jkwNMfo2f}88r$L+Q}Gj&DwxqC?1d}ZAutF(Y~mi&mZO!Q;$*=Vj_gLMp7L9HdH%sH;K zgiZ(8DQB3WQgNcqg(-K@?iARHM^R5su9Ijw(!!z~fJe|vyA>Ovo%RO1j*3T;;6dCB zY_|LxQR8M2S>#pis+}FxiHDj#DLRf!GJ#y|ZSacN>*yp`@M#}%KjHS=*L`G7l}}|B zHnwXykDj9pU5#!oN;V@!;+@FMynKz_$~y8pzUvc7Cp@VQa2ks8_Iv!_Qb!bu)*|bN zH$}9JsuVFQvQu;m&krSxtfEfh9nxQ2$AjI&M<7|~2`r0CY<$wWGre@7^x@GFf5_8h zN+dO}Iz41YuEMr%1JK)q;W=^!iBz~Ifzx_)?GteDNXR#s6`kNh*%`jX1^LR_M{SAX zN*FH-qO2??Tj;gffFHq8Z4ItsX{^;C;ZMj{^f`s+fm|i97NVee&rhBPn7L7koi}I0 zGdv1H@(_kVFIZuJ!yx(=SWg|WVUA6v$Jj${c|_90JYff`(U}yX1L?v}zXZRj2Q~Tu zgKEq$gy^HrQ+yg%;L26heklrQjq$XYPK_FaUw0<9RF(BiY*Zt)BJfv>>A%`XNggT7 zWb}qU{HT+XuIZyw8eZRVa2S_7kZ$l&N+>~ZyhzT*@4|M^;5FZer453VN5wx%g8YhyRe19=oh6WW;4B zd4dGiu>X-QOHMBa)kz}t1sv%xI9mp}T0X!JVYA$0?xad(ftPb!4zNz&$9v6_g55x8PLCP1dVSNY(huz zS0DlGg2`|U2BAht4`-lg@K>0tZg-f(5At{vW!?*T>8BXwQ=$~eqU8nWc}GV1AM7;Gz`9%l z4|ck9jSR<;q%+iJKhqZ<$R1)4-O&mr$Lw@O|4^|HJGP`ixt=ng$SX!c?D1;5Dw_<9|cm6(l} z$h=kpaaF~KHQqM#6g|Eke&a6jn?^K`DiM7$rh9bVm|M|Xr0LK0`ojAiEWgu3u0;X5 zz(|5Vx)aaK>a%!P5!PSMS-go_sF`ja+=ULg&qdH(LrzmCc1Km|Io{x}GR-o?9HO;aSg6r7AId|F7;$^ z-~;(`wE{zpjc{>a^4bFeo6(~F8OUv(p$eTr>vNq?>3~?LZ?Y#-wNXlyrH@`5` zekFOv7$7S0U3fVE&&Hvp8I0mF)VCGZI3~@IoU$3 z+#MEa5VF2~QGS$v{g>6A+DHd?<&gQ8SSE3@boaOiBaEmP(ThXHBl<^G_OgarhKIrO z?BZ>=`iM^IxW3A{8#@Jn{Z@hT*cBh2RG~}Z`d(kZQbbAk|CznEvV-@?pT>^HqYv4} zrfD&E<{QzPB#ZeZakrKy#L3O8=Z(wvbk6UZ@kD z&jnm!;*z^@gne;JoKHgVi#BneCgpiFHD@u3&PQAk8u!sUlpRS>g`}X)%;b4;y3G*J z|Dfl2O}8@vH=$$VZ+ZZdufSD0^Ew&LuCUxEnp5y<%S=`AU=}rH8k$OV62UKA<2izH zVbqtVhxzsY9#6q+oaJ1mV(h_Pa1J`Sn!Ksa63lU0_m;9v=v*2FX%I<0eAGWFb~Hv-M@yXJ(XS3uP{+TesC5l zPmaR|m_NtZPJR$!O@ayh#+-X6yh6l)e0;%UKMnp}?O=#ZB=umW`-~}|lsgIc$>gEU zb`00ga!&cA?o!a&7b*#-ND{9S`|Mp_VczRpl>&9~Y54$#>Nk|`n^4V9SGT-sp=q2? zeY~M~gpTkU%RO#B|17R_-TYo4w@tlx((o4fi%Ayyhh&^HqP0qCuY_|wOC|7|g_3$T zLN~&{+lAyLbapYYjyE!|HF4wdNugZTH{oHPnR_`O)@gBZ< z4TbL(^~=c>JQ}dCbNSE086)>cCWt8z^G~K*Y&UMZre4arD?SAhu_HbO*6|~*n8}!q zpTf!aQ4fvb92u;q1Jxf1=e4iQOm>5GtIGw-^42mvU8SoD;Jn_L>X41i+7lc~*W(vG zo{aKUXf*#duREq}iE|wJz#w2HN$5V}DA||y-%cFY7TO8LdN^{8oq>UyY!CYLuV1IO zBr|W@7uYO3x9{V86}x--feN#V^Yy-%0pm78uEy=>xzm{U^FWTlm*KP2Kn@$jg?Mgv zqH9Wn3%3t9U=YsFbHo$5$(jT&db%|j{OXv!LJe|zxhcYNBhRZvB!A{M&x;%~*10G- zZbcEe8wFsfrjzOIS@Iuw%^Bssa06~#IF?;uO9uVsq~mNx9-v^pDIHH6l~O%$+jNh$<32Jtl?MKJ(-7$P;y6mp+2Q&HA zv``ZiVZ*^@8W+NUWO^qhu9dl+k6c~lNH2aT$1yMOkqw!c#>m9Zb6H%aV>dBTbO;Jm zel^Lp+=n_WH9br|_{@JX8KtlmfV3o_hUj!MlbL8zP+Jm%IZQ@*@t#g5983s8QUt#yF?@*VZ= zCD7xIxO^Q#+j5Xht4&m@1mrVbcM@w?%xg|3Eo{XhQ?d?MVjrzK^{p(=FY;Fy7793= zsb)K_ew*FzCiM8}Wh>M)uBb-WSI^l?!cjv`$=2j7Wp}>AhAJmYyT5qrwS2fee9&TG zzpvNRIdj-wPk|jAi7so7?{O+$aO)Y8FM> z!*)e&Vf`(ZxS6dyLg4hW30LV7Y_w;(CAp&Rkc0UUZp@Eney%HxjLr_K2R!6KZcF`? z^-aXXi*JFbXc*+*>82+-k?7P4X5V=8EXZ+L_6QE?dq-e)l@OWmoqBBT;rSW1mSbfn zYoUz8Hf^AIDqCCa#Z|WFwL}ZjVa{3E#6&Rv0ZiW||MNpA$kzBqz^2aZgkvD^uj@m1 z(JPRLyL%j(=r8uXU^VtWJ($j#i}h5tcq%B~fbV9M3(>Ni6ARULcL2_!Pv{e7iA%8c zJ~)lYXC!M|{$k{Cu8UKk885{F=9&>~>UHj>Q*fE`i_5{oR&O*u`{|r}GKrtIhXt-N zF>OW#f8QR(cImvmA1>w~SqpXaaM_%#L49HB5hV8&V4B+{hw`b$mFJD#6pGwZnvL1!M-O4sehSA zmdYmfcp0PWl7hF&ZA<25QSkP$Y!o}2w;eJY^itx2XhOD20$B^p=N%YeOy|F4`B|rXKs;&Nvv<17m3ho31x?#`te;N`rQ_e$eLU|9f1_ zPG_={Sue(3`;qHR&S-$RAMX`-%V3dVemUaY>8v$4mCoL2_ zO-w-Dob_y`7J%KQ#B-yTw!pHOlD5LMiNpUhL``(ZqAt%QN0M05Mf2fzeS*37C$E;4 zd997Ugu6Lf|8DKT-FFA5#XBa$6zT(v!`HGU$j%Kl+P(^|c^2KlV3keeS1-J>vH=^S z4fa(gqu$OQP>4|^=Pq;yvkAVhez)d`?dpcJ9EbLJF-$hvAR2_yYgCs+-`WokT0Du}0gd?&w*A*0hJd1NM4y@tINMr|qlUJl|?($`6b zcX1bIw|I{WitST=c-PtRL+C+;+0D$7 z6wPL1I+?pTB-=RaF5owx!D`D0tM+fU24!JZHxYj^;m?;vow_hgPmnQv+U*KQd6v7J zjho?4u*Qp&sAWIWe_b<9I#)y?Go7d*>x$?2OV@`hI9kTnPQz%rhEw+{rveqM26%4~ z{@tl`Q_t8RC8BSs3uc~Mk~rXe7Q5hk4`Tm#92I9gx`<8krBg#~Qy7H9lhX7XOR1Ye^!n6$A+)F63is%EP#)E&MvKTyC>~Wqu7?usih4m0bD!LnBy=*J^_%Dd>ghhtu}zJ0&H(w&Sms=njr9UtHSNPE z!_6a0`fdEJ;gX@u;h2bK>Z$kHkEefR&rm>*v@^)cdTM8+D5-aV|5-?1O>L1n_YLXK z(IUwA{wI2_RQ48d@7GQ(bxKW^Y2DlG{)U3dkI`E&^}phJK8reR4^`zi=9)8jxqLOR zfU?F1!z>I7@jP3b(|lUxsUb(W6L-MYkKttB%s0N-T*)-F4E!@p53q*0<|Y%tDv0_LF2PJiZ(JLKH8;;-lpd%?vmX9o_+SMW2x zET7|&l*v6Da>EheTcsj##XK<7I44PgPC;We$dYWk^787}g$;lbH1V(fo zx5oHjxe4g^%Iedt$#|`Fv-Sv&DwK+y(PsXNRJaz8@Y>tCNpC-93iQ?xCHMa#juW7TnvTzKh=38eJ>m7MTf#q)i%5b zdpSu^i9MA~@r>FeX1klc2sC9|T?yy;f!j{BbE|vv*kn%)=Ze}E*&?QdmyLY+drm48 zNl(lJY^+w0f&R1lYSd%HSOsU!66hZj!{OL?bd9}UtZP$6_)h)cs_hMA$BAnLyTW+*=cM6|&Vo-@2yVa(xJwn7 zWJ@{;QA~%)5^6?YaR6PpPeoeD^8w_@-voWV%4Vl7ubJN|$&N3psK_b2O72Cm`%#sX zJE_Lk$oTxrt>9de>DU(XESFKu zOKSs7;OO?{iYY=?SD5Z=iv9=bow@jX%ZSpbRu9XxC{Yi&j{isKO+=-VFN&=!TBOjQ z`KK4$8gVnCYfKe2-fI$i!>+%Y`;xxy4s4>^>ZqF=M)WYZo@k6Ba15@qN%4!;!=Pu61K4i@wk)I%eOcgm2j6b4_6512E8pG zZtB+x^$MSLG_=ESwM95M)zfR@>YrC1=!|FgG0|Bda_dWQx(n-*VE`u37tmMV)Fte= zwRkZ-m4DicaKgW3wiH8TH%|N+A~!g|e7q*sQ2{D}rEI32Bw}W2^k2tP9&e>XIF@|o z(=E*@yajiTE$D6|K~eq&KiR{h4HB}RI&+8~CWiZN0srqix|(X*H~bBng09RV?>jvm z%Q}$N(QIDIvuUc#1k@Lu>2K&Eeq}@51I1`6Hb+TzfFfX}r0zsJnMS8!Cy(xc{) zRkadaaf7rWbVME)#Wj39Dq6RlX?Q7o1T!f|h3Txd_O|tTpHA{)U?D60Qq;ZzR;O1B1QE zoqh=Sxs@Qj@wFYyO5Ld{NyKGxD57CR)B(-%sU+WUgE@l^#^kTd&CJ+>({lmuoYZ%H z2kyLkwZ+y3(UsR<#BWXm^ImRDr#CKRtyIUn5%}gQd67CyPV@2gHB;Do zI)wbZ;v**I)a(?BvZ)y(7oimrp{#B%uUYt(97#gdRGbQPxkurkHu6q$KF)NT+ePRA z_t;I;SNEDdS&c#a`Inp0ua7pYxOzmNlLIAGQ<$RzY*RHLk*5slR~PMr3LMDZMh#!& ztW?Y0TFwl$A6>&p7=0v@ia2aHM$3!xh&4*SqMpQ+{Xjgfpf_x4{mPuuiBCAGNC4`U zABTYkbR*Ys%ZW!u{A1%IJ}S3Jf?ds&l#-c!2K`AluGpgRrgpQxJC8f1hU45x9`^=@ z@H^WBRj5nz!A4%%OYz-J1|w@7vsDdT(VCEF>fxl36TGayM58QIy30i`*kv10wg2YE zfpxTwPxZXhfcpO$uGbqnv3|_38F4!5tB=8bXo{ZE$stw*hoP$804~`XC$tx2#m)my ztRgjlfGOf&|tjr z29qiNk9(i}*x;m#t3UR@`;)TpGuJ85r9O&25qA7SN zzS{ReLgF#^)pvU`-~Qk|5bfRkK8Zl?b=P%z!}b4wA8#wUQVepHw?e)Jna^(&7l~vC zeIH#)PyLRwmn8c2;L`c^!PJwSoGjx+ZG8uRclBW}EE4Br`;W<!7t~{_8`oQ!8TNj4h&)yr%x7WMdM$Et+f&gSEJZ ztx#gR&1rNd^U=|J^ft?xeeSU1>#n`EUeUv}rG{uAEx)4UXby^-7G7K%rxux6u`JfC zsL0#mU(p*(<{qiw-vXzM5^RBes?Gm@EDZwOlwI;@*;wmHA2nVZN^g*xs`9}3S*vZm zaC*`Yyk|nG#!p&mKf_5lPEJjF)Rb57Cn?B1Sdskqj$#ygr4-VEKR-&g##?|)Ox2o- zbi>)FC59)tT~F!EVADOEe%#Vt;E)>T@tRXr`wm|>t9F|?F}`*jj5m#Th<+xly{31L z-B5pabS)|M?G@MdEp$>zn3eAF8Lp>S51^LK%U<4bzH5ExSgPs&fVG#_27}dP05{G? zz0Ge;$J_8I%&yAl?z=jZ>HZs|lZthBduRrkUWbARmsDk7;2&1Gt*$727L#e%Nj)Vg z^#<2%WqNNMW5pR6WgN%9p+4G#+~OvRuqkpKlgL;#)hvg)WH5?-6!K;v@j~u*9-}|| z6)%p8;cxhO^mjjy{CHU%b)>tTjbys;rEo8m&OeTxtD|b`J!QuH$)D!5QWLyaAi|4D zutKryrevqy%pGBGBJZdkod399QD+`bLkXG7TH{gn5{&kgvzLU=RxtcJdR5_UXK{O2 zLJm=ttth-5s=yN)t;#!D)jz6^(*V})awm>Dq3*EtzUFKWL@@y!1IN%nR|>;qn}(wB zDJaJd5Vbk~C5~-{v6dQ*&s1jHKLcN2H!YzG{lqmE*UFERit&gK>3;K!8- z_rKkN&iwuWc0JFTWm3Y!+=XY_UHkW7W2?B6pUoP1j;O1;t5Iy=X2T{bjM8@=dYnYE zEK2QSVgc-e*a(F`Lmau>R;F&t2%p5sI66qD}NR0 zzqWE9>f_@0OaDUGaoU)U=C`obQNBavmCt`dvhZCt)K&B+&PFj@KZI{vd3`k6w|IJ8 z640J%7P{bC`V40lwWT*GOEYTDJTb>vhgR`ENN*xF)-cgHm(-JxfAmb-Mz@w023=Dz z85C}!t`Mk2b$OhvXbt@Q-#8a#dGvZ;(479{%p`5( z;-Q|Nhzekoa}@qnY8-*4aCgMRi7_?@@B;mXV1rN)4c$6A#)>#gmjvlbh^~LUZsA3` z9d^)5)T1L%CAPCh!H7GFFP5#s&U&@et%+uMh}X->hC*kl(~?PNos(I;aI3QSP3>;P z_j?LnqvfcyP1uIJ%RAfgmMNs&;dEOW(9Gn{3LcYysI+2wT|s4tLk&rXU*E{UC*z(y8wBPp3ZR}$ zB|YG_J-55CDSl@k$FDdEJBVT6z5AGwl9AL_2FCsZP~RV%8TgB)bmsEPDc~?<;c*u; zSA4ulEAfAQ$KUuTX9k#{yIOCRL+xl1@U?Y~(@8iAjmqZi+b#3mCggx-LS6n_& z7rYcqP)*4q+@RKZ^PD=Q1s*jwgB*5bLw#BNVr+sT6E>cZa^)In(MOju$HP(iK?K;g zUlxhn#oh-y%sZ@Y&yn*vLt`D@GB~r;k8qrxvk{wOtblLx$fze9%ckaCIYKToN6KT; zARAHO?~%)G6@I9~b{ z4mX%Z_|yKVPS^!h7j+M1rAN+eE|CPCR65@INTaHFCX!g4&;l%E7qDIQV!pnGV(|=q zk`g^B{twnV(NoqomXLt(Qmbhf5-sTPe-w{!#X04qr;B*x9K*ptBTYULIcR%9fOo?T z*~6W929E1uoZF}38&*lX#VpVl6!aho-6N<@nS!_Bw}k0ls>E)ZjNkz8kbTEw^cs1MtYi(j(#jSKAeM>30YL}dLT77CuZ5;KVv-io*?=;gFp*S9HY^Qn@ zfuq%wwA>l?92^yTQukWg`BXMH75+|pNHZxwj!BqWR7a%JPSTUyrjqP(e$eiLJ00g8 z?g44p35WLzzLzI(+gQg;^qQV+51XSK=4IYPV&2>DY>iWidNgWrL~`w4e8f{~sX)lb zqTlcB41z2E(jH6gt>+{*Du7O()?edX_%{xyy+}-c$Q>Dk<&}e3>43GAPI~|y?_vFg z-Bem253w2RhhZPo1s&e5O#D*sqkVpat}r(oyLw(t?$apuoEd;Qwi)yo$5cAWf$aA? zGH;d*c4yDh7j@wVHGqz#qTiZSrk!#wZWR{O))%o0WorpHJ{~^D-5Xvbb1NISGG=p} z0DmQSsfes6vv>pj{Z2-y+|)Rslyf)P9y;sJoE({OE&i3w$Xn8xtIHdx;rip$gn@!w z$$OXs=dd&1j-h9E+JX!pBkOhm^`rn3>vHf77ajK`S;sC4E36#2`Frsg-{vVIcBZZb z>%Kbo_+Q-bX<&oTr9K}C{7xO%jQ>(n9Er104MsBQJg~>(hxvwirwSEj1FFNgcyw21 z4qD0Yyti)zYVni1U@vK?S0$-1%fWur@ydm${38NEc1)d^d|Ts++zbZh15!xBxQu>) zU0#&wxPfyD?5~*QeW`nHtg35?7sV%#!cD4IwMW3yGjQ10gFb7mD2K8kUbq#Uv(@2| zOjYaMn|3R-S(BUqbtwi%u%qr=CxUa~mED|Ol^}^D((mI=RFD0U{v37M|3wzUBT&Fq zE9_o2DpT1?n&ZSLd6}e-KJI(mPBy}$t3m~y1&**A#_xLWyXBnz5`N|+bRzeS8+5a4 zou{}{_Ar)P-Rk8y`X;p=ZiqNJMZ$dzePTQSEKR`r7HEq;8rkS`9;4&FPr7#$is@R$ug>2n z1V1r-3@43lKR9p~YbTtL)8OG>g4>y2X5iZDvNIh-&$B=;WY+`{x@naVmqDtxlcs%# zy-$6840y%Q+GY@&Ct5bn`qp}XPKp0L-qQ=tx!q7bPnseoTX@FlAy}GMQU(uFZ<+fchS%GwE*DWDDPo9&eJqi#)?|`W9~K zl{BA-gz7L@-vKcKj}B9%b=s4&RG(9|naT^Il~|k%mS*$S1l?;Lbar?UsU`MdnU8$H zZ=he(P%9LaE!gqJk=68nxk4)Aq|}rw>EdWB4*pkL&f zkbo)a6JDWp=t8I3+raMjJ27k{ddWC!c5XRy0(bcgu7M;cVD_I%6)J$*3lB8#^hACx zp^+f=#AxlTkf-337n7yP`8_xJzUj0DOY>_#V8m@4*J!zXfF1k@yw5YVZUr37O9E;N9#2+ zPCL7$YUAN?u6B4+VP|QgJnuOnN~L}F-`r*2!G$V`IPOngGQ6`2$Q-csD&fRlTn(nG z^!GOSmVeXF5mCcU?$-(bp?Z5m!xho6C-dgu?4Qn$@{5G7_>a(^?w0`+%4=k0&>LG_ zv-UFO)naq>hp0(rJ(#@y%sC@^Q)kwQg?3q78Wt(b-x`|ehC-tw7OE2d&*1_xllP9~ zh#t5;ma-?}+_>J{BZ|uIMn!Q?m_`P&mU0{M#Bos*UE4)5IXDoMGFPww8jT7}nwv?` zTYz?>e6YDShpd(jpf{6XZuqztj->MJ4Q@rp(U~mQYbcJ-TNOnE?WlPXRbX**DyRP` z^Eqy);F;jtv5yBq>t@SU^fa$XLY!m&E*k0y?4lwmY>F7sUC#i{KSj^YU9b=b`#NMD zenLlC3TNi*;=Eo1Jy&TIVQoYg{RFy(J6d5hXrIlqB92;xia$Q*$SGJ=QEYuy$!3}k zGW>}dXB$1qHae57bSxX4X*j7)Chub|HDLn?$uh8`sc`E4rf*s8T%y`+rysj%9-|`t z$lse;bLgWoGa22d3O%J3J>yK7C@lRXsY)R_s|zGkys!t6c=8qA)^dK*S!~k^Si?jG zxPq?Yo%ZCy(RX6Ytnu%p?^^X|=68psLPXc>+eLPht zzy3GxsxKL!e~CJNYZ*!I6i$rUM0Hl%}+{NZ)YB__R&5}l}iAkI096;pRt+ga)i?pw^!1zaX8F{2h(`m z8n@EXtbq@30RPfh4R)&!w? zgTC`P9{jiMoVco%vPkh!V_*@4yj5mT`Cazs6`Dz~KvY!CQFBye`+OJ&*9K~hKc3Hj zGW(8L4RAcwj;pH|&i_5w=`6Q#S3z&SRb5l#WMXfiH{3hxufg%DSZH!YG1TTa!lU3X zmXEm1r1~~g1?RZ0p{ z9{3SQ{W|i4eiBB|E|uC(?{D{)g-=9&2vv(LAM-f$TjaOs?*7+swdgUS3K82P-pr2gU43E0iX&6 zf*m;pa*%1b0hjer)Su$~%fdp3h80&gL!XoEJSwQ$)g-NKQ3fb&4?Hs0fJ zG~CK1OR8&`UuT_>>0`vBsEN@pNxH0x20uGp*$8$Pdt_HDGn(AmDDg{iMNQ!jZW`>v z1TlceQEVT&@_yJ#%PI+il9qn5cv$b0SS345N5am#_VCiglq z30A>BgHiJaJ#P!n&x}?boa*MVUH#FkX?}1Lsk&Nscx@^1%=t^~X4ni_awytMlUR^WX-5e)VVoykU+w!H)2s4#N_mH*Qf zUNo*#RSv?v?;AKo$1<5p63aX}j>d2ilPTvFsS5wv7fJNaBF02^kIofSBhUOiS@UGo zU%~u}O+E@RZT)2=cLoP;7_aR3Xio~msNBLgK2ZyR`z2uWZOBCEPHU*qRQ&XA9Xun_ zdppTZ9_!XM&N?~d6BrerWL{20!)t+d?~%-eZ&Es{L=E%>tMxnfd9+0pLEuw>m`z7X zv>5NN@78*m5D&cwnMZ9>#r-~^>MGh_9L|jEM-n%duQ*@LfHhZHHKK2Pi;5>Nvt&1e zeS>_Z-Nt1o9_p|{oWiZ0s^X=!M3Rpz3c1a_yrDkfH=&AAuOi1q>ft5fA0sQdXW0+d zLr;D=T#?7q;au)|uWop`e1P+93x9E_Oh^Zt8}D5Y%?VZW>*u_msYUp3xIiem*OYTt zu@zd+EYOVf%uF~;KBt@RV>IQ8ohbfw-Eb8V$CKgW;-*^~p70CrQK+Yy1AFJZF+pU* z6|k{LCWq0tybDWFLO*ih&)hbW?UsA9o(q(TU~l!atQOoqohhHTT-P9EyX9Ckpn z!C`g=FQUjx01k5#-AyXkyiNF~da?V&&LjKfpN$Aj^E~BX8)F< zeGy(n*$O4f*D60Wd^j>qrd#?W=LVTR9i8LitKJR1)=#DoF>)kMDf{FcvS0%85dFzz zklWKRC{My>*~08|&isoh=_q(hA~r|6Ne9L!!}$r8-7_Zya(e(gIW<#NIjtb<{ZBX~ zSju*$pae>*=dc%xi-Fbb$fgA+S%;h~dQq{F49SPPLPpo~*iQsmWP_-deNq?C&h9fEG@$u8Q)iJZI${Rm@z#{&&0) zqA#6qq!mYSOX~@*b&}qP9%eYx&2?%i+!lJ#>wMQkKq5AxUaUrc*pL7E&G`mX#ZgUh zI!&nRJBB;bt7Vho=AH%RE9AX1qzI|*VuRZf)!Rdyjk~ZP{~|x~UoW}2>F_G3tHFgN z67B&xp(g-${ShtCX1KV2GL@uu`UG~&p{v0cOBz?sqet94>#cU&u#lhS^i#n!#0<4OgE8;472)W=rGC zbRN&2=D7~%(sIts@gjS57TioI8SGj5|18>7z!-MXybb5@G7&KmVr^>TaqBSXET)<*7#d>)Z5@=U~Qe?jPJ zL^KMbqoM0+7`^dP@0PzL+&EMqd^nUelps7Qbj|Lirl?(LL~ehVyCPA}isR^HJ{v z4xFE9qLUbgw|PqZ)|T5pgM_RjeW-?EgUChGnM_A9)Q1x`V{inIiRf4+@h=YJ3ak*^ zg0^~EkW&ei^%J|=N9?i>v8S#9U#=*6=lbktKaky291cticGJ1|HuG|~W)=T?JjeOy zfMhE4s`a%_^eQFUK0kD#m>`b8qN%{%yss7?rQsy)cRV^W>CrHJZt8JS*|)$e1NTvL zHC1rD-#fB#s4dPOSL7dhBQ|IU^(yStZtGcjzSMY%hgCuNuGQr|oa$f7+bFNEi3;%C zl9KAX5|+zV*gH!>ax8c;?KF)}=@mSRmrfO?uvX$F%8tb9JdEDs>ZLQ7ea#}LB>c*k zu=nH12(~|Cnbxn<`Fz2zY$(0Z1Zx#L^j4@;8fy>1!xN(;I7YI@a=5{*I6WVT;;@2? ziMnus#={xTW=|6Xj9pwai}Xi!Tv;3^!xJKjeSx#AJLf_@ub$nI9qur2;r42iIa;Jw zy+G`vWihib2wrWoC91a{t&^N|&#hFdySoI(-B<1kJq8tLCtSiFlWcyJdAS65;1PPl zsi+T5X|rL7ZPwSY9asyZQU!)*eVk6)qr@qT|M?SChlMQ*Udk_2*sx^3MF-ef+{0UU zxd9g$XY968Q%at5tM~1ize!(sffeR`NWaFAx;s_~D#08f=zoHl~Wvy3Vc*XbWP!TsKot@IsSa8;N#_FyL6tNlm%aX8K`?dG-VY@SG4WS^dtpEZ8?)|S0)%f zr{!#K6|Sl6z2&5)9`I5)t-)X}lAADF7007smP%_zp}st4ki0Alva6`AlF_RiQ7){= zx!%-FTQl#;;)L6TPee5oQr-3*nA7>4A?7{YmKGUT$n2Y8jE4(%+z84T`I~u5PL|v8 zvYRRHpec(_GF>I{tE{K@cPa@7)#_c?1+Ue8Ug?|PDfA_BL}Y`=rV;ZaoXEr8bH75w z;)r<>dn4=k`9t6RIT7E(AHp4TyvVXSW@dzptQRhARpy&|W+s&F)JCl$vvUB2%QUuW ziLJ>X@dj0nA<}D?s63PO!A^IvTi=e? z%bNPGgjFPV_YdV>z-cRs_|q~MP{4u)0Ok-uUTjyGe)c`%lN&Rruv=U_9VoIPERCo2%I ztzb@Ys=rZ>t^m=>%G@28;+A%Z^x-*dQD49%8)AQT<{7aY z!=~mbkyRd{O4pFTppHBud)sT|FKVuRK&FO;`=`8#f6IFEc`w0cN#GWQ2~*5_r$;bB z|6)w$|6M7jxve}4+_S&G1x`gp+27bA>fs7|T{J=oRe_yR19mQt-JaS!kiuJ}Gl#gB z2a)-H6f8IR-<~KjZalGw%4#;n3BYe}gS||^2kbIA&awG%Ye^q_%bw^A2+1lkN8W?n zCZ;020@00w3ZN*@Kl5rysPHb^=aVRsl5$nUHN(YaEnCHZNz0D*(m6kpwN(oj+SF<( z&gMlz=V9|cz;i5`Zf7qn?&Qv<|2EVef}O#~QtGwn1Y77^Ksid$t6anl<{S>jXPG61 zH-%18^IZH$JFDhqA}Z}5kih&nX=f0Bz-G)Y0&siU!Yv$0KU3NrW*1c1(YB{lm*j4_ zmOa`$ud;v93po4KYB#^xmW}6UBcqTqGmMcFB7rfNP3~xODLb8JW+7Q!I&?vC??z@qO%x=PdongYMQESyal!H!0u)r7*rW+Has^UY~*9W zr`p7)dW=Q;-;4@l2ea|>&w)6oBpwGYz^Ayvz0ikm_9i&Noq%bL<|o;$!E_hH+0uUv zw6XBm2{hw%9tY}CNs{84z5(RqPf*+tnF~u{@-GL085B%K=EBq9UGDu8^iB)m`A>mO zHwtz9NO&(6(@^1HO1_|~y9Hh*B5+LzsKQgxQI zwTAiwP181+1^>MX+>e!UQ7)#Pu$qeXfwlBq$#KUT1Y$!=cBnX-$?z;VqZLf1|urSgw@ot-i1z zgi{R%=mMnPS5xVo7*$4{vO}t*`fe&VjFIM2c0H@WsDtmpTJnAo)LlHa5_hNbg8EmAwKh(R#*a9zC%i;^~y!sf-S^GyIDcWazx`OZ(@%iOxY>Hap^Q zmBLGm({d~E1Dd!jBqgPy+llqs{gFNRXLeOb;UxU#EY>fhZZ3kO*9K9^&WMLnADqWS zq1tY?(3>#5FL?V+;}jF_BG~wSWE6XgRw}o74=&Ik+$lfed~!(USJkZ-=mN6gLcCob zg>%?Tg`6%VBW85E!cRMA-ry7NLl0Ap{Hb2lu^1-6W-vw)qDCBMTyTzx!Kf`h$Ypj` zy3<{vHEA=waWB>Vq+*oY*E@(BJ&gBfC{!-AAj{>L?a_;}SIRsurb~|FS;lAjEyt@Y zH)2x7B#Ty2&-}kb*CI~%Z9_F9ACeK$Jlx9iP(lSzY79kJ^#~ui46u3@(xsH`y3yTid^&TO7civwW}@ z+_@nE}9rqsf&_Ga* znRqF@a}u#b$%RWsCtPq-GBw1A?PSLc!3C%fm9rImzh}-@<1kzP8>pcN!b@L6R!brB z7jmnwoHPy8E4IaL)KwUj5?0cCc%~Qe;V&f@>POj6b;4oQ71P0d5{fG96b?FV@Jh|) zY@-7y$8>p{$!>9Ax>YGByfP=Tuxre9p`VY1)WTLTtaq>h2WKCaq1psmz25CRFa__ z7zlsk7@bT%r#f1{5jdjU67AtZ%@WP=H~Z-P3mb4hP7afdVjzbrtoC?i$4=Z^+!3;~ zH_!VRwJiE!^n}RTQM;qBv$^i1dK#(lMZ6DA@{5`(5_VCs3~X%o*KLUNiH^9^5r0#C|e%=kty1(>mf6`9WU^SG*Ki zDjk_vtJrziTF%D9e1yFa?~Bedfa0J&DE6P~HWkC<$}G>s;c!)Crs^g|_4!M%J+2Au z13itycE`YUc*orXvzc+~Q9briXZ{~cXC2&zkl}0lct+wv-jMYGiT1cq=SEiV<*|reDa`Y)Q=*; zpTRJ(QBo#~JUdg(y=G*NZP#7s=MA`fa6 zP_;2xQ_iM;Je!;-2cW)g3m^ELNH3IFbduc2H6J8hdpg<12`?rp>kB5c;+Bet~!hEN?{z*{{MIzKccdt1N}8A z(7=oaTlTmPkRSbZJ*IyrJKj6R=##xl?$a8UwFa@7A3IAPrg-NCyIRmAnGfAvxTt|w zDs^!9#;-6xSLg&hzzpRd^i-B~hZBz}V58s)9Oe^dMn98}-mW3y4d^-qWOxAMvCCN$ zRiXjahjsXIEq#+7JTjro#xiw_5k!0f_bLUx4LMLE{enBXM1J=ZPCJ{($qc^1LdYrz`gM%6UPn=()hYz1R~M^;1Uwo9yHmn_4s zT2r2c$*+woyGHlWcKDW_us7NDrZNr}N(%?Vd;xcVmfpb`sSLb?o8!R*n)}kSy|-*o<>>F_7e%mO&1d>+s=hPhNA6XtNjG>>!?rBTBidp>LK`h(<=p zhT3NyuhZ`0*B7fr@xx6L)(+1Y|6!R_($Zn_b z^bMG3ey|4n&y+u{Z*~`Jho_FWMecRE76TZ8e|-xUg|H%OfkN*b?KuXaS6PW}e;L<2 zCy&<=u}6a2r=6c+Sr0KaK9_i;P9Gt`T_mOjApz47`I_pa^aU!366RW&Tp;2Km``A zwWB{f+6KpV_N<1E7I>Csry9j{X1KiNqNH;X@zP0RhKAt7Wt<;eN)p{Ej&X-47Xz5f zxf5@MEa1#1R{{O4XifZZp8W9@Yu`d$JdN-B96T(JFV+-2cQb*;BZ=V` zapsMv-KxSmhHGtj9HzA-WFR5A^V5Ku^ewM)R3{+{<}hIe@DgMn0GrR0*cH z?g6jthGu3U^SQQihT6(a>bI-a;jnR~s7w_B74x~97-?Pm-TRC}tOOfPW-uyff0*8V z^rv{ZMz4C9QN%r62jH>}JP%`sw%BPdz9xYFeMP9L%05 zsu3N^1=N+2(ghO&dQ4z~NiU|<&B3)}GPz7e`tA$eOY|wU3X|k0@}ISq>Mu zH0EsMf%G?3vd5w3KYZ?f;Gw!jw36}Q`xNPA=3OXJ!5w0=)9iF1%#T}wrUi|k`-1rD zxWHV7y;OGGT)mfS4z6l47}sz|8tTRS@X}AjPS+eqg>&#H-(hsq6O&8<2kwDCsWo2> zIy-U5EHp50K%p=E@{=4VsXqJA>4CnI`gI&Q^9@v44#yi!Y!gNFvICvWbA1hG`^w|Q zoOvt%>%*2JXm`V4Y%J;vn_(3ux%bFW97I#gTJ#51khAbG>4Ss)S~V7xW?^-Zu?Xh> z5(-{7&JuZH`O<@{g>W&gLymtEO=CN?ARKHY3ac)71=pf>ds<87W}o>l?91Ah8v3F?6Zzzct8!1BP-b}Mg1|Jrsh;+7PFMFLL-tS*>6 z-+`<-o`p#us=fB3ognYthXc^hWY7s@; zn*o+-f>wok_hVwDitreNL_gvpRfeN;7*AhS0QIle9A}9i;6QJpk#M-6%^*fsxVzOL zMsYN^^YG!6>{BlsH$i|LxKfOSZ#>TDU*%X2wrl}U8lo57hI{C8$7bT0gH$@2f+u0T z<~!&7k6L^*j(i&u)T-U#!9wAhD!Hf7H?bP5&gXLKce&cbsD7@;iTAZphZ$`nWq>l4 zd4tc07?&I0TxE=L%&E*p?vqp%_zixfziblIodz21L`(G*EK?u#9bIOt=;rvw#JXnm zx2C0sx|+-(RBM1e3@#;?Sx84iZpP!QP}VHPw5h^Y8qGr|$ReE0s>vqW3i#U?bvU`z zcJ(2Z{Q^{=I?3Pad(l`vrqec!`9>@fWsM$En6L2f3WM$0itp1$ zwBoa5I8F=sOojUFXK~Frn{QD9tZrZ&M~6Jm2mvkMFy$~BhAImTb_1?edsHqlph_xY z>`rhmryZ@~U9Zsl^H3Mgt}wdi92I!}5Q^MCAjwPcWEmZ=&xv79qh^_jcJ6}iLy14& z$#bx6J3L9eK9-2(GDzaalYbt#^n(cJFKDv@|JyroDPPH<4#C5yR2g=orky$PzL%-prg<9-IYO`E zGoataob2(p&7ro_5yiEHLF^)s=xRXf%r_DOL)0|<(m>rq;)7IP1v)vOM zSlqkM>kZoLeeTWb9qQfX>1_Hj0r-b^iT{d#rMcByIfFj?YXNus>wAj$C-OE7m=f3_ zXao#TNBcDUtjFj}A6!{98!oCIE_WYU+H9(P(f8pyoe$p6m)C@n4drmpVkgbPo&3|` zAl@tE?CjnGw&ZXxVZ9fCuV_p6hpz@yjcDYBqd)tLKND#?iRNlaJVr858Eq>!<3zBA z{(`%3pC?eT)iMPhA0^EGRNrTsUh_F_?iXMqGs$Pv>9(U2%Y$-2DAVz8yn_4m8zYrc zhcmgA`s6`g*F}2n8pVIm&> zQSfyGxr>$%&3@3cz}r+ZqR5+yGP^JXb8q)BJ88dYmf(Pw)mfQHm*YXvt}k$QaW^vZtMos~f=U-Akj%kJW+A;f zy;1V)Q{HmV$7vl=hCW3z6HWB;7k+Q2(i{hX`br|BsYru{FhW#TtB7Z!r53_H*$jue zB4#Sw5r2yFOap0QZbYU3*=|bpCmk*&2XJIc1h)JWhfvccl@rwxpi_JGq!A^mb0=?+ zS=>wIJ6K{UYQ;8p8MGX0 z$m`-U_+ z^<_qTtuPq1Q@c!#*&SYM7)V}-SpWl>vHnP8!s)IVnt@VssXU`yfY;rFYxgFmM`e%$ z;bHO;=Zr+XKbS6wl(IL~=VY?4Y_5$Wf@()!c@cAo=)gRPab!UGJ)u@f+vTa{>0=M~ zgawreOdC)t@2Y@{0UiDGdMf*G^Q7};Lr<5>zkz>RYcGA!y^JbmU#l0Yk67WscjdP} zT04m2X+wL5x1)EOJBAtGceVb^N9&HiL|^#r{NRS^EbCrF#Vs#PY$D>5d2}Tghj%SQ zM|lJl=Zj2+y5(v@{1(dHK2VHTgYmpIQ572I4RxumPz7)rHdTp_dTV)?zR3tPFTNSCV8FKP6TuVTaiCg9zeVqooBJo$*gj?8^#A4?dKjdutZT+v={}uur>Pt@SkGkgGKH+&6HWX~lSFTo(e4bgj+q8lq@B#A;_)pmC|A(4)rE~&N%WFRgo%nwq}nd-siEwLVA#6(9y%=~65Vs*i-fUC2E2CLq#C=ao)sFeF$IzGN)vvRnOHy6UDN`^pbGG@~8O%zLUt5>}bBaE09>PnoJc0zS7Am9k#OJ7y4{CSEP8u7Xcat**eYbsX*)_o?4? z!|CM$n3J3i$44;bYl(JV>F3~TzHrP;uXHW6Otp<_aQ@Hn{~s-)@Ni24yE0Jy!2Bi; zo>7K%Sx;`xG>@Q36O^Ci@|EGzKZ#o8J_CqY7R$}@ zl(rF^xuA`at3jul@VyhXA_@AU_o5E?Ge``ScSQjv&J35SZ00~Q1!R-GgX;boJJLJB z|BnBd+-U;G2Y&RN^tb~eWO=)~r8oI>KT`i48Fhq{g~ z>;QE;+NwV4W*%23>zqWCZWt{@1+vQt!lu7>G(LE>jgF$YTES=q>t4F4o za@QBzU{~BshZ#eM?_H{T_0cr?j!T)Z+L8GtqfpepW=2C0OlEs>pHXB$_qZ$ef-Rqk zhrKWx7gRs8lB?=Js*q)glbfks3|%%1_!|%&R5svEVE;VH0}ckh&-2?qPtdtMu>lc5 zPxAcqdIBfpuI`Nq_>xQI%BKlRr)`GYFNq3wB+8t`a7wqGqv>3DOb&PyhlHpYoiZ%?{L?Iv7jw z3b?HorvFIdNVAas*}JZfaDZo=$KksUIZ9D=-G*<;YvyW|q1M>|7tOs!Eb|3IaJOsE zsw}5yxC`!5SL5$A(w$9)QyHAaU7eiz=35*U-r?#qUHL_qVvz(kU>I|~_t3f7T2Omp zu22`V9!OhAcE<%ehgk!ieO9@M^IVp*dB$y4*B8hcRue5Ls+BiJ7Wz&ey4TVt*IEYK zlRZ(!N4iGND=BdB2qPERN?!TbosB5;D3cDV(3v$<*4G6;Wl#%)w;AUlegY5Mr!Mf z<8m7NP(yn?D%h`f8v8MGG1j1uJZ#OS*YXp}rOm9$7a(vd`f5{yu{Y`TDQ@ng$KEY> z;BqzvuI~*|Wna_|SMbLOLxX*mx{_}a^dmK!5e;LL6IaqBpwvnD^RB4M{ZWGLr>|rs zRkIahfw`4EsHoMMZ2yl4rh=FakKeBNEVYJT>WZ6sGvfTy_&RnFZ^-4dHK z&v*q?RnHhl#T2cR+y%$-6$M>Mveyi9tdf$vCJ%X22s5+Vl3Q7@!2`v8)JMmdJ7dYo za<^6)Rlp~$0Wr@vEf)y$MN{$dUPzU{6W9_YvNA3FG};MxEGxv`#f-%hRta|`lQDn0 zpEC&-2M+7YL-y66t&@bpC zeT%Vn1$o4LVGhRMVH@aON=(GRuBWVTE>og#h`O&Fg7Y83UXV_D)Hu2qd&yOH9s8m; zeL%LL2D%f+q__AlbrFLUfBO4&C?$>mxJz5GQYtdbco;L&gTREc>K)EA18fOTK=Li% z$p>`BzMM1`y)JyKkf`qWie+jH{L&ll?Q&uK7+hu~U@)K$|v7IirGD z$c%|jI2_g#dC*H7z?c31j{+6%GTEJbCbtaXUY&?CGXnl{HEiB<*tzpWv#-cLblf?1 z5VyRdr_3aaU&k?z+6lIyjsN0+`9VRR5dTp@$ITC(HGy$rn!Um6htEK)y_&s#tF=(vs^SwSOF+@&*`(Z^TcXvF;JnYc)GR39-lawfHy z=OUbM{ZQ;DIyx!b?0-E(3#Oqt@Xy&sm3*G^4R4&EiU-Z`5%QE$#2#(*6Lbp9(GTPH zIbDBB_471d&BdIznaj2w2Y`6^fy_+W>O|%AwjRp!^@(0y5phq$zqdA3vq*M|iqwv) z$&1WUi?T;(KXDW+PVd+~wE8hz{~xZWdLY-iw=qCeRSSVC9qALSNLBPdoDw%`-Nbge z*8PEQh5YVQ$RYa>L{)0ui#hj3C zN&vgTn=zZpvpGhaG@f&LJ!3?I~Lt*aR1*siV4q6i!4+wYp`+)A;Z&yK zHW8{krI&q{%Y$QB5p@Gvf<5Re8;epfHkGO4rzG+jqwdnz%BDDPe5Jdsx>lG^m|Dw@ zA4nJc{f;{}s$b!1mXl>K0$X0g*d)>J!tN&3V#$wwu!|jstNEli;C?&B?~<5l^5>n) z=%(Ds+??`Egc(c*w2XJ@DV*r8vdI>xWyT0aouiKynUtiuE=NW!YM#~6Wc#(MVUOvEtbP&S6iy=X4ekOhRB)$C>J2zG~;ZqyD;PA$Lx$=P~!1ALLLJlKzZ&I_rPY`x!>3G5KmPvxl+* z&)7b6I=^A2(jw79UR3Kb8!l@CU$jOIpqkkQ#ZIK?1y-CB-PF(ENlkSTmB(Q9EL{fe z)nxo{9$phB2C3U%6nCOjj1n$-sm4=_d;^|bQ*Ie;+1ckC%rsTz80AG*Wf6}zpj4?U z3V>7-|99N2x)JRy47Q`*fx!d=0 z4NKq-kph?0ji`l3DR=4APO0Q0Kl+CQNgmY8)!pr>zemHXzjEf(Pcc=tAycG7Q6de& z1J-Z{!jFwc&AtVW@rzyt&0$_+Fss8Y!d(AwFK7vSP{JsnCS^*+Iy#6ddH1X5jHdE~ z5{kR&Lpm|(Vj&}I48lz1liWugSA@JHt(YTCwAXFWvrUA*s!tEYadyC|#uIr2g~l?Y zC2Opa%z~T4He!|i%=y_3UX;N}`@JzzHdgPmUo>Jmq{ZZ^CNK%{j-J%%Jj_HWiQ@VQ zTAf-;R-faOjSgRNR^S_T zhQkWQSE?Qp#_1N(s=-P%$%~P=ygk>T`B0Q=w53KfY*HuhTml}$aXwo z?xE>@K`kki&ZGNuE&15p?d)H-s6a{WGRIfsvwP9>#_E&7pR1^A9;&yfgQucC{)d=3 znbD6L!&o|Jx8Ov#lv$Y_g@Y>R1tXrW-j&8g*q9f_X;xDf7~Ps^^;46jZXwfYKvXaQ zWm;dkjEJ#@lz4fC$YIQ$8v)5uFE)f0AosXMn^FW2M5wT-AXBN)|=3Tue$i6VE2 z41p^q+d)C3TN{-NMha?M>0q8q=%al3B$d9Ta)ee8)w95df8I&Ut5_}-io{W2M%NDseU#>5B!FCwYSj+j$&8EzflcCH=Y-!j>&`6 zN!N)S@Bni~5)fmtIIP5xTa8sVqOA&1mKmMNkJcF-{&&2^{@YKaLcf$lgn%Jk;5~}6 zgGA!dG|m`~)~bvguI4kQ$$`!rX#Z^D&uAk+DMk*mRViQ$62LS8HfwOlOo1&d?yOF3oW~lzy81Jfr@-Tzz29TWR;();CZg=dby zNxci}A~ju>O_*-5#fX(@>{mE_4`4cTfVeA{YAK1dFM_2Ta2I^ZT7IqW1UYk3$BGvh z>9cDCVmH@=MIdJ=E$gU@>8%(@2k2QmAEx7mv6`MxgWj=AOyKdeb*Ar-eC7Br;TK$*i1SFdNcF=CHm*nbXcdB zeR0e0BOen-5xu)CqP0{+CPAnWZ>WFtbgqx+VsVJvc7r`8%++ z*?F9VOf4(k4!iOEnW^@`kL4Ol@Aarg6VW>`AFMb@b^9^LS=6kr@HjcY+8F+Nl3&dM z&RoK+?iulOB68c2y#5UD?!qf(EHO{Cz7z-D_RJS+M9uU)J;H@VVlvr-bb3~x8hhKB z6Ft~5@1d)kC4SEYb5omNdu9xqq?roHp z+=g5sS5xC$K|LiMc#;{t=_J`yt0$_0LM1_uCi+A9&U{3ze4$wqH9~V)S?_E1v+6U9 zFqOGS-A>?aNn z7p=ASAYL>)SQO2t83Qf{EHm1g2kj)eCg;kM>$6?o^U2c;SGAV-fxjT04+~IBboTZZa$gpb)5PWelJ38&SzV!<_Axvg1~O#nODrtE2A!< zLnn>8pB4H{X-emm3Ky}Sc=Z-pn6928lTy`i_ST~0O}j)jHG{YTYZ1wM&&vHX1zbA9 z^Z$R$4g&0D2FEjM1l3%z?6Dout{=0Dm>29n{_TvjW(iMGuEte3u>!Kbe5$UIbC@S` zSR|58BCDk$16g!8k&XA=PArs7oFK-i1S64*jA@2wt=19u;fhD$cyLddYK#&M)iLm& z-<1IJmq6dX$n!~=v^x`SsygaNKO(LbYF|FpHnj)eF<5QHCoitL@XAO+J*z%AxkGo; z;~5QqQkZ_IK=P>>JpM@@lwHhXcJMv!Ko@$9x>UxGGNJFedW3nPU)8gEv||*V&<u zZgmWI#VNG~zd|ZpN&=-nwd|7Y$VbIgxk>2-ZkNQZsGcm39<`K=C#za+jCPDCB0Y*z z(;vM88c4(N!rr75V~mXEII{yQXq}wq{z)7@n_h_0%!*BhpVce0&8z5`%c~Vc+gTf@ zm$fKT9_w?-WggJ`oS5spflAO#uK7tanh7v+QRwb+Y3KAB%072_5e36I!F){j)?yLI zE92qu{_q-~1Jw&~#t;0TPk-VqDh1iWjamG5S2<5&?Krx-IFM&2s1;4LQvz(6&gZ|X zXHXL1av+s!_@jQpD;mhfF_orW4uGw^9>!{Sn}Caa+rKe zhP6=sbZ?WHnICimH^Nk^O(ZbP9U;$J-P~80HR*(T`^1jlOSoiD{Nn1#Ztm1(R?~!6 zjld15B)?IF@to}GGd1Ax^rPK1$Cwvg=iQP##bKn9t*H0qkpbLwhpY|airL>@N>yf` z8KJI6U$IdHRFM(L&S0lFPGeW z&f};(6U*spN#dM^e9l1G1BJO)7Eq(`yL-=PTL;TM0{6=m>TkMtSEx<71BdeXzQ_l* zY-W$U#`!Ixh~2Dx!CJ0F3@}LaCC0e)zoVZzk(J&-Ij2t+)8IXph+Xy`Z{mP-L~EC= zr}UdYmc{5{iIbVNSa_`$Y8%-Wjn*4p`AA7i-)^k3hF$xC@(Ayv4D70#aqn!07tK-F zkH+{%1;aCEX7a*iv0Lc~FHue{M(*~PzxET~lpG+*5iqu{SfTu&Tf_}g4HVb#;kgL9 zj-^wqA6?vQMJHTg$HHiR27IPP8F47WaN7hr?bI$bYJ?bx+@$L@C)gdpe@(0&M_bOB_ z2eb!hwO!KL+f=V~`2oksV7k)$$Y`QqLVl8iJV#}}Sh>ude4R-@F0+T;S4J@l;GL*U zm(65zk2y?^u$EgN@$L>ccW}3F5`P_yP}Z#`a=1rC@dI9N7U+GQ^_~JuSjd%+p$p&> zHH3@QpyTzCuA6Y!b=1o~pBrata@8}{ z9+>%e?o8qX{vN%UYj8;%qrTi%w1zo3NzdsBrvI#!CET0DT-l1gwc&COvs33ue~zPN zBKJ;^=N-On!|<8+Jrj*ucmRFM-^2nUnepU1OuRrFa7wR0=VKjKN`5n)Ue^q@8oE=O zi_Gonbguk6bqzBh7K_(-EXOm0yooiFShtnu#UqlybU8S7gk$*$db~C6iZ1F2 z^5W^N{j1`Z+DB+IqdE+f_=_XK8gYTn)++E$?_fTU@!87ZNHB^2ng;)wQ`pR5DJ{Z4 zp4up^+Vj_EYFF?ov341K(LEGgfz+vsQwgob9g9nV+MZfcVpRoI%F&g)8m=b2c>KR( zA)=bUtmr~yUViMhYef<55H7@;)`)s*v}zJ#_hf4A3v-ovN7yn_T?cMtQdhy)JW@`B zD#uyl#X*UxMtiX3tT6|_k@xhCE`|pv4U2;RwfLv}BM!N!v@)0njzh;V-s>Y%A>ygC zv=ax}!5fK4Bw#XX|}t7*BO8f{60o{rww zy*S_vK~Ggj45q87j#tDtxw`?C$B|_9u&@GP~NCn&3udKMLnDxZzb56SYgYS>~nR`;;|LkCTz+S@jCp z)*>`5-%uCyCeogu3XvYaqb;z<$JMpO8}s>PnQfxZ<@f!fzJZ7AK?OVwk%>k8*0dj*;mu)vwjNu{(8T)qkVXhGE0)AhawbCeo&%wiNA^v#_i}D5J zA3!z~iO1bb@||FC<05$Smnda6bB<0VV9RAtD-BuF5;VQ%$hq=q7r?Xz{IctniZCq} z(Mv@UM9r%Nohf6e)$XH8xCF({6ZozBxQZ<`GUBIEO;psH;#tvL%S&|kLmNR(v;)q( ziTte%NB^JIJskw-M|FFaOy*ufpW{8^z%F#A@1>LL3$>4~vJmRTnX(2E&Qxaq`SdWU z+(+dGncUqC?#04o@vb?Aj`d8|Qe0n#(J{Ik#m+r#zt~HTRgyk{54t7OS)r%}nhTS9 z?jEty_+;8vyj9$@-?J0fueRo1_gQMt_vJ9FmOac|%shx8?!kEV45piY5A4Sh_OhYu z>~W$ib(~nDhy`#XpOuzmB0pGPTSOcg!E#p93HGVk`BKChs`JN)IBA+C?O@8z!ev0{ij*$F%Bkbab1PBl>dCV3xiG9yl$Zq!Em(9|+SI z<&TLADYflp9otZjg&LXGZf&uw^^BVKd_YdhCxsd9QdB!W}`RL23GC1m%Y5aqUS0gc&Fq(c&9Q|FS4KQDGqR3Q*%QP3 zq5oz8?)2yN=~Rw>8%xw~>|bHT1Ksgoolm6q(>0iUYK7Bu*EKHSV-t#2WT&ML>YQR^4A(Qk4x75xn;UixvT?ExRo@Ga63q4m|ms519Q88crFP{WvxQCHdEz6l$g zmwa=&8lp#ubnagUDs&^Cnc6yRY+*{#Nd2y?X|7S*a?Sndyk3NwJrIvc_#Tk44j_pmjA$@ez_8(?5gi= zhP%;B`HNk~;Pqp1=**7W@L`lE>0nV(R8C(csC&3!hKGu^{?Osj4L1MexV z`3xDz{l9`mAE@#aB$G;kc4!Vgr!iWDNNlzw+PQ-&bv@b86Xh1Vz8kO{#mG}0ff9eH z!;TQC)S)mc(d0V~Veg+Z+xG)Z!Upi?H1X1Tr5&~APiPDpgGI6ELSvP8aGs0F1?s{K zzv7v7^sb)5rQsx5-bK2l&Z8b|1!C7n13Zpy-!*y`9JE%04;7WIT$6AmA4s*EGqe%c zxj$c_0xb=;9772A2x1|}{Y zV0z#+RL5^zL)|mPS!%oSJa!RTz0!TI^RLO2?vt&T<4(As{or?3$cftUYnCwbfrQ!7 z9qmWE=q6`8OJ9!))7TmG8w8)GsfTchKFb-$%0E6A1*?GmO>QvPY7fxnIngmz6h-M~ zn#9`Q0Ut8MF_HBij$cZ0o_WvY_ZoC4FufGli~M4S_R-iSFB`|qB%Tm)$I4=-v$}YO z`7bmqYr3a{Y;6a72C}0)6ldrl7zhv9ft6AMZQchm${;ldpv;y*^5iV6Zqb2 zFdvA8-UBg?CaOX_?&6VIcY@&kp&;bIq)kO$o-tEvK?Oe7XM z0=weUPVv{IpvrJqode*B#c#I|%sB}m14;mPqd!QO_VB3_2;u!PC%Teh_1UFb64d*lbk#@@+7Ly@!LqIc>vFPMkI*-o z3>^Fa<2SVvchNF58^y@WRv4LN8g&=PVpP5Vh@|>b{5FQ7Alf5Gpl{3~|IoeO7i>91 z75F>6YfrSn&0sC^ky&ge^BPOmb6=T3e6mLQ3g#qH4;stBjjk{)@k$`<$`2(Q+0$Ft zmA+tFGM<@6^im6-x<_P5XUG;~#CMqCR|3yZB8r{hZ8R160^oB?Wi~2MU!PiPaArT% z=gZ0mcJ_uuP=(+Lzf%cKh3Bd?CgHNx!;qnU%yi3tl$+#9HyYj7Ry=)d@=zjvl{ zk2b2)pHRY#v&MM~1a@YUTFs!c)-rGQfOze!IBp%lhY)XHba&%(-yb7x5CadP)_jzT@NTNf-}FfE;fo%Kx9}}P(?)ZDCW420Et*q1PcPduy=8>a zg!;}kB8uc_R2G0UzQ;xaAFF9M$Y|1Nm%#I9L^U3#KflEsvae%u65X$d$i;GkD$_uW zIR3X1yTA@dTbSWsBFwXTdormL;7l@YKRirvxZGK|R9(R3Th&hUSRuBV%qKjqXH-YR z496%{iS9ltt-=4PN=bI8Q?Bgj0cJDv>w&QjzrmNr3fD_k(`cm)oc{Z6Epp ziz-_sdY@5Y%8yz&5v;-7?SbSwH;6Qfpra0ft)){@hKl{>DUc+MJ=1z*rMF`(-7aot zL1XgTZi7c|HP0#gsFmAJZucfa@KthO(j6bGv=tHD8!CJ5c(e__(AN5Ev*G#%P-$%f z+mJ*qGbWg;tRy(i{jx@D6O7LCJool;syV5s17?ID_f=675$F77H=io*vKNnM?Re3j zq@|V`PF=V%kMmLWhyiE*Gfwk(5K%`<)*Ul}L65Jj_b24t-N|%*qZzwQ#Fdyn)p4R2 z(efsky)neYU8(zA0e_Of@-?CceNEH@bFSjpnVz~sCNi?a#3aw@bs7f`ke*8S5V~~7 z!3O_CeRqc4xh?9%ue?iuNQqv~cMKyM$u8rGrk=nLeAln>zVnD@7NC^K&AjzYreRH| z9^nUHl!v&*joQRlugQpyUkcRbgW;2}5~t;-%Vs~{_cGO!r^;Sq4AIC_vfyZ?3taAL zDi=%vK)ce4iT1BSgLqsqd$2ZBp=Q~tTmdcbfjNU=aj$^1ui#hT;bSnL>#!c|It*r} zBIaqxT$r5ny@u-<(HXu(OY;cDT{pC*7pdOf!;djeuML}5m0rAUT5h8ox{izb5_}$7 z8%vaOe1?bUY@5;>IEN1U%W7}3>_J?Cyz~$ib9XZ?qx{#{Q^MTqU~kUSaaoG4iHg=% zeZ1^rPM|CBsASGD-Co7Dy*PV(g9C3wzkh3^w)ks!%_P=T<00$)J&aserr7i_ir^Fe zLS&-r`JvcoHgx4+dcsV)oLU)mU_f@uNqS~nhOXn8Gld=~KP{!P0FV8bME;M#$<<5% zpJTj*{dCYr<2J_2!#E@)qB}kZx;0h(PUlT(6kLt$>+X;8xVe!?RfRvS26H~0j*Q_j zdtGqm7)5PeQzIJFCJn^+yF~ z7|or7h;Y&{A!0aPw+^C|g)pnv!2}CT2muYw>MtB~iFwwOE8PMy@aF>=3X{2<0C{ff z|J)B`A!`F3YUR!KAjV^`r3k;&1frK~L}J6`DtHbn3Ort1`Qn!|Cl{*&AI9VWcJ`Fkxw5i7{#3PO zPjwjC=p3ah3aBKCNhV?_@u;U(py<9Yp5fVh+!`x~nvboKAp1FVBUgxZd$Q`E;zL!0 zxf{P(L0idsp7BkS@T*J_iNKR3)J>|OElH^kK~Gy5=JT3ph9}!SQJP~Eam8@zP&)dJ zjhwRt$WxzefG8y&F~&(EjhaL_so?A83PFT3k1Een?vrxr476u%wIbD@ylOjM zc?eMzA~2Cj0wDMAwe$PkW-2n(8s4 z%|g8UEMX8A4FO52p{+dzy5e$8SI!i=Z|cJ&ze2&%kP6oWa^znyy`9MuUvNJhA_A$z zwbD8BUeqt&h*ho<#hoQ?iK5OzZzC-CZJ3!F#4QTdTY9h1wA_cAI?lc@43++V92+vy ztC$0Y!v*&^V->sbLw&Zh8GVRpl>TU7#uI;j2V3UCW?R6V;px& z8AX&w?w+VV{-bV?7w4k#Mph<9+_iMQviu{*G9h3m>XE}nG3tX?Ip$M$gNiHFc=EWJ zxQF%j4g-;ESS8p~b6eNBS65pFMFZ=ky#XKiqxL{o7P`*6;CFppuSvH>xKUdx%e;Ue z%Gj8qZKvZxE9^;~(5uMCB zqmtScUMrRR9~{DQ^c%Hg9{OIv7A1>O6sO|9sIWAqPYsi~iH$nTTiSl35;5&veV53E z|3+>jIKjbb9eww2>C=pKe>T>G6d}Ur{1#6a{9)?&>xoG&fGrAjBHx6O7?_qo&|@V! zBA;J(HR?Y5K#99_CYB{{iUC_*>-WhV6PY>GAbr2s%#<{rf1M4UOa?>F@xMihc~+ve zyTTdDgIdF=Umpc$vT3`?lCpv@GwG*3OLUZ(=O<9-iqH$GZP5;uK#^M=M)#MyFjJy3 z!SlCGs22Gq!G%+g7UXyLi1F6oH1d|~Hkf{g7$Y}zqOsD?*o^;SPGh*)+T291U@x;V zvFQi%F%C_6>@D7co-cOmfHdBXo)&@Eyw5z@0=|2iddhqKJoD}K{s%mPo(us$J?%YJ z0#bOJcw)U@>@#+xlx8NYILgljumPjQKbV2p#QEzIeATY1H9`1+MyMHMm6spQe)djw ztAA)+N}!C&LJ#?Il=f|@+O(xk9ZOWE!z6rynG7Zm`V5QmQz=X4@(zFdO++7@(#VU#US?7n#Vr2f(=Ir9QG3JSh&Il!n1r!x@Hg zw$^YXmtk_ofiR!xQ=7wDZU@4YM+twHYGzJqI_;@ReWc<%64Y9wKBV7gC)Fs679&D z4sTqY8tyxE`}e8)w7{idH@f{&Y6W=UrQ~JNL}){)JMZQ??<31FxB?I1y~$;%5IKlm zHW0BaB!{X@%#g~cB?jQ~w2NNRZ|JpZ&_h&@&YI4+&5v}AW%lWI@TY+|M>f=in(TN~ zDDM+0gCVF8Gs=4G4x>=v{-*!sFB1Vm#Um8CiNy}|SEG&F`YYMh`c6;ET3HqK^+oxa zp4_ZvPPY^E?xpP}k{p1)ho5|nw&xwyq8i3*c~E{g_jvMIz3fN+744oL85kz#S}nbe znN<4@C%}bTsIif_=slAz^Jy*U0D6U{8N=`ejT24W$H{X)!BQ5L`M6@K#1F9gqo_=W z{7_kkOec>_q;>T%*xbXn>F)eO<>(4EwI5_#mEe1WomI2|1MUY;r0-zP6VSk?mz%>% zKTmGr2k$XAAtzc2hCD~*QUaw)I5Ej{KC_?p0Bye)7G)va$a(!Nk;zrMoQm^&F!;d! zOlS6ObrUuV&0>eJB`q)xOM=iV63?5jA3uv1(_KcjQ(MYMBL_o`L7 ze{!qssfN5`>S9~&DBrxN)Hp}E#2@_Xk1`9m0SHqRrQ$if<=e3PI9&D6)P=xdofeOb zNoIZXft}gY%T8>?S)o=>yP2KV9_86(x3+cvE}lpB{eU_+DBKVHjc-Dop#1)-w|U@j z?*`9T?+8y8PaaPwD74XY(H>-9^MCL0dOCW$+5g$)$Srq@EiQjUqaWd^(S@%0$tWAA z!S)}`wf2fS>2~&djeR^4tY}JB5~bYJ zd!tj$O@CQhDyNI62YuHk65AwaB^DxLIZI@+8EzyQvlbS!HuJG|#=(%JlwDCQq($ej ziIr6q9`zbo%uJv55}Y{=w#)%lDxj=8OvR@rS@1VSVWsD1Ek}q2XzhNB*=iY(VWFA} zre!Nw@eNL{Qi*6GM$bn+G7!E+Vb^A^1^G)T_2sR+@)wG)vEZtc zDIOP;1w1}M22_qul9){Dy0VXW!wEOlhx5dP5A(>0a)Y9kh`t7Lx33`)M{DOENXO`GcQs=v z-3OK6s1NZOo~Xy+|IeU*9)f3JBsq0?YGjX?&Ar-vo%ngWJIwe>ZbfYuN6;C@Lw5W_ zq8g0TbMp|sxZ|bbZiMdvY`8hrIxe0O)1GFwNQfLoO(BlRsteKDH_Z~CsZahg3wqzv zZ4%+X%|7CZ3#cVpnq55?@XTDsT!yC0AKV4$|#h^h20B9kPL$t z%0z`dn2hx`k@I~qnVL;$yajd|w%kDUm0vc1&G{s}&W2h}kgXXLZx4V2FZC1jDYr1D zvv;25p8QLX;If1mV-^|EG0;F|)$hU2|Aih#r11dc5o9-W`TVD-@?235f+3I91JsMN z5tA(9cnjTr3LcLqyYV4ODKe%xu(*%NeJT?Lg^)vC(0hO(he4Q%uq!iojV~MW*ABtA zK)*52ATdj zy*C2}dRKb%z{%b%o^$@kJpP`F-sHUIsb?LilGdBav%sF^t?s#RN0J%!_T&&JM38v} z?;yNtUEhhOveWM|!}9MC0GUX%2 ztwbz2h&LiZs@AOT$GEsW=2()djju1SCTlG}HLt^*p%;}V7i(}o-Tg~ocL%A_F#l7C zPo5=o+?=QX=meUaIx3!+Fx5^}VL4FO$5QFe!#idqx`+kY#*=aU|1lRaRZ94W$*>SZ z_&i>6gnI08)7Yc?nvYRj_|q45fp`C+j3p*HM~qRDU-KjSu02WyqcY!gCo4BGs<};Y zK4I`JZT|qlkpNR3V-Z7yUrZTno6u#I4e4Yc+ zwjkQ+0Wz-@i>PQHLG@aYGjCSrqLeGEguv9brtVyryfu)uT!#B9g*fd@>#l+qZm_!_ z$H7#HLR>Xb-g=0#dW&i3u^)@CD&LCz`X$<@_v%5oyLGNiD07yh60c`uRnil^ta4Rj z^3_0UC_Ri@_#wQ(S?Cd+K^FBy;yNv@oIpkL5^isn(ZzgXUe*7}e`bDtmuza@!>8qs ztiepOW_EJwh~vH8t>gAq|8wRE=7f$l=1ATC=2%bWcZ5hF8GXUu(DLj6ACjW&FGj?* zk(?q05m!b0*FCtQh#tiB-0l9vUvQglYuRr`F#`Pv)2tR>j!N-pCKjxyMp@K zJED#jI9P6g4=-pWC!XGDzUOy(CTlPOBsH8&K3I_jbp7~rEJ5JO0`iw=*1NB}u)pID z+>3A0tItX7hZ;$ClbWn&2|YyDh*E0tni+hfv!F*^P+}#T#{0a!7&+7|aw{K;8%Sib zoDKpXyHbP4tH_?tgFF@Z>u~&bj?;1ABug5v6i4%4ow=X&g+?~yyNjCB?R%WsTyrtf zbqKeIq^=S0^Bw32h&Rr={P2y>r+BFXTvPpxF0vLnmXfB+o57pTbHZQoR`o0(&l$p34NL9!!*XF+;0(ZvqQurbth}E3ink#SqS#^ z0IcI{P;oDM`}Fi9oDfe@-AtfLn~WOYd3+w8@QXiVC4OTbO%K*cf7~Ui!sLN< z*(7RkPM_mh1l(*xIM0j3Grr7AOVF>DfEhk1(y<<|ki~q~V?nUmyrwc}ttcbrVjlLJ-EM07H3;g)|8|5T)Q`Ib)JNOJ}Z^U(wr=ny&DZZe?TRJ%IC;!Xhv zV#xlwqQ9+$?%hR4!#)sfC;0hCIRvhLq*G}=c=(5C?UvHq=ng;gj+`hKrlm6Voj+8Z zr=z>eCnhI!8n#yZlE_k;N?Lm`oky zKHYzh^*ikHzxDG}ng61|@1b>pDea4Lbp^S=L;VEXQ5pv0jxJY_joKsQ@Q-*Of*8i;`g>4nkR!V^n6# zxSG1B8=vuHAS)%tIF+Dx^(jnTyzgRWqCI9hb64W}PHKK))W*s!cM?&^*#O*I4vTvU zp2Xm|j8z}Qs&{}bQ<%Vbimsne>K>vOH^|c(aDAlOho z?P|viqL-y;9ru&(Xv93r9VI#EICL!M&?7bCIGn3-0YoazJINX1 zO!@l1sw#QNvX>}r;qGP=$<%{&If{<1H*rfGuE-iniLLr4dCcraCrjmkP0TcG>sh8J zwdPp)@s6);rDu*|v^~_SWG}KVT7|%d>flCoyMbq|UEU72xyo?5RqRJrHM*2@+O0ho z>>hRo?>{@lZr~s4DermZ{ceA;Q+k(^PYw1Iv?tpOSxcR*VdStYL@~P5H{+gqnCj{| ztf&cwbTRS%9naNFIM80*U3BV3a4{ zh(Px7`cAMSXW$3C#x<|^KIK5 zfKwokN8qvc&DiT$L1j6Hip)>_3OMiv_FyXS{DR$F$76FKQO7ite(`id%~fycRUEU~ z6E?$LeAKU!H6+6Iw~VGTE3*qeKzW@#aUfrh0y&Oe8;b~MIef@Nu&5%);Bhu3&Uwr; zKJ7|+Egf3@uXLe?;B@NiNA@{v2a|E!A)iv=V^<|`s-Zj|r(Y-E`AUTF1CHf5TDynj zGJn*ayfOp;6e+trasDR){@H8% zr~3yP11yg<-qjvH>6)5I>T<7{$bM**!e^z3_0imE^|5|i*{KJeFi%(`toi0MtB#e( zinf+m#q3J4{U_- z(lrh@r-dk0dZ4u^%GxAgkG{aSw!Lzh_>6rL-n-d08X zeFwH@3t34{dPcgVOum9sVrnK<+#-sYM815N7@z?$iN^lE9oBRs*y2!I!X&#;>n*3M z`-j^1KQu7Qhz!H@BTT*DtM7v^eNM!Z5p`=gaYa-5ed?nD{_wx!B&tFcWhfpLjbt#7 z|B23;ftskc`ztx)X!yxX^gSgP$EknKq5gFpY-va(C=XX)2MAUN9sVPth;Zs+ZBf(J z2Vs6Go8ie*s>Q@^sztZJ83U9U2TG(PL&>3LrIYBWa+AHLgR%(3$jmd{eSJ54((Kc=1>I5vHZ%4EGyUJ7;iCkVmtbuQ)oN$ zNhjLwjTX8G=1Am%ttf9!!_#;uTuL-q@*DaHlAud}PL2B~`GOnolr~yJ91Fv6P+840 zUs&n?U{QP%nV+%CJ7Hip5Hoy7r7w7Vjn_QTZ;&IrWA{%*#&H@>^);(FC(q9W2d+}D zX$Z@)1P=BZXNXsQZU!sp;_2_$OtvzR*y$1o;EO93aUP%2J_m>q2HHH~Y=gNDbBQQo zIA>$<#HUE{akCY`itUbGT-ymSLN`(8{!veZHmNwnP~w&gpioL)g(|no;5&li2VqG0&R?WoqlOd01Ako|{eR46I;Y z#|bMyK38|^^cX3J-4%td%)`gbaz$!$jU=wU+B)jph3IpChMM}T(OFqWcD7A%!nKAg z{_Lt-VE?+at129$*iSpNBO3{kzs+mH*ekoi9wfyDB`xc81?o71jHaTRU3^2)s-TaJ zP$t3w-T-;Fvl1(U2Ib*K{-NQ&M&)HWQ`^F2TINc`%MvW9s9hllcJJ^B=4Q4AhH zXJx^o_JVah4hP!;HNy0(*&n9?&Z^hj^wPI=6@5WHNzsGnv{IGKkK^ zMIYcuUy_xa)luRp?a*4da9~*v|NIChrwIAfx&IxjlaF5pW1FD%>C1|KAr>%wER^_v zF8x^}_;weGl=6y`pwwO2>xrW3o*NF6%pn|IayE~e=#f!>f2N*a) z+(7x?A55-@dVeg+A0Hy+g~6TxZr?&L*G8E{P8!HFgYh02LFPCIh9sPB$D^>UAi1@E=uHOnv&rFJ&`e?js{M^BGBEO~1lO4@mBWBAgVz$)P z8Sp&iP_yh1h4Bb#fp

`tDJ#(j}sTz+IU*_LWYqpIT-q@W0w3(#daDL(f%D8BrUZ z!~j=a_Z23ul>oZ%b10S z+b$z19Us~F-*T+?9jw;h@TfkI>ou&+*JLR9$Tc>D4|iCD3afY$e9leKsS4P#n1~{p z)%pPhmd;v4lX-VvpK^NMaTFaum&tv8fK;pbY`zT014FWsD;x=8ILUMtCumn@<3bb7fBB-3 zsYD=0z_h$@wiEe;$H{#(-fI%raweho`3S$D!FL(TdA9R>MY5rNe6~CP$J1Gd#jSN+ zyzcIkWRjdD<3Qc0ySux)%dNXBb$3eLsRC7M)ZN`(Z(YA%xbO4*lZR&*G9i<*&)RFR zy*9kDgX2dz4ZSdhP|ekY;qL+Wl1))q%T<{^e++&pH+!D)npdPcY2kC~t3~)IFo!WoBFi8iOb%T5clqElQ5v$vmaB)~Dls5T&lc<$f8M zb}G)0Q$$W;$ZF9Yy-FjLz#f!4SC|N&#uy0pXu|WkS7-9MGM!w^&Mc--r`XBf`UxJg zCA?*6;>k@S!#}Jh3G983ewlX0VMXTk)p)pdMAbGw9y7dPEw`h}^M&0u#%O_(JdDV& z1c$?a%)&4RCCw|aH>ap^rvYi)R7wy5ob&>}H*TWE9ZFxrd{n+p5KgF-Q9I6j@*i{^ zI;_+5CY+^aS_y8avy#u&XraMdVsug?c%>)V%^9lUDX4pHp+D$5)!VP^_3gN-Pjp!p z6%O;ZyOKB7yVTR&H_~6xlfw7LHwd+cowZdoubclJuj8`Sfa>!rSU#__ILIW4wvSJL ztd60_yd`YOJ~*Gxpq)R;9`4dFWe%Cq8s&{%mHLj2-&QD}Kp^W;krkoJe2sUyU^C7V zL;4!GKse`!jdo5q1GcRex#TYp&QP?tS75HnD9hh z?YERjtbG(| zVnl$gRDbTXit`a?=7D@3Q3*;*B)LmJ@*7mR-{~xfAZk5g{VJ$xhjQji#E`7~JcoO0 z;}8dNU=}|gC3@IOw%IVQHoIc;wEd`(A0-aFBD0$S@;Ry>kI%CNk#S37{3*H+GX*%?K)^%};eFIQs!kAARO95w65 zTuEk-%q97UX5l*6l0EJYdrb`0|2C+1h8h33wz_xF6<*zygwC~V#(gFQj4(IwtZu?7 z*J6hnpgv(ou0g$jB^{1s(fd4RlK37`M1D%9S(wS2mzkb(3Dx3h)OWv_vaw35iC1KE zw20kMMXzTkU7$~4fAoUG`?G_01WOb|XZ4PC(E%hUZPX@7n#d-DpPAgBc-VL zOs0Cc$ed>Uw8C63JYL^T|52l%YnHnkos!Ee4Hw^}RyFY6JRGIEpkX+KrsW%}av{|m zK9@6(rTYBQi17XOHxB+35*G3zBs8R8u2;F9Yt77n^@z1_+x+i%lE~K4%nJwm9i}%s z=;Abe7Q668Z-)+~47g|uRqx}(w4i;*qXkMBeL3LiUzXjwTvE9^+OV`^$W=v3qH z;T8HRYB>9G2eQM0?!+fy26dm4)WJs+VJZ`C`jMLyz}x2p*-a+uCEY@CcTn#I8}OJ(epHjCPt5N^ z2k3RYvP;Wl^+70Ju2OHBsMHmAMP<3CK37C2{X{TGvW9Wo=mWw`iKece*@>BbbIl<7 zV(Y*IeixyZ!###?{I)SnegY%3hc);cF6bGVPBkf?T1c#A*Qkk4!h1Rit}xp)JD$ls z)n0gq+Ah|+=v;0jZmO5XIP)?QHJQ1T|Lb9V#CXc8Oou*XCMwozpqC>2ybw0#9?>-| zD{&lg<}B;FH2eEXM+aVC&U-Ghx_zwZ)#OtTI6eijOh>181unM`XPC;`x2ws1GN2Je zfa7|1bT1pgJU_@#2C&jk5`Qwp#~C{&4C1qPu=e+IS37gJca!7D#Em(yF?We4b-^wB z<1?CxRGH849RvC^%&PJ$xc5qHN|H~xq z$uUrw%y~v~J^S?$N-;2eG36C#K@uy?7@Qu1+zma?J@35T{0o^>yTIeajVHMwa0pLo zJZ3J;E_^|gXobWDsfgAS$E~cm6-|*lYVX7`aP2YUEj7GpbaTuTy{RXzrC+Qvnu?#W z4b90uJ}d30m~5ju(3OlP3mo8cFnkws|0iVGSC!!~hsm6i*_+-d6F@A9h%Kc-@^kn( z6V($kBe2IHdfAvHW$ea*XE^(5H={rGi(bZPw4IZUNMnVwx^W8K>@N7p0ch+G@K>Q^ zB|XuWJfb?2#E7F;s2WU#3Fq*XJOw_H93d@rqMY>HJp%){QI+f`3%f)Wb*nzw2v(ZG zZBAexUud+1tLbkJ@OuN*18<5{Df+OO(|;w9D###nKWbj4CT`&yUkDe?&p2fbux6sN zsBbMMgK(n4Jt8)n8XR9Ivys1RU{}!N+@o@y3%=;@6r3Q}3e7Y_t!?yUPNWafKAt1% zk<1sU)0fd}P=LtrOY>3SQcUlN!2HcQgW*_Sh<2J4;{ zE$HREV~o`cTg$BV)?{}ZZ$^E&xyUNOx4!_V=x$7ydd!OZt9xOG3RA<9iHd!2pjv>U zzO7mkR5ML2g=(~o_MMqK?bKSLqtrvKP5ypK&1}@5pZwjP5tUJ@%Z6H(uR z1pXz$EGBl{#LIL9`rkpIzU{=Bq^#(nFukY9S)Nb{`UhTRKdTFQvQs0bIMzJDZQ<=)m>Tbwh*~2*- zZg~w|nsZUt$KVYAN|}zn_tXw|_ZqlC8Kui zfT=&pb-vNhP}L0~w)|1LqHDTDFYRHnkG0^3bIcdMX#Nhm6Wk=^LV;BUzl5gtgnM&& zGtGa}71_^l*&z;d3_QlQ_VtVl=0!w$0rhcZl)E zGS^)96Frrc#C4utfZsUY*vAwws4t1D4d}JJ#i}`JG$I4NVHQFI*4fMh;tHqMGmOY^ z1(Z@6y-HWsb&OIEZfZJMr3A6%5&Zl?r3U$q15W-t-}q@9u|_)=(%Uka8uxYh*!&>I zY*fj6gV0uh1G~bL3wYo!DB|*w`-~?8>WyA5nX@Y2W+GnshYaf@JpyT!F~pV>%4D*V z3d$zkXMCau?Iw=#-9RimSXs84P%C!iTaLZdeSSE`ff$>S9$S&W|AO)gr{lF&IEZ_jAv&OS0E zQ1yIsA;vAKc^z7O@*{SnW4VlDjChAW3&^pBtbDZm>D2XQj1D%td zt#MROa+$};)4S07mX@yMF0lGVs4s-$<1t-X!IODdnM&q&L~TTsE}hCyux+YpQ6Q+703UV1Y-nz8hH zZ!`zvJte!k;k>ue+)l=n*i1r)Q8axDUtu8AgRs|-L4?3xwcrfT=qDP6CZ`gg=%sUH z9rf{?+706l9p#HT!!NBfZhIT`Jl5;~9ZTapKg^87_n@s&M9qrpOAHKidMg$_R+!Du z*2Yn3ssMht3CmOwj1dJBdr_GILlUi&LJPN^2vd#-c^~~}BVzJfW2y7EVd7!37!GOz zeFS&Oq)gbV>`sG-v<1yub$k-;Qz06x?87ys0@;D>om+)$E0SEPU_2Myf?j~NI4UGk z!t@6s6N=L}q5!xqopA?;#tve%)P|0~e7I8Y7gNbQ3L7pu0dpIZ@pmd@`{a=CGsnvq3yftMu^tV*WZIMLm!OfphhJ`Hr@+y! zoic*z;dOSCNZb(yx~F-smhDnW4w z3SCjt@Si-w{LpavS(qCL5_qf^chmJ1pnSm#Jx_AaInW~_|;UR7xk=MB9-5LQ1GBE6N60#)zv2Abl)|u=E2uhMVI?}*>xU96?D|VmGMcSK zhG%fLd5AeH;Y%-)Ya|6Z%py z29?VlqDL?>yg&G4C$Z&yJf6uwRGGusP7v9g@wJ`~ppli1X`FK-z7{vg%Tyx6D6+FH zxNu)|u7>Z5a;_wg%R!_X&UvCiS?P#T!(rg<7?Ly|&wS(l#1fG%f<;xL*%A;@H0b4m za}x}FNn+O(V%J-}fL58x<}>!|3g!>T0D3BYa4BBnyqd%)W;Mr+Kg3!^Klmdv6g|0M zx^yn8FxRQY^rzRANqugz#de2?LocW$xvd6 z^M*5mzv@JG-GQfi6_}*0vk_5$rL!xqEMYIO6>%FuQ-z$HQQG_>AE|~aV=2c=@U(r6 zgY@RFVUofS{kkYmFHANjM4T0=$*M9NDd84Qid6U?mecLY#icYS^WW=gbwpRGwN{t; z1y%JKVw!wbzelur%6>N$Ek|u54YlU$Mx@#d{Vly%)S4&4-rR!6P{{&%;w^ZNt7?Z| zN(@Svil|1%(f4tIy{{_TnD=zQJ;cu~hdCB+lPB~pDQH?!!9_$+8=4EFI}9fIDZP`0 zjN4>wSBYi?)Qohi)d5Y_Gs9q1dhl}t^cbI=1E>mJQg2eF`tNv4uWfx{65kMZ!!B?* z&yAt_4)Zcy|FKj?TEQ=!Ld{Yfe*1#*gS+&DsL~t8dkaV-N@)QHwMoe*s-oX(Z(Ibi z)TdUm*eRR2%-YU@#xrupm*hk@$(XvsnHK|RZbg4si!QS$?#~KpKzZTzOTxhXR(?}A zj8mfNg-^(nRLba~4Av9lj})%|ONVkB{XLrPZup=ULeYIdbXGp;(R4#E)f1X4!3L9z zGIFe15PoBgY8u;!GeJhET!h}V9}*M!P=JpBnUrO&%?z^UyV^=)gfts|V;Pg#n~Diq z2Sc^WvZ7qLy3JrJm#Ns3TYt(u^;^ON5)L)$6zLqAwa}RgyUP?SnWSdDN>SyC1pNx+ z4B6${=6U8VqmMTWyY`v?cq(~Dp^9u_?w9lH2dG|cXI4UE$0DA}7<3)a_3O$qoY0s* zh12smR@@8L?{RuuMmtsze+Eex$W1>}nXt@P_{fBrJ@l)_3G|KDNC*3R7aTH=`esF*>L)~u45%R5!I!3E z-z>pSJq>r_LR25-I0LM;+1#oAR5Rz`qc|o$s?_A??(E|y(AW&*KOD;XU8FYyw=d$e z-Vv9YqB6b1EBC1c4a9Zxv0lkofLB#(QA7Hq`b2YPt%isLQWCu;epa8fW%x36gaceC z&7}vbkE~ED86Zp4n{vrt(dsy4w|*41c?ua*LirwDQElZx`et!n_K07ojthz8sFX(t zr_q^C9GyI(swlvY-vrEX0|sCe(Xt3RNGj&1972ndi)>=7a}8+Z37LCYT--GjZg$t< z72?tmn58CYY4#CUQ=v~=>?})Owg<)|9g*aXQO*?LpBAQ59Zn_aa=efC0C?%0_&IDA z3;uVEQ#a#D@=H6%JRPs-Xl-@v7U!+^t_8ThBsQz#KX97Z+ZFy{7a2-f_~)N6BroBM z8gSpbvkR;PlN3e45->J__ez)^@}MjDyOxB5x{iwY3S76{(X)+ja4z5eYUM3RvLReY za?qwgeb)uu&LSAF&%~||c;|+qE8YulJdNs%VT@(w*Ku)0%A^e!5%NVXFL?Eo?lG^* z@99=fA>BvsV|Ug((X&zKO({0eoAXoOA$6ixW;BldADOGz4?OdTE}paSu{E@R#SC(g znPM;Xl<#_RHH!IPbMOK>W%`{Hcp5h_2ko3lCP%9q#R)l~eoWK_P31B+h2}5pEf`s@ zNU7KoFQ_k@x$jYhj3FxqLeL9hw*w zHM;|4+BxSQy*+nqADPG(>PkyNZZOu46`-AA;GY8^mYT$k;hcFrXYld!7|y?)xZ&k& zVPr|$`K(Grn@CW~1+M%j(P;&*93xVd0@XC6<8r&6+cA?|p$Ba37UEcHHG<=7GNZ}- zb}!kJi#%=x2+BrWM~GU@9AlkpVTw+3g$Gb6$LY_Qy!k?ZOSYSk8SC4rzh`2aTRM?b z@=S1l(|)8Wn`LxO`4SHSvM!PapEeDa4r%`dAdLKlLH7uMR2>W2nO} zAui1(I^9u*z}BYczbXhHcgo!)KW8C1`4#zbjJK1qL<+--Wcr0hb%Vj20HX0fESj^hX%)5F>glb!)*JN$>W=1k z02YABi6h^!T`S9hRgw}94f?q4^BCZeEqr(7(bCM~d#R45WkMR*_WX2$U^BUnv9 zC8Y~}PnW=c8^m1VNeFRijPjbep;14YPvoD6{%|$8s4C3NVPbW0_Rg9{Bl6t8s6K{K zRXHbThk;MQDl*KRbk<)tVp)|TB9~)2dCwg5RA+c%KT1Y^@z8y0j+p{QFJ4z#bA-t3b-cdz3@?4Pp%Y-+rQ9nh=x=d?ud!aU8 zqoLYuu~}IR>TVxM6m&b#JakIITKR|QnV14UmA>Pp)*@@HJ5apJFD@v$)i317MT_6e@ zv8+ddf1LYVFdj?zc;`Z@+ZRyT4dN5S(4xe0Z*^i$N5>U*@7}nqtRQCEEJ_$X-3Q_$ z#s;EL3^8XAy#_ltTRP6UAYKC#MiqVspLm_)Os>|hWzU02OG0({y{<~hw7=TM@S3iw%qH@eC10+?e3@JDt`8L#eIgBT35Z~_Mm1C-yPZMoZrAvZ zcEO6&<`_fmBZ{m!1-K$9O6bnSlNGN%b4-=3OZ{HLqBB{^~|gxj4S>mNE&iySkuUmDzLC@B$o3zdun? zzNGCmj>*56X1QNhVRClJdy*gB)R8!{)?V`QyHkHRdinByE}=dr~@%-G)&F|a^bQ@ z0y&*F0B6E&+85UNEKTOOTlMqC5fmfQVkC8tL)4e|vbQ$I@pU7W&=by0?8qNk#ob{Y z53}2*Qcuy9^MDMv8`;4gvW5b14iYl~^Nakf?c>I4W<3|NB=&<0_}fedSYZTw$7th|5e=U4EZ-p}d;l{P<2_4QQ!n&1a&r(`Vl5xN z38i`Rn!>qdvf5a&%);nvZ6&I-vC6^yNLF8bcSl)Y<;Tpm2_wg`9VQdOb<8C`Ttu}} zkUVG%*-aF8CMi*(0|>#67{8s7#Fq2mlcB_!SlE?{FyUiC5w@NrDGW&@apNBRMltm= zaW5nBBobBaIr_4jf`tyFg1gT7i}UkrP>tQ6XxEM>a_9F`Z;B(PEh5To=bV|;ZDeOR zh!|}hyZMerptajZT*?MZGnowQAn#Af`Csr^Pw0lIMy=`zDCL%(5}xX(UPQ{Qp@^6A zYi=gX3mDGMQa&xe7z#i3iA-l5(Y_D}Ap{kAENeVHtlbiL%SGh=hv~05z|W=7W^8Bw z-AygVN2cF_=+TY+aVO7M3<}rk`S7UMM|riIfcbI{1No#HIbx1lUMbI;Z9V+uIex zMbcDQ%1&~7rgROH4)b$Dxvjp62^S~yJ)(eoL*FWz$yvo-k%N_(kobI=>iGtE?-{T+ z;T-EyIhur{-72EDn|`7`@Fb<-NnR?&M0@&o{(uVpJDw%qEJ$5uEI6d3vnExZakyqB zL^Y=qXYM*j;crt>jYZQwn95f!)1!T+kJy2OXnS~-1Zbxl;$yj;=yVKcylG4U(9lDk z#%<|q{3MH`@HW4x1dV408BPWlWh7D`Q4y=IdhwZAMg?jB%A6PImU9y)@6$DJ`)fJT zvQDAC(;Vbbof*wpa8oI%^%Z6C7VBpumE7o5JA<5eQ+K$?UfdckG?M)~FSSt@^D-Rb zgJUY|*LGn&4118D2s4@A+Q*!{QWl4`cg7L*yXa?T1oyV)34BTX4{tL$LKjH+sh&rONyp{^`ex}Y@g-ZWlx|l6-k5!pf94ABhM4b8MwCl5pQ2h<&N=);TV(5*G_0n0jtT>31RcWJ! z{tC|4&idBli<{4yU?rjhf3-D8yfB-<=oT;d`IB(GVEF84H7 zI1DcLq+<+qnNyA#to<8~9q_%!L4CF_Za&^Q7W}i92zH+wr$1QnU_4fdq-y*SW}-Q0 zZK2-O5ld|EO)OmxN|E`Lb$kmu_!Ku0W)mz>6q(LUYV;YI=lDxcCOs!t)ur@WVi^1E zT48qBvYH=mvyc`HQ&C9EDU!pS2$=a9}*uOL)7G=2jATi(N12el{dOGJ#!ci%(>_&-NejU zh3ZdzJlY?jhfHr|K2^3hx4x|+pIZ@y4p!DgIC)zAmM-$+(& z4r*A2(O-_%Pcf0<6@8{nq}R;8OeHtwIrI=`j*1#`ULr|jJbZYP<-;hZ9?IGD%hWY~ z;lEpqY%7yd5LfvsqMB5VyO&e$0D9>I|5JhNYz^q;g|toU#pLdjbh$2;nrhYgZC8#n zzz_6lQ9&L9`*Z??FoFK)eK4`3`MIvs*4wM-xR#-=i6rKD*`0^}|1koUppFV<2K;tg zc$@m@if@r=PBsp3wyG$^dNFbJ5V@5LwQW8S5egut7<@-z_!eam&O!7xmRI{xSNh@H zj84^_x}L-Q=q$*zi-Xi>o0<33oXkN!p#Bx1OqF_TSe9ahSf8!!#H()RN<5m*z%`BK z`<*QxM2q@B_Up`%CMI_>5~IdnMLjMz|Mh68I_zr~^eVg1!p-36Es1mWYLvGZoXc4k zw!$ij987PKA@x=>i;3|(WIxu_DfVuAN`;@UqT3?9{t-R?M5O?qxB`T^OiWkSYS&>x znrThM6Z&(LiWE{pEd>gzN5pm~6U-Yh={TE~6&-#9t+Mz7GoRIJAm_z<>M)wX$#_SU zMG4TF8IE1VBnQ(|smNqd(+hhhY-&)-19r=u@GbAz_f^(uL#aJFmcQZ$&hE|30#;jB zf4LsJQX?=&cO$cY4PEqoy|016Vx*BLs5EaI@{eYO#4cJ z6LKSTStdJ0H5_Ym>OVyfCS9HrF?2G%6wBj}S)G+}FB}ec+=*IG4zm78bXFTsp~{Xwotuh2PMQU`$_oUp0SrS5R8g_vMMtwt7FH};6* zs)L!iW7HU-C=Hlc`jnOO%dphq^ui3LTQ-Y1iP@JW%tTryvn;xiHfBX;YJ8?5^V!+Z zcx=>Ss$&`E%;ranIZLSs+Bf8l@VB+NKfyS6Eg|0-EpJ9`B~kfGYxE&AIx3DyeYENH zCTQAxqQoL*P3~nK9W`QP7nR0J_!M<9>f;aa94*HZx;sBet5{Pm5L7_y24P;*Ycmzy zjf&$p9w0AK5xmj6O7+=KQ_9!0mVDNFYPA*NScY)NPLZ<|1ts=IH!_BJl849;4omYJ zrOkiG(as%2&fa)azbA&QHh#d9H#F=y)T4+nb?M01OFvXND!~RsfaUm5H8-x3rCuZt zZQ)#LJ4mRPaFpt$r@|>Ty>6K=%{p2d^RT%ewQ5GI5YEP9%t(A5VvJvEQaoX%8CCEz zuB~S=D_Ef@^epp?yoN5JFu6SY$VBCvp29pwO?jD7Ksrx%$TD=`k68t0;REM@Qm%up zkAVL>gH^VG^KA~Hi6fdUJ_3ctExj-9bGhL(dcvzridX7}qbs}3ik^?IHIzQ9lU5eh z%f41cI>I-Vffe0CjH6}*!$GVrC4Qr}YD+xtz>YZ*ZFz6eQa(h?8A;Y(Ms#!z=WoBD zNsz6UbOcNXMGq!ZsSJO}Tq?&#uCSq_4_YsKH-E0zbIe7h))SuAUf2J^{H!ON`=IBO z0%-rbYh#&GRF1v#hdBWr^{l=?4%7CCduk#&<0|Nt%=_jG{RXjTKW=6Ht>>bQHPNao zKAK&uiF$t%y%If^*XTZYZmc!yD=*;&!$AlaQ2x{g#YKWrPV$K@xC7yi<}kgZh!8z@i!u`4GxW$To=!l?idIqaL8dY9NBF_`YbmHMLp1|8=c`uw- zI9qX8nn~!{_EKeDNc8B#x3gStMO0aecIC2j4WIRjc+!EKXg!Rvt#Szi9c_dwt^z-^ zgFNV*o&ZMTjb2j9sY%4`I+{zAlT2qvO zRBb?(Wa3^j2L0b~_?9s0I|uQB$wdA>4g`Bu*$~gG97WU8Movw|a4fUzTQdcz7I~1) zl+|X&D(N)!*fvs6CgrVX2KOP+QW~d@7nKzcU2n5c zvl!`OK0u8urEvj=#)@Eolgd$ek}!EbnhmcUqMxOfG+O)3EWcfJNJlCK^?lT2O)AuH z)sNJ7#w*i_y{}>8e}YBV^ZiD%g6y99OdO-h;*;Sl)L%^_G-(~2r&F5DdKn=H^a!%H z66n*eQVR>EALx>6~+ik9F7AsZ)+{?11*zzuN9U5Dq=OO!cT^oQ^>!?l5MkGuG*XGT3{8x_N| z`U9N~#mHD~r{6ImEfXg@7-!6;%$5oeyLKo!pFB4; z&ecAxptwVBzSY=>(@sHazVuG5OxDwpNV1Upy&O;MI->W#Xl9zjXw9J-m_yk_zuF%8 zj5dyFQ&+2o|3)}kzPIXD+}P^CoWB7r|I}CFJFx+s(jn=+x=d7Z)FQSFqt1Jad?7nI zML*aNyCQr6c4u&WrFAly%T;)ct-S7&Z>oLKJY>;2Gx4Mx->;C?vw}*2a;h^UV4~P$ zd}3Y<4sf8Kr$$pWrIXE^`a<(x^Ef@%?dWVBMK9P%Shc%GQ#n$dA;O6#Cpf=G_O~To zse6g)?LD}iX`rSBRCDayFgfhXNO~e{+>!(Aao4er_gr(_Cc>;FE?of!+KS(m@FDxj zkCMW%L=m;l@a!ezozv0zA11?k4~AJvWh#zuDhcnLM`m<_SF(bRMuCgA@mU(&%OrkY z%kGd|-2n0_qwb-bdkAsQi?7#vP+WcDNqZ^4v7nS#OOBiRzto!VIX0mpK95fFk^UJx z`AJWRF6f({lL~SHoHyFRj6Fdm)s!lajXDye9Ld97e-+QjgA%ZB z&f^|A`G3a-bSOmNZL$R~-v-V);IRg* z{R?n4W5F%8=s-UXwv>!n)T-vdwcb_w!^9?~hj$!V)fJe}qIg8`*Gd<%s_INJm<985 zg4ai*;TcKYzZkjLwRm?dq5#Mzj0m#_Pwd-pYenf-Jw+A!2f4{sGV4L&6j-#rSw^~v z5>R1YYD)3Iv5x0zDo<=2s;Ut3{e0-nBH(XN^W0XX;yjI*la0N2B+vg@*jg8vk&0_> zdJ%`OOC50^Oqkc0qFxZ!%{us|ZV?G^kxD6Ir7(WmK|ZGS!F~FQwp-Lkw-yDfvzFW> zm$DobxTj1b#54G(kjcgbfsdn@prY9N{bM$7dRpWZjvySe; z-*iB=VxyOvpb?wfe99_&OUxO?incW;_o+TNbX0<=I08HIk+ocjHG7Jk;=Ud&WyF&+ zL~5#<;tL)oZ}bl^As_Tc@=~(wyzpa-@C&u;ZeQuZT_+^TB$h6L&9*xr@_|&y}c3juER1 z5sNO6G2Nvq!Kx}cj(jC!k=W}PM$Cbg1(bstsQZ+Pxah4DVj=UxXL*<9Ir>QGZ*oE%fRua_CU0AR$y0MPX%!d zJL6w^YQ}M|XW*zGM^w?Aol&i1an?rf{TUBK4;j{YI^V}I9c(8x%4u|??~sOR!$exD z2{nzu(i}Cpk%7BCmpv;?t;?jmVfuM-M#)P(HykI%rcAFGPfg0h^B+gedMcjGr_|E; z%Y_Ce2YLl1%~>R*R>()|ygR!mx$%-Y8t?URvzeu;&xPO2uKHm+W2kX_#aFg9ec3CW zi>VQ3q*neCgi{7z{U~(=Gk-&=k`!d_Y89TwyZGO}GY-)GSJ14DnqrNa(Kuj5yQ&zc zth=rwTtmRjDbK^Z^|ahZpKY9_%df9F8H6^_$j6jGl}X=wi7@XSk#I)|q$Y3zd)aXw zu_u0GHy=jL(cVq&l0giUhSQ&NNE)HFg8Rx&?P(_8V=3XIH@u(7qcj9H{dc^l)Wats zAH4cxP)%2q`3f~|y1-yjB8VY3;Bam!W67$XDPeRB{R{4z$c+1eRGqSe$|e&}N*kA% zhmzVjLJpOHzikZqZ%x2 z&)6MWh!RS;*28!yEmUvue4e6;XM4!Ax3FXGLLrtM6rB_IwiS9P)4?LBIES+vb^@hG zQ$M}JaWrV*2(fn~@wTaaPCLO&qGo8nhbXyt_G-hex!~tRw7l#=E43k1{3Dn&QHh?x zD`EqD`%mhdX?R-3C@o>h(kme_?a7$^5XIb$KTIrdq5ANw+e`=C4^a|c<(6KK6@7}X zr6J&uhK~8LqqB)5H~~{1-wi4_%lEPXw&xA`!U^{MndBf>h=F%tR%}Jbel#ShShH^& zpvVZYMlPd@tLdo>3GcX15foY)GT3q*1~wT%yRy( zTl`;#oO`HzJtuD4-ZHjY*mk0GD3j<~?X4tZU%rb^R4Ax;Gtr|wc};ho)nZg*UC!~i zb8I5!Ea3P`pM$%po0!}etk#)KzaM%wjp@4i(O?fiNAX5EM5QS^^IQgq?6L&gSr63u zQhb(5!^CdF?Z;R&NO_!CFUO6~P%Q@>Ex6?cBrRA=Kz<7NF{JhtBxL zuJNvup43dGJm93PgTR&o7_;li7x1iGx*?3*3T3hH4|)=ZA3yq#Ij zxM3A>^>sbNowFhCnNU&AO0M8El@Sa!91U9zbpB`PT&_-~X%}(E@BD(BWqLYgMu1i3 zi-GvMtROxVph~qGUCIyg)-ul9sF;dTleK5Z2qJkIa^dda&qY+sC%`06CBj4#TOx3w z8GljJN(&@v4{7k5dUDF(?u z(KRJkGKo+;94d&OMsN8oeQmALt=$zVm3GYPDj_?yZlWZ$;HPF<)ZH=0Z}l4*lgf0x z^>A$=CS(zl>DhTtgpVS^x8Qh+IPV}Es*WP}2K_SUxE~YYQtqQKwP!w#gHyHpebR$2 z>>Ab+*0}9RxraVVmv{%_3KB!9l4oaaR_8p2$O!0Dp_}+F7$7&tsr=XH;(3ze#MQG@ zct_B2v!B2xKK&?Y!S--VBsC<<_|5MB3jMqTY!CyUxya9h zh)C;*Du;-5jk&i=xLR97TL|oN6K3xkQK%4q+ZX;MmgxPJr>imSpMfqtfu@ti>;{MA z#%tGR0!I)7qsYaM&ff@cs-bB_t<}#{G>Tz3J^u*R~E8N(NGwKE- z!Ybj~WPE3?_bj85tDd`{mdVTR2l?^a87C-Wu}v(0v{NZ6;eW=&;1T|uR(%+@qA zFw5y8TybWtqlkbpC@qq*UwFiN5cNqig*)WG9oRiD!mUjLL98M#X2J>2n9VV^;`Va6EUKc{b&pC(Ya6>o$xXgED~(+BzTye zVjlBb&w^p9(x-43mx=E%X&RNg5qw|KXc}{)O6tpQQje(82B*qFVA1_#Nt2!XV3kYb zxI2h`=b}WLw#p2;s&B)$|0A5RaHiQ+c55Z^Jy-?5IvB1`2ff(-B-26fS3oA!cmijG z!%q`u{=)Pxrn~J3O3@O$-i@95l%Ca@0mk*G<}=4x36(0Si5AHj$cAdu=eSIqkz1o( zoU54TPJCbj#whw9qll9UaP!zFCNsr&I<=wwYDdwP9VFOXME_=StC(7p=^Mk92<LP? zoy=u^ymH2#IT%HpVNN)jx0GrOuiK8+G2|mRK)hA>tV~ifR(&COxewHR+LFr4(x=|p$%9N5}(Xbgg#E$CuS@9Y9Y`&yZZcDE+yNo~Z!B)396yI8AAPwOu2 z0~t^fdPd68w;o1i>J0V$;ZhVe*~!Wy>e`KUVQsM5h|yM%>#~-dnnecPZ)Ua9nA@44 zea~oa8E$)i#uIl2cW=*lpK8r?U-cw2BCVg+MbAIpX`Wf;7N#?9WNuvo*EszinB=(_ zYdx|8!B70-ybnVT1SH=MS8dNd&u`t!aNcRf25UtFLmclFt#VW(IThW@$D_(1&h%k#;7wG!l?n>jn=Dy;IV_xJB z*FowESV)deq);txf*!N%k@0d_>;pE&8KB@}J;d|(FJj!o6woo#Q7NEqUq7GI5uV70R zmbDWe!i$LyWyyw?!pV4yV<61tD1ir|`M*jek%%YRh$^j(BB&yJ!vilvQ6o`NYDrgP zTkwxP5veIyh?yU7^?$&zf8@$!BK5fjkGQ9d)SsaRnyNiE_VHvs)YD2GKx_vsa(T2xjIkReYrm!B~9>f{)PLclS#e?dK}t%Nozqc}YDibdI`-~1MDjx@+mEpdX0VEHqTG8-X4DJL^AfKpuqca(B=)qg zjKsZR?EcqL>DT3*YvDugQMc^{FFgu8vkROykZku79{8_W*Clx8bfV=+SlRNdtl?z7 z$6;@Z5u=u)*s(i?KCot&5p6EP?PTE2%-~)fBeSqOqixOG8GcSgzO#sGcoZymX+Aft zG>Chv!x&s5p7bR0#O1LHp7Wdv5q-BSCfgHF|f5Vij+O% zS?9=}h7wN};+waP_uG8|#Qv{^kY7aDft+k5mH8G$RmvbFek4gG9>y)+6EaD#N`7J(KvfHcm zF`wde+JMwe~$=^snKYjNLWe<6P&f9l}+fewDvTIv4nmc>GxHs5L!O<{XC;hWc5 zFJ-N;{s`HX-F=j*>oeCZ5pDH$&dXl+&p+xX2t zOuZj(Ri1^))&y z3W8*o;k>n(J@m02gcj~Mt}IE^wV<+OtiB=f9V=ll2KFk*2-2R$$~~%==MG#Hd-3Ev zZN{S5Np4n^$7myj$*l8^!jJBy9o=nx z^Qe;C;;Ql!?H5z$x=J3DhPAmKcJu~VVhTCVT)5N6DCGaZi_WIzbsk0WXLi>p_@Wo= z^=Vm;cD=^_27i!eEsT#XV~JNbO3BMUK8OGM3=zh|N?*#Ja+dQHVMQ;b&byaSPfZ-y z$ERFkRo3Qnm$SxCz_*yhig#Se9sKvI@>dZ=fM`%bZX&}%K6f(_z^)u^ptH$#l59eZ zvZKe|_*H)p&)zPaa|bNVG48*YZ=@3}-*TedOVC$0aPoHYrl-!uMBQk#b4_@<7RTR# zNMcAru+S#1)9!LA!?*STPmJ^EU0W%P4jFalC??feOt~!XxbcyuhRGLDmUkuZBnM1Aeo<6>~ zz;AC0zZ4we`REiz6q9?P;SznDDE(fvWsrO+ZMc{kjNX~@r zcb>BTeQwzs>D%Ma=Jy5i`;+-Q2Q~J$_YV(ZKAt~MP#*sfe{A55|Fi#B&|F_e|DQk( zvX`#DNUOBxo42Gf)b-YVQwY~@*GeXJHgjhcZE!dgVyn5?+AIcJ&0Y6IGS^4f5$lZG z^u{sA_ngOxW5_m8%_Osy^++5u=U6dl9gbKX=&K)P%{6jbTdYK`P)-1Mm3OR5;ThY z$wm*tjW;1z+6bfDgXxevPz89YuzQuYXrcQlKjF&y5SL0BbEthc#63HNm}9WB^rt$X zTiCv!U(ng+g0IeOxXFD|Fo`=1Zoe^pl?RNTu)F2S3!1Yt)+FQD3{FVO6W*WIbQniR z4}3xxnII0A@UiQ~Q@RZ9o5f&U8<^eDKR*|>aaTAjZoy-hM0u51ODoFBhx4<{f*D6R2N)U7;xPQ=xprD)|-40(Z|7W!uP7ZPUBrC1EtXde4 z)6!af^PG8KoUk5P|CrBR(eAbG#NJW<*PgfDTY*QeFP=+Yjg|bvH{RddUp_c{$j0EK zIa?K|pMP?}UHP)){airx-}dQ2W&L;jKLZbZSpp{lWBdyOc|!_&JNk|X8oTzo6<-l+ zu${rQNMuDw;c6cd$us;& zS5SL;#3>wyW{BMKFZHVEA(z%3i~4vDY=WOzrMBfBCPw}E-|;@rNfHovepr_POdaR0y+42beu!vEDMFVO^6X++kt4PCfmwHVNx~MXP zX*fIJG>(b=^bY5NF)K^1k(JE1CArT?T%Lx@Y0(e{+tWNF`SxwUG7qb!H5o*d-dNs3|JP6^DU`O(!^if= z4|y!=Xs`3LmX5Badmxz^Qd_MoYL*RJU6j(hiC2T=cj^K04Q)m~vgur!Ph5B0P;-b; z&eK9S&*`~v(0Yl#gk*Nq_86I6;Zi30i3*Wj#__p!?$aHm$~9K~Lt^qwc#ro)ZxbA{ z5XAnBC{cx~&?*YN&9^V)f4K}Q7{vP{aqhj#DptsS2C_nrfdQ_O z^$g=XI>-6_M1X_foNMUWB6Qc}`uXqg8jkDdw8T%2@<0PNen*C@6(f$+oq(9Z1Eg+dxM5HQ?NKnIIUbl60 zKVh2(apq0L%!8cY?i#pEE$JAjtTsKu=|o-0q64uFaV9N1@I1~RMg96LSCx*ZVkW&v zONm`;xT>CXJ8sh*&Q`cIUvu8mXBbnRbMThz3z{6sPPa&}i=SZ|cB{V37LoAOjUdMl zNKN6!PqTw06I)R+brUJ*I5v%z(h&8AXznOPl)6o1>m$Bt_f3WQy4C$_0-JN|xy$5r z=PVfVA!ip~L;slI_Eu-lTyG8SnHgcp+DWs9<#7%5RQKgITDnfVGiy`KA!bFf+3Ms< zZFO~j^<1=ay1)4@7^|#y-YO!eRoq?NZFwvEefb*YotpPkF<+51p(m={s`Rzo(4yB0 zEh@6g`^1|gsAsOHIS1sa6`VQPkt=U-h2ZizC%T(@KKOqd6|G&~r_3}N>PzYS?0Xt? z3-8V_e<4?T{Fx)o!LAkV!R7yb>M%;jdWUVLr?e;pQ4P^CVo*zSV(oIi{#K0 zs+==W4PB!C@=;Gg$5UeQK)R=O5sT#@IXLU4rX%!$xYRWy=t{eSczC6B(Ay7M!5 z?^xxb_Cg$_pS==YsHt>=&Y&84A+d?6otH$Y{9fH4HiGxcqIFq-Qf@BYoyBk*>Vm8D zR(Qgb#(qZxoy*0T5pW2!e;7nDhTZu*@$C-kwvp&nZo{;EV^AQ(sK`4v(yjA|;~bb1yRvhFefbHoy(@dPJyHD|acnGT z9K4l^?`5?c&PfVBxIy%5PU-Bu>_}sU8ex4$iI7_rS1>&lxZX?6W$8j^UE0!w{ zXN~VAnw=#=>Ri=+>f>;#a82Q?`Z%y$Pk7o@aLcdx^d8g;W`gTJf>Ort{jMdpd?xZN z;tuSfino=jUmeM#WfB9W8a!zm_}MLrp}b8=O|t{n5DhPcN}0bs&V7F7+{&H2&zYxk zo~?RGXGiJh3<|tsEE}=#%G3?&y5id{SUta8^;o z^~5z*sezhwgj&VyXSqF9d>8%WJh^-i15?f6jL%w*M`b1C@zX-etuI;<;dlK~A)5;qW z2=`C%#{?De&G!ustTn#4R=W;abKTMIFe{Zi%w5EK;QHq7XoOigT|LEc;!IJFKC6`Z z%TiqZ-03`5+_&8yJ&nEB)P3TrDNDFXiA+*WwGGp4s^E^8P-?6e5Rch63*e{v30II+ zJPCL4dkmrq@R@Hav#20_c2@h}F@-!v#m$ATK)0~~bx;9j4IjauEhCe24>K{rV6U%D zuSBdUO-I8KkmOix4Em7Iu$^h-V){_g6s?p2!`w~FPcOqJP~}OokJju#`_vG8XjAZ< z*xD!Dj={i>VGH-M%iDSPNuG1N_PUZv%54z7JvDd$iUP}+j*e)gISxh3V51f)7mF_S zPV|_phUv{`>{4HX=?tx>k;`g>6Z;uaKAt70$|MmPt#w*(OF8E%GUy+2V*1Au$-VV% zX!}p-oy2sxw3siVJ&Ap7!PXXVQt&lU(e8Gm9vNp26-Q z#=!y}^&Y%OZ*a>F@}Sq`M`O9ewzG6bFos>@u;=)cWM`j7thmI!Z*#5Fi8HpMcoaMJ zIQZ2#{!@*gce97y=6|;W^mR7pd^G<1nnRYimh0KUId9{jvX7O%4dfF8BX6z!^l`0qgktf;%FQi>2S{63XWwN(dHF+V+`kC8PDGA2931m)3ZxiwC{QY=Gtc> zs{f-E6U$NOW(7-6BX?Vgf@M2-lb6hH8UMS@W9K4Y+X26HfNOTcBUN-3B;)>vsZveo zG@Plo2SrDrt=+FA6n>_mZ6m5&L`geQMDtg#slhq50A5U8nMBqEY=1^X@tt~b(r68` z4W)OujZuT1qc+A0a)Ebd9pk&Jue+h|m48O?UH4S)dVdi;tF_*B)49($&g7rh?hD?k z-km-vs3f(UAFgvaAg#0>nWtSHz30rDu0_67R(IDF-)pm}Yp$=VHPJQQm)APuTIZF5 zUj~&5saoQ4QAg;9sz1t?FJHWTg3=jEwj-Xb%G)<6F|j3Y{?~b<^1pG<@wE4K@+9-l z3#|4Y@}BV@@$~c_4?OoD@Mj7h=h3{={qMY9--Ezx&uh;Z|9IabU)G@KzJ&gJf#SY> zzIVQ&L309C1L<8Y-Ss@Rtt{>X?%(EN*T3#nro+|M{nNN=jd3M5H(CQ-9j(glE}pt( zD_1|yarqTJmBG?1)zaddcf5({|7 zs z!2Wp*ok2^9exp#wJBa63=}rC#&YTagyOQ{s6pptkK35xwEiFMTZbx?KD7}=U26xOs zoEfWEaDINJ%`k|C)K;7q#{7rj`1JZAG zt;iwe;%U6iuKHdNpuu+GQ81Fd(kp3rf(<0MO@Lm#0CS?Bp;|9t%#nL(Z^b~lI|_tw zSmM4$BUs%LRyo7vI`3(tUou18NmZQ7(6Jn$%Qmr<+g;8VX*6;5_WGHe{>(awm*WT1 zZ~bxo@tpEZ^mYs?&g|E{{(($peCNw%wR7$AHZqo5dEA!&r!Py;o}ffQ4fC`NIuVEq zo|b!juJpMkjK(#>GfEip3 zz0BL7vff8tIq=Gt&R;2bxp%(zqkopq;XCEe;#=sA^mh$<6Nm^JmGgFRg^(J77XCbe zN2cN~>RF_pvNl?=)<*XgcZ~JXwagQvO*ay{Qi&zjGFNHbWhc z&azP4io1JpSfqG?;_mJghkn25e*esqCry*dWZv_hd+xpGuAqZ|0(~prVCmQ8)M7Kg zawc(@HS&WsZXRcG{jc%YlH9ANdsr;KXOZ^obnT_VX?#5@=xaK-7c2GgqIrb}Qc9*| z#%WpU!%EPu(Z|!;8#Yx=?S!*vU9pj>+5~Y)*rZzG|KTyz;0(R}QxX=OCl2!T1Dmw+zR>sX~6Mfc$xEjOA3Yp&W<|>|w z3VMvA!g`;=)id2_bAl)Tz%H(H4YTsN7g3U_J!Xf`$ACHUL^Kmvo7<=WdBNU21jLBJ zXX#%wRkJc}3|IRo#}HP{4C1~7)=u+eowg#cTnZwY-uqp6=aXpiPrxD@oFs<#KZ@H) zZjdAw{kE@Jt-Iia@0o6p0Kzoj)4xzdc$fNh@y~IkZ&~{qzK8MkE_QHIE?9)NEw>;kgf6iG%g*(wyJEMWj zr*3MDJcn2{L>{GPwq%ieQZHQ-4)YJ{$|^AVgS?d(ss)@X81*cXKFtjDA$qJ?l|*<^ zzhoBIgKKNf3fV}!R+zhS2;b3EfV0?RlC`5vT$%bXDANI$Ou953Ew`!^h0{V|YZkP! z6=0*;b9V@BuQZkUHAECy_0rsFi|~0HX&q?aC)&cS9;p>*r+D!8(3K&fp&1GVB7cUbh*ZKO zBAWZx_##4zySunM1b_5w@hn;xrcD4f5k+jP;H zyA{`k&Ny|<17Yr4Z&G19(%Mq#BC6m{6H1iSjeg$(RE;)qH$vRwW-zt-|1-KQB);6<`%+gNS3edrE$<~n$sn7(du8l*PzFj9vkV7q%d(*W8;*WrdKzS8ozCNzUBr64DX-&KJ_f6757$}+uebxwbS7q5PJFaa9<7$J zOeL^I6!FhTT15V?+e}vEcK`v|#2t0{*u8X6>JQUr9yhBY( zU;ejBa(q-DzPA9Zh?RGMOV?l>TTlXzf^rLq>E6*9_la7&V_Y9o!(U6B)rDATPjb!D zI*$K>)pfy;;lUZ^zh{B!`@3*Cf;S6zTD&q9} z_M$M8v3LlS#?|Dvn1>UuN1T8o4Ht#vy;O_svCU9kiVn%pkpT##bY`?3m zdrt5!l-nMmfB9+zck=HEzUMvS|5i3-=@q4FH;=9tU9Do{m9=_RDj4YO_l11(EcL7m z9t5i7_Fwj`3jXRV5!mXh7})IJ=t~>&D`HB-n#l8>{9eVE=>6@j=^q)|J)~l2Cs#eP z;)9)OU86kLjLgm#?veTxBj6gQ4{=D&wD#%vB~FrFqLBZA`Wnu9?@z6usZh0nz5D?y zsE12lWqRleQ1jCbkDU^{My1F0AAAE^!LX85@-KfC13O0VVpP;__BfeWdamQLOKERWwi%5GA zPA|(<8!>A}r4{`YEAa*WCAv^4x`TVH0P8d(H3?ls5}pq+n#EpPyGwP-Hf1vk(g>>M zr%OkLi@y6O!a^>9Nn=G??UZ$mQWejr*Es5Dv%Hcvp)MXH&#;he(Npq1@t{f{ zcJQ0r?N`_V&5F32$%ti!2~ zs%%!%nRnwPau#p6o_SEr2BF10Ak#-*F+4c~7&3uM17Qm&pKyaYNXJPSHSMvY71i3q z^tJST{2)i!O1Xdn&=|d?1>Q}UaGGsHPg!kSL$IYNuir0DN*j}VkuuU>d_gj_{n`Ps z+^FIF={)Jm6g9RE#YYs$Q#hl)xGyqf z4=TnM_eBwFRC6+yPoL$;5uD0b-1n#RjI*q#QxR{W>fyhOW-go~;#B14@aJJC1D-&Z z&^6`PluasGsCmH#6RO>AmZn~difMgEyyt=^`7FLT|D3SFAvr_B1CM+$!RG?G13|$P z3m%L}6@JqD!rL`?nr}vM3SYE;uWyq-(KEnXEwIG-9_@L#HpeL9SmnIy%uIc4icovXxhF(J5md&ssmCPJFPA#<{mD-fmSz7Oe8wHbLD+oyiQUD`TXgxO65dc72uM)7<3e|CHN`jQST_ zJY62AZ5!0IT9ow$7`QXJYRgHamZZLrLdZk?18x-7K3c1)y{Q|lgzB=&7N_nLo9S|L z;9|-QOzK#UE8WCGR?si@)?jim%`M%?_Vy;;xsY7nb)L%n>nNiA=*@Y??tBom=}I@& zCD3Lc-72wY1;4?Km*_KNSuY9jeA9EFKfL9?Y3Wy31ub}H3l0yL*z0R2tB#FXJO52u z??%?lvoNACZ#9zD6Mly5d|AJ>O2LJzLZO$g&ZN?F&FiMHqDEXq^a zaz?bFYVIbUB#k*vg1uZ6oowxiiCR)4aUPFH5BM_~S7(*EFpcOKT0$+Z>^z~rb9{9z zay}z>Qwyx|x<5KbyKZ`#87-ZW?%GAV6%HxdxXzoZugeG8)^75m>Wrd23U`cH;C}6C z;AH?HYKRFmC9gTH&)yK{-I z#tkDIlM(W13Xb0klBWsG7B<^!`~hxR_fbE#**aSpNv0+m@4I#Ihi*)$tOT}HlFQ*l z*8rdYiP|<>O}dBvJ0@Avs(YD-8-ZJ9Rk%kD5NVw9Hyq;^PE%=cFjJWHSB{F-7S^Kl zE-eGcBFRiWrBmPnJLz}sv>WV+h2Tq>aXvZ4)aJZW7NKjWY#neQ-)gJF%+q_ej_O(P zZI*f$_o~_I8&9x~F}64)GB?lho@VV5<^i!wMf!kgQ5DC^M5*%f{rIZ%sYz@@DLbH_t$40P6 zU!mWjCS15S`|=sqW>@Nydr(z-iuJXC{j?GLavbaB5U(6g1o4n);xn1qXmZWZiEHw4 zZx6QYm(9sL8PPG;gB53Z{F%sTBfMoV*wT&X7ZCxSVzru;X8%$X@)BNDkiGbCw8|Uo z-3EUhNJeHi>XyoX&F1x|L9q8|AnnPTttU>oL6p%AC37wN{x5lRvbwULSwe@v^Q&OX zFL3Vx%%lSm%0{wOelo>}z?lPd6z=C6F&hzeXgmH?2T{+q@qM3>&pO9J`K?odOU=RINg&T| zc|6_d^W?!`qs_j6D^Q5@UjheP#`WoDUj-gc0zI;!;lE`k+6iu)B42i&D%?0Y-aVK} zB5HFraCke>cRa{bnQO9)YqE*!Z>lS6iGy~*$~J*WyZQAGas~cHjTuOkoly&two$P# zHMt8WD_snd*aV+nh!1u@+{n@~l{F1h!2ZGmCmO$$NXK;X#pvv8MX&KgJr3XO%y{hg zz;UGs2y>e|sxe4nc8J`-^W-1pDLTwpaHb7)5sS1UN_OT$WwvkCM(OMvOgR0dEYxMk zd2!5G#tiBd?l<0w&LHYjGL=XydZt*`YQ3Tcm!4KPxLT3Qp9{T+m=s>eU&jA2u)Szv z5v6ccxzT0bmpN(Vah`LP@t5#l4Bk_4Ze)hAS_RsLl?ZDaoF_OYcvNsZ-w*%G;P$>k zf%2Zxp69^@g4dzqj_}U&?((+zgVa@c*#$8Z=aKaQ$n(IulYH@L$#l&M#s?-|Ybl50 z|5nF#nwsbTj&ADQ4iTBk>RJ^0?6@_~#`pUz7=Da;`0Ca-#I4=ImOsQq>mX|9dg5Z5 zpIYBCxHW#^cMldr$x|H0?X8hgRP3uaG8qGL$H_S=$Y~89w*^8bJ%}xfdE@r zvxA8jj)MRXSoOW&`R0t#bRfn&)RhZFPXTy)oaH2IY9E!QmwAWMFqSoN6*IHb2qfvp zr)-1sctL}~pwKb0!zt`$9c)#e=}sKFoz-pDr1b_d*0Jt7^3FFc^Xz7=ovE*y^>};u zT$M;_9&w`CWAzw>Tuxot39xP{QB@r5Wj_ikSyPk^)6r!OeMWmYpRXXsU!4B}&?bp} zp$8iN40y{yo~eK;I}(hgTb(O&jR>d$kN;q5+ethLYLLh2&b8Vs4@6^0kOzTCPe7rW zs2p>^PgChNr~y>3|#yW5xC5pV^4<|f(3{bb*6@=AP)Y-Pa~tF)h7^G)2n27(FQ=o~q2 zZDW5fZu86LW?g8ipD7p}nBNy6d$s9!()QHv8#U$9OypU>9MVquJm%j8X~l?%{uQmk znFmydPE}rumA2_rMn&6(FoUK)>fZ~|nMnar`Wp0>bcTazWmoZ$D~IQE5Z=*;^d;Iw zql{~xvmV^!SLDTrgOOVcGzlwGpnXVK;7@=1ko1AMf!Be!z{$`g@1Ma>0_|KC++Ocu z)_+5PsRA{_>J;zRp9tO)NF8_;SmmqgYY|xKkM}+FYa!|Ub^PlJ?TUOB zo*42Wa5Kd18}1A7A3(Ep_&yeGUMM8;TllxI(&4v*rw7*y40P@GeD`)!vr@Yfg~A-| zN}x)33w_Wp9d}%H$YE7;7jyjO{N!Gsr*{71N~eEf2I~?g$F0}%Ni*rhnQimn25{KA z177jV`h=`&f^`xOEt8lIPz!(Er(E-+#6gYeIKHe7CClSx?dP?fr04O!<663h#?#|j zg7{?x9yncbaU8>}u@-pZUBl_|r@bsyD@8z+NmMNs6oYUYED6KuVQ)t5N@kR_p{SR~ z)VtR0oWliM6thj=b7am5JIi^F)zqW-6$I3sqA$*+4`G;J)pyzgGG9fFu5>2k(T@`4 z<<`1NqrtQW(gbSo@<>x~=jZ`9+au=KDyk<%X=xcXQE91>%cI#*rwwvFJ76!GEs6f1 znl`)GL6uH#aS#>v3of*!m~s-Pr;#Qo`Ly$NFOqpi8TiItA5ErU7xh;!(Nu2}CH1$= zgpKq>LpVxK! zRZiBzPyr9i2IL{$emnN*l)ya9i)nL&+P$C=0W?;p5ILTLeIsCz_ zjGaydv=s!H#b@nI#-iCU7PG7IK9NpO)TBw^l1wyIhtKMTeliC1h@u8>n3}`#lDuvN zQDH~b1Cwb+Pv}-qB@Df{AyM0J@Mjf%FMoqqjj4+pLq>io7VpXtA&%fog#cft|knzLowKzLEYaA=~_y{Jle}_!WPyh}f`hVLOAh;JW@+zM_6l zAh%?n@#RUbmd8IiFr#?u_@NWPc?(>;Fg>==W6F+5OrBy@C;Dm&_(@M9i`KXw z<4l9I))`z5BB>C_NDYZ!(U{!b1Z`&?>PKU76vqCOeQhVJ@2Yhv|C?k>tF9y7`HD7s z%a$4*b&aWTyF~|FuRn>`=()m3P}bt9QWb5q26G>}(kVX}+(^X)gsB`Or49I$7L$gf zw7jx4<+$2bj(Dmc-eMV;ymnF@&eVqCs5cMU4VjZ}8>Mc53oWM0wT|?SXwgbuysa@! znam{dcS;m0cpdv~k=Y2vt*EW&h&$XND#|{>Og3=*1ba89kBx!JoM(0KK#l1{9Y`E{ z=6x{7?D%_(Dq>cu9AItUhuc_*eP)v(y@i(R;rS`3BIiJrhw>(lbf$nOm*ovaJzqhU z=IrL{IHm?SdV($HjGpZFmGoWaw9jM5AId6g2Lssv^5h{_sz4Mo2ZUM4u_>%$0q15q z>5T!+*27!cv%Z?3axCE-D!>>gfq1c;Z7yPvEohtPz_C}bkrwv%g^Es2w@=W!2{@+qv>f)Nam0P9eJCvD8%Q;Q*l-S?-i>c}jY{JqWY3%PjgDFdQ@3OKdv8Z$F*_U& z!rbDBhISDjY0Lz9L@%~ec}GWT4dn}6077}D2Iw?-qvp0gw|}KpXDa{okaxaA*0K)o z+#ejT3OsVR27!XYuYTxHGa#!B37%u5<4Eg@+Uz9`UJ? zt$bw37Ty}(;lU%FiLMCm5c0&IysM~e%NV@Nn=yF3cX)W(0&7Da7XG_no6wA5iT*jh z+mU(0zW8Sq3=duIdmT|IEGo2Gge~l5=+6QlL-vIZ3U&oI@s$iI7dR96VqA8fak-2+ z&Qkp5ieKF=a`hhK{^-*d+qmHpk%r{;d`Sb~n)XsNCtTVm)2Dz=> zPTAGlJ=4?MIO*u^T7tV>Ha*Cm1_zjj_L15wqn{loK0~u-)fyQO9Ny@1v?n!4N~C@RmX;C*Y0Cn7sxv%*zO zPvat5B|Mt?810#Duv0%NRZ`RAu}}`Ih>@x?UD2%G&xE(oTcuqSU#^Ukg9mm|L!dgrFQZX2g`onnW zc%vc-qHSA}i6KuqzO>(CQCVLBGk#5VT#bgZd? z#5=3O7Sp||Bgf6;xSz5T`w?fY;Y@0>`Z^}Rrar7`8Ct8Udvt?)tmjw)=#(HP6ANuobm{m4bFEyO3ac43< z`^g0Lvdo5`{DE)wSeVam6dtq7awO+GpQFW60S;5d(gp3stR6HUn+Q_}bn44@{Kh-< z<@%s?ftJI#Rw4GmIN8nyKhs#3*iWGRi~wiE3MGMaODT-LmO~$GD*;!TXUjxHFvpe)hsMVA&ps0iZ9AC6vBh?Veu2Zd z(SFyRxIgE2B#G8W4rd+jBhNW+zXG9QGXqrEyB<2XlpI{_QsGf8k2Y*l%@HyxP%C5( z4hXqDEwv;gg>$!Y&bi8&!IP^ z!Z!+x41|{GUuabL{$hO#b&T|eWC~0O+z8zmIO`t}JkVRsQ&_9$*x|^;ocAt9ajmT5 zf#V;hlgAjp)dt26J9C+wPnUQdnfIa(b3)vtQu$UgH!lq`)Dn}Xld`PZ82)wdubhw^!5sL zKvtBG;^+_~*>P|F?|7fsXP}mnyxU<>m-!v1(Lx{NgrnGQuyRvb8_~&_O3FrblL}NH z45}|@LdQJYLFO{=;GVptexM`%gZjZ%no9d)baUWd#l6^<2`r67e*J{18}#O(pNC0CHPqVTkkqSZ8Xg-iuc)=F!6{blI^oa7Rm zqznB)NldORgLg+&t*k8_(?Be?*Y+cD;CAS5FJRPzgplUbqaCC8^<{cz$p;GMur08k zVvfugSp{2CfR<)-^Mvee1^YL06E(=cyg)fjPdAo1^>r!H${kq2efb6%QbBZo4}~YS z7zWe02CCFW_gKq+-GTE&qBky2u0k2fDvo2_rDJ8aMZ;N*F7Z^}4j(Y5r8hynG3Nof zi67guHaDPo{z6+G&HHbL37BrxZHU?y@tP<6bp#w{E&ux`D(ina*|(O5VBiNbDQ{U{ zr9hQUthEiWg)789&G@XD;Kof*r3~>)G)l6;nLK1?zo{DN)qQY@nGWlSLU-!UYv$mh z*PI%jQ6SrWVwwJ^E{oB+XTnu#pim7XCYl10L|JCYL(q78@D8O>yt~7N4uMf)iSkAh zRk`fZ#5)!2V_+bIz>s`!-Dh-mF#my`%4r;5T4wRBv%z(*s-@@$`KJDX<4_u*TTk)I zY;-JsL7|-uCtSq$I>VVy;(ewOE$$%_EJ`fdhF@(K{K`T`Y!-U&YW0Ej8u_5++zE%Y z2XvZz7hP6xr9j`1#KPSRW)2Up=q=l?R6Ab}U*163 z;0D2s{A)xjF;r(?(bRA{K4^)KInFYAA?G^RT9K1jX`!-Szp01gIWw8Q?cC1yj-ih1&I-;b zx*+`eN4d84m)_J~N81jbw9#%FL+mZI@5Ta9<&!bd-e0@N^F#QA82fDPsZkFU+F-cs zjkUQ(bza%?|Hn6SLHei;%Xh^DeK@mSD(k1Dl;SI{;7(>MtiVm>0{-@s&}KuVaU8Q! z3He5ZNw1hA>69MRpB~3w{aOKXfxqc+&mwX#ZzWO8WhO#x?Jg7N^WsGJuY^+<6W%9+ zAuEV_lEfk=*`;ObWeE_b75wBGu6Os*Z*!w)RMjT)OcK4!5)LrulsnQ4-`T#3QR@YGZ^e~GGYt$iNrxeiQ^tnJ$Qns;XOP1 z6Yh(#$u%vT={QJ&r8fl|PoNooqqDCB(cfb3M#J9Ts7B z37J0C7g+`MSrLcPJxuol0YexCZtP=Sq#>hWYOM!Z7hmO-uou&R#;mF}eGKd9ln0RMB`RHA|gU(*OiOQlgC?v&sej%u}7M2Ibi97I2B6Tc_InREazu!Ka9_>NsMfc%U zgCk>Q#~E_=5y!&*8e7p zx=w3i|Em3~*TW^ghW>^Ml4jl~u;_f+JnJ`ny>3~f=!{F09)U~!nXB$?O25oI zOw}hyXPK4Ooe1ayS&DKn-csu_?@Ex+Ua^@hI5QFhv%+mrdM_~a-Z-#R*&c-j3(M>W3{uKanVuI z-Pk$E^~@dP9N=2(Y3o_%dF8!Wa#OLL;bW`pEMKPRs3Kh=T?H~k91D#I$rn~ z+2E;++EUaP@TVyh2)`TFv2@*{g9-=yM}zkU&(^9ifh3)t&+*)`Nxh>-8(vi7ZoVPj z9KJz*t3OxZLWpi8I{P@U+Qanv`g`T4o}gES0fZY(nH?LepHph;myL~zS8wfjY>(Db zInpUBwXgJjFV<9t%A^UWqqK5X>+N_Z=fOF19aUuK^bI)T{;OY-e~JzNKgQuG-cz3{ zOIlrh6!DZxZ;tcrd9>f{ObPi@rYePbbBn}#Eu;MoH7BNz?LfUB3h6N>hQ!e~+K6g{ zF8V)YYL9Bvtc#ag8ln<|31|*+Tl*@H1fwR?r}SO>6Nd4ZUXB@>-?UJ=5#MS{lndG} zeLTNxs8LDDg}Qr5PEcQHovc2%Q#5m4>JYUp#HVz+r2}0a}sn1*5*#N5~}_zvR%8$%eS=!Q|r=(%HhjY6}GS!r{}Q()mHmino}Qn&^w)S(&*Un;L|gcT zZn=vy`GIbE6V_A^Br&U14}pGHS(Uq~={>^wx(B!EhiB$Y&Mg-z#|*G&J#o!%P@*@g z(IP7F=fgb;p;7k-ZN_rl>hSz9;-hG}Eo#g&D)ZaHruwSYU?cruFKO*Vh{CelN1?-< zA`?`eSuKC?U$5XQeR%yC7}r~#zd&F7a`cmMxI{(js2}247sqwTV!g=s3bG!7pPZn} z>;(M|re=E@uJtc9Dsw@YX1tp*jh$W!Uz>JQ?Xo>Xi2j`fl&id*oF zW+&&H<|Sj-N-AE~ay@$z6X94T3tZz$6QRG5OK4lz%NppXaN3zjWqw0By;erQXJZZo zJ{yghQ5l1VvxnTob9?}1v*WB3YlzO4(52`gW>@S7#Vze240$jW8c(R>9f*h6M==a_ zH&HAm`uMFgu$m@LRyzLt0u7O*e&pXRF zLB(8g-Q#HDd@n8<369TdI-Iv7mA|yC#!`_R#x&Qt)Ro!0z;nnm+PBcr(^(A9xe#x9 z-?`Axkl&%xi}WtIv`~eD)x*1n^$RTZx%}yUwSrUomWK2Vqz`y~LBSb(2_ZuR*#ntM zh82Afc`qVo*iHYR`fEoP*J4o?CxMyDDE**b&R;*!E%ai@y^!SvCIp@YzK5Q2w07Te z$Jl3Sh4tIY0e!iiK+n`wy$y3W)(~r?!UyIcIrm~{Q>pZ7MnfjL6*p#s=6m$_aE9&r zAId^a7;gJL(0hWsh3Pi==)m&p1<3p!`~R^r%I8&mF#75kq+6ZqX@I1En{E zwL8x=qnkTFHOZgIo`+k1qb;YbE0MLdS3OaMvZi8ahA z%>BW%P1J|%N98Du=FpT}^K`V?PgJaJ0jW$bRGzAmCalwF^4gooG}R#L838j{!?6yp zA4Y}GGNPMuT-$EMYlq<>U8qJL1GCxzSNRj18H3t$fHiJTmm36I$YLJ>i^yaD2TkcY z(bERbXTDmU?vGn)SGo>At1YQr`K;Ea-svU2+-a28%mFbwn;vjhr@{XL{8ERAk4&6t z57*j(#=99N_7~qS3X~hlSq(!^Hs5~+`Q>gzLlv8sRREO@ckH^)=%Nz$oYKr^WL9$Niv?zLu9y5V`F!|_8%wT;%U%(3T(X(v{ zN+zaoURUzc+jx}6_2_(gCf3QFaCGa8|8`Sn3ua$-a+J~wI%YeYX+w=Ej`P|cV>9ce zEuL>vm77{qqaf=dhp|}Mr&Tj{X@~K1xJ(SQ#ogc8$#u?~+II=~?BBa!v7sJ3l+udMY~hQ(04joY8QXB97?U z99eJ*_+rdd_vnj^T&`gH55{=zc`y5}IB&WO1kWWhoZ}p<)X-(4xN;Alc^#*xn)+F0 z(`3-M(8-ge|7Lv-Gki*M&1MWyv+7aCC2?BM=&*{udZgnKF`e6y5r30Tj`t$HG2M|% z^rn`+EUvsw{(oei3jPg;!~yCRYx37_jvY(~>uD@gN@!N29IjpkjrU4*>5W~ou8m=SN=40YMA#Ck-@hxTfw7E}3|zaH z&|%$9D~Ip&1y=MM`5Mf>w|tved!1|%O{g%r!oGP}KF-=%NMuozzsA5u4$6DDSG&M5 z{$M3~sS_SUKU6$;Vs=&6r($*w_k0HN6z-jlyQn6!%!1)DEtz~IF<&nyU9L;S16y^f z^*+#hJ4fsSN%D}D&P}a_={I|tPdSbTJCV5HE~~gEt9v9okKi2AS&vfNA4Es$eZE~!R?{L@hCx;Csj~CQsLN!7bUsEaHeBtW6EkL zYN9gZxsg{%Np)p5<+~bUoe%H&z;E${sHZyrJDSML%)%dmpG+hcGyPuffPTWl{DHsZFhhz`$6pUFuk%Ucp3CbwwbsC9|z6xPn*p!sip%ovKY&7(Rl zrSy|d+rOEfnOY0bkDo?UVZSe#z#gM~qtZJk9%UbGRq_44Z_7u1;kQkIMZav3JYI$B z8>GEaUTTdUXGOR%z_A|mxTpV8lC(Q|b=J`=bRI7=6LI6BLS{G0gRaj-+j*~5up3%l zriA$P0S0+mBfm3~-p}#QS>Gr~eQ!x;h-;OnkK>zjuqR1WHS)Mh;~O%|*kaG41sRpq zT>2>Eq_7x`9ktaceY(+?j=s{yD76)M(nWmMHAfn)mvPH+QOoPN>TInycMNpiGFm(P zIdAGu9UGm6v}49+$5FMDZgYf~)1!@DxI(3HWEX?<0kEtpdKyO$y8h-G@70!iI!9Uc zJ^g)al`b%ZE%1{|dL=xPzUUk5+sU(K$8G6@W=DOv_5WjPdY)x{22~5!$=1o_g|o{c z%+!9G99JHZwkvta>C9*1Z6-NMsi$>P8foQ>RMzsu7qx9$sg-@N)S~CfBlj1tw9QgS zkqZoT5raBy8_C&4Nn6x9c>9+UXY_N{eq7_qC!Zv*=9O#Z73#m%1bcNdgME~{;$LEu->kJf?4Ge`eW`@$XOfy2$AjLI z5`XDzbetU`3)W2ep&ijD)7{A&6L>;V@?TYz+F~ttdkV3dJNYB(!ELZ*0BG}syLU2} za4IG4Z*H;)Rx48Asy@s!HOTCM-S>-=JSl{BstvDhklL3#ll;N^XJz)#Wi>n1 zL>JWGaF2NA(U>)C=G;;{-6eT&Tsdj2%pBW7b|kPpU~ZOG)cM+`U}ZzDx;UW-W}9p<8-SfAl~(NPCDUb0jl9rqYj= zQ@cVA^eFWe2kZl>xQMsUr84SI>gBpqf13b$Y#;(EE>$2t>MoIW(<IyAF>u+>&9#_lhXJ9EUwH0~> zyyjD&RAkT=G3#)ac*mTwPE0~vEstVO*;MA~q}QLxYsvgBmxqg-`h7g8S1=nr4{pzw z>~0jxQ$jbYIt;A`K6YuS1J2}nq%U{GIlJM<(%*GSpW|4HcV%@)2G=N7U9fX7j+6mB zpx!b8X|bqfL_0#o4gISTg0rs0F;bQEIAgzZM$2V{5jD)#J1e#6+Vd$jQ76uVem$At z_?-D@_32@rM4#^f<^i>m!l)tX$4>oF%#n1e7CTDcmE8E;-Xi-`T6&K=N=GRlN^Cpn z0)CJG9Y=s3vNlGVri7tWjZw1j{5t-t5zqgo?{6|!rigSo{iG6+G!^kTiqzJCRrnJvGjswW*Yl22*ptexq!xhqosYiqvU5zb*VXH^E? z(h7BG9FKp4CtZmgr@}fe5i9ipndeg>P=sixHI*tuxSox{8MEViBv^EfTK$=1m*#^c z`A{%^qX-?Ns$@7!EMWc4Rds=ZA5m2DDDSz#{fQrC9Go7IHNI)aj9@iBb15Vt_%8sZbbLJjc{G;YNt z#29-&riiXbYaK6rqvO61T1u4BK$@-|(DK`Q(Pxy&)|p+e996hi#h|}&eYyQab8$F?kQ66PkGakUMlJxATzz_7k>QYpQ=ITx@jWJyrgq~0tzq{(X zf@9HQeJ0VrDt~lEp>7C8ogUNk=bsQmP`A9@ljQm#@ zIE*mbDj_J?U*sH2KpVqOxj{=|uO{v>Q}PNE=Z`Y+u#(oE3c$C_FRn%gb{#Wqbxo6A zD)$5trCMJ5iTCw6trZpW-KZAJLj7u4bnNG%k@TFbSq14L9Iq8V-<@$<=n3w%|$o-*Kw63cuL_(gHAq>x1K5D>81mscB3})@HDj!1G6>c&_JY zDOTx>pMN~sb7PxbX{NQLZ~eA96g6+4*d;GS)w!xv(R&&Jr7hEZzRRt}O1**Y4YO+7 z;vn=1U$RK5?E11xo(4%)6B&iGx2^?Yo`W#8*;O~8Kz8C74*tXteR$y&OIhi+xmO2* z8+O>_a>sg`u9Vf*kXv~VuPBcG`Iaj2+~7tl_{FA_-i$khi%C&1ZEvw66*03>gcY( z*nRBg(X0!zQr(=4aFndaJ=pswIL}wIA>TlN^Z35{;UsyeNX)HJLrUxuNsZ85@@W-t z7s(B4$^$3)o2=_Ol-1Paphv@}#sTfvR0mhy5Bkc27R=(NRQDZ{w7bI`1wp~;-4XK*vM zuhaMzwW&*at`-5!ezIn7F~Ov_RUvAtPhC?INRkDFDNJYmZ`{39t>&HYTB4IV*bI2f^W+YQ+f=HY%fzx zaF|`7@jAHQ9yH!P!4Xa2@t>mEtU8leywDFy{rRQc+w*&#%E$i*oiV zVh89{2L@Y*{q7|?Pcu9c%TjICOt!%g8Zf6Znh@^N+2I} z0Qa+NRP(082{R|PRU@Ria9x|!LT!NpGnu@z9Vf;oxGzTIKD_|F`x5D#NtPMmcaGL90yuw7NDzzE526L$Sg@y5X4i1t-r@>Yq$1 zY!0uO&gw3#HIoj4gd3&LOv-o(yC|WhmczN{i^`>`2{E}r1?>WArwddmjI&l#bnJYv zi0Z_8ZKz_+ukDljFeM>T?k*C@*sf69GexMqT8_CtA@oO&kUl6W;2~F(x1y_b9VKcL zEH0pZkb8^!%oUoW{-OOPt*5H_zvD1z9BgnZ^SqeIAjK(PL_Nt%4cjuQ18BTPN~LDy zaR&0jkZc9BkQqAt-o(L4W(3LF{NnsH<$G`c4Lfx{eU9}nVA%MO2-#14LgPx(N!Re?Ka zpNNHvzSGX&+kaWRYAdEb5xK2NM1+%3U2cIPZ^=-Zc=ej7$8?f3>pf!GTh02n5v<35 zS(Q16TxJs!U4Sn|p@*0|=@a()uOLaNHJy@81gKHZOuwg(c>%+$uk13j1JWu6UOGK_ zy{TGURBhHMFl{v7ZWEf)UU>m5;t6c6E`2()`P|=Rlly{Ti%_M?f-O@}hBxv!40h2o zx!Tt3mt4(N%0~RsnLN&P{_7Qr_+a?ecu;x_(M@OifX47XrZ++~RleiNznK5^6K-~a z$z|JE`)Oep*Z3CmLEudMg5=uB(&j=PN~`Be=5E_>urFNT``$*6y#jBE0cTS1xfhAB zesVoCz>#k9&Q~~xUBo^+VS{7PUQSz{!d1@ER~|>^eFy(_ieGsr7<&#(ehjMAWH%{6 zq!l@Yq^&j{kA-_;jBO6}Q8R2S z!HVxAXAV^+WHY66~SfbCB1mFP;oco=ABAA z%S4jw+8Nt+{&$J(7!kp0xu?ji*A`Z^j^e6AzsAgnIOcWSR|ZhW`$IVnduagsPwm)Z zWOx2>-quGr^0~Sbxy*F5x1SNc^z0I@h1xIaHC>iZr6=N!CQIkRifK$)-K*ubk79mE zb)xitv|b#0G7F4(E%$HcexOi?KfGs!8}1gWB^gB(s= z^PJpzh%Japw&B)sL?kJ#ujHKgLG`7lCcPEV=a5M4eB=gzdDwT*{k9iTR?LRD)@2C#YPy z4%1#oN8~R!NoMZe8mPE;&_d^kIO?XRX$z^Dj0fL3!Bw83LZl(yuK^alB&U-dXX(=H z#h2NIXNvdMA@~9f#3k>ExB{2z&n!8KnN}Bg&n%)V)n#xnszEMO$!5-OUQX2WiY#qB z@!Dp({BCmR2heh_lM#N1PJ0Vg#{f@$tBQ3wn(YV6dej)1>_bYChZrz3_jwnX)jF7! zIZNml3~hBXOG!-5K+OiVb`Y(^f%hwk%Z`Hp8Cd&UL6r?eIyX=|2f<91qMWG7y2>S9 zIgay*CU=vYe9bU0$DI3?2^5+O;+a0!>3Q6NcxO6_P!N^;r^$2eBBMPDJaH4PJYuG5 z3gr@ry`Ko^b8=5{T4J1I#6$<-1~Pd2L@fwbeCB%<1xeBn2i+zv_*h}Q1T!fgK#427|5M8?zHb`*4Fk3fuLxOS{Y^J&Gmn+I+$0ZC?qCvjB8nE7i{%UunhVHO$8q$!n< z^bv84)HTqp2k58t{3K2fEZzWRntQ)A}y?xzmHF1-NS`Jj0rnO#Yt*MJCFyNF3HU4J3;N1tR+&vT8SLlL@K^g zgGDLy74}-1UGK!byM`&KrC=K0!Nw@G)6C3LDJvI8BWf;3;YQn6&ZQ2}228sEJwBZ@(b|B}5TvtdaK8A|6MJ@nX4LUO4osynl#x zgL;ye+99PlTJ&csY>R6fP*k$~|F{(1@|7;a+~Cq`a4#6X6hy7;J!V;&4ofBBFqgSc z#;_NdUM{~`ahaJI)rk7}_O=K(&vSJ66QYN$Fz*v-JTO8QQUg;(C^4vfL2 z{|Hr2Y4Cux(to`UEutN|oL4kqjjLpcmV+uu>}VhlT4;K%PXTt-ht!bd;VPwJ)jTJ1 z`J$#^hiWA<5iP1jeCO1ATA1yevXi^<1)U;Y*x!G!y3JXW>(DdPh{3Gx!`#nRh?PE* z8%1j4bMx{3{b5I+xZA(M-kbCOn_)87L5XVY)O(Y?TslyPW4c0R05=B1dG-=7^kRLj zBuiq>TKXlg;rZVjO&5AogT0TQIU7DRmt3$pgQEkw>)-SRB+9Ef9)YErzxX;LspmW% zO`Y&eVzd%OMjJrUg*=m<$Y>_|^L);&HD{t*kD`-gWc?~Nd=yhCanYahVPT*^IiUQek`JjAaP2Qyj870ApvBv8S71N}+^-!8ymOh>+R zRPdTvwz4mA_3)O7XfN?}BhJ9hax2J_O4|kZ z$Ot}cL0dR&+s7KY17f7mzEPvDX%$%!4XJF2Ar`1ZhtX5ANCSAE)#W|wk2qlgQXEfLfOPU{U66`^?+XAF^sOZ1&)vURL3}18MI)(qn<=QPfIIjU~b@e zxwn>I_kt(!_`CKHOK=w-EaqveB#r6G(Nd`BO0VutwSiW~R$TGpERoZ`Q{3SW8=_BD zx9BU4<77;N^na!1s6c0=)~G8#U@uWxX}KuuWf`^9lbCt>2*=3VXwl8I1=im9DQ!o4 z6zswum|HlX+DMDo$zA6py8MXllF?e1X@Fj9FCzMk=rH4?OVl4Orxqv|$XNzN$}SIwLG8u8_PnTNEryOd zklbQlu?1E+PfID@Fokk7H4v(v&$f(Oxf<5|tn_#kgBS4jAK=CSsv}Gnrkg}Pwcsi% z$W2@YTY8ZF+(&j}tsF}xbccLU?ZJcszc@hFX}dTseIn|a3)Ak+EZVK`ha+?_)QNooXF8M7m`4ulAeE)XK&OrDp=O@i z^kEpq6<}gES@@+WAy=sD=+2cf{lGtgGYjA_XxA{MgY4sGJ>DId%rzdTfPs7=Uc13+ zH9MVe@wrEFSgMV3vJ-Cz|{`cc0_q!(fX?E(@|GK$*qTz zcaE@b;@S=ZZ!?O*=u6Ldw~u7a1hubE(R1p6RIxCYP5c%MIscdVL7k*la8fd3`3v1U z1@%97QD%Da9nZi<%&8FFi3Il$4Xx+fz2PqUVfo2l zSmqSRblMJ%LD~i~M?b_a+d5uziV4vv$UYxY(&El~0Om8F%9bA>%t*49RnUYJaA6(I z;}p_~Wd5@R-cp*1iyLB!UDhfZ!;MPJR6I^L`H6cuHKtP>)uf$NcHEL~Fv+^QeYqZ^ zpO-giMfG=5C$W^Z-<#Q~ndNPGE}fLpF*|Z1n_4q1Abo+8JmorHC1>%7y7lSSA>6B< zrLT1E*r;GXNp_5BRYWC!QG>RY-n}Sl0;{8G_~oWlWxSETkQHw)`Qc-!tx0lbu??1+ zS)6Ch<`Wm$*Gf}adSA9PF{Coc97$%Om}p5|RxP~y5|q`bSO@SDUWuCbO-(}UuA$Y0 z8NAXqN~@`U-6RbMWB)tO1ZO@mr|c)y`wzgnGO($k^qU9ClZo#P+izmKj?ArItL||&T7@Qg3YLDC=i7pTv0%$K zcz$RqU;|Dvm8w7STB%qO0baL?PuT$1zFgZSyCK-XBW$znqoorvk|5j&`!cp=a`NaIkC6tfC=x%fHn2w(ew9522Qn!rA6BXWNL_Cp!qdk89E$|MN^@0X2JBczs4Ckv!U3 zR=1!N=Ox~=xYU!|J<9PGEBXMe<2F~d6z`U#4j><-kVAJUxp2{+h$reG=9J|FX=8a! z5_K$(sLLG(!bsLR;8*}H?j4zHkMam4dCqy7b-AZdd*&o##Rjk9T{cB7T&WIYp^lD+7|SgvP7wE&|z{i8L$R3i7GOgYbDjT>9u(F zuS}?L7r~8_aF4XwN_M($B8IBDU-T5jf-Rqkn{sRErQ1}(WS1WD%nD+5(~V>#wRR0` z?Lm@m)ZKj~{^|)A`i_6>ZV_w0MepQoht+kI(q&YXnYQWp2CRb#o4qRes1cnBr_DfY z9s{DLMt2C&RWd1!jbJ&Y*i4sAebtAjgc~Gz#^X{bFhfvr2jkv37oF;%93<9Ly^usj z)+b`3o8XlUw%i0~uCa>=@g6pqACJ&>%19=*FXc+ELcvI6zYD@scaZcBAMCr*MCzGW zOWU}HbEF>RkN!KxlLL}qO4X@s35Q{Hhd~|TTYr&yvF`Rzk+MSUB6l z3P{WP+5`tEDdxytYTI(y*5DPEpEVK?!{tm&AA4o@X@>E|x}IpuMpkHrIK=tX)G~;6 zdc2WU4ya=_gX)Vc;=APt*ZBt=KBc&CdBMHtX z(q?6=7R9~wRo}$?*30}lkI0F8a2}g1`q9C?9;eWF_}&M0{F5R(Io|?cR9oKv9Fcog zc+1}~w4AI97ZKTgW)_U1-@mYMvIhOa!t>e4UyNk0KMTIShTC_5?LXptQWD!<K%y0Dl5$RldM`UZnD5JA9@o zTDp^X%lJ2JQ+_~v)eA^S1U{0)QHEsQFjZ8Q_0wp0NT_RX|3JtO~hq>yB*>MeRGPI znK})db`zXQqphW?|0~FI0FCz)HNa`mVUBbBjB=ZS4D}mcpF+A!{&Xub^*7O#N|@J7 z95$y0Sjl%B6q|@Edg~wbV~z!`E&2vWO;>F)M*liL%QMLH=CsF(3))6h-{o2le5;;o z3+0AdO1%kl)gPmWw9+i7wN~v56((I!L7h^VurQ@!4X!B#tPN1Gk8<6FSiqkCjVt3N zGOaF$il3TKd8QRJ>f5JMy}wW%Br53pQBYfJDP$l0P`{*bI(bi97r?;Jfl(>J)>25m$>Xa z+2m{?M<8!PG%-6hg{-W0#< zCd4nrsiLuyr`txf`;I%O5i2N~ii=AiNfCbiPH>e2!~%uLzl=j^IfiCCRo;Tqa#r4t z>X?mtI1#4ZOthBHke|OpH{2cd8%{F)n1{GX*z{6FUL&+7piMe8i{4SG zjvAANm6aK`@Ex}BmAfuAe(%p&mH9Yk;&`96c8ryhMBPY9?)gnr&)(x&^c*|CYFj zzXhL)5S^9g@m{!hW$PDa;!JNq(r*221J_?4XLGv{0dDih= zmcvX+p*!VG?izSO#BqXOAqNO)&aaFo;%@q>~N0Y zD!0>Z))ri`CukaXnadmu=dq%VB%+z5SAOws&p784e7lt7yw37IFX%(OP1b1>pA`*< zw2I=qe-M>yabQkreuY=u8$-!$J!XRZIsW$t_>gFM1lpVfJ!0V})4;KstF2t8x> z*$+?K#Ey`UUnHK+qb>YiA3zuxHIy2@SRrW~xG{3xp*;Wit1{;TMT#`{5x zupg)X_P$Yq>8#_m-$q~O3H330$Vg=meUUlsapJI6kea{q+G%D6Z$WEqCq|RWy{kp( zYvt)$34Mh8msW^A-!Lt;epnhV=4eZ$#-g-VRB9q};3hGdIA{@9EJ#eEqT?+&h-h@z z`KVGes6{zX9JEdDp#|xmr4_t?9{LN08v<7IL#s`;I)nCrm6%JcfUA7xH+lv-{e=Hy z7AMIRxI~}i+LD=Qq4$VHzVRs~En)InI!Y!{D-z(-v*5_&6)CivRQE~xEP5(N>mN~W zGidFp!ma=RV>USptSLmULqwKFZp>MYV*Sq};&bCUxs7aZU(I0~YA>nXrrvd$IE(67 zSQJQBDOd6vJ^vp|XB}5n_O)>g?Cu*U_SuK9d(^SVv11&&+tE>H%yI0oyE{;9vA}K- zMO0Ko1r=>y3#9mLa)x*u19}JQtsfD?5K$U*vBw+YOH+#$83Do4?UNoA=#fB+#l+((P1+5 zF-%UlENZg?+RI--zw4-2{lFg;&Qkz9DWql43veH1@(!Kw3wE)IsN@V-m(Drlfkm%^ zHAPUbQiWe!hFXgifEcf+D@%eucw^)F#8&D|ta|cXT*XH!-x8_*TEpGj&O5KjrB32A z%~8lzw2Lz+htoO3QNmzNR=~=RfNv>u{uUB7?FWhM>QEK0;1d^9!4k}UNX0s)qd&x8 zP4Ot1Td>X2d`}7SGydHmkN+4%8Uafe^cnTSx=Yyy5m|h~&we3Rs?Am0C;R*eobljw zFtO5oe0@*wIgDs4m=3|iXgXBrDGCFUrr zLDbMz7tMJd9ODDB~ zuCklzM{7<+Q6ix8+_8g1Lxbowc!)CZPsH?#b{plnHyNwO%r3L`O>3fk)w`0fdBQH4 zpNLHx4@PFb|Q8`dpv{2@`!h`#Y@Lyl->klsC>*` zxQ<$IkX?{|h?!(U|&~W zuorw+`p?J89%Rnm%Y4df_GbM?T}`9^KR%^)*NJi~L2QlOcMIaOPGs|w*+ao@xT6MN zAtTcpJ$Z4?6xpuSt5`cDzva3Xv)W@g)?*dThqpZ7wS(2Wi8b>KmaQi}zuem^ApdbC9R1K8CUce?P+>8UJlAd7GIM{0mSP%F{}^o~cP zj>PV+gKyQrt2EHS>PvEm1%1KGAHv43W6M@GY%dfKD;IMcy|gR$B$Li-YZ^>D{`w2M zbH?c^Rr+b@C#SfRN#xU``1J^W%hmpo>&?YIx1!GDpw=t6$_wI`+4zz6_Cz>jAercP z#N#h`9S@R3p$44h-iF&=frr_)Vp=@sd=dV35F`nJCC1`$mchq-@IfXz%zDuN2&*VV z*-bS3YUn z?YS)NHdry}sJjDS?MGzPiIsMZXV(NZW&jlwAv~j&Ab)q5=Q-|13vwczV4IsA1?dkO zp=Zi2W^bysGL1_pD%o@d_)^*83$|p5fSf+)o8*;TP<;>b{sQV=u8T!P3;n5Bx$Zv2 zv$fgOeUgVRGUXa`slHaemLu4y>kK{G*Qrcvq{fL`2D`Q!pN!wBc==7PA_slQ_=%YV zFO7OiH$%vO(V!-Y)y^;`q4_wL|uB4R|WPbqja$JF8v8FU$ZL?qPQi znIGbBvtb2O@Xkq85cL4<=h0tT-0-EUp$yj>i2|I!-R>)vkXha;x;qbPe#T2;^KT-S zmDUcg`j(vO)|@$M^QbzDB|Ec%IH(AoeT?u`ZZjLds76Jvyr%Uw*2%`~0WeF}Qfw%+ zg+Y?~|9?!R3!pZtQ9G(xOuE!M82>6O(42>o2^(N;L-$^9TteF^C#HanVW7l0B?4yt zGuCAF>sbt;BoXm7e9c=}L2=fbwWp^qdFNHEA1eFX#{YkLSCA zoM{wm{wJ&;k~NkN>W=2VeZ;!IgF;KNaUbdp%HuVj5DQy-^F2ou{Da@R#l4MXWfo%% zFTv~G=Cu`DSyN$elcjyBSiQIP)lVSLZ`hhu;eUay!VfYTshy|+AES4~QGs>x9{w|q-Bo6i#c1!irTVb9 z%L9(h92x2?YI=vD;Z#N$$Oq~!1*1YhzZdA$jXB#p{MK!9Ll^nQvew!_kDUhVcn^lW z!{e{we#XG5?(@mnpt_Y+xy3yhfxUl3UyWn!wS$e+vaf)RlqM#6%{%M){&HYN3g4L* zX84IHCYs|rKA()bl>lNa=2sF>Q6lxW)TQL5m&aAB!#)ci@eZ;4dLqcM7rzrj{yG?> zO2h9Q%+X-yvO>IhIt9pTZ6*uv1Co2}m1B)~y-l{XY zrInRF2O2lQ3-ks_?%|Kxz;cK1J`)>27!RuKu8Ynd_dKS#M;gNIvfDKBxlN4uOj>x0 z7Pk)nJdWiwV0 z3@z0tXa*@k)V8@eFsKv~*FU+)GCx+X`G9SJ>)nWsM z!`_d0a};)$!qXXmu6~*M)qje)tm7t8 z&qcJ90%Xm?nIn6LSalbZu{TrS>gT9|-k3}IMPGuFQl8`Z|^UyE_{jiNfofb);rqaC12P=uF?nIe#R+Na8&AW1)G3i=#c6?o^&sMPid* zm=e$zJPE-bt=aOE=)`%5<$i#-?8VX_p+hDqcUkT2K!cx&h6+)$RE*uG&Vah{s1OTb z<9Y3~h(x}id6tD&&xQjOv`-@vnM_WzFP}d`<-h~S(D+1vs(VU4jH?o|)D+*_Mdk3!K zC|4avgmR0OtlQ_aHe%=$9?mpTCp}F^**$6)y>f@?m`i7Uek8x(rtKHE*ux@9)OI|j zqRE%sSqIj@7WjNMFfE@b2BxJF0oCRgL%H@S;-H1Fo!rzXPZs$>o(weG%`lA&l#rJw zBnROkG4O|HFs^;vjR>+`cR6}t%SEUtNI|7pfsgBq0`nc-V)ZXqV<&}g=-#j4ONHQy zaaj6WP$J!y!Z$y&Jpz9$z4R1lQ-jxk+lRu_3OJlxT}}Hj&hrs`;1wF|X6{LpEsT46 z1W$1i_L!)6;bj_A<9Crr^Ela`!#O#!z8s^t|Ndx4b2-nBT<<5C?H4@qpG0;ptby9h zsmsGWgiyRsIqj~#ll_9T$jDqHFVo5~5e3!T`58sgFnfCz_qe6rHm10BCwKE&e1s*g zrsrrO=-~sBG#3|PaZPd<@9!cEKh=e-VsGXJ-2rWy(tTM~3leiYr&KClB~mob68MUuo4GMH1CzZ>o5>iNLj=DyA&Kp$xCG@oln<$XDr$pa0{ z?#5{IfElRyGH>JnyY=m7idt9Wp|M@GwImW>XYdV$h&hiLuN<+e-5Bl&P+#E1=BOuOaWm2N!>N(JM6XaceUWibsVHWU zIlW7N%_@}Y40b^7OT6?A=^}HT3QOkICv7CbAU3 zw%-`rl%4uYW23g!C});n2E`WlVfGPl!^`;nf<&fO@WpZ9Nj2_&h9h~Rf2Wi8rxY#Y|@Ji&NP(R-6#*%^z7Q4&qmf*9I;X{tl$nAgwLqZui<6~ ziRltSt#}afFnvj{VQW^e^hCH~6x^m2HhX~laxqjvU#xO9wi6D%1;cRu=2-`zCv*U# z%JW>#)6sABwydHurxiQ1OwsO%cw*iI!i(OD8Pq=ypr`BvRY@;ILq|Ti&@r;iz0q1v zlZ$_0ze*)_WAbr@sP%0N`}$5CVW}v;gBV+>TK~-CtKm$+`bwo)EQsMv%u)gF5fAcQ z<0{g~-bSP7Msa^!(a$r`Ko-FC3gobw;viVO-WAr7n=I7Oy!r zk=&*HRGno}U1047atnre6~%ZA>u9C@UsR*RXkQi4pti!4BG7MRxrc+`7+x@ti`+K> zSMh_N1fp@ZMrpYM)}01t+{qtZ;{7`y*kX>wQH}010p=9=)|70Ni#1t}II%i9a6Esl zqs4MoPhmk*sdM`z$~(F+i`B~i-e-TkC9sP?d%Sv9tl)JD%0M9avj@F5w{aZBW*wYr z7Hs(e>#dPE57Kr6Pv(OyPk8#h#3!z(E9+zl(*kO-!^l#(T_0jTaoeV>qC+{4Gm6Nf z3%zN#cy`UnsQ%7G?>{{rx>fgR;aQ1H^Ip?O`$wLVsoDW~UB-!#W?Qgjh#6vb)3zBW z{YRz_4bzf$#ArVtzgI(JZ3(aUI3Poi%7~1J30(MEFsn~ zXW%0-!ZU@8lYI-lFYIvx7b`pYR#*Xu9X|h6PMYJ)Q8o0OYgJ2zx^X*GyS8TPnY)wD) zIHRcA!I&+Rlx-qGHo>Ah!5#+^r-Z;6@}RocAntwxuV}=%_2KDHC!14)oaSP#cRBjT zT|9mW_Fk2k(aO@ECSP^~9-fCea+zdoy-)+%if?>dZZz0wSacy)_*Xda5bj1%Z1uMB zk7F>?8zNcBuJBb4EW!_)ddW%-hnpo+rIrunKR5Qfi^%;ZvDhoRll-}pzUY=!h?^Yt z#&qdcw+|*}DQ5p2Jp2PaX8`Me5nO0Dekv@-dw!(aq!GSyHHvOHTq`e}nF11rfGw78JOU4U5u|znmR{y5Hzz}P9VWJ%_scS|^OacX zd`;dzHc|Qb?{}>$D$r;+YpYKpz{6PC6XvOb& zrzlwQmDuDf*c%8|ys@nZVJZ->{XpUQS^Gw0WI6C%Sm{mIco=LEAgXm@*%Em=_%UrPN9My5w zPOKx*X>Sqj2BQ3o0wtp1ldt%#P#8`kRRlN4gTLV1JlH4ntKN{@P&(IQXdgKfhn5Vt ziQ;beM{fxMJp!oh+~RnEGC7UCgwm<7^&wlao2qzr>7`q_0(!+$;+H1i@KdH+6k_s5 zB9(hqG&BLUzsr;A0k(`JTNDLi4+5nZ5Q${lH^7CqI@$M;dfgK2vDJ&6Ci+m3QlFmO z&CI0k<=&DhE9X55(V4i*e5*}mulBFnBqn}8)yBz#a*LiIlVy}XkGXB<)avAYE-5j> z$3T~)@?;>rJT;9~iW~L(E7T%JEc=^Grg}4*NqU=6;eO!Ly==pXdKS?a_K_Zt(PVLV z!|uN6-u7UvzgU3pivvjt@Xju38*+(VFzR@HG}%Uar(okhh%#b9n?rE;B2*P-D%s?c z;!v$;)5{z$7BP)58MJ>5&shfNPX&?EQG=IL-!(?ew>RTHc!4-wbvc!U1IUpnbRf>7`^h?fM~|9oL^*fqr;YQ@MS7xf3sroS zv4X1Ci7>_nZ3O+=HTmxm9t(>=QMR;=EGt6NSci#~O7 zb}t3{$QoXgi7dT!DqmW5*38|+o_nGdtc}aq zdjh=TDfch{t(dvZj#~Kr#b9X-;wEcaOC&h_Ouxc6=c6L}AQNkf*z18RL#W=roudyM zbkZ$=NuK2lqp|4LoXIvS;zGIS16a?yiHshDEmjxdR1oYj_qi6{c_+1PF+96=*wGG9 z?By`jh9BWX4Z{?4&;DNVcBd>E} z7MK-GGM&#H;@A_6D7 z%?v!nAnIOnqbz0OUn)^eU4*Nf#l4EOU596$MB6!uqB01U;!CdT3^DRZSj}~EQddxB z&Ve&wAYK3pb}C5z!Il8F6avj2Q?vcImRIbsx6~St<{Ux1Tzn+{R%lRfGPAY+$ioVKUVj*bZbg$!aROWM!bN{B-G*_C9^sVxx zY^kr3kK|Z=zKoZ*weB*I8N7eUV@z(WE(@3yRh#j_Sg9N*x01ok*kjClc~8c?oAR?* zYs^xk#8c)e?Gs^!hx3^JMLZ;y$s;D(>XAp7!mc-wc&s_3nwKQ)A%a?~otp!$(a)&vmp1aZv~&iMvbmW_{FjE4#% z7APQkf>T$h$zR2}`D3Al$Q=w~C2wN7&_u&p?G<1JRj|z<_Jimql9)du%u|kr>m8OzI zoJfD!5!Bj8#5GNMI(xXTQ20p&R_tW5c-Oekby(Y1SW(vOsYZC!lUT$%BG$h+lk>z} zKj28iz&$TA!tdc!Er>Xt;jb%lw$@bga6Vs}j_igY?G6+hOADPrRwxDT?M^gZ0X)eB zpFNqqkP4d`M15}+^ostxzZWFz3BM}FUstkDYSKx)0rcp}^_oo8yPQ)Ga{-NO2KjCe zq48H&5pDAZF*3-)zk*k+;B1?dsm~+w5gC0UqKV}UD-!P{qLAOje(!)nPti^np_f** z2k}JKv3q7^xQ8{lrG;J5KJyI@5Th1~ZXVdtdwwy^wwq_IVwFXS@&eGPufU>vb0#^Hp%BEY*4+^kj!OyHow`xIoWWJF-QeP>;hw(Y@qrtf@^msW9Kj>m=BVKf3N3 zR{IaSgf{a2)|{&T*Y`EjkQ@W*F2dUxm39&)){_EI=tnrQq(wYlvRDG$gQdbfGzu+zY`6$W41|Q z`j@F4C3|v@oo-aPcLn^>6fy`a&`%<{hb8f4g<)UZd7_3)?p^}rv(@3Z9q4T-!aA8| zn+jHp2a!VIFd(h;Awn(FkxgH_vVpa{xZ;7aY~d^~_}k zLM_?YnWk+qBB{+@E{e)xrp=8#Z{0mTKag3@FM~ifPx0Mekmqudj_Kk|cYK5Wtz|xW zxcuKS7(CAwWgLak1LxWXD3|CT?4%sPr$^xTk8+>W;LNuB7}lth)hPM|KV~IfEs>qLOnQ-Y{J0 zB7|IGOmMsJ@iQ89U6Xxbn33}x#U>ZgbY12re`L*{CgwT@7FEXXis4lUgLO5D5o(CM z_C|Q>8*q$ly#V;RLKI+%<_yEnz7tkC0JQX@0@j+L9YNkWMaTzSh09S3|MsB-}ssCFU(+wV#jYBK9*4SC@PFtG?2LQm>gN{Qm+roWPLUJaM{ zNEOXXe*HLprWH)Xn*VkK8&9;I;+-UT!dqhYWoR{Fwyk_Jmfq(ls4GcS^mXEx8w@!Q z$M#_6Od1ty`88__O*^e5kwPSR@|@g|)nT-VUu;Q^yC8Fko3e5~pu5(A^E}~-j-o#K zqqhtJJ+{K!4ujP-xX;V!89Yohbb{CUQHi6#lci`ZXTjwAoYfhkq36USR=w_3aL=8Z z`TTVJWl)PXk;>aF>VHaT&&Yc348(o)#VH}GmQnGtXMTPnXwlC|uIwe}@SN+AAiEvn$Z{vb=T z2^&18SCZe{{M;INO!B;^JJ400Xlr392bpA)C70^=vAnlLL!olDrb=JxJu1oRvI6rM z56OmFEm_v=ud)G=%uSZ}iLsN&=!|iS`Qaa^U-c9Vsim0@>#oYa_B*IcDWFZnn%1nS~Nn*6s!$uBtiBYi`9Hmzj>aipd^NU@uWbUcOW`9p(-s@+;r$ zDMUb-#0xb!tGDVs@lLNR_n3FoVY;_0MNHI_%uyY*zPDtT9*F$TidqMwH_y92J>Ayv z37uyzGH*EiA>|adPyh=$$$ieCKI*?5wKj$-ozp};x3H0ZTvrS_m@88Y zwqpBv6o36HT|XB&|0#|>bZYEz{Dh)@4^&JcYpLK3@8HuP>BCX{RPojMhu+HwP{#8$XRI@-ld?m!}XLLpdOfBeWz z?o46gmwDLfGRF#SDpdpqPiHfT^qg;Y@R=L1olKPNi(q36O2R_s$sfj(<|7Zd5x=t& zep|xc8O!yB5By5^!DF)S?rIng_YQHZ7_B(_oO(}He6uufANdu$t!)~dF>!_ zF^P=6lc7GxF@k!PSe}9p*kW}e+(gsc#hr|@1%bMrSZkWz2DLdaSa*YGRFLmaLpOfI zH5Rtpv@bkyU)V?*y#OCUg9_k=fD2{l3sI2VvCt$ql?(s-l{=%W`{CzK)sK7SP$j+5|in!o- z1Tzu#Nj#A{c%@{04M>tgr{Nt?;sDia=~Qu?gt1JgXZa;e<~gqesGiNsj(97Gm1dFU zNrmU^0d4-Hi#Z<_eURFQb^P@+(}xPdXWCnYo*^rk<@8--ocA#sZHMft)nMNrsSSY1glku1W3#>} z$POL5^u5x>9IvGt+n6qUlz!g|$}#=C@tO+z2F!ANqB@LGb_E9IO%*~hxkYp|JxzDL zvD_*TkbP#p0er@ew%=RpjPksnIASdI!!GPpyA>6uGy2SJ>QonytF>kz-=n|g9@S`T zsit_S|3xmKJz0`^@chHh9ePn2jmA2Jloh~EL$Q}4Ttyx7LDoc+3Lxh%*q1d;s5mw?CC3IE;b*Oyu}`oa z6VE<^YW6Cq&5_){aO^q_FZMGl`6~YUlRm@w51N&)R#MKB%oIWC+716(Zge6yt2J3JQy3lp7q6c31Fp8YD zf5#vq{bgtx=cv8;8GG@?hu-FOZPxP!kmoGZQEn(F`Fsl@m;HzRmLMlvR5T$=E6KbU#)3A%WRhTCm$30pu$0Qwd=|!Xw~%#uf=yX4)m>2?lXNGlOFr({BGB9Fsk}@rN(Ca9%S6*5tc?vYlPjEI5WHm) zXm*#L=!Y=9y}bT}{u~LXI>C2Z{p=55UvVhy105dvdiw4!=qHSLCh$x}OS}q)_okb* zqS(Pse2L-)*pfk=MmS2^U#yvDVj-`SI7WjZK6Dpm6Nzu(9dC9<%jO*~_{>gFs30}B zK1{*A@7Pam!yu67HB}sW$W}Xfrc*>5nPOM-it*Vz=k6;enN8ht=|g0&yiBBXO>WkA z%OCPL=3Un?sg)qtTt~kQ+RWE_(bfCAR$H!;rZyDqrL}G&Thv0dHrtysMHN2zRDUk* z=0k0`FgXlb zsVkV7|B!fRDf&PWaNdvOTfEN%yxyLS_O?O$-FD(gJK zxt5+yKa{RLGR@)CZN?a-npiGRI}T85ybc_*^ed~w?kHz+4F)oS*Iwu$ck!*G;a7*T z0&92pTKsh<)hU;WU#@}V>&fO^BEM6LEQUX9=Q5tX4`&{Zf|N>@XC77=2`jT0*9@#U z4v(LWy$r)bt*ZNYta2pR*cJ|1j;^8;+@H-k6}-NnQy5s6Nq&3>SA7*+ZHpC05{bBo zpHLASh*(E0W|ltZ$&MoL=8ny-1XXU)%Q;ioLmkKgY9M>lZF)6l2F_g+upJ=b-kgf! zv*d7Jb1jwlW(y^vL74U2`Dh}z_h8o!aKHm@Qyd*;8|x%D+5Bi&$775HPk7l|r9Eo~v*w zqIK;D9MZ1E!RTHR`MdBK_V<^GgG8B3!QU>(lt_B1%JIsq*j)^RX%2Gj3F+Y!E@<@c2t;IA67rqHT2tbafDcj zX(5in`X~1Ih(Wcxl%uX(1z{}S5(kppBU@x6*AtAlT14!SiLakT7t$HJo9%QaKIdqu zH`$^i3UIjiN=~PW8KnO%&$(qPDdLR`R9CZy*9xr~G0p<*ccPvs;+=D{fxbdUOE3M6 zENE6`SLlawlGa6fQH5JiE|tx-e^Gv!m}T}h2a7J|ujUNlVK!q&o;qfIreQaw_qma? zhqerCsf&7F%I-zQY&Uz9(qA+om(kZ4CAONKOuCBAf6RZi)9lU`go=`%OnOTq?saxA zYNfW2+5E^1lfOWi!OUHMLO$Yx@&e^BoqS4ta>qx=d%6(+PJjdV5u2RLn4R)QnZp#m zHqHd{Yd<;CsG}Q8w)baHYa~9dI`{SzGj;cfA8Tv4I!1Y z4yF1W+37C!%2qaqYPV%rkUKp2C@O0PKC3U+conP6Mdqaet08$;^^!ykw_@GPZs4LNbZllKJ6Fu4W;{efI2n==snZcJJWJm6{A9xkVH#>-` zZctwu&F_xk++Kp=R(IhF7*P_ao=hFfHll{NOT#d#S`^dooFB6mY;2jTuT(6 ze+ODF<654<8Qu`bZ2*@_F?ljC+2jJ6LN>u&s|bhuRVzcBl@4c9(A)}wGogHEI@SAM zQFXrKUvh&#nXtk2U`rTEuf?@m^KBaOOD1~kCSHfZ)*f;vU&CHvQEQ*UuTnvwk|=?Z zuoPD<20RJksnthA&P!Cn9=k+pd9?3jxZ=RlXIy&#+EEWsvX zfocCw#YO5&@|jy@m|LoQS*D^T7~`<7-E^6OlC)q{$^uyn9$8COrkp^i<&K|crR)Q z#|aoq5b@3kV(J!(H7nx@dge-|`!<(lj3RD6rXL71+zeASx&qfSQ?a9{MDC_DwNKBq zIXvI`+~>aZ{)FT4+wydL@VO~4@ETzIaTMUZ;y78S!r~wJ-cULe%F$;uf?C})uTZiqe9 z`P>%%@R=3z8Nc02ZgvcUH%8M}mGS>aSGZr6@t^X9DYC;=)tDmhY0YGc8IFJdNdEU3 zUid96H@7I~ctMsd3Hz$d`Io|LkKtMGCQca%dTgX>E*hVHfVK3P&&29JOcV*A3!^j2 z$#_Q#eld`Ba)7GT@>*-5DSpggiE%DRX~|1J=~`nEQNL4^K%GoCyv*+IHjn9Io;>At z8a2u4GWaQ{YRi*M#46&!RJ?ft-?0m1yNlJ&0vRq~iTB~*t?-5qxXLk86 z2n%|RuU`O9^Cx%t4lleLd!NAf-)6#I0LbtM%IyK(N#pfGp7IH@+vQ+W=V4GO^vihg zIxji&GE~rQp*pU*vn#69Y3dN3Q)70Qi0nFbtp&*?%|Y`Ig^d&fy+^Q4tj_KfkYNr; z{)l%9i&*|@p@%g;ybu(PG-_HJ(o>m7(}d1@1lTs2Ct zD~G?w06mDh(S=S!gc#i&?U^1H?&wWlOf&V6a5aZ&4p3#3Ru?20r!|n<mv_HZ*2%UXa}?4bd%{>=3MhP(ZlRw_F=z)n&u?!oUvO5!xMJkQ+6mOwL|er zYZRVrrH7c}{K~Zct7tgi0=IO59YPzG%>|&WxnPr3K?Ii51hu%)RUQ5TLqi) zAZM4Yt`*jvTL)w@>ZyhkFJ!{;SF@8~ER|aG=?FN7@|lZW!2Tgh=t0GoN_S@Kf6_w1@c01#~N=+0yWEQCYui>q< z7r(IA(of7qSp*{*WBkU>jI-tcj*r!BW0EXN)m(Gq75qn&U(|Q(!Zd^#(Ge)0spQ1l z*-iANj`m@&j#FsbNnmUfR&^rBlA<7$T?P^Tc-ZoBYDMB<4t0p2qw#cuncBTwog^|~ z@7BDgzNk0e_@<$v5|cAK89(fqoT(Sug4M0;LPqjC-mtxBXw|*Rr|5N>agG&hRhwAs zVrmYaOu~;JrM6jxOMCI10sP_s@MJw&N&w%Z!ueMap`7IP&%8g1t5`>T5vuIP_Txd7 zKZ!xCsgm!>VVwtmLRtSCU;>F8KVaQmh!iZA5(3-z=1Onk-=nadqwtwH`gy;>+(xqU z55S8a@y!<$A7YtQReEFy9QlQL zup|2SxyE2rn#UmFMbIFQcOtpN_d)Dvbg)M-!*Sf_+C*5ns8(`;9nI#sJBh*y!&N@n z@@lW(GZ*=Wa%A3P$exeoJIj(&i>7+#IA=SR80QUrJ=Z`IAF$;snb2U^$a#*@Xuo$+ zIITD+lPuCzYx)^D#*uBs)He?lMgKW(;<_&;H1xq_tP_P1%;+%{I%HS~s~<*4JheF*P6#Iwl8c zeW{@9pdXgC%&P2$_NTcB9k>e&WRl6QLagat@-CX>RI%7Ll#EOT;-GTG$P3VUKB&12 zQ=S!l%zEZD_OhRAE;b^~4dyxVkJ-}PNapL1NP&+%5EY0}9*Iw69KMsM=&EiLgIV9< z#y00(ruQF2@%w>daE+>!CCnlS#ot|}d+`LmH6K+XSICo$AQ$k5sKQ)`qvT(9g1YS- z_thA&+ddV&bdmixI72wO+Q;HwWebXEVf`0b*1f3qyZBocqu25$vxXLC#F&GqEe|n< zGTEt!v02R$mCgShduyd-9N%Oyu(mSp$2)8S2c*!uMdRsx<*}J6Uxm;nNp6&`7tx zSMQ;CTRSd#qxf0f+|e+Qt<>#5ATOVdw$Pg1hZC@l>O>O@P=F5*%k(84UjuGjfPtid z3=7~+0o<$2V9pwFAb?|A&}JE`ls^?muIgS8TTrhx2KMEJf^ichvGxhw%=aHrHW8D? zDCcsjTrR`426FyZN839x&Q&?93+Rv!(PXA!5%wM+{HN%noqLtmA z2J7%g9gRkb_2t)Z^L|md&~~a3E~2GK&T0>y-KrNF4#N0=Fdy`W)TMZ$Y;Ik# z!HxXX4^(Fo)Dv{(wy=~T#6%gawD06`YJp=9Idki%h_5IuDyS9B=MDQUUgsjN&n=Sq z-4vp>FL0#$VAOMdWg=+&1pJI3Mw`Pr$wL;KT4h@noFp#_kA)r;QEv)y=5f3q3i^E~ z2N%kf{>m{IHH1&#mPdIV<#1)M!=14Acao2l|;jIYeD9KN@(iXDwa%*h_I?Q=(AiQN7Gh0Tm z<5+#OFB(gGvzJ*(IM7eV5kEYX$LUDA!d*Jc4kvZ&k%~r6ya+tMjW{T?WxV*+{N2oF z%rU#0)r~dgSaTDcr;gd1`I*y1X=)q(5bx;I|3>wuLcHt-=Wii0>}$Z-)69dN%9C2i zv)F^b&LG25h81gJ%YCBk68M_KthaaAj#ZcS2+PXLQ!55Tu#?SkP%D)W&COwV;OUOQ z7zfdbvlL!5k4np6k%eb*rMi{feT+fMF>!}|&HmCOsKPBowLmQUN^fLGSL;|7Odn}B zXHrc?X);ftu`KGyBA+#snKN&&3s-DmJy!e%9pDb{%&>=%3+qRGngzD}ga_Y=s*s9h zjled~;+5W$L+OsEE{>L%r5~aT?=&4=GpL{ZjUJ*l#z!nJJ4a=Gq|YT1JVosA0L0x* zo;nEsK7o~Xo|rwHit#?+h7Y-tTUcvr7|4A1fVHoOrXJT9!wLf63H897by#IMzdMjm zE@B$LACX8MCYWs|9=S;-I!rmuyo)o6H+lb4;MiZ(6rJF|itr5qC}YpD>;C+10>79} zUZXvD;!ms*gI)L&-#x~rzUj|xm$8UDwlFeEhrpIQc!cttVJZ7h=nsk14=2Oe$8yz| zn7Hx^oT&<`Ea6uU(!cY8_dny=^NM{u({#NU^(WRusXf#T?}qbO`xSnq#x)4tHC>;@ zo*mOco^@zz*1qj2WNjv4^?wsJ|0X7I6%Y6oC*Qf2Zq1hx1sJw znbrB1{6df6Sv1l_P%+c?95iqNe@yBJf>?FoC`+HY#=}IoR<+3wDtniM8#Z*6zM?*v zqpo!HWWid8FiUD0-3QfBR;!^%4z^?nGKR`(-xqs^wSRV+DCgMqflUcseAF%&r1(8Sl7*EW;3zK z9BFnI3(XN`5z*4@KtB41{MGD5J;76BiK95@{JXt6HTGw~=KlHiwzqHXJ+5Vq4klL_4 z+7No`hiXgVy&LJlPsJh%u)50fEKZ}ErjYyil~r*Hzh6p(@w`^D@2$7i-Uy>EWv%E- zO`6HGwrbk<((iddY-BgI+suozG5Ir`wXsEvc9hf~iXln@6WZjb0Jw6klPjdFvj);?L7(1y=gPR>E zU-nM93_3N22^2)vN@g7vz`I%KVRbvYVWHNpJ-&SN7j%q&z&amzi;*Y-GDL3W z5K%*fIo3Vh0yS|U49ykF&|vw0w~?b_|;jC zSYwVehza?X(OCYI9(pu%(6JVJe~fgV1K?xYm@5PxFs_X4_ZHxPCH@7R%9{lChEWaWZH zvNC`ya8>=8@!T9w73>C?O~&~R{?`RtI7PQYIJ4rfs$R@_enK7QZ_Fd=htgOYeeXTG zv@ZHnn-R?ZAk*O?|H0cop@fvfYL0OgmzXMV&2n7ojACL*OLaRQHn-gsZ)NSr7SqBG* zL^|elto+Jx4pHYmqM(IK$n zDX2abU%8LJ-sivivhtsy1JL^q+YE%oRVLmUMSr5T8~c1XTQr#_y4vkg)Qw+sKGN?S zr?B@saGvELPZo1%HE|tu%@0e-LR*U_r{jvHp9O8Mq8GiS+t&wn^Z=#NA9ZUf>XtPv z`IG*OV;a3$&8S!3hK)BS0$R_VbVvE0#6AB253C^`5@(fUAK1C{mqno7yd(PP$8Q}6 z$E+E@9l`m6bTzt(Z=8Qsyx0oV;9T}=AW44uplkCiKGAj96|JT@YtPWS^8S6k`5p0K z3(zB!zjk0vPGG&Agga${Q7u8a$zVw?oDv+Nt zb!41#qdwYrt+th`iGz};dl{sq%UJ2AMZsQn@w%A#o8G}3%+Bu%$q>EOJCb*vEBaC8 z(pUF3wo2(71HWSLNTs+~#TuQ<#H1I})9i1YFlU-oja?=?{)p{x9^D8v=bK;177a7Y zpw6^0&cPxybdvm9TVjD8SoahA1!8AwmW~rfIt1K)jNd81>&|e|PIO}xA$Q+WtIMpL z4cb3={43g8(5AcgH_ynE6?F(TxFh{pU5OkAFhS)O(XNfisHVfgER^-iT=9h6Njm5? zWw0X+lpg1>_6TzlV_sxx;wDzeEvog>^xvpUJi}R8#}m$v)V@wfC7Xa2R2r?gy#581 zW*U1zo;NbE#DYZk^jBeJ#jrBVvnQft{saPa<36_sEd$lRMFKc8npN#e%>D;8X;Ii+ z0DV#mi3noBgxak3JakZ6a~+e=uEWraJn0g0vX%}L#azeBdx8c#nZ6Q84450O;{-a! zbt3V4c(=Kz8Am{&d{}flROR>Bcy0K}23vD-K>vU$$Iwqosk=cNYns_)a=1Pqz-N@( zp{Of6U@VPc5qmkq6U0I7`QLLyL&wo<|Hd<302v;zk}Z_j2m87Jrj6qq0`a+zVKTRg zxjxvePXD>Eo%7_L-ovAoaZhg1KX?wi{z}L0L?*A!Bir7DNod2w9io^w#3RR0MFPOd zLh#raxZer>>m7Z^>)|1V>623UWG`&(7FtT6UI`B}jyka^D5d{0i{rNb%^ARClELZ# z;VVy5%{y1LAx`(>N}s`+8nP2>Am6EgQ3J`xZ6KlAY(-zA^}>*5l1uxa`aTR7~uI$H4nu^DfYKs4GObc!K^=1I2bc8%Z^4Rhc$_a(<$XbuvH0NyzAcKp*cM(t11vWb~eM`~ZGlVRq9iD$nETh%9LQ zuJ@3;WF@_hnJ(w6x*W;`IVlIw2mKI~sG;v70&1YoF>9DV>l4gs<|n5cSbOK50 z*}o|b!~v!#uKotgNesNMaU^G@s7#Xq^wXEHU z7TxpBDGIB)gW(BJo59z*Ai=_=y5q6G2QI@(E9EY%e2G)M$s4Gi}!aCy{Wr< z33Dij-?2p2j9o>)7rFN2?)W+Ge^QIfU)@<}E#y;aWO z-HKr2h3(c{xHeeFH?-xFIWbE$KEInN!w(guFxWB)W%xL1?FD5GvozN$%ZY|ADCga0IH+M83Z;Ybi~C<8Y_u{JXus zwh3;q5AZf= z6JN^tL}DpAzw#d(pa%YZCAf47WMd{62ono~O5y2jAXL&BLQqKI|n>_cIFkWB! zi55|HY3%~(!nNeHR|U@l$%y3_V~C_Wf(Cz*f0pOpVhlFlNcHV?ZM_k)?gCnhr4oq06uVwz}V9&#IN_?mxuEO%Da9~d#acN4UaOt}}t{xq3sqwm;(-34aTqp2R{G-BFv})5_>esoT%=Snb}#{R-+z z8}pfSkT_seac*S_WR?{D@nU8(se3$1sRcjaKm4G_$losy5rmS341 zUY3kkcl6tHu#`{Yra8$xD7?*{Aj|{uM75dbz1}Qs1e+7h9byQTcz574{md+JmQQ+; zu%di8feW%wpX}Vei)Q8v5Zv@>zMT=6x% zIc|Z&yhY7;3!gtr)Q|!vNmic2K=Lw8Vh$W~J5>-v@NrYAdVLK_Y$wwc$0v(Z6;m6( zwu*HaN_=61-(Epex=;M_m6&21{4EfKv9w%^30&oWKNH(cA$B+nub?7YKMi}Y4>#D2 zSM|YO&ciVV;@wt)5(i->>FBfrIk)Y48#IuG)H!cama-#isIn4_N>PXY+S-y z{IRc#WNoY}^sBa>#A3Ut{kwrh_eK$V1SgLoCp!(so(Q|XA}-piqbk@@A>Q!*AW-E5 ztmYQyuY*xhD9OqC5iF}0xsuCR(@Cr&2A1`jI=w}(xDZh8B|K*-8NeWPp0;3%55E`& zCiFp}vSNocj+3!FZ$250m&pe=%Wof#LK5n*>A8#y+a9#F4k#Yhym}jI-a@V@gnUyw zFsh3EPwtN|*__p^oKBoW0?N`8o>wGyxFf7JH~yt9=<$j(4@cRs_B)HWkAx?W#=l!# z?bW!GCG7ojuDvdwyv@A|<|!8k!zU1-9EHpEg3+yJeIDjMA5d27o5-b(67^sUi_inT ziKlX*%@zYyI&tRrsn#xu&NdNM$BL4ic*!>8$DJ^N(x^-?>A=lT*4u!&t>X+2;>AL8 zvS}}fj!$w|Z?ZOC@Vo8M!A|hJZt`@F;;pvQ0XJ3bhv%;l52al7eu#BC4L^Z$`G^)$mgup_;k%A|!~m;=d-Q1UxBP!EbzljDJ&b4ELCOp`y82`Xdm z6D!Su=1H;39A$=!05U{B#AWkebFAo1R(hyVU_O`m%wW@%-RPH_zmcb2YJS%ejGOW| z>f#oQL-aGgV9rIA!0RT9E^Om{H~F$=4+ zF%^VaYiw4U8(WMu^lQ&CBAp(hi&0hmkDS^jWea<4nA$2?%`8m?@KLFJy1 zvb?jNcxzOnryvOKXB{(P^1ei-f6!0SS*cG(sid~T=qo+d$5hkZ1`Qs92C+&Q$70ya z3%qb%R6akrPz-jRTjX(st0U=c-^P^72>ebFR!A+7#|viiCTD_BQBl#cQk{&qy{nwS zH~oNTH+RI?-4SccFYU*}sb*jTU z_MlIOf}hq-J=UHV&*2G)VDee?tH*j1$13U>ij!}CY|Mg_jHhBY0?prwJ>PTpJb5}_ z>7pu6^-C~}Wjs8;rdW=j@CoR%Zxkv$z%(e8c_yOlMX z(ZyCm2&$@9=@rS*@1^^|(uY@2A$$qecN^TE4gcIpTv-up`5TVC2K2t5%!OkY)2cG1 zDvsYZcp?qSf*Gt6FMc;JCu>|6R+EknQwx=KI~vhcI=(-F5??{r=BRC_bFx48Q2vv^ z?TK91dyc7SFKgf}SEwV3LV+2;bAJQcM{~XFVPA{w^O-q(ok?_S&?*bjX`4+C)JoLF zF60h&8wsYr`!_SreW+)o+SI6Jrh@s`iF$%zCJ|)e4>7YVR;x&b)m!x#z9SYc@d*F8 z7nbrAw0UojAy!HRZO-$Z8gr;NIDeyZwH4E+-Q|9|fZE7fMoW64N17YlHoN&%y5m{b z^OQN=t+rbo6igR$2-Pee<^w8ndz#02e(7?B<0Vm*hs~yh(wiJgO!UCnkexMB)Oe$b zEUtBsgXC4skL>nRIt4z-a-uUGj)TM~vxV7GG&gIR?6_|JVy>jmsH1rkyB=h^8&l00 zrW;yPWBC}|=pp`~SMED2IhNxt`0Y2ei21}Om8jKh#SD|3te+`7>no@Vsb~(bVY!)1 z322I62yy6okT?WaDQoOdqQp2O+4%+@cweb%6qEa%+r?yKw(29@WH9GB(JY|_%h_^~ z63VX7Gn6}GoSaBq!6D`mS;ywo57#Fq%113kQ6@8FivHSOc1|Y$>}6@@&NV{IPd8>}x&>?FHDlfU#`An_mY>B!681OSkqm%?A#2hvDxB6Tajy z7FSl?X3oUQAC+J#$RC`~9(q3?z)uF#%efirII1j$`P2YOX5i&Fa<Ip}tixbbg&1sq8c6b-X^Khu6{68e?B=i! zHl9d@Nh%dS|A7N}VFLNcxy(Xm=tpMCA5@H1eAq?Bs@5BA^9Lcvz{GZ<*!1LgS8%+* zu{SZucB0D{;EXj5cpX0F8F*EKtJ?%Gx{2=|LM)b-*fTG4q7Gxl!B~1YwpoqicKBo* z=;z?|Q1al1K=1lQ7TZz9-%}-coXJ$v=~)XQ=F)hMZ=7XBC2Bt!qh8t^+tlggtqv=N z#ZvflDlu#nefm{wGZd@#?-M9jpL%j1=IN~kPb$%)y$XK#0G#j2e_1=fXA--0gs+x_ znM@)&>khZ{07LfWWWlUyHj~Ks2j)bAFIW$cSTm8JN-_$q4F<9X)#xTQ0L{4eN+>yQ zSiyddiMA80qVqhLG!Wj3m|WnlZ;6q2vJS5i4Xptyit`lSQqkds65>YpeLuaJ%rI}6 z(eAB0UDQ-@Li-j9-6v2OV&OA4SiO6B{RUQZ zfh^boCby&z4Rv+sR8YKhdV?)Jhy_B8URs3FO`dTZ<=(^XuPSdUj`Y}GZCK@hJuiVE zS>|)4v3Oyy%dY$`=TYhzW{@KHo5Q;lnCP3bAU)KXHD zuhm|%h5V#;k!@seZI~P)v$S<`zdWEP%Op8aG^Hac6)x1>%wvo+yO~#n3Og#PRAu** z{Pqe&qtCDrSCrN2#FDpQ@`X`F&J(K^C-dt;c5f|qI))ur_h5Zi@4q#FH64DN0h76e zUnoP?X()3V>rgf4%?@riMOW!Yh3yuJ0*I#6R7)mfwM+|;rOg-WD%o46tNmpqxkfq8 zEQlKR;duQ29nVpT@sb?n80`Hi%zw9NM$~$U{>^H}e7P4M7>gp;m|BPodTLz7OLVq( z;LJdt?;*UHH8D?vpA@lM`)=f--qZ6g^Ewo~ zwR*~ybKgS29|e?~N%eIA(cf;qIe_oULELv3DZ#Nzb`sA@)=pBTnfj{|Tluh} zGIR21an!U}yTV;1B$|yE`BfBqsN_GMthEdSb7+EavsLA3si{#VUAOM>L1`pfVo)-;#t#c?fes7uL@`z5QA5_lVQ!|-+`Aqcy zRgS6UK^UhRYNRl`yAxeGUDO6f0Xb0HZ;X@s=%c(U{}m<3I-S<07>nf!dv~%b!|cRc}(i+^pCSS^P^_*A%eRj_3@Q3gA)^F*>( z56Q$+=a@h^ljw4bcx^mXyp4i#t(tDkB+uS}>b?S6KRUtptE#LgpQ-|EY3*!8chVic z?L9VS9Zyri@`4rE8Q)iyYU)bjg*sd`l)o@N`G;}XUWIywNicyoWMJF#yXWw*kMN|{ zF2F}nLI1a}VhaANCDykC)qR_YWB;i~Mi;c5&SE;4KvvprtyO;%<9P7^o%ShKZ-<#I zCvp$M>MLSR6Nr^gU{ju;!9w`P8PG!m0hVw)#Om#Uy>GX9!|eybVYb6YDr2D&unRBh z;=U^fShrT*X9xH5I9_rjmTOH2---5En9mHrBhLe?tH2pFjWI~hBG&Z9~-hhGR&dKTAB^Dd1 ztVd7w!OM>&zS|5!4#|nys)2-asl%}*g6Bb4+y?c5-X9%lH>{>McRm)}%LJWLvE2H$*2)&8}PH}w3 zaWDC3tE1$D62j-d!(pr)gztdlqtUYF!gg-+TLx!xfU^zftON{gI|%%oySj{T=x7@Z zsWx7dSML0B@_p8viZoED9n7Q-mU0)5{4;8j zJ4Z!pguNGpvHa@zjcLPm^p{2#bF4h+*2AM2)wVs&p;UV8zz+tpdM^@T)hDZU*ve~A zUssbF-2!Af1~A>!h3O4HsMGMLHtC4CO8vkg;-hesLqB-SK5C5~f-pg_nq%PDYvLw1 z$1hA(E#rKumo~XYO<-UtR6DbnFCM| zl_MXqaVhxDuCW=|EMg#cO&`?m73Iy>BFactt5fwN%mas6ImtrRE`u8iEoPW2A|MAkl~+ksu;S zBqF0B5^9PdK_!R;#UqNC>wWj_`{#b{=jP_zefHVww|?ul)?S;rsz&K@*Txey4}eKF z0%w}RXQEjrYMqbfd8M(^Ua>Ya=dW6^c*Ezg^hRV7d)wD=_Qz#=3AOqw^gpbt!rRyl zPDUHE#SPig_=UaU!;Q_tU6$&Zbm{z-K0-}g-s%!psK=^K#MX|XJ@y7+vRr#Z3fO;9 zS1dwB>|iazmQRq$EpX)vYM==$~>gz4V#_B)xurzQ6hHwIarjzO5P1`sSP3xWAz5}dIlCT zhB!13WbO(7*+$Qky{_zoI%hBooBACer_fTJT6hU<_8OU^A9E`bUGd(mEU@Uy9;B@oQ4h7B)S|zjJBTF8z9wq@F$*c zUI%9;fW2!O$vH+c79ObHUqyTAfmRd6|K5O)1+iKVFshFjPYA5?D68}%&^Vo^e;u4R zSsO7hVHL5~GL(vSnW#SueTHbx7R$4q!;aB|M6ADZ<)*eD_0{12St2hARlw&AqP#oY zMPHtUFE#BM)H%3Q#c@H4!G}(!`fHT+lATZ0=I}D^X|iDIPMHdYf~c>9fQXayp&m;>7|~;z(;Ws_$g+tx?8_&&Q1i^tZfE zOyng?42{2jQ|XaIjPs4=2ZpPTmnA%%JLDgCQ$JAGN@vZyfyY<9 z95b2!cc`2Uh86he?X3r5kNv6{CH5HaqvbX*o`{d-6r-0MD$9&FsZU;FLW=tL|smdnUfSAr?0ky(N*AtortQkYlYPXWfqVwvhE4%Nkbk(HKU)n$8uc z^k3lZ<>ZD2^ST^Xa~>P-YTKyK0~ODLJj=*Qts@>W;S5{gBpbQY>EKT~I_eRyD8fj0 zF_uf%?0hWoBf6YTrRG!yWl%D0mVJ>Ne&5GFVwY)r>)4MFI-*S(v~;?ji~9q)v!XJ!ev2LFf6! z8YsQ0t6mlV)sFibL`<{|?D~jVodB;$;yLz&J&q<8i^twebU$YP0hpt9hGxT*P7rO4 z;PdS;95?!T1Pd`QO0TVyw>sbuYoEvc%2I`O=47U!H6 z5A*b5?x-&JJB(V04DNI_EGV8>tc>iH(&);mseXixZ3DyIh_deyYro6NOv2v1(E)yi zUGD?2@0c+tq2s8mcQpFc>qEm`*qpuTXOd)9+eSNrK$S&gk*aJZMQ=R(N|y5g8T?)X%+CC5~cJxVR8?+KDe%-87+&VB^mXVbkT zMJ&`kt!Z>i_>}HaqqLT2iBGWQ`j$F5w}zF&ld8o&;(qYVi^Kx#Z?gZ_u}j?7H_%h! z2KD^ohW>E!|F3b>gdF4f zHfJ4fWKEvq>G$Tf0a?qIWEC6JBchb*j!~#{_o-o+0?y=tA>OQz6x57zJXaG|@(Nb| z5txQLk7EtY;S&73D+tp-i^Q{cWB+3k`sXL~t!m=oPTjIY^XKknfDf@CfU0&=75Mwn z5)1UbSfx6<`65@o%x?V`U~Dv2@Eabm4-rjYtY#62RLclAKSO(2>hi+jSV6VE4CUbx zPjv`3z5vAQ!pa{=B}OQo-Q8G&4UQl_I!iy3EEJ@D@c$g}B%jzP43=`+b`_tS=dsd^ zC*!O}7zC2fD&(tH(I<#H=kLVQC-Qk1I{#KIZVk^dk#80=dk(nJ9C*uquu+|jvH^_B$Lj0z zTS4TuvhZB17{f+bSrSp0Fn%?=!(L{ZPIQ~aE`-l8z0e1ifG}s^7~{eGBpAyGUYGNW zYCq;sem4SdLWKvIb_ZTJ9@JI4z0y%yR1fns*6J;ytx~e0`FOO)E=BhV&)UZ4q4=p2 zJlPw1DY*R!yuEaBRbL!y?a!))V>_w21tqZXw~?sSQ5IX{vqn@m4f@liwDd*iGpaQ)QseDdXZ8 zxIKnS&cj5|XYhA#qLcK1)qJIm!^SsrYDi1@s&P;F$cDW7%N*llx&hl9KgbAjWS`3! z@~IIo2Fec9ac-4gX`d6ZmV#7w@xrY^gTG%hhyoOzG{>@Hz=uLS`{%r>o~!QGB6v|v zn9p!BM7N zh{o=o^4g8hS7FQUWPipI*{vtn{0zT82;LIHNYr_t>V(#n@PSkMMAVPT#3?Jl)y>2~ zepux*+wY9_9E?SsYBYzj?Pkmw#5G*p5Y7BAzS9eANhSJIx@8jh zS!BxxVU(I(jutVOQ4N48t;7-!!KMac(W)M}9ltn*Y-}WvQEO0f3D?;L0?Y&@Hj#lk z3WnSztMwsQm<#8rfccCiJGKQ|u4k-bo;$%PW)MH-ptO9)$OBjjF3zKal3!m76Hup8E(UeeU@zrpFX~+B zvs{_(a46MJm|Gj~4-u)=$GbI$b?5WMU$a)hipxP80b$`6P$aW#tMLo~ha;boGW>5gK(}t(vnW*h39hxoeAp z!&KWg5q05z)aBjPj60PG4g`xW6^u{)qmEf%+6nldaKV&Qq@yLQl1M zoWvIlo)n|N6q7fMqn73he5RPLj~mz@@e!zUfK~lA8K1hGF|ihuh?HB&#Y9=Yth(#w z7j(}_a;w&Dk!svQzx6Tl#Y;i{lCz<6jZ)5?--*W3P(I)uTFVm0 zV0IAKFegzx5^AosM$wb1IoJ}8nz>oh8QFSW*0JwLae7JB%^~(c?Q#rcKXnSV*)jSY zD^8rzx0BT(%fsm(wH^N?x8=qQr^Rb1Dwf=8e83LMtH#&tXgI{kePk#IStRPP`#7GiKAniFF0*2*K$VuPfxln# z$(dfkcl52O(yU0V>OBvUkgkl^`v{iu>l*5Eeu&J}A7kyoB(y{P8$5^YH z8i`;|Q%a|{=N8=lZJyV0{SYX3nqQfOU2No6R9(t=5GIlJb{7R;1paOX*wqt{76|Gl z;4!`Vj@n0E3-3IDr}HL%vD(C#1~l!BMvhxgCL5=ZiVrTyl@wSGWX zDFk7Tvb$#HP1zd~+$tpUwQd%4q#j zeX~IWRl69-7}!HVzW5BWQZHi905C8TJdwsSm{nsiEdUk~&2_rKvI3aZ9j;iikQtba z-m;!~$YY&M0k48tGpWqmau8-O*_?~$cm-fZ3h4J6tEekFUOI}|UyS(?O6CFX>#=!@ zc@{LllgMQ_aZn0#dkW7|huI6~xo@L(Xfjux#@P+qIkRRf zolbS54tnNfy~=7YcWSSi#nc1mQ%zS#)67^hR`Eux`3S#Pf@UETmN9)e=pj?V_t~a^s=5HYU(YlP*Iot;17jn_;RZ36mrmgWk;0f z7SfIV;OF?=Y|i!g3m^1?r}l4bc?+wnglF9wM*8>bZ)7@@jdbHR!WG?w@SO`}nf=I5 zHFm`XpJ@R+@wL>Ko=3r})=?YC5friZ9-<4WuD{3Gi`R+REk$!DF20E({U5Y_I~Wp* zS2_%P?~LC`;#bs3h^=`|X5ALCx>~?$Cb0{w7Zpmw=!y4~ldAd=6%J#!zCS)U0{f~1 zCQLxv*@Rtqv(iHOubuGx_Sn%D)P_`6<{Z@BIFKO~`)YwL3?T10A8Y+p_jhGCmJ@^Y z<$rHc|CUTnc>`X15BfkVK6eW5^U3p`M>!A0I?_-x3c;M2;LLd{wXWhxyWy|*5EGT6 zY2*>{-GR5fVg=!O51`+6L)-b(6`gek6_>z3a`c5DaD7l>8rHgr*ry56L^3$G2VdWt zkxwM*Ni-dxN&wMExIP`#v`}9}|Kv)Lvj>@-^&o>MBh1C(E)e@H0E^;?eUh+iU#_qW zoqsEc{S7#<7W_$Io_|0UiDR_cSb9e=KZG&t2IU^>YA`2(z0USq|uA zxtN}C;Yoe)WC7T+ivRtA>#v4mWHNh0$R#btUu1y~OSp0%*GcA+F6h#UAXOHz&pf`V zp2&V;uFt`igLsPj%%0lC8OB@{qQH3Jk5u;aGUL3$C)>eZwfn<|?>O0!WTUdKlC}B= zS*NqiKn-Tk#}d zeM;FgYF6qWxdezRP@0pu+R~Ob6^ux=hzx}QmFP4%$=_jl7 z6g1t|9ur_6-Pj}Wo7IsH_y;(($A|8e+l({xtn+txJDtuDM}chRe3$OKs^8;UIhFq2 z2V`URN)MGbXE;4!uhEa!Pvl8obF4^{waq{gEo*`;VPuY^_>`^%Tg+LaGd)%!#S4#m zoa0$)D>j~)4?T9%-zHzb3~zC8Qgb!Ts*@E)$NpgJV=>O&#(IxlxQDG(qC5GlFGN3& zWOIkr-eV!PbIDZrzeRoi1WtQ=u8$DMMTI^BkA9SrIC~3!u zN~&2`o>nfiQcxnB z?~jG&T*Kb4f)y=YmhwGo@)oS%DlDP{mcEFHVvIVeiV;NyK z8R2PIoZ4x!A2nFxJ3o=#hyh!Y&~?Yclvd$ez0g9Hg>Huf90sRVtgZS=O@-}m#@_c5 z!TDk%$FR^+be;n2=nl2Am2_ND-9<8~<5G5!4+595tG5gl=pg>LooyyaBB*AWhi0;t z$ZII`^9#|$O(LCv%+h*dB6sd$9Ak)P9>z1;dCbNRBDiNnQmx4I{({O8iq_GT%=K)t zMJMzHs74L={z?>;LNLECoMa029>&bi=9Bg>dIS)B71X9 z)d6om*}9p``7$Eh9iaLkJjEuwRT5Y5<;sB|gQ}MZs(-~pIOH(qIga}+fe9>UwHziNRLD~v!8N}|%L-?1_v#x# zvl`rQp{=%~zcs+_MnC)s#y{90cv{aDLe^jh_aqdR59k8%m#vaY;8(OVD(fEEqUq3h zkNnXJdTmv~MhdiRu$ueqQ~tnfY_liLH@1A$Gtq0R_dPM9QG$0CouI0mGpUQNvNpT4 zmz{XPD0oW+H8Az4rb{JHok6wvG;)Rg={j45$G>C@lXK|T{h0okMeOP8DFW@C&4r?- zF^7ECpZ3dGWrqEh7{?hF4P=7&*o<>VIV(hqM!t@#CcGFPv+r{^hIzw6JmF9Xk6Q69?%y!47JBTdNDNWoC2n z4m-tW;Q?dehy6JnIG6LJR+!0FJI#lT(`>D~8D#lu6U{}OTe;+m2#_I zRsB^Z9Hu{UaXBkib)f1(rdw4DSAse2@GGV9-k~l+os6ui8ra2%3U`|*qZv4$Dme@B zfEQ6*kI(V6hSDt6lRav3&#Iy@7h97IbGVr;0BZA6mp( zok9jPoK=^K^?Bkk)%jT}N3#s}p3I6pfX58reY6WzzK7dK5OFA0uo_ED;=h{lUY&6t zj_1w8c2?jkckYNP37uBvjgt~$bcL{k5%f>aSUWPSATb%;y4e+;LFuJFI>-v2X;l z`aSm&!+-7J`aMCd9A@}1m^KT1K5NSaG1QEe@>kU_-4~=yg7H28AN-h;y*!bzti}h7 rvw?M;XY?4oy-?rJy{P`BaXg)qTz@Vj??oPW`N<# literal 0 HcmV?d00001 From 637419ff94a858a3ecf5297fcc0772baedce08fc Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Mon, 24 Feb 2020 12:55:28 -0500 Subject: [PATCH 55/94] Pin jupyter-client version for comatibility with pweave Signed-off-by: Jason T. Brown --- pyrasterframes/src/main/python/setup.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pyrasterframes/src/main/python/setup.py b/pyrasterframes/src/main/python/setup.py index 26588565b..b33c10446 100644 --- a/pyrasterframes/src/main/python/setup.py +++ b/pyrasterframes/src/main/python/setup.py @@ -142,6 +142,7 @@ def dest_file(self, src_file): ipython = 'ipython==6.2.1' ipykernel = 'ipykernel==4.8.0' pweave = 'Pweave==0.30.3' +jupyter_client = 'jupyter-client<6.0' # v6 breaks pweave fiona = 'fiona==1.8.6' rasterio = 'rasterio>=1.0.0' folium = 'folium' @@ -187,6 +188,7 @@ def dest_file(self, src_file): ipython, ipykernel, pweave, + jupyter_client, fiona, rasterio, folium, From 40022093a9f18c45999c143f9d05854fec49751a Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Mon, 24 Feb 2020 15:23:54 -0500 Subject: [PATCH 56/94] Add deprecation package to requirements file The requirements file sets up the environment for pweave to run docs build Signed-off-by: Jason T. Brown --- .circleci/config.yml | 2 +- pyrasterframes/src/main/python/requirements.txt | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index e085851d1..cdb675084 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -76,7 +76,7 @@ jobs: <<: *restore_cache - run: ulimit -c unlimited -S - - run: pip3 install --quiet --user -r pyrasterframes/src/main/python/requirements.txt + - run: pip3 install --no-progress --user -r pyrasterframes/src/main/python/requirements.txt - run: command: cat /dev/null | sbt makeSite no_output_timeout: 30m diff --git a/pyrasterframes/src/main/python/requirements.txt b/pyrasterframes/src/main/python/requirements.txt index 5cadcd11d..fd7e7fac4 100644 --- a/pyrasterframes/src/main/python/requirements.txt +++ b/pyrasterframes/src/main/python/requirements.txt @@ -11,4 +11,5 @@ descartes>=1.1.0,<1.2 pytz matplotlib rtree -Pillow \ No newline at end of file +Pillow +deprecation \ No newline at end of file From cc50e63780b9127a4d7686ee27e02168fd2e933f Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Mon, 24 Feb 2020 21:47:51 -0500 Subject: [PATCH 57/94] Correct pip install progress bar flag Signed-off-by: Jason T. Brown --- .circleci/config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index cdb675084..4805a508e 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -76,7 +76,7 @@ jobs: <<: *restore_cache - run: ulimit -c unlimited -S - - run: pip3 install --no-progress --user -r pyrasterframes/src/main/python/requirements.txt + - run: pip3 install --progress-bar=off --user -r pyrasterframes/src/main/python/requirements.txt - run: command: cat /dev/null | sbt makeSite no_output_timeout: 30m From 22f349f24b4c86c8214834a04b282bc7c5ddf912 Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Tue, 25 Feb 2020 15:58:24 -0500 Subject: [PATCH 58/94] Tweak python GeotrellisTests for CI Seem to be hanging on circle CI... Signed-off-by: Jason T. Brown --- .circleci/config.yml | 4 +++- pyrasterframes/src/main/python/tests/GeotrellisTests.py | 6 +++--- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 4805a508e..cb95598d5 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -41,7 +41,9 @@ jobs: <<: *restore_cache - run: ulimit -c unlimited -S - - run: cat /dev/null | sbt -batch core/test datasource/test experimental/test pyrasterframes/test + - run: + command: cat /dev/null | sbt -batch core/test datasource/test experimental/test pyrasterframes/test + no_output_timeout: 15m - run: command: | diff --git a/pyrasterframes/src/main/python/tests/GeotrellisTests.py b/pyrasterframes/src/main/python/tests/GeotrellisTests.py index ccfa31082..c3fd9a5bb 100644 --- a/pyrasterframes/src/main/python/tests/GeotrellisTests.py +++ b/pyrasterframes/src/main/python/tests/GeotrellisTests.py @@ -26,7 +26,7 @@ class GeotrellisTests(TestEnvironment): def test_write_geotrellis_layer(self): - rf = self.spark.read.geotiff(self.img_uri) + rf = self.spark.read.geotiff(self.img_uri).cache() rf_count = rf.count() self.assertTrue(rf_count > 0) @@ -43,7 +43,7 @@ def test_write_geotrellis_layer(self): rf_gt.show(1) - shutil.rmtree(dest) + shutil.rmtree(dest, ignore_errors=True) def test_write_geotrellis_multiband_layer(self): rf = self.spark.read.geotiff(self.img_rgb_uri) @@ -63,4 +63,4 @@ def test_write_geotrellis_multiband_layer(self): rf_gt.show(1) - shutil.rmtree(dest) + shutil.rmtree(dest, ignore_errors=True) From 12339a0c478ba0c20da89a8c89ab4463c0c0dd51 Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Wed, 26 Feb 2020 16:47:23 -0500 Subject: [PATCH 59/94] Try to get hung test on circle ci to finish Signed-off-by: Jason T. Brown --- .circleci/config.yml | 1 + pyrasterframes/src/main/python/tests/GeotrellisTests.py | 6 ++++-- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index cb95598d5..ca815e37f 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -222,4 +222,5 @@ workflows: - it - itWithoutGdal - docs + - test # - staticAnalysis diff --git a/pyrasterframes/src/main/python/tests/GeotrellisTests.py b/pyrasterframes/src/main/python/tests/GeotrellisTests.py index c3fd9a5bb..a739a4d22 100644 --- a/pyrasterframes/src/main/python/tests/GeotrellisTests.py +++ b/pyrasterframes/src/main/python/tests/GeotrellisTests.py @@ -41,7 +41,8 @@ def test_write_geotrellis_layer(self): rf_gt_count = rf_gt.count() self.assertTrue(rf_gt_count > 0) - rf_gt.show(1) + # maybe CI is unhappy about print / show. + _ = rf_gt.take(1) shutil.rmtree(dest, ignore_errors=True) @@ -61,6 +62,7 @@ def test_write_geotrellis_multiband_layer(self): rf_gt_count = rf_gt.count() self.assertTrue(rf_gt_count > 0) - rf_gt.show(1) + # maybe CI is unhappy about print / show. + _ = rf_gt.take(1) shutil.rmtree(dest, ignore_errors=True) From fb64bec7020d57cde55522755ba34bbee451d2a1 Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Thu, 27 Feb 2020 13:33:43 -0500 Subject: [PATCH 60/94] Move Sqrt expression to its own source file Signed-off-by: Jason T. Brown --- .../expressions/localops/Exp.scala | 20 -------- .../expressions/localops/Sqrt.scala | 50 +++++++++++++++++++ 2 files changed, 50 insertions(+), 20 deletions(-) create mode 100644 core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Sqrt.scala diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Exp.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Exp.scala index 4d0c1bc5a..b1fb4d714 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Exp.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Exp.scala @@ -113,23 +113,3 @@ object ExpM1{ def apply(tile: Column): Column = new Column(ExpM1(tile.expr)) } -@ExpressionDescription( - usage = "_FUNC_(tile) - Perform cell-wise square root", - arguments = """ - Arguments: - * tile - input tile - """, - examples = - """ - Examples: - > SELECT _FUNC_(tile) - ... """ -) -case class Sqrt(child: Expression) extends UnaryLocalRasterOp with CodegenFallback { - override val nodeName: String = "rf_sqrt" - override protected def op(tile: Tile): Tile = fpTile(tile).localPowValue(0.5) - override def dataType: DataType = child.dataType -} -object Sqrt { - def apply(tile: Column): Column = new Column(Sqrt(tile.expr)) -} diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Sqrt.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Sqrt.scala new file mode 100644 index 000000000..f30580897 --- /dev/null +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Sqrt.scala @@ -0,0 +1,50 @@ +/* + * This software is licensed under the Apache 2 license, quoted below. + * + * Copyright 2020 Astraea, Inc. + * + * 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. + * + * SPDX-License-Identifier: Apache-2.0 + * + */ + +package org.locationtech.rasterframes.expressions.localops + +import geotrellis.raster.Tile +import org.apache.spark.sql.Column +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionDescription} +import org.apache.spark.sql.types.DataType +import org.locationtech.rasterframes.expressions.{UnaryLocalRasterOp, fpTile} + +@ExpressionDescription( + usage = "_FUNC_(tile) - Perform cell-wise square root", + arguments = """ + Arguments: + * tile - input tile + """, + examples = + """ + Examples: + > SELECT _FUNC_(tile) + ... """ +) +case class Sqrt(child: Expression) extends UnaryLocalRasterOp with CodegenFallback { + override val nodeName: String = "rf_sqrt" + override protected def op(tile: Tile): Tile = fpTile(tile).localPowValue(0.5) + override def dataType: DataType = child.dataType +} +object Sqrt { + def apply(tile: Column): Column = new Column(Sqrt(tile.expr)) +} From d37ee465538766e45ad849615bcaab9bc2a050a3 Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Thu, 27 Feb 2020 13:39:33 -0500 Subject: [PATCH 61/94] Add rf_sqrt to release notes Signed-off-by: Jason T. Brown --- docs/src/main/paradox/release-notes.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/src/main/paradox/release-notes.md b/docs/src/main/paradox/release-notes.md index 16f9dd1ee..32fffc1c7 100644 --- a/docs/src/main/paradox/release-notes.md +++ b/docs/src/main/paradox/release-notes.md @@ -28,6 +28,7 @@ * Add `rf_local_min`, `rf_local_max`, and `rf_local_clip` functions. * Add cell value scaling functions `rf_rescale` and `rf_standardize`. * Add `rf_where` function, similar in spirit to numpy's `where`, or a cell-wise version of Spark SQL's `when` and `otherwise`. +* Add `rf_sqrt` function to compute cell-wise square root. ## 0.8.x From 01b9014ffbb3279a0f67e9f49839c2f64aff5fc6 Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Thu, 27 Feb 2020 14:16:54 -0500 Subject: [PATCH 62/94] in Sqrt expression was doing 0.5 ** cell_value; flip operands to cell_value ** 0.5 Signed-off-by: Jason T. Brown --- .../locationtech/rasterframes/expressions/localops/Sqrt.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Sqrt.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Sqrt.scala index f30580897..3f5ea2d2e 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Sqrt.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/localops/Sqrt.scala @@ -42,7 +42,7 @@ import org.locationtech.rasterframes.expressions.{UnaryLocalRasterOp, fpTile} ) case class Sqrt(child: Expression) extends UnaryLocalRasterOp with CodegenFallback { override val nodeName: String = "rf_sqrt" - override protected def op(tile: Tile): Tile = fpTile(tile).localPowValue(0.5) + override protected def op(tile: Tile): Tile = fpTile(tile).localPow(0.5) override def dataType: DataType = child.dataType } object Sqrt { From 0db172e00638f1d7c5d3cdc391e2d91ba64a7599 Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Thu, 27 Feb 2020 15:20:25 -0500 Subject: [PATCH 63/94] Bump sbt version Signed-off-by: Jason T. Brown --- project/build.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/build.properties b/project/build.properties index a82bb05e1..a919a9b5f 100644 --- a/project/build.properties +++ b/project/build.properties @@ -1 +1 @@ -sbt.version=1.3.7 +sbt.version=1.3.8 From 98781d171e2dd05ef6c5c2a7d2d4edf296bf2f31 Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Fri, 28 Feb 2020 09:47:52 -0500 Subject: [PATCH 64/94] Revert "Bump sbt version" This reverts commit 0db172e00638f1d7c5d3cdc391e2d91ba64a7599. Signed-off-by: Jason T. Brown --- project/build.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/build.properties b/project/build.properties index a919a9b5f..a82bb05e1 100644 --- a/project/build.properties +++ b/project/build.properties @@ -1 +1 @@ -sbt.version=1.3.8 +sbt.version=1.3.7 From a2cc84979f245ca8286b3e8b9f8a7cf72bf40f78 Mon Sep 17 00:00:00 2001 From: "Jason T. Brown" Date: Fri, 28 Feb 2020 09:58:03 -0500 Subject: [PATCH 65/94] Conditionally skip GeoTrellis tests if on CircleCI Signed-off-by: Jason T. Brown --- .../src/main/python/tests/GeotrellisTests.py | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/pyrasterframes/src/main/python/tests/GeotrellisTests.py b/pyrasterframes/src/main/python/tests/GeotrellisTests.py index a739a4d22..da7373d54 100644 --- a/pyrasterframes/src/main/python/tests/GeotrellisTests.py +++ b/pyrasterframes/src/main/python/tests/GeotrellisTests.py @@ -21,10 +21,15 @@ import tempfile import pathlib from . import TestEnvironment +from unittest import skipIf +import os class GeotrellisTests(TestEnvironment): + on_circle_ci = os.environ.get('CIRCLECI', 'false') == 'true' + + @skipIf(on_circle_ci, 'CircleCI has java.lang.NoClassDefFoundError fs2/Stream when taking action on rf_gt') def test_write_geotrellis_layer(self): rf = self.spark.read.geotiff(self.img_uri).cache() rf_count = rf.count() @@ -41,13 +46,13 @@ def test_write_geotrellis_layer(self): rf_gt_count = rf_gt.count() self.assertTrue(rf_gt_count > 0) - # maybe CI is unhappy about print / show. _ = rf_gt.take(1) shutil.rmtree(dest, ignore_errors=True) + @skipIf(on_circle_ci, 'CircleCI has java.lang.NoClassDefFoundError fs2/Stream when taking action on rf_gt') def test_write_geotrellis_multiband_layer(self): - rf = self.spark.read.geotiff(self.img_rgb_uri) + rf = self.spark.read.geotiff(self.img_rgb_uri).cache() rf_count = rf.count() self.assertTrue(rf_count > 0) @@ -62,7 +67,6 @@ def test_write_geotrellis_multiband_layer(self): rf_gt_count = rf_gt.count() self.assertTrue(rf_gt_count > 0) - # maybe CI is unhappy about print / show. _ = rf_gt.take(1) shutil.rmtree(dest, ignore_errors=True) From 7b65fed59bd453a703953b0e300a3670eb50cc7a Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Tue, 3 Mar 2020 14:28:09 -0500 Subject: [PATCH 66/94] Updated through override gdal bindings. --- core/src/main/resources/reference.conf | 3 +++ project/RFDependenciesPlugin.scala | 1 + 2 files changed, 4 insertions(+) diff --git a/core/src/main/resources/reference.conf b/core/src/main/resources/reference.conf index fc76eb5a6..a9a7cc743 100644 --- a/core/src/main/resources/reference.conf +++ b/core/src/main/resources/reference.conf @@ -22,4 +22,7 @@ geotrellis.raster.gdal { } // set this to `false` if CPL_DEBUG is `ON` useExceptions = true + // See https://github.com/locationtech/geotrellis/issues/3184#issuecomment-592553807 + acceptable-datasets = ["SOURCE", "WARPED"] + number-of-attempts = 2147483647 } \ No newline at end of file diff --git a/project/RFDependenciesPlugin.scala b/project/RFDependenciesPlugin.scala index e64d46d22..cf84a7be1 100644 --- a/project/RFDependenciesPlugin.scala +++ b/project/RFDependenciesPlugin.scala @@ -71,6 +71,7 @@ object RFDependenciesPlugin extends AutoPlugin { case _ => deps :+ "com.fasterxml.jackson.module" %% "jackson-module-scala" % "2.6.7" } }, + dependencyOverrides += "com.azavea.gdal" % "gdal-warp-bindings" % "33.f746890", // NB: Make sure to update the Spark version in pyrasterframes/python/setup.py rfSparkVersion := "2.4.4", rfGeoTrellisVersion := "3.2.0", From 492dadd7d1007cef435c089f225b00623d99d19b Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Tue, 3 Mar 2020 15:21:45 -0500 Subject: [PATCH 67/94] Initial cut a maven->python version number converter. --- project/PythonBuildPlugin.scala | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/project/PythonBuildPlugin.scala b/project/PythonBuildPlugin.scala index c05950c04..f8a0b6317 100644 --- a/project/PythonBuildPlugin.scala +++ b/project/PythonBuildPlugin.scala @@ -37,6 +37,23 @@ object PythonBuildPlugin extends AutoPlugin { val pythonCommand = settingKey[String]("Python command. Defaults to 'python'") val pySetup = inputKey[Int]("Run 'python setup.py '. Returns exit code.") val pyWhl = taskKey[File]("Builds the Python wheel distribution") + val maven2PEP440: String => String = { + case VersionNumber(numbers, tags, extras) => + if (numbers.isEmpty) throw new MessageOnlyException("Version string is not convertible to PEP440.") + val rc = "^[Rr][Cc](\\d+)$".r + val base = numbers.mkString(".") + val tag = tags match { + case Seq("SNAPSHOT") => ".dev" + case Seq(rc(num)) => ".rc" + num + case Seq(other) => ".dev+" + other + case many => ".dev" + "+" + many.mkString(".") + } + val ssep = if (tag.contains("+")) "." else "+" + val ext = if (extras.nonEmpty) + extras.map(_.replaceAllLiterally("+", "")).mkString(ssep, ".", "") + else "" + base + tag + ext + } } import autoImport._ @@ -121,6 +138,7 @@ object PythonBuildPlugin extends AutoPlugin { inConfig(Python)(Seq( sourceDirectory := (Compile / sourceDirectory).value / "python", sourceDirectories := Seq((Python / sourceDirectory).value), + version ~= maven2PEP440, target := (Compile / target).value / "python", includeFilter := "*", excludeFilter := HiddenFileFilter || "__pycache__" || "*.egg-info", From cc219a280fee2490b6bba2bd18f08d40c7a6b67b Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Tue, 3 Mar 2020 15:37:32 -0500 Subject: [PATCH 68/94] Propagate translated maven version to setup.py. --- project/PythonBuildPlugin.scala | 2 +- pyrasterframes/src/main/python/setup.py | 11 +++++++---- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/project/PythonBuildPlugin.scala b/project/PythonBuildPlugin.scala index f8a0b6317..a223e1ddd 100644 --- a/project/PythonBuildPlugin.scala +++ b/project/PythonBuildPlugin.scala @@ -114,7 +114,7 @@ object PythonBuildPlugin extends AutoPlugin { val wd = copyPySources.value val args = spaceDelimited("").parsed val cmd = Seq(pythonCommand.value, "setup.py") ++ args - val ver = version.value + val ver = (Python / version).value s.log.info(s"Running '${cmd.mkString(" ")}' in '$wd'") val ec = Process(cmd, wd, "RASTERFRAMES_VERSION" -> ver).! if (ec != 0) diff --git a/pyrasterframes/src/main/python/setup.py b/pyrasterframes/src/main/python/setup.py index b33c10446..5ffc4b7d1 100644 --- a/pyrasterframes/src/main/python/setup.py +++ b/pyrasterframes/src/main/python/setup.py @@ -20,19 +20,22 @@ # Always prefer setuptools over distutils from setuptools import setup -from os import path +from os import path, environ, mkdir import sys from glob import glob from io import open import distutils.cmd try: + enver = environ.get('RASTERFRAMES_VERSION') + if enver is not None: + open('pyrasterframes/version.py', mode="w").write(f"__version__: str = '{enver}'\n") exec(open('pyrasterframes/version.py').read()) # executable python script contains __version__; credit pyspark -except IOError: - print("Run setup via `sbt 'pySetup arg1 arg2'` to ensure correct access to all source files and binaries.") +except IOError as e: + print(e) + print("Try running setup via `sbt 'pySetup arg1 arg2'` to ensure correct access to all source files and binaries.") sys.exit(-1) - VERSION = __version__ here = path.abspath(path.dirname(__file__)) From c6977d3daee0b00a05f5605f0e2d3792d27e892b Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Tue, 3 Mar 2020 15:50:01 -0500 Subject: [PATCH 69/94] Removed jar from pySparkCmd since it's in the whl file. --- pyrasterframes/build.sbt | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/pyrasterframes/build.sbt b/pyrasterframes/build.sbt index 707f6ce14..ce8797e17 100644 --- a/pyrasterframes/build.sbt +++ b/pyrasterframes/build.sbt @@ -26,14 +26,13 @@ pyNotebooks := { lazy val pySparkCmd = taskKey[Unit]("Create build and emit command to run in pyspark") pySparkCmd := { val s = streams.value - val jvm = assembly.value val py = (Python / packageBin).value val script = IO.createTemporaryDirectory / "pyrf_init.py" IO.write(script, """ import pyrasterframes from pyrasterframes.rasterfunctions import * """) - val msg = s"PYTHONSTARTUP=$script pyspark --jars $jvm --py-files $py" + val msg = s"PYTHONSTARTUP=$script pyspark --py-files $py" s.log.debug(msg) println(msg) } From 2c1b24dc8c15ddfc6197e21fddd32279ed43e062 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Wed, 4 Mar 2020 10:32:23 -0500 Subject: [PATCH 70/94] Fixed handling of release numbers and added better docs. --- project/PythonBuildPlugin.scala | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) diff --git a/project/PythonBuildPlugin.scala b/project/PythonBuildPlugin.scala index a223e1ddd..03c7af526 100644 --- a/project/PythonBuildPlugin.scala +++ b/project/PythonBuildPlugin.scala @@ -40,18 +40,29 @@ object PythonBuildPlugin extends AutoPlugin { val maven2PEP440: String => String = { case VersionNumber(numbers, tags, extras) => if (numbers.isEmpty) throw new MessageOnlyException("Version string is not convertible to PEP440.") - val rc = "^[Rr][Cc](\\d+)$".r + + // Reconstruct the primary version number val base = numbers.mkString(".") + + // Process items after the `-`. Due to PEP 440 constraints, some tags get converted + // to local version suffixes, while others map directly to prerelease suffixes. + val rc = "^[Rr][Cc](\\d+)$".r val tag = tags match { case Seq("SNAPSHOT") => ".dev" case Seq(rc(num)) => ".rc" + num case Seq(other) => ".dev+" + other - case many => ".dev" + "+" + many.mkString(".") + case many @ Seq(_, _) => ".dev+" + many.mkString(".") + case _ => "" } + + // sbt "extras" most closely map to PEP 440 local version suffixes. + // The local version components are separated by `.`, preceded by a single `+`, and not multiple `+` as in sbt. + // These next two expressions do the appropriate separator conversions while concatenating the components. val ssep = if (tag.contains("+")) "." else "+" val ext = if (extras.nonEmpty) extras.map(_.replaceAllLiterally("+", "")).mkString(ssep, ".", "") else "" + base + tag + ext } } From d615d995d6cdb8bda43f8d278128ea58de195b79 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Thu, 5 Mar 2020 14:05:01 -0500 Subject: [PATCH 71/94] Moved geotrellis settings into application.conf Fixed gdal version in noteobook. --- core/src/main/resources/application.conf | 19 ++++++++ core/src/main/resources/reference.conf | 19 -------- .../src/main/python/requirements.txt | 2 +- pyrasterframes/src/main/python/setup.py | 43 +++++++++++-------- .../src/main/docker/requirements-nb.txt | 2 +- 5 files changed, 46 insertions(+), 39 deletions(-) create mode 100644 core/src/main/resources/application.conf diff --git a/core/src/main/resources/application.conf b/core/src/main/resources/application.conf new file mode 100644 index 000000000..3565f4b83 --- /dev/null +++ b/core/src/main/resources/application.conf @@ -0,0 +1,19 @@ +geotrellis.raster.gdal { + options { + // See https://trac.osgeo.org/gdal/wiki/ConfigOptions for options + //CPL_DEBUG = "OFF" + AWS_REQUEST_PAYER = "requester" + GDAL_DISABLE_READDIR_ON_OPEN = "YES" + CPL_VSIL_CURL_ALLOWED_EXTENSIONS = ".tif,.tiff,.jp2,.mrf,.idx,.lrc,.mrf.aux.xml,.vrt" + GDAL_CACHEMAX = 512 + GDAL_PAM_ENABLED = "NO" + CPL_VSIL_CURL_CHUNK_SIZE = 1000000 + GDAL_HTTP_MAX_RETRY=10 + GDAL_HTTP_RETRY_DELAY=2 + } + // set this to `false` if CPL_DEBUG is `ON` + useExceptions = true + // See https://github.com/locationtech/geotrellis/issues/3184#issuecomment-592553807 + acceptable-datasets = ["SOURCE", "WARPED"] + number-of-attempts = 2147483647 +} \ No newline at end of file diff --git a/core/src/main/resources/reference.conf b/core/src/main/resources/reference.conf index a9a7cc743..af46605aa 100644 --- a/core/src/main/resources/reference.conf +++ b/core/src/main/resources/reference.conf @@ -7,22 +7,3 @@ rasterframes { raster-source-cache-timeout = 120 seconds jp2-gdal-thread-lock = false } -geotrellis.raster.gdal { - options { - // See https://trac.osgeo.org/gdal/wiki/ConfigOptions for options - //CPL_DEBUG = "OFF" - AWS_REQUEST_PAYER = "requester" - GDAL_DISABLE_READDIR_ON_OPEN = "YES" - CPL_VSIL_CURL_ALLOWED_EXTENSIONS = ".tif,.tiff,.jp2,.mrf,.idx,.lrc,.mrf.aux.xml,.vrt" - GDAL_CACHEMAX = 512 - GDAL_PAM_ENABLED = "NO" - CPL_VSIL_CURL_CHUNK_SIZE = 1000000 - GDAL_HTTP_MAX_RETRY=4 - GDAL_HTTP_RETRY_DELAY=1 - } - // set this to `false` if CPL_DEBUG is `ON` - useExceptions = true - // See https://github.com/locationtech/geotrellis/issues/3184#issuecomment-592553807 - acceptable-datasets = ["SOURCE", "WARPED"] - number-of-attempts = 2147483647 -} \ No newline at end of file diff --git a/pyrasterframes/src/main/python/requirements.txt b/pyrasterframes/src/main/python/requirements.txt index fd7e7fac4..f12870aae 100644 --- a/pyrasterframes/src/main/python/requirements.txt +++ b/pyrasterframes/src/main/python/requirements.txt @@ -1,6 +1,6 @@ ipython==6.2.1 pyspark==2.4.4 -gdal==2.4.3 +gdal==2.4.4 numpy>=1.17.3,<2.0 pandas>=0.25.3,<1.0 shapely>=1.6.4,<1.7 diff --git a/pyrasterframes/src/main/python/setup.py b/pyrasterframes/src/main/python/setup.py index 5ffc4b7d1..b4be59eb2 100644 --- a/pyrasterframes/src/main/python/setup.py +++ b/pyrasterframes/src/main/python/setup.py @@ -132,27 +132,33 @@ def initialize_options(self): def dest_file(self, src_file): return path.splitext(src_file)[0] + '.ipynb' -pytz = 'pytz' -shapely = 'Shapely>=1.6.0' -pyspark ='pyspark==2.4.4' -numpy = 'numpy>=1.12.0' -matplotlib ='matplotlib' -pandas = 'pandas>=0.24.2' -geopandas = 'geopandas' -requests = 'requests' -pytest_runner = 'pytest-runner' -setuptools = 'setuptools>=0.8' -ipython = 'ipython==6.2.1' -ipykernel = 'ipykernel==4.8.0' -pweave = 'Pweave==0.30.3' -jupyter_client = 'jupyter-client<6.0' # v6 breaks pweave + +boto3 = 'boto3' +deprecation = 'deprecation' +descartes = 'descartes' fiona = 'fiona==1.8.6' -rasterio = 'rasterio>=1.0.0' folium = 'folium' -pytest = 'pytest>=4.0.0,<5.0.0' +gdal = 'gdal==2.4.4' +geopandas = 'geopandas>=0.7' +ipykernel = 'ipykernel==4.8.0' +ipython = 'ipython==6.2.1' +jupyter_client = 'jupyter-client<6.0' # v6 breaks pweave +matplotlib = 'matplotlib' +numpy = 'numpy>=1.12.0' +pandas = 'pandas>=0.25.3,<1.0' +pweave = 'pweave==0.30.3' pypandoc = 'pypandoc' -boto3 = 'boto3' -deprecation = 'deprecation' +pyspark = 'pyspark==2.4.4' +pytest = 'pytest>=4.0.0,<5.0.0' +pytest_runner = 'pytest-runner' +pytz = 'pytz' +rasterio = 'rasterio>=1.0.0' +requests = 'requests' +setuptools = 'setuptools>=45.2.0' +shapely = 'Shapely>=1.6.0' +tabulate = 'tabulate' +tqdm = 'tqdm' +utm = 'utm' setup( name='pyrasterframes', @@ -170,6 +176,7 @@ def dest_file(self, src_file): }, python_requires=">=3.5", install_requires=[ + gdal, pytz, shapely, pyspark, diff --git a/rf-notebook/src/main/docker/requirements-nb.txt b/rf-notebook/src/main/docker/requirements-nb.txt index e82e45453..f4ad77f5a 100644 --- a/rf-notebook/src/main/docker/requirements-nb.txt +++ b/rf-notebook/src/main/docker/requirements-nb.txt @@ -1,5 +1,5 @@ pyspark=2.4.4 -gdal=2.4.3 +gdal=2.4.4 numpy>=1.17.3,<2.0 pandas>=0.25.3,<1.0 shapely>=1.6.4,<1.7 From 98aa5ee0938e017eb0e7374c2483bd17c26f7c11 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Mon, 20 Apr 2020 09:59:51 -0400 Subject: [PATCH 72/94] Upgraded to GT 3.3.0. --- {build/circleci => .circleci}/Dockerfile | 0 {build/circleci => .circleci}/Makefile | 0 {build/circleci => .circleci}/README.md | 0 docs/src/main/paradox/release-notes.md | 2 +- project/RFDependenciesPlugin.scala | 6 +++--- 5 files changed, 4 insertions(+), 4 deletions(-) rename {build/circleci => .circleci}/Dockerfile (100%) rename {build/circleci => .circleci}/Makefile (100%) rename {build/circleci => .circleci}/README.md (100%) diff --git a/build/circleci/Dockerfile b/.circleci/Dockerfile similarity index 100% rename from build/circleci/Dockerfile rename to .circleci/Dockerfile diff --git a/build/circleci/Makefile b/.circleci/Makefile similarity index 100% rename from build/circleci/Makefile rename to .circleci/Makefile diff --git a/build/circleci/README.md b/.circleci/README.md similarity index 100% rename from build/circleci/README.md rename to .circleci/README.md diff --git a/docs/src/main/paradox/release-notes.md b/docs/src/main/paradox/release-notes.md index 32fffc1c7..97cbd81fd 100644 --- a/docs/src/main/paradox/release-notes.md +++ b/docs/src/main/paradox/release-notes.md @@ -4,7 +4,7 @@ ### 0.9.0 -* Upgraded to GeoTrellis 3.2.0. This includes a number of _breaking_ changes enumerated as a part of the [PR's](https://github.com/locationtech/rasterframes/pull/398) change log. These include: +* Upgraded to GeoTrellis 3.3.0. This includes a number of _breaking_ changes enumerated as a part of the [PR's](https://github.com/locationtech/rasterframes/pull/398) change log. These include: - Add `Int` type parameter to `Grid` - Add `Int` type parameter to `CellGrid` - Add `Int` type parameter to `GridBounds`... or `TileBounds` diff --git a/project/RFDependenciesPlugin.scala b/project/RFDependenciesPlugin.scala index cf84a7be1..dee32e3dc 100644 --- a/project/RFDependenciesPlugin.scala +++ b/project/RFDependenciesPlugin.scala @@ -71,10 +71,10 @@ object RFDependenciesPlugin extends AutoPlugin { case _ => deps :+ "com.fasterxml.jackson.module" %% "jackson-module-scala" % "2.6.7" } }, - dependencyOverrides += "com.azavea.gdal" % "gdal-warp-bindings" % "33.f746890", + // dependencyOverrides += "com.azavea.gdal" % "gdal-warp-bindings" % "33.f746890", // NB: Make sure to update the Spark version in pyrasterframes/python/setup.py - rfSparkVersion := "2.4.4", - rfGeoTrellisVersion := "3.2.0", + rfSparkVersion := "2.4.5", + rfGeoTrellisVersion := "3.3.0", rfGeoMesaVersion := "2.2.1" ) } From 118027fcf507682ae57e78acffcd5606f500185e Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Mon, 20 Apr 2020 10:00:26 -0400 Subject: [PATCH 73/94] Fixed regression in toString of RasterRefTile. --- .../scala/org/locationtech/rasterframes/ref/RasterRef.scala | 3 ++- pyrasterframes/src/main/python/tests/RasterSourceTest.py | 1 + 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/RasterRef.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/RasterRef.scala index 36bc48e1e..8b5af22e6 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/RasterRef.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/RasterRef.scala @@ -32,7 +32,7 @@ import org.locationtech.rasterframes.encoders.CatalystSerializer.{CatalystIO, _} import org.locationtech.rasterframes.encoders.{CatalystSerializer, CatalystSerializerEncoder} import org.locationtech.rasterframes.ref.RasterRef.RasterRefTile import org.locationtech.rasterframes.RasterSourceType -import org.locationtech.rasterframes.tiles.ProjectedRasterTile +import org.locationtech.rasterframes.tiles.{ProjectedRasterTile, ShowableTile} /** * A delayed-read projected raster implementation. @@ -73,6 +73,7 @@ object RasterRef extends LazyLogging { // NB: This saves us from stack overflow exception override def convert(ct: CellType): ProjectedRasterTile = ProjectedRasterTile(rr.realizedTile.convert(ct), extent, crs) + override def toString: String = s"$productPrefix($rr)" } val embeddedSchema: StructType = StructType(Seq( diff --git a/pyrasterframes/src/main/python/tests/RasterSourceTest.py b/pyrasterframes/src/main/python/tests/RasterSourceTest.py index 4687864ad..ec0877486 100644 --- a/pyrasterframes/src/main/python/tests/RasterSourceTest.py +++ b/pyrasterframes/src/main/python/tests/RasterSourceTest.py @@ -66,6 +66,7 @@ def test_strict_eval(self): # when doing Show on a lazy tile we will see something like RasterRefTile(RasterRef(JVMGeoTiffRasterSource(... # use this trick to get the `show` string show_str_lazy = df_lazy.select('proj_raster')._jdf.showString(1, -1, False) + print(show_str_lazy) self.assertTrue('RasterRef' in show_str_lazy) # again for strict From 0ddfef129844bac68912560294281e26bb18b633 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Mon, 20 Apr 2020 10:37:04 -0400 Subject: [PATCH 74/94] Attempting to purge FixedDelegatingTile. --- .circleci/config.yml | 16 +++++--- .../rasterframes/ref/RasterRef.scala | 4 +- .../tiles/FixedDelegatingTile.scala | 40 ------------------- .../rasterframes/tiles/InternalRowTile.scala | 2 +- .../tiles/ProjectedRasterTile.scala | 4 +- .../rasterframes/tiles/ShowableTile.scala | 4 +- .../functions/TileFunctionsSpec.scala | 9 +++-- docs/src/main/paradox/release-notes.md | 1 + 8 files changed, 25 insertions(+), 55 deletions(-) delete mode 100644 core/src/main/scala/org/locationtech/rasterframes/tiles/FixedDelegatingTile.scala diff --git a/.circleci/config.yml b/.circleci/config.yml index ca815e37f..7a2cbbfd7 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -33,7 +33,7 @@ _save_cache: &save_cache jobs: test: <<: *defaults - resource_class: large + resource_class: medium steps: - checkout - run: *setenv @@ -42,7 +42,13 @@ jobs: - run: ulimit -c unlimited -S - run: - command: cat /dev/null | sbt -batch core/test datasource/test experimental/test pyrasterframes/test + name: Scala Tests + command: sbt -batch core/test datasource/test experimental/test + no_output_timeout: 15m + + - run: + name: Python Tests + command: sbt -batch pyrasterframes/test no_output_timeout: 15m - run: @@ -69,7 +75,7 @@ jobs: docs: <<: *defaults - resource_class: xlarge + resource_class: medium steps: - checkout - run: *setenv @@ -108,7 +114,7 @@ jobs: it: <<: *defaults - resource_class: xlarge + resource_class: medium steps: - checkout - run: *setenv @@ -141,7 +147,7 @@ jobs: TERM: dumb docker: - image: circleci/openjdk:8-jdk - resource_class: xlarge + resource_class: medium steps: - checkout - run: *setenv diff --git a/core/src/main/scala/org/locationtech/rasterframes/ref/RasterRef.scala b/core/src/main/scala/org/locationtech/rasterframes/ref/RasterRef.scala index 8b5af22e6..8c03c8427 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/ref/RasterRef.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/ref/RasterRef.scala @@ -28,11 +28,11 @@ import geotrellis.vector.{Extent, ProjectedExtent} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.rf.RasterSourceUDT import org.apache.spark.sql.types.{IntegerType, StructField, StructType} +import org.locationtech.rasterframes.RasterSourceType import org.locationtech.rasterframes.encoders.CatalystSerializer.{CatalystIO, _} import org.locationtech.rasterframes.encoders.{CatalystSerializer, CatalystSerializerEncoder} import org.locationtech.rasterframes.ref.RasterRef.RasterRefTile -import org.locationtech.rasterframes.RasterSourceType -import org.locationtech.rasterframes.tiles.{ProjectedRasterTile, ShowableTile} +import org.locationtech.rasterframes.tiles.ProjectedRasterTile /** * A delayed-read projected raster implementation. diff --git a/core/src/main/scala/org/locationtech/rasterframes/tiles/FixedDelegatingTile.scala b/core/src/main/scala/org/locationtech/rasterframes/tiles/FixedDelegatingTile.scala deleted file mode 100644 index 5bdb7d258..000000000 --- a/core/src/main/scala/org/locationtech/rasterframes/tiles/FixedDelegatingTile.scala +++ /dev/null @@ -1,40 +0,0 @@ -/* - * This software is licensed under the Apache 2 license, quoted below. - * - * Copyright 2019 Astraea, Inc. - * - * 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. - * - * SPDX-License-Identifier: Apache-2.0 - * - */ - -package org.locationtech.rasterframes.tiles -import geotrellis.raster.{ArrayTile, DelegatingTile, Tile} - -/** - * Workaround for case where `combine` is invoked on two delegating tiles. - * Remove after https://github.com/locationtech/geotrellis/issues/3153 is fixed and integrated - * @since 8/22/18 - */ -abstract class FixedDelegatingTile extends DelegatingTile { - override def combine(r2: Tile)(f: (Int, Int) ⇒ Int): Tile = (delegate, r2) match { - case (del: ArrayTile, r2: DelegatingTile) ⇒ del.combine(r2.toArrayTile())(f) - case _ ⇒ delegate.combine(r2)(f) - } - - override def combineDouble(r2: Tile)(f: (Double, Double) ⇒ Double): Tile = (delegate, r2) match { - case (del: ArrayTile, r2: DelegatingTile) ⇒ del.combineDouble(r2.toArrayTile())(f) - case _ ⇒ delegate.combineDouble(r2)(f) - } -} diff --git a/core/src/main/scala/org/locationtech/rasterframes/tiles/InternalRowTile.scala b/core/src/main/scala/org/locationtech/rasterframes/tiles/InternalRowTile.scala index 72f5631ae..169166ce0 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/tiles/InternalRowTile.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/tiles/InternalRowTile.scala @@ -34,7 +34,7 @@ import org.locationtech.rasterframes.model.{Cells, TileDataContext} * * @since 11/29/17 */ -class InternalRowTile(val mem: InternalRow) extends FixedDelegatingTile { +class InternalRowTile(val mem: InternalRow) extends DelegatingTile { import InternalRowTile._ override def toArrayTile(): ArrayTile = realizedTile.toArrayTile() diff --git a/core/src/main/scala/org/locationtech/rasterframes/tiles/ProjectedRasterTile.scala b/core/src/main/scala/org/locationtech/rasterframes/tiles/ProjectedRasterTile.scala index 9a822cebc..4427a555c 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/tiles/ProjectedRasterTile.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/tiles/ProjectedRasterTile.scala @@ -23,7 +23,7 @@ package org.locationtech.rasterframes.tiles import geotrellis.proj4.CRS import geotrellis.raster.io.geotiff.SinglebandGeoTiff -import geotrellis.raster.{CellType, ProjectedRaster, Tile} +import geotrellis.raster.{CellType, DelegatingTile, ProjectedRaster, Tile} import geotrellis.vector.{Extent, ProjectedExtent} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.rf.TileUDT @@ -40,7 +40,7 @@ import org.locationtech.rasterframes.ref.RasterRef.RasterRefTile * * @since 9/5/18 */ -abstract class ProjectedRasterTile extends FixedDelegatingTile with ProjectedRasterLike { +abstract class ProjectedRasterTile extends DelegatingTile with ProjectedRasterLike { def extent: Extent def crs: CRS def projectedExtent: ProjectedExtent = ProjectedExtent(extent, crs) diff --git a/core/src/main/scala/org/locationtech/rasterframes/tiles/ShowableTile.scala b/core/src/main/scala/org/locationtech/rasterframes/tiles/ShowableTile.scala index ccec3a340..ba241b914 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/tiles/ShowableTile.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/tiles/ShowableTile.scala @@ -20,10 +20,10 @@ */ package org.locationtech.rasterframes.tiles +import geotrellis.raster.{DelegatingTile, Tile, isNoData} import org.locationtech.rasterframes._ -import geotrellis.raster.{Tile, isNoData} -class ShowableTile(val delegate: Tile) extends FixedDelegatingTile { +class ShowableTile(val delegate: Tile) extends DelegatingTile { override def equals(obj: Any): Boolean = obj match { case st: ShowableTile => delegate.equals(st.delegate) case o => delegate.equals(o) diff --git a/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala index 7ac72dad9..836561c1d 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala @@ -260,6 +260,9 @@ class TileFunctionsSpec extends TestEnvironment with RasterMatchers { it("should evaluate rf_where"){ val df = Seq((randPRT, one, six)).toDF("t", "one", "six") + + df.select(rf_render_matrix(rf_where(rf_local_greater($"t", 0), $"one", $"six") as "result")).show(false) + val result = df.select( rf_for_all( rf_local_equal( @@ -271,10 +274,10 @@ class TileFunctionsSpec extends TestEnvironment with RasterMatchers { ) ) ) - .first() - - result should be (true) + .distinct() + .collect() + result should be (Array(true)) } } diff --git a/docs/src/main/paradox/release-notes.md b/docs/src/main/paradox/release-notes.md index 97cbd81fd..4f89c72f6 100644 --- a/docs/src/main/paradox/release-notes.md +++ b/docs/src/main/paradox/release-notes.md @@ -12,6 +12,7 @@ - Update imports for layers, particularly `geotrellis.spark.tiling` to `geotrellis.layer` - Update imports for `geotrellis.spark.io` to `geotrellis.spark.store...` - Removed `FixedRasterExtent` + - Removed `FixedDelegatingTile` - Removed `org.locationtech.rasterframes.util.Shims` - Change `Extent.jtsGeom` to `Extent.toPolygon` - Change `TileLayerMetadata.gridBounds` to `TileLayerMetadata.tileBounds` From f58bfcea2b539ca44ca334842ff815c3a56720bf Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Mon, 20 Apr 2020 10:57:24 -0400 Subject: [PATCH 75/94] Attempted fix for 'implement_array_function method already has a docstring'. --- pyrasterframes/src/main/python/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pyrasterframes/src/main/python/setup.py b/pyrasterframes/src/main/python/setup.py index b4be59eb2..7531f0cba 100644 --- a/pyrasterframes/src/main/python/setup.py +++ b/pyrasterframes/src/main/python/setup.py @@ -144,7 +144,7 @@ def dest_file(self, src_file): ipython = 'ipython==6.2.1' jupyter_client = 'jupyter-client<6.0' # v6 breaks pweave matplotlib = 'matplotlib' -numpy = 'numpy>=1.12.0' +numpy = 'numpy>=1.12.0,<=1.15.4' pandas = 'pandas>=0.25.3,<1.0' pweave = 'pweave==0.30.3' pypandoc = 'pypandoc' From 145da52e51cf25164482a80337b25f1df5e503a2 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Mon, 20 Apr 2020 11:04:58 -0400 Subject: [PATCH 76/94] Splitting up module tests to see if memory footprint is less. --- .circleci/config.yml | 12 ++++++++++-- core/src/test/resources/log4j.properties | 3 ++- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 7a2cbbfd7..6ae2f0f16 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -42,8 +42,16 @@ jobs: - run: ulimit -c unlimited -S - run: - name: Scala Tests - command: sbt -batch core/test datasource/test experimental/test + name: "Scala Tests: core" + command: sbt -batch core/test + no_output_timeout: 15m + - run: + name: "Scala Tests: datasource" + command: sbt -batch datasource/test + no_output_timeout: 15m + - run: + name: "Scala Tests: experimental" + command: sbt -batch experimental/test no_output_timeout: 15m - run: diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index e17586b72..9dbb3d54b 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -46,4 +46,5 @@ log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR log4j.logger.org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator=ERROR -log4j.logger.org.apache.spark.sql.execution.WholeStageCodegenExec=ERROR \ No newline at end of file +log4j.logger.org.apache.spark.sql.execution.WholeStageCodegenExec=ERROR +log4j.logger.geotrellis.raster.gdal=ERROR From 0c8107781118d1a88480a915772d5876e3acdc80 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Mon, 20 Apr 2020 11:21:54 -0400 Subject: [PATCH 77/94] Partial revert of `combine` and `combineDouble` for `ProjectedRasterTile`. Not sure why the fixes in GT 3.3.0 didn't obviate the need. --- .../rasterframes/tiles/ProjectedRasterTile.scala | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/locationtech/rasterframes/tiles/ProjectedRasterTile.scala b/core/src/main/scala/org/locationtech/rasterframes/tiles/ProjectedRasterTile.scala index 4427a555c..b5701b095 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/tiles/ProjectedRasterTile.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/tiles/ProjectedRasterTile.scala @@ -23,7 +23,7 @@ package org.locationtech.rasterframes.tiles import geotrellis.proj4.CRS import geotrellis.raster.io.geotiff.SinglebandGeoTiff -import geotrellis.raster.{CellType, DelegatingTile, ProjectedRaster, Tile} +import geotrellis.raster.{ArrayTile, CellType, DelegatingTile, ProjectedRaster, Tile} import geotrellis.vector.{Extent, ProjectedExtent} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.rf.TileUDT @@ -69,6 +69,20 @@ object ProjectedRasterTile { val c = crs.toProj4String s"[${ShowableTile.show(t)}, $e, $c]" } + + // Not sure why the following are still needed with this being closed: + // https://github.com/locationtech/geotrellis/issues/3153 + // Without them, TileFunctionsSpec.`conditional cell values`.`should evaluate rf_where` fails + override def combine(r2: Tile)(f: (Int, Int) ⇒ Int): Tile = (delegate, r2) match { + case (del: ArrayTile, r2: DelegatingTile) ⇒ del.combine(r2.toArrayTile())(f) + case _ ⇒ delegate.combine(r2)(f) + } + + override def combineDouble(r2: Tile)(f: (Double, Double) ⇒ Double): Tile = (delegate, r2) match { + case (del: ArrayTile, r2: DelegatingTile) ⇒ del.combineDouble(r2.toArrayTile())(f) + case _ ⇒ delegate.combineDouble(r2)(f) + } + } implicit val serializer: CatalystSerializer[ProjectedRasterTile] = new CatalystSerializer[ProjectedRasterTile] { override val schema: StructType = StructType(Seq( From 06031f6ebb37177afabdcf4d2ea10bb698882e12 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Mon, 20 Apr 2020 11:51:19 -0400 Subject: [PATCH 78/94] Refactoring CircleCI build to use 2.1 schema. --- .circleci/config.yml | 304 +++++++++++------------- pyrasterframes/src/main/python/setup.py | 2 +- 2 files changed, 146 insertions(+), 160 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 6ae2f0f16..8989bad94 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -1,205 +1,192 @@ -version: 2 - -_defaults: &defaults - working_directory: ~/repo - environment: - TERM: dumb - docker: - - image: s22s/rasterframes-circleci:9b7682ef - -_setenv: &setenv - name: set CloudRepo credentials - command: |- - [ -d $HOME/.sbt ] || mkdir $HOME/.sbt - printf "realm=s22s.mycloudrepo.io\nhost=s22s.mycloudrepo.io\nuser=$CLOUDREPO_USER\npassword=$CLOUDREPO_PASSWORD\n" > $HOME/.sbt/.credentials - -_delenv: &unsetenv - name: delete CloudRepo credential - command: rm -rf $HOME/.sbt/.credentials || true - -_restore_cache: &restore_cache - keys: - - v3-dependencies-{{ checksum "build.sbt" }} - - v3-dependencies- - -_save_cache: &save_cache - key: v3-dependencies--{{ checksum "build.sbt" }} - paths: - - ~/.cache/coursier - - ~/.ivy2/cache - - ~/.sbt - - ~/.local +version: 2.1 + +orbs: + sbt: + description: SBT build/test runtime + executors: + default: + docker: + - image: circleci/openjdk:8-jdk + resource_class: medium + working_directory: ~/repo + environment: + SBT_VERSION: 1.3.8 + commands: + setup: + description: Setup for sbt build + steps: + - run: 'true' # NOOP + + install-envsubst: + description: Install base packages + steps: + - run: + name: Update apt + command: sudo apt-get update -q -y + - run: + name: Install secrets support + command: sudo apt-get install -y gettext-base + + python: + commands: + setup: + description: Ensure a minimal python environment is avalable and ready + steps: + - run: + name: Install Python and PIP + command: |- + sudo apt-get install python3 python3-pip + sudo update-alternatives --install /usr/bin/python python /usr/bin/python3.7 1 + python -m pip install --user 'setuptools>=45.2' + + requirements: + description: Install packages identified in requirements file + steps: + - run: + name: Install requirements + command: pip3 install --progress-bar=off --user -r pyrasterframes/src/main/python/requirements.txt + + rasterframes: + commands: + setup: + steps: + - run: + name: Enable saving core files + command: ulimit -c unlimited -S + + save-artifacts: + steps: + - run: + command: | + mkdir -p /tmp/core_dumps + cp core.* *.hs /tmp/core_dumps 2> /dev/null || true + when: on_fail + + - store_artifacts: + path: /tmp/core_dumps + + - store_test_results: + path: core/target/test-reports + + - store_test_results: + path: datasource/target/test-reports + + - store_test_results: + path: experimental/target/test-reports + + save-doc-artifacts: + steps: + - run: + command: | + mkdir -p /tmp/core_dumps + cp core.* *.hs /tmp/core_dumps 2> /dev/null || true + mkdir -p /tmp/markdown + cp /home/circleci/repo/pyrasterframes/target/python/docs/*.md /tmp/markdown 2> /dev/null || true + when: on_fail + + - store_artifacts: + path: /tmp/core_dumps + + - store_artifacts: + path: /tmp/markdown + + - store_artifacts: + path: docs/target/site + destination: rf-site + + save-cache: + steps: + - save_cache: + key: v4-dependencies--{{ checksum "build.sbt" }} + paths: + - ~/.ivy2/cache + - ~/.sbt + - ~/.cache/coursier + - ~/.local + + restore-cache: + steps: + - restore_cache: + keys: + - v4-dependencies-{{ checksum "build.sbt" }} jobs: test: - <<: *defaults - resource_class: medium + executor: sbt/default steps: - checkout - - run: *setenv - - restore_cache: - <<: *restore_cache + - sbt/setup + - python/setup + - rasterframes/setup + - rasterframes/restore-cache - - run: ulimit -c unlimited -S - run: name: "Scala Tests: core" command: sbt -batch core/test - no_output_timeout: 15m + - run: name: "Scala Tests: datasource" command: sbt -batch datasource/test - no_output_timeout: 15m + - run: name: "Scala Tests: experimental" command: sbt -batch experimental/test - no_output_timeout: 15m - run: name: Python Tests command: sbt -batch pyrasterframes/test - no_output_timeout: 15m - - - run: - command: | - mkdir -p /tmp/core_dumps - cp core.* *.hs /tmp/core_dumps 2> /dev/null || true - when: on_fail - - - store_artifacts: - path: /tmp/core_dumps - - - store_test_results: - path: core/target/test-reports - - - store_test_results: - path: datasource/target/test-reports - - - store_test_results: - path: experimental/target/test-reports - - run: *unsetenv - - save_cache: - <<: *save_cache + - rasterframes/save-artifacts + - rasterframes/save-cache docs: - <<: *defaults - resource_class: medium + executor: sbt/default steps: - checkout - - run: *setenv - - - restore_cache: - <<: *restore_cache - - - run: ulimit -c unlimited -S - - run: pip3 install --progress-bar=off --user -r pyrasterframes/src/main/python/requirements.txt + - sbt/setup + - python/setup + - python/requirements + - rasterframes/setup + - rasterframes/restore-cache - run: + name: Build documentation command: cat /dev/null | sbt makeSite - no_output_timeout: 30m - - - run: - command: | - mkdir -p /tmp/core_dumps - cp core.* *.hs /tmp/core_dumps 2> /dev/null || true - mkdir -p /tmp/markdown - cp /home/circleci/repo/pyrasterframes/target/python/docs/*.md /tmp/markdown 2> /dev/null || true - when: on_fail - - - store_artifacts: - path: /tmp/core_dumps - - - store_artifacts: - path: /tmp/markdown + no-output-timeout: 30m - - store_artifacts: - path: docs/target/site - destination: rf-site - - - run: *unsetenv - - - save_cache: - <<: *save_cache + - rasterframes/save-doc-artifacts + - rasterframes/save-cache it: - <<: *defaults - resource_class: medium + executor: sbt/default steps: - checkout - - run: *setenv + - sbt/setup + - rasterframes/setup + - rasterframes/restore-cache - - restore_cache: - <<: *restore_cache - - - run: ulimit -c unlimited -S - run: + name: Integration tests command: cat /dev/null | sbt it:test no_output_timeout: 30m - - run: - command: | - mkdir -p /tmp/core_dumps - cp core.* *.hs /tmp/core_dumps 2> /dev/null || true - when: on_fail - - - store_artifacts: - path: /tmp/core_dumps - - - run: *unsetenv - - - save_cache: - <<: *save_cache + - rasterframes/save-artifacts + - rasterframes/save-cache itWithoutGdal: - working_directory: ~/repo - environment: - TERM: dumb - docker: - - image: circleci/openjdk:8-jdk - resource_class: medium + executor: sbt/default steps: - checkout - - run: *setenv - - - restore_cache: - <<: *restore_cache + - sbt/setup + - rasterframes/setup + - rasterframes/restore-cache - run: + name: Integration tests command: cat /dev/null | sbt it:test no_output_timeout: 30m - - run: *unsetenv - - - save_cache: - <<: *save_cache - - staticAnalysis: - <<: *defaults - - steps: - - checkout - - run: *setenv - - restore_cache: - <<: *restore_cache - - - run: cat /dev/null | sbt dependencyCheck - - run: cat /dev/null | sbt --debug dumpLicenseReport - - - run: *unsetenv - - - save_cache: - <<: *save_cache - - store_artifacts: - path: datasource/target/scala-2.11/dependency-check-report.html - destination: dependency-check-report-datasource.html - - store_artifacts: - path: experimental/target/scala-2.11/dependency-check-report.html - destination: dependency-check-report-experimental.html - - store_artifacts: - path: core/target/scala-2.11/dependency-check-report.html - destination: dependency-check-report-core.html - - store_artifacts: - path: pyrasterframes/target/scala-2.11/dependency-check-report.html - destination: dependency-check-report-pyrasterframes.html + - rasterframes/save-artifacts + - rasterframes/save-cache workflows: version: 2 @@ -233,8 +220,7 @@ workflows: only: - develop jobs: + - test - it - itWithoutGdal - docs - - test -# - staticAnalysis diff --git a/pyrasterframes/src/main/python/setup.py b/pyrasterframes/src/main/python/setup.py index 7531f0cba..db9810464 100644 --- a/pyrasterframes/src/main/python/setup.py +++ b/pyrasterframes/src/main/python/setup.py @@ -144,7 +144,7 @@ def dest_file(self, src_file): ipython = 'ipython==6.2.1' jupyter_client = 'jupyter-client<6.0' # v6 breaks pweave matplotlib = 'matplotlib' -numpy = 'numpy>=1.12.0,<=1.15.4' +numpy = 'numpy>=1.17.3,<2.0' pandas = 'pandas>=0.25.3,<1.0' pweave = 'pweave==0.30.3' pypandoc = 'pypandoc' From 392b31741aa834673193176396679a159ac1c361 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Mon, 20 Apr 2020 11:55:40 -0400 Subject: [PATCH 79/94] apt-get update --- .circleci/config.yml | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 8989bad94..3a6ca05e6 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -17,16 +17,6 @@ orbs: steps: - run: 'true' # NOOP - install-envsubst: - description: Install base packages - steps: - - run: - name: Update apt - command: sudo apt-get update -q -y - - run: - name: Install secrets support - command: sudo apt-get install -y gettext-base - python: commands: setup: @@ -35,6 +25,7 @@ orbs: - run: name: Install Python and PIP command: |- + sudo apt-get update -q -y sudo apt-get install python3 python3-pip sudo update-alternatives --install /usr/bin/python python /usr/bin/python3.7 1 python -m pip install --user 'setuptools>=45.2' From 7dbb02fb73501aa45e8a6fe6c98b503ac2b34e58 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Mon, 20 Apr 2020 12:24:29 -0400 Subject: [PATCH 80/94] Memory control. --- .circleci/config.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.circleci/config.yml b/.circleci/config.yml index 3a6ca05e6..2a6a5fb28 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -11,6 +11,7 @@ orbs: working_directory: ~/repo environment: SBT_VERSION: 1.3.8 + JAVA_OPTS: -Xmx2g commands: setup: description: Setup for sbt build From 90978bebdb9d0ca3597323beb2723ca5a77b83cb Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Mon, 20 Apr 2020 13:09:50 -0400 Subject: [PATCH 81/94] Reset memory JVM memory settings to isolate problem. --- .circleci/config.yml | 18 +++++++++++++++--- .sbtopts | 7 ++++++- build.sbt | 2 +- .../expressions/DynamicExtractors.scala | 1 - project/RFProjectPlugin.scala | 3 ++- project/build.properties | 2 +- 6 files changed, 25 insertions(+), 8 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 2a6a5fb28..6085005d6 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -11,12 +11,13 @@ orbs: working_directory: ~/repo environment: SBT_VERSION: 1.3.8 - JAVA_OPTS: -Xmx2g commands: setup: description: Setup for sbt build steps: - - run: 'true' # NOOP + - run: + name: Setup sbt + command: 'true' # NOOP python: commands: @@ -51,7 +52,9 @@ orbs: - run: command: | mkdir -p /tmp/core_dumps + ls -lh /tmp cp core.* *.hs /tmp/core_dumps 2> /dev/null || true + cp -r /tmp/hsperfdata* /tmp/*.hprof /tmp/core_dumps 2> /dev/null || true when: on_fail - store_artifacts: @@ -112,6 +115,10 @@ jobs: - rasterframes/setup - rasterframes/restore-cache + - run: + name: "Compile Scala" + command: sbt -v -batch compile + - run: name: "Scala Tests: core" command: sbt -batch core/test @@ -125,7 +132,11 @@ jobs: command: sbt -batch experimental/test - run: - name: Python Tests + name: "Create PyRasterFrames package" + command: sbt -v -batch pyrasterframes/package + + - run: + name: "Python Tests" command: sbt -batch pyrasterframes/test - rasterframes/save-artifacts @@ -140,6 +151,7 @@ jobs: - python/requirements - rasterframes/setup - rasterframes/restore-cache + - run: name: Build documentation command: cat /dev/null | sbt makeSite diff --git a/.sbtopts b/.sbtopts index ca8c83416..f82d4db67 100644 --- a/.sbtopts +++ b/.sbtopts @@ -1 +1,6 @@ --J-XX:MaxMetaspaceSize=1g +-J-XX:+HeapDumpOnOutOfMemoryError +-J-XX:HeapDumpPath=/tmp +-J-XX:+CMSClassUnloadingEnabled +-J-XX:MaxMetaspaceSize=256m +-J-XX:ReservedCodeCacheSize=128m +-J-Xmx512m diff --git a/build.sbt b/build.sbt index 8f1582e35..2450ceb6c 100644 --- a/build.sbt +++ b/build.sbt @@ -134,7 +134,7 @@ lazy val experimental = project spark("sql").value % Provided ), fork in IntegrationTest := true, - javaOptions in IntegrationTest := Seq("-Xmx2G") + //javaOptions in IntegrationTest := Seq("-Xmx2G") ) lazy val docs = project diff --git a/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala b/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala index 398becd95..dfced6c14 100644 --- a/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala +++ b/core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala @@ -137,7 +137,6 @@ object DynamicExtractors { val ymin = value("ymin", "miny") val xmax = value("xmax", "maxx") val ymax = value("ymax", "maxy") - println(Extent(xmin, ymin, xmax, ymax)) Extent(xmin, ymin, xmax, ymax) }) case _ => None diff --git a/project/RFProjectPlugin.scala b/project/RFProjectPlugin.scala index f15e88dda..701e8dc78 100644 --- a/project/RFProjectPlugin.scala +++ b/project/RFProjectPlugin.scala @@ -36,7 +36,8 @@ object RFProjectPlugin extends AutoPlugin { publishArtifact in (Compile, packageDoc) := true, publishArtifact in Test := false, fork in Test := true, - javaOptions in Test := Seq("-Xmx2G", "-Djava.library.path=/usr/local/lib"), + javaOptions in Test := Seq("-Xmx1500m", "-XX:+HeapDumpOnOutOfMemoryError", + "-XX:HeapDumpPath=/tmp", "-Djava.library.path=/usr/local/lib"), parallelExecution in Test := false, testOptions in Test += Tests.Argument("-oDF"), developers := List( diff --git a/project/build.properties b/project/build.properties index a82bb05e1..a919a9b5f 100644 --- a/project/build.properties +++ b/project/build.properties @@ -1 +1 @@ -sbt.version=1.3.7 +sbt.version=1.3.8 From 0ba9b1870c03a720957d14ea6286d8096fd027ac Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Tue, 21 Apr 2020 11:24:24 -0400 Subject: [PATCH 82/94] New docker image for CI environment. --- .circleci/Dockerfile | 123 +++++++----------- .circleci/Makefile | 21 ++- .circleci/config.yml | 2 +- .circleci/fix-permissions | 37 ++++++ .circleci/requirements-conda.txt | 3 + .../geotiff/GeoTiffDataSourceSpec.scala | 2 +- project/RFProjectPlugin.scala | 2 +- 7 files changed, 108 insertions(+), 82 deletions(-) create mode 100755 .circleci/fix-permissions create mode 100644 .circleci/requirements-conda.txt diff --git a/.circleci/Dockerfile b/.circleci/Dockerfile index 9bd966e64..01198c1a1 100644 --- a/.circleci/Dockerfile +++ b/.circleci/Dockerfile @@ -1,89 +1,58 @@ FROM circleci/openjdk:8-jdk -ENV OPENJPEG_VERSION 2.3.1 -ENV GDAL_VERSION 2.4.1 -ENV SPATIALINDEX_VERSION 1.9.3 -ENV JAVA_HOME /usr/lib/jvm/java-8-openjdk-amd64/ +ENV MINICONDA_VERSION=4.8.2 \ + MINICONDA_MD5=87e77f097f6ebb5127c77662dfc3165e \ + CONDA_VERSION=4.8.2 \ + CONDA_DIR=/opt/conda \ + PYTHON_VERSION=3.7.7 -# most of these libraries required for -# python-pip pandoc && pip install setuptools => required for pyrasterframes testing -RUN \ - sudo apt-get update && \ - sudo apt remove \ - python python-minimal python2.7 python2.7-minimal \ - libpython-stdlib libpython2.7 libpython2.7-minimal libpython2.7-stdlib \ - && \ - sudo apt-get install -y \ - pandoc wget \ - gcc g++ build-essential bash-completion cmake imagemagick \ - libreadline-gplv2-dev libncursesw5-dev libssl-dev libsqlite3-dev tk-dev libgdbm-dev libc6-dev libbz2-dev \ - liblzma-dev libcurl4-gnutls-dev libproj-dev libgeos-dev libhdf4-alt-dev libpng-dev libffi-dev \ - && \ - sudo apt autoremove && \ - sudo apt-get clean all +USER root -RUN \ - cd /tmp && \ - wget https://www.python.org/ftp/python/3.7.4/Python-3.7.4.tgz && \ - tar xzf Python-3.7.4.tgz && \ - cd Python-3.7.4 && \ - ./configure --with-ensurepip=install --prefix=/usr/local --enable-optimization && \ - make && \ - sudo make altinstall && \ - rm -rf Python-3.7.4* +ENV PATH=$CONDA_DIR/bin:$PATH -RUN \ - sudo ln -s /usr/local/bin/python3.7 /usr/local/bin/python && \ - sudo curl https://bootstrap.pypa.io/get-pip.py -o get-pip.py && \ - sudo python get-pip.py && \ - sudo pip3 install setuptools ipython==6.2.1 +COPY --chown=3434:3434 fix-permissions /tmp -# install OpenJPEG RUN \ - cd /tmp && \ - wget https://github.com/uclouvain/openjpeg/archive/v${OPENJPEG_VERSION}.tar.gz && \ - tar -xf v${OPENJPEG_VERSION}.tar.gz && \ - cd openjpeg-${OPENJPEG_VERSION}/ && \ - mkdir build && \ - cd build && \ - cmake .. -DCMAKE_BUILD_TYPE=Release -DCMAKE_INSTALL_PREFIX=/usr/local/ && \ - make -j && \ - sudo make install && \ - cd /tmp && rm -Rf v${OPENJPEG_VERSION}.tar.gz openjpeg* + apt-get update && \ + apt-get install -yq --no-install-recommends \ + sudo \ + wget \ + bzip2 \ + file \ + libtinfo5 \ + ca-certificates \ + gettext-base \ + locales && \ + apt-get clean && \ + rm -rf /var/lib/apt/lists/* -# Compile and install GDAL with Java bindings RUN \ cd /tmp && \ - wget http://download.osgeo.org/gdal/${GDAL_VERSION}/gdal-${GDAL_VERSION}.tar.gz && \ - tar -xf gdal-${GDAL_VERSION}.tar.gz && \ - cd gdal-${GDAL_VERSION} && \ - ./configure \ - --with-curl \ - --with-hdf4 \ - --with-geos \ - --with-geotiff=internal \ - --with-hide-internal-symbols \ - --with-libtiff=internal \ - --with-libz=internal \ - --with-mrf \ - --with-openjpeg \ - --with-threads \ - --without-jp2mrsid \ - --without-netcdf \ - --without-ecw && \ - make -j 8 && \ - sudo make install && \ - sudo ldconfig && \ - cd /tmp && sudo rm -Rf gdal* + mkdir -p $CONDA_DIR && \ + wget --quiet https://repo.continuum.io/miniconda/Miniconda3-py37_${MINICONDA_VERSION}-Linux-x86_64.sh && \ + echo "${MINICONDA_MD5} *Miniconda3-py37_${MINICONDA_VERSION}-Linux-x86_64.sh" | md5sum -c - && \ + /bin/bash Miniconda3-py37_${MINICONDA_VERSION}-Linux-x86_64.sh -f -b -p $CONDA_DIR && \ + rm Miniconda3-py37_${MINICONDA_VERSION}-Linux-x86_64.sh && \ + conda config --system --set auto_update_conda false && \ + conda config --system --set show_channel_urls true && \ + conda config --system --set channel_priority strict && \ + if [ ! $PYTHON_VERSION = 'default' ]; then conda install --yes python=$PYTHON_VERSION; fi && \ + conda list python | grep '^python ' | tr -s ' ' | cut -d '.' -f 1,2 | sed 's/$/.*/' >> $CONDA_DIR/conda-meta/pinned && \ + conda install --quiet --yes conda && \ + conda install --quiet --yes pip && \ + echo "$CONDA_DIR/lib" > /etc/ld.so.conf.d/conda.conf && \ + conda clean --all --force-pkgs-dirs --yes --quiet && \ + sh /tmp/fix-permissions $CONDA_DIR 2> /dev/null + +COPY requirements-conda.txt /tmp/ -# Compile and install libspatialindex RUN \ - cd /tmp && \ - wget https://github.com/libspatialindex/libspatialindex/releases/download/${SPATIALINDEX_VERSION}/spatialindex-src-${SPATIALINDEX_VERSION}.tar.gz && \ - tar -xf spatialindex-src-${SPATIALINDEX_VERSION}.tar.gz && \ - cd spatialindex-src-${SPATIALINDEX_VERSION}/ && \ - cmake -DCMAKE_INSTALL_PREFIX=/usr/local/ && \ - make && \ - sudo make install && \ - sudo ldconfig && \ - cd /tmp && sudo rm -Rf spatialindex* \ No newline at end of file + conda install --channel conda-forge --no-channel-priority --freeze-installed \ + --file /tmp/requirements-conda.txt && \ + conda clean --all --force-pkgs-dirs --yes --quiet && \ + sh /tmp/fix-permissions $CONDA_DIR 2> /dev/null && \ + ldconfig 2> /dev/null + +USER 3434 + +WORKDIR /home/circleci diff --git a/.circleci/Makefile b/.circleci/Makefile index 57cef6b1f..35d44a7a5 100644 --- a/.circleci/Makefile +++ b/.circleci/Makefile @@ -1,2 +1,19 @@ -all: - docker build -t "s22s/rasterframes-circleci:latest" . +IMAGE_NAME=miniconda-gdal +VERSION=latest +HOST=docker.pkg.github.com +REPO=${HOST}/locationtech/rasterframes +FULL_NAME=${REPO}/${IMAGE_NAME}:${VERSION} + +all: build login push + +build: + docker build . -t ${FULL_NAME} + +login: + docker login ${HOST} + +push: + docker push ${FULL_NAME} + +shell: build + docker run --rm -it ${FULL_NAME} bash diff --git a/.circleci/config.yml b/.circleci/config.yml index 6085005d6..f330688a6 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -6,7 +6,7 @@ orbs: executors: default: docker: - - image: circleci/openjdk:8-jdk + - image: docker.pkg.github.com/locationtech/rasterframes/miniconda-gdal:latest resource_class: medium working_directory: ~/repo environment: diff --git a/.circleci/fix-permissions b/.circleci/fix-permissions new file mode 100755 index 000000000..2a2bb9d7d --- /dev/null +++ b/.circleci/fix-permissions @@ -0,0 +1,37 @@ +#!/usr/bin/env bash +# set permissions on a directory +# after any installation, if a directory needs to be (human) user-writable, +# run this script on it. +# It will make everything in the directory owned by the group $NB_GID +# and writable by that group. +# Deployments that want to set a specific user id can preserve permissions +# by adding the `--group-add users` line to `docker run`. + +# uses find to avoid touching files that already have the right permissions, +# which would cause massive image explosion + +# right permissions are: +# group=$NB_GID +# AND permissions include group rwX (directory-execute) +# AND directories have setuid,setgid bits set + +set -e + +GID=3434 # circleci + +for d in "$@"; do + find "$d" \ + ! \( \ + -group $GID \ + -a -perm -g+rwX \ + \) \ + -exec chgrp $GID {} \; \ + -exec chmod g+rwX {} \; + # setuid,setgid *on directories only* + find "$d" \ + \( \ + -type d \ + -a ! -perm -6000 \ + \) \ + -exec chmod +6000 {} \; +done diff --git a/.circleci/requirements-conda.txt b/.circleci/requirements-conda.txt new file mode 100644 index 000000000..17c4761d9 --- /dev/null +++ b/.circleci/requirements-conda.txt @@ -0,0 +1,3 @@ +gdal==2.4.4 +libspatialindex +rtree \ No newline at end of file diff --git a/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotiff/GeoTiffDataSourceSpec.scala b/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotiff/GeoTiffDataSourceSpec.scala index f8d4ebcbb..7d74e293c 100644 --- a/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotiff/GeoTiffDataSourceSpec.scala +++ b/datasource/src/test/scala/org/locationtech/rasterframes/datasource/geotiff/GeoTiffDataSourceSpec.scala @@ -269,7 +269,7 @@ class GeoTiffDataSourceSpec s"https://modis-pds.s3.amazonaws.com/MCD43A4.006/11/08/2019059/" + s"MCD43A4.A2019059.h11v08.006.2019072203257_B0${band}.TIF" - it("shoud write multiband") { + it("should write multiband") { import org.locationtech.rasterframes.datasource.raster._ val cat = s""" diff --git a/project/RFProjectPlugin.scala b/project/RFProjectPlugin.scala index 701e8dc78..5ecc83fb1 100644 --- a/project/RFProjectPlugin.scala +++ b/project/RFProjectPlugin.scala @@ -36,7 +36,7 @@ object RFProjectPlugin extends AutoPlugin { publishArtifact in (Compile, packageDoc) := true, publishArtifact in Test := false, fork in Test := true, - javaOptions in Test := Seq("-Xmx1500m", "-XX:+HeapDumpOnOutOfMemoryError", + javaOptions in Test := Seq("-Xmx2g", "-XX:+HeapDumpOnOutOfMemoryError", "-XX:HeapDumpPath=/tmp", "-Djava.library.path=/usr/local/lib"), parallelExecution in Test := false, testOptions in Test += Tests.Argument("-oDF"), From a4c8c57267096fc6ba412dfff76db09c9e3bedf3 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Tue, 21 Apr 2020 12:54:23 -0400 Subject: [PATCH 83/94] Adding credentials to pull the (public) build image from GitHub, because GitHub wants it that way. --- .circleci/config.yml | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index f330688a6..e4a5b373c 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -7,6 +7,9 @@ orbs: default: docker: - image: docker.pkg.github.com/locationtech/rasterframes/miniconda-gdal:latest + auth: + username: $GITHUB_USERNAME + password: $GITHUB_PASSWORD resource_class: medium working_directory: ~/repo environment: @@ -196,18 +199,22 @@ workflows: version: 2 all: jobs: - - test + - test: + context: rasterframes - it: + context: rasterframes filters: branches: only: - /feature\/.*-its/ - itWithoutGdal: + context: rasterframes filters: branches: only: - /feature\/.*-its/ - docs: + context: rasterframes filters: branches: only: From 98edbeb91d7233342b9de4c45c828f3a4a634507 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Tue, 21 Apr 2020 13:09:49 -0400 Subject: [PATCH 84/94] Dropped testing heap back to 1.5g. --- project/RFProjectPlugin.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/project/RFProjectPlugin.scala b/project/RFProjectPlugin.scala index 5ecc83fb1..08566a503 100644 --- a/project/RFProjectPlugin.scala +++ b/project/RFProjectPlugin.scala @@ -36,8 +36,8 @@ object RFProjectPlugin extends AutoPlugin { publishArtifact in (Compile, packageDoc) := true, publishArtifact in Test := false, fork in Test := true, - javaOptions in Test := Seq("-Xmx2g", "-XX:+HeapDumpOnOutOfMemoryError", - "-XX:HeapDumpPath=/tmp", "-Djava.library.path=/usr/local/lib"), + javaOptions in Test := Seq("-Xmx1500m", "-XX:+HeapDumpOnOutOfMemoryError", + "-XX:HeapDumpPath=/tmp"), parallelExecution in Test := false, testOptions in Test += Tests.Argument("-oDF"), developers := List( From fb434b01fdd6e71f1a4ccdc8c67a92d9ebd2fdf7 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Tue, 21 Apr 2020 13:36:27 -0400 Subject: [PATCH 85/94] Trying pre-installation of pyspark to work around execution bit error from setuptools install. --- .circleci/config.yml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index e4a5b373c..dd727d05f 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -136,7 +136,9 @@ jobs: - run: name: "Create PyRasterFrames package" - command: sbt -v -batch pyrasterframes/package + command: |- + python -m pip install --user pyspark==2.4.5 + sbt -v -batch pyrasterframes/package - run: name: "Python Tests" From 7eb0dde012073492af8ab5676d61e429a4c0d4ad Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Tue, 21 Apr 2020 13:52:30 -0400 Subject: [PATCH 86/94] Applied PROJ_LIB fix to test runtime image. --- .circleci/Dockerfile | 3 +++ .../rasterframes/functions/TileFunctionsSpec.scala | 2 -- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/.circleci/Dockerfile b/.circleci/Dockerfile index 01198c1a1..a15fd242e 100644 --- a/.circleci/Dockerfile +++ b/.circleci/Dockerfile @@ -53,6 +53,9 @@ RUN \ sh /tmp/fix-permissions $CONDA_DIR 2> /dev/null && \ ldconfig 2> /dev/null +# Work-around for pyproj issue https://github.com/pyproj4/pyproj/issues/415 +ENV PROJ_LIB=/opt/conda/share/proj + USER 3434 WORKDIR /home/circleci diff --git a/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala b/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala index 836561c1d..2a4277cf7 100644 --- a/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala +++ b/core/src/test/scala/org/locationtech/rasterframes/functions/TileFunctionsSpec.scala @@ -261,8 +261,6 @@ class TileFunctionsSpec extends TestEnvironment with RasterMatchers { it("should evaluate rf_where"){ val df = Seq((randPRT, one, six)).toDF("t", "one", "six") - df.select(rf_render_matrix(rf_where(rf_local_greater($"t", 0), $"one", $"six") as "result")).show(false) - val result = df.select( rf_for_all( rf_local_equal( From 110bc08a9e62e817568e8621c25df2b841799c5c Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Tue, 21 Apr 2020 14:01:41 -0400 Subject: [PATCH 87/94] Bumped spark to 2.4.5. --- docs/src/main/paradox/release-notes.md | 2 +- pyrasterframes/src/main/python/docs/getting-started.pymd | 2 +- pyrasterframes/src/main/python/requirements.txt | 2 +- pyrasterframes/src/main/python/setup.py | 2 +- rf-notebook/src/main/docker/Dockerfile | 2 +- rf-notebook/src/main/docker/requirements-nb.txt | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/src/main/paradox/release-notes.md b/docs/src/main/paradox/release-notes.md index 4f89c72f6..c67c53089 100644 --- a/docs/src/main/paradox/release-notes.md +++ b/docs/src/main/paradox/release-notes.md @@ -30,7 +30,7 @@ * Add cell value scaling functions `rf_rescale` and `rf_standardize`. * Add `rf_where` function, similar in spirit to numpy's `where`, or a cell-wise version of Spark SQL's `when` and `otherwise`. * Add `rf_sqrt` function to compute cell-wise square root. - +* Upgraded to Spark 2.4.5 ## 0.8.x diff --git a/pyrasterframes/src/main/python/docs/getting-started.pymd b/pyrasterframes/src/main/python/docs/getting-started.pymd index c04044a34..11d3c8363 100644 --- a/pyrasterframes/src/main/python/docs/getting-started.pymd +++ b/pyrasterframes/src/main/python/docs/getting-started.pymd @@ -127,7 +127,7 @@ libraryDependencies ++= Seq( ) ``` -RasterFrames is compatible with Spark 2.4.4. +RasterFrames is compatible with Spark 2.4.x. ## Installing GDAL Support diff --git a/pyrasterframes/src/main/python/requirements.txt b/pyrasterframes/src/main/python/requirements.txt index f12870aae..1ea3ac8fe 100644 --- a/pyrasterframes/src/main/python/requirements.txt +++ b/pyrasterframes/src/main/python/requirements.txt @@ -1,5 +1,5 @@ ipython==6.2.1 -pyspark==2.4.4 +pyspark==2.4.5 gdal==2.4.4 numpy>=1.17.3,<2.0 pandas>=0.25.3,<1.0 diff --git a/pyrasterframes/src/main/python/setup.py b/pyrasterframes/src/main/python/setup.py index db9810464..4649787d3 100644 --- a/pyrasterframes/src/main/python/setup.py +++ b/pyrasterframes/src/main/python/setup.py @@ -148,7 +148,7 @@ def dest_file(self, src_file): pandas = 'pandas>=0.25.3,<1.0' pweave = 'pweave==0.30.3' pypandoc = 'pypandoc' -pyspark = 'pyspark==2.4.4' +pyspark = 'pyspark==2.4.5' pytest = 'pytest>=4.0.0,<5.0.0' pytest_runner = 'pytest-runner' pytz = 'pytz' diff --git a/rf-notebook/src/main/docker/Dockerfile b/rf-notebook/src/main/docker/Dockerfile index 5e75d8a9b..99d979577 100644 --- a/rf-notebook/src/main/docker/Dockerfile +++ b/rf-notebook/src/main/docker/Dockerfile @@ -13,7 +13,7 @@ RUN \ rm -rf /var/lib/apt/lists/* # Spark dependencies -ENV APACHE_SPARK_VERSION 2.4.4 +ENV APACHE_SPARK_VERSION 2.4.5 ENV HADOOP_VERSION 2.7 ENV APACHE_SPARK_CHECKSUM 2E3A5C853B9F28C7D4525C0ADCB0D971B73AD47D5CCE138C85335B9F53A6519540D3923CB0B5CEE41E386E49AE8A409A51AB7194BA11A254E037A848D0C4A9E5 ENV APACHE_SPARK_FILENAME spark-${APACHE_SPARK_VERSION}-bin-hadoop${HADOOP_VERSION}.tgz diff --git a/rf-notebook/src/main/docker/requirements-nb.txt b/rf-notebook/src/main/docker/requirements-nb.txt index f4ad77f5a..929ac3eaf 100644 --- a/rf-notebook/src/main/docker/requirements-nb.txt +++ b/rf-notebook/src/main/docker/requirements-nb.txt @@ -1,4 +1,4 @@ -pyspark=2.4.4 +pyspark=2.4.5 gdal=2.4.4 numpy>=1.17.3,<2.0 pandas>=0.25.3,<1.0 From 8d2adfc62ffdf37daf7d96cb5073522bac1981d0 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Tue, 21 Apr 2020 14:15:51 -0400 Subject: [PATCH 88/94] Moving sbt memory limit to CircleCI config. --- .circleci/config.yml | 1 + .sbtopts | 1 - 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index dd727d05f..dd53f0ad5 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -14,6 +14,7 @@ orbs: working_directory: ~/repo environment: SBT_VERSION: 1.3.8 + SBT_OPTS: -Xmx512m commands: setup: description: Setup for sbt build diff --git a/.sbtopts b/.sbtopts index f82d4db67..3f292a6da 100644 --- a/.sbtopts +++ b/.sbtopts @@ -3,4 +3,3 @@ -J-XX:+CMSClassUnloadingEnabled -J-XX:MaxMetaspaceSize=256m -J-XX:ReservedCodeCacheSize=128m --J-Xmx512m From e75906b3be5d0af0d8b260503c760bde8246e7bc Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Tue, 21 Apr 2020 14:55:54 -0400 Subject: [PATCH 89/94] Added test setup step to make sure spark-submit is executable. --- .../src/main/python/tests/__init__.py | 21 +++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/pyrasterframes/src/main/python/tests/__init__.py b/pyrasterframes/src/main/python/tests/__init__.py index 330857c14..5541238f3 100644 --- a/pyrasterframes/src/main/python/tests/__init__.py +++ b/pyrasterframes/src/main/python/tests/__init__.py @@ -23,10 +23,27 @@ from pyrasterframes.utils import create_rf_spark_session - import builtins -app_name = 'pyrasterframes test suite' +app_name = 'PyRasterFrames test suite' + +# Setuptools/easy_install doesn't properly set the execute bit on the Spark scripts, +# So this preemptively attempts to do it. +def _chmodit(): + try: + from importlib.util import find_spec + module_home = find_spec("pyspark").origin + print(module_home) + bin_dir = os.path.join(os.path.dirname(module_home), 'bin') + for filename in os.listdir(bin_dir): + try: + os.chmod(os.path.join(bin_dir, filename), mode=0o555, follow_symlinks=True) + except OSError: + pass + except ImportError: + pass + +_chmodit() def resource_dir(): From 9ad13f2fa7e0ae2b1206438d2fab0077daf6ad6a Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Tue, 21 Apr 2020 16:42:31 -0400 Subject: [PATCH 90/94] Tweaked filter rules for it jobs. --- .circleci/Dockerfile | 2 ++ .circleci/config.yml | 28 +++++++++++++++++----------- 2 files changed, 19 insertions(+), 11 deletions(-) diff --git a/.circleci/Dockerfile b/.circleci/Dockerfile index a15fd242e..d498294c2 100644 --- a/.circleci/Dockerfile +++ b/.circleci/Dockerfile @@ -10,6 +10,7 @@ USER root ENV PATH=$CONDA_DIR/bin:$PATH +# circleci is 3434 COPY --chown=3434:3434 fix-permissions /tmp RUN \ @@ -40,6 +41,7 @@ RUN \ conda list python | grep '^python ' | tr -s ' ' | cut -d '.' -f 1,2 | sed 's/$/.*/' >> $CONDA_DIR/conda-meta/pinned && \ conda install --quiet --yes conda && \ conda install --quiet --yes pip && \ + pip config set global.progress_bar off && \ echo "$CONDA_DIR/lib" > /etc/ld.so.conf.d/conda.conf && \ conda clean --all --force-pkgs-dirs --yes --quiet && \ sh /tmp/fix-permissions $CONDA_DIR 2> /dev/null diff --git a/.circleci/config.yml b/.circleci/config.yml index dd53f0ad5..e7c0e3527 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -31,9 +31,6 @@ orbs: - run: name: Install Python and PIP command: |- - sudo apt-get update -q -y - sudo apt-get install python3 python3-pip - sudo update-alternatives --install /usr/bin/python python /usr/bin/python3.7 1 python -m pip install --user 'setuptools>=45.2' requirements: @@ -161,7 +158,7 @@ jobs: - run: name: Build documentation command: cat /dev/null | sbt makeSite - no-output-timeout: 30m + no_output_timeout: 30m - rasterframes/save-doc-artifacts - rasterframes/save-cache @@ -176,13 +173,13 @@ jobs: - run: name: Integration tests - command: cat /dev/null | sbt it:test + command: sbt it:test no_output_timeout: 30m - rasterframes/save-artifacts - rasterframes/save-cache - itWithoutGdal: + it-no-gdal: executor: sbt/default steps: - checkout @@ -190,9 +187,13 @@ jobs: - rasterframes/setup - rasterframes/restore-cache + - run: + name: Uninstall GDAL + command: conda remove gdal -q -y --offline + - run: name: Integration tests - command: cat /dev/null | sbt it:test + command: sbt it:test no_output_timeout: 30m - rasterframes/save-artifacts @@ -204,18 +205,23 @@ workflows: jobs: - test: context: rasterframes + - it: context: rasterframes filters: branches: only: - - /feature\/.*-its/ - - itWithoutGdal: + - /feature\/.*-it.*/ + - /it\/.*/ + + - it-no-gdal: context: rasterframes filters: branches: only: - - /feature\/.*-its/ + - /feature\/.*-it.*/ + - /it\/.*/ + - docs: context: rasterframes filters: @@ -236,5 +242,5 @@ workflows: jobs: - test - it - - itWithoutGdal + - it-no-gdal - docs From 2fc268105503b98fd57b5bc419fe6a98bfbc350d Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Tue, 21 Apr 2020 17:10:08 -0400 Subject: [PATCH 91/94] Refactored scala compile step; reduced integration test memory. --- .circleci/config.yml | 25 +++++++++++++++++++------ .sbtopts | 6 +----- build.sbt | 6 ++++-- 3 files changed, 24 insertions(+), 13 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index e7c0e3527..7af2462c4 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -14,7 +14,7 @@ orbs: working_directory: ~/repo environment: SBT_VERSION: 1.3.8 - SBT_OPTS: -Xmx512m + SBT_OPTS: -Xmx768m commands: setup: description: Setup for sbt build @@ -23,6 +23,14 @@ orbs: name: Setup sbt command: 'true' # NOOP + compile: + description: Do just the compilation stage to minimize sbt memory footprint + steps: + - run: + name: "Compile Scala via sbt" + command: |- + sbt -v -batch compile test:compile it:compile + python: commands: setup: @@ -115,10 +123,7 @@ jobs: - python/setup - rasterframes/setup - rasterframes/restore-cache - - - run: - name: "Compile Scala" - command: sbt -v -batch compile + - sbt/compile - run: name: "Scala Tests: core" @@ -154,10 +159,11 @@ jobs: - python/requirements - rasterframes/setup - rasterframes/restore-cache + - sbt/compile - run: name: Build documentation - command: cat /dev/null | sbt makeSite + command: sbt makeSite no_output_timeout: 30m - rasterframes/save-doc-artifacts @@ -170,6 +176,7 @@ jobs: - sbt/setup - rasterframes/setup - rasterframes/restore-cache + - sbt/compile - run: name: Integration tests @@ -191,6 +198,8 @@ jobs: name: Uninstall GDAL command: conda remove gdal -q -y --offline + - sbt/compile + - run: name: Integration tests command: sbt it:test @@ -208,6 +217,8 @@ workflows: - it: context: rasterframes +# requires: +# - test filters: branches: only: @@ -216,6 +227,8 @@ workflows: - it-no-gdal: context: rasterframes +# requires: +# - test filters: branches: only: diff --git a/.sbtopts b/.sbtopts index 3f292a6da..8b1378917 100644 --- a/.sbtopts +++ b/.sbtopts @@ -1,5 +1 @@ --J-XX:+HeapDumpOnOutOfMemoryError --J-XX:HeapDumpPath=/tmp --J-XX:+CMSClassUnloadingEnabled --J-XX:MaxMetaspaceSize=256m --J-XX:ReservedCodeCacheSize=128m + diff --git a/build.sbt b/build.sbt index 2450ceb6c..faa2b4cf7 100644 --- a/build.sbt +++ b/build.sbt @@ -117,7 +117,9 @@ lazy val datasource = project """ |import org.locationtech.rasterframes.datasource.geotrellis._ |import org.locationtech.rasterframes.datasource.geotiff._ - |""".stripMargin + |""".stripMargin, + fork in IntegrationTest := true, + javaOptions in IntegrationTest := Seq("-Xmx1500m") ) lazy val experimental = project @@ -134,7 +136,7 @@ lazy val experimental = project spark("sql").value % Provided ), fork in IntegrationTest := true, - //javaOptions in IntegrationTest := Seq("-Xmx2G") + javaOptions in IntegrationTest := Seq("-Xmx1500m") ) lazy val docs = project From 2c34001e99e6d48917a2943d8d456910eafd8120 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Wed, 22 Apr 2020 09:12:38 -0400 Subject: [PATCH 92/94] Throwing in the towel with running IT in medium compute configuration. --- .circleci/config.yml | 2 ++ build.sbt | 10 +++++----- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 7af2462c4..a0ab4a13e 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -171,6 +171,7 @@ jobs: it: executor: sbt/default + resource_class: large steps: - checkout - sbt/setup @@ -188,6 +189,7 @@ jobs: it-no-gdal: executor: sbt/default + resource_class: large steps: - checkout - sbt/setup diff --git a/build.sbt b/build.sbt index faa2b4cf7..7662baa5c 100644 --- a/build.sbt +++ b/build.sbt @@ -113,13 +113,13 @@ lazy val datasource = project spark("mllib").value % Provided, spark("sql").value % Provided ), - initialCommands in console := (initialCommands in console).value + + console / initialCommands := (console / initialCommands).value + """ |import org.locationtech.rasterframes.datasource.geotrellis._ |import org.locationtech.rasterframes.datasource.geotiff._ |""".stripMargin, - fork in IntegrationTest := true, - javaOptions in IntegrationTest := Seq("-Xmx1500m") + IntegrationTest / fork := true, + IntegrationTest / javaOptions := Seq("-Xmx3g") ) lazy val experimental = project @@ -135,8 +135,8 @@ lazy val experimental = project spark("mllib").value % Provided, spark("sql").value % Provided ), - fork in IntegrationTest := true, - javaOptions in IntegrationTest := Seq("-Xmx1500m") + IntegrationTest / fork := true, + IntegrationTest / javaOptions := (datasource / IntegrationTest / javaOptions).value ) lazy val docs = project From 7df6d21fb8f75e82d62045006393a8b4d631fe47 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Wed, 22 Apr 2020 13:42:36 -0400 Subject: [PATCH 93/94] Misc build tweaks. --- .travis.yml | 36 ------------------- core/src/it/resources/log4j.properties | 2 ++ docs/src/main/paradox/release-notes.md | 3 +- .../src/main/python/docs/__init__.py | 19 ++++++++++ .../src/main/python/pyrasterframes/utils.py | 8 +++++ 5 files changed, 31 insertions(+), 37 deletions(-) delete mode 100644 .travis.yml diff --git a/.travis.yml b/.travis.yml deleted file mode 100644 index 9b6f44ea2..000000000 --- a/.travis.yml +++ /dev/null @@ -1,36 +0,0 @@ -dist: xenial -language: python - -python: - - "3.7" - -cache: - directories: - - $HOME/.ivy2/cache - - $HOME/.sbt/boot - - $HOME/.rf_cache - - $HOME/.cache/coursier - -scala: - - 2.11.11 - -env: - - COURSIER_VERBOSITY=-1 JAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64 - -addons: - apt: - packages: - - openjdk-8-jdk - - pandoc - -install: - - pip install rasterio shapely pandas numpy pweave - - wget -O - https://piccolo.link/sbt-1.2.8.tgz | tar xzf - - - -jobs: - include: - - stage: "Unit Tests" - script: sbt/bin/sbt -java-home $JAVA_HOME -batch test - - stage: "Integration Tests" - script: sbt/bin/sbt -java-home $JAVA_HOME -batch it:test diff --git a/core/src/it/resources/log4j.properties b/core/src/it/resources/log4j.properties index 1135e4b34..94c1d1b92 100644 --- a/core/src/it/resources/log4j.properties +++ b/core/src/it/resources/log4j.properties @@ -40,6 +40,8 @@ log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO log4j.logger.org.locationtech.rasterframes=WARN log4j.logger.org.locationtech.rasterframes.ref=WARN log4j.logger.org.apache.parquet.hadoop.ParquetRecordReader=OFF +log4j.logger.geotrellis.spark=INFO +log4j.logger.geotrellis.raster.gdal=ERROR # SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL diff --git a/docs/src/main/paradox/release-notes.md b/docs/src/main/paradox/release-notes.md index c67c53089..1ad1482e0 100644 --- a/docs/src/main/paradox/release-notes.md +++ b/docs/src/main/paradox/release-notes.md @@ -23,6 +23,8 @@ - Revisit use of `Tile` equality since [it's more strict](https://github.com/locationtech/geotrellis/pull/2991) - Update `reference.conf` to use `geotrellis.raster.gdal` namespace. - Replace all uses of `TileDimensions` with `geotrellis.raster.Dimensions[Int]`. +* Upgraded to `gdal-warp-bindings` 1.0.0. +* Upgraded to Spark 2.4.5 * Formally abandoned support for Python 2. Python 2 is dead. Long live Python 2. * Introduction of type hints in Python API. * Add functions for changing cell values based on either conditions or to achieve a distribution of values. ([#449](https://github.com/locationtech/rasterframes/pull/449)) @@ -30,7 +32,6 @@ * Add cell value scaling functions `rf_rescale` and `rf_standardize`. * Add `rf_where` function, similar in spirit to numpy's `where`, or a cell-wise version of Spark SQL's `when` and `otherwise`. * Add `rf_sqrt` function to compute cell-wise square root. -* Upgraded to Spark 2.4.5 ## 0.8.x diff --git a/pyrasterframes/src/main/python/docs/__init__.py b/pyrasterframes/src/main/python/docs/__init__.py index 0f728b435..0fa3d800b 100644 --- a/pyrasterframes/src/main/python/docs/__init__.py +++ b/pyrasterframes/src/main/python/docs/__init__.py @@ -20,6 +20,25 @@ from pweave import PwebPandocFormatter +# Setuptools/easy_install doesn't properly set the execute bit on the Spark scripts, +# So this preemptively attempts to do it. +def _chmodit(): + try: + from importlib.util import find_spec + import os + module_home = find_spec("pyspark").origin + print(module_home) + bin_dir = os.path.join(os.path.dirname(module_home), 'bin') + for filename in os.listdir(bin_dir): + try: + os.chmod(os.path.join(bin_dir, filename), mode=0o555, follow_symlinks=True) + except OSError: + pass + except ImportError: + pass + +_chmodit() + class PegdownMarkdownFormatter(PwebPandocFormatter): def __init__(self, *args, **kwargs): diff --git a/pyrasterframes/src/main/python/pyrasterframes/utils.py b/pyrasterframes/src/main/python/pyrasterframes/utils.py index 806d7015d..328a62ccf 100644 --- a/pyrasterframes/src/main/python/pyrasterframes/utils.py +++ b/pyrasterframes/src/main/python/pyrasterframes/utils.py @@ -68,6 +68,12 @@ def find_pyrasterframes_assembly() -> Union[bytes, str]: return jarpath[0] +def quiet_logs(sc): + logger = sc._jvm.org.apache.log4j + logger.LogManager.getLogger("geotrellis.raster.gdal").setLevel(logger.Level.ERROR) + logger.LogManager.getLogger("akka").setLevel(logger.Level.ERROR) + + def create_rf_spark_session(master="local[*]", **kwargs: str) -> SparkSession: """ Create a SparkSession with pyrasterframes enabled and configured. """ jar_path = find_pyrasterframes_assembly() @@ -86,6 +92,8 @@ def create_rf_spark_session(master="local[*]", **kwargs: str) -> SparkSession: .config(conf=conf) # user can override the defaults .getOrCreate()) + quiet_logs(spark) + try: spark.withRasterFrames() return spark From 9f6b87636fdbe1824c8d7b4afb5523142e212507 Mon Sep 17 00:00:00 2001 From: "Simeon H.K. Fitch" Date: Wed, 22 Apr 2020 17:41:04 -0400 Subject: [PATCH 94/94] Release prep. --- RELEASE.md | 9 +++++---- experimental/src/it/resources/log4j.properties | 2 ++ pyrasterframes/src/main/python/pyrasterframes/version.py | 2 +- rf-notebook/src/main/docker/Dockerfile | 2 +- version.sbt | 2 +- 5 files changed, 10 insertions(+), 7 deletions(-) diff --git a/RELEASE.md b/RELEASE.md index 613910283..745e895cc 100644 --- a/RELEASE.md +++ b/RELEASE.md @@ -8,10 +8,11 @@ a. `clean` b. `test it:test` c. `makeSite` - d. `publishSigned` (LocationTech credentials required) - e. `sonatypeReleaseAll`. It can take a while, but should eventually show up [here](https://search.maven.org/search?q=g:org.locationtech.rasterframes). - f. `docs/ghpagesPushSite` - g. `rf-notebook/publish` + d. `rf-notebook/publishLocal` + e. `publishSigned` (LocationTech credentials required) + f. `sonatypeReleaseAll`. It can take a while, but should eventually show up [here](https://search.maven.org/search?q=g:org.locationtech.rasterframes). + g. `docs/ghpagesPushSite` + h. `rf-notebook/publish` 6. `cd pyrasterframes/target/python/dist` 7. `python3 -m twine upload pyrasterframes-x.y.z-py2.py3-none-any.whl` 8. Commit any changes that were necessary. diff --git a/experimental/src/it/resources/log4j.properties b/experimental/src/it/resources/log4j.properties index 4a81f524a..cbbdd4af2 100644 --- a/experimental/src/it/resources/log4j.properties +++ b/experimental/src/it/resources/log4j.properties @@ -37,6 +37,8 @@ log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO log4j.logger.org.locationtech.rasterframes=INFO log4j.logger.org.apache.parquet.hadoop.ParquetRecordReader=OFF +log4j.logger.geotrellis.spark=INFO +log4j.logger.geotrellis.raster.gdal=ERROR # SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL diff --git a/pyrasterframes/src/main/python/pyrasterframes/version.py b/pyrasterframes/src/main/python/pyrasterframes/version.py index 86c68f9f5..11da96a82 100644 --- a/pyrasterframes/src/main/python/pyrasterframes/version.py +++ b/pyrasterframes/src/main/python/pyrasterframes/version.py @@ -20,4 +20,4 @@ # # Translating Java version from version.sbt to PEP440 norms -__version__: str = '0.9.0.dev0' +__version__: str = '0.9.0' diff --git a/rf-notebook/src/main/docker/Dockerfile b/rf-notebook/src/main/docker/Dockerfile index 99d979577..dba7f9c0c 100644 --- a/rf-notebook/src/main/docker/Dockerfile +++ b/rf-notebook/src/main/docker/Dockerfile @@ -15,7 +15,7 @@ RUN \ # Spark dependencies ENV APACHE_SPARK_VERSION 2.4.5 ENV HADOOP_VERSION 2.7 -ENV APACHE_SPARK_CHECKSUM 2E3A5C853B9F28C7D4525C0ADCB0D971B73AD47D5CCE138C85335B9F53A6519540D3923CB0B5CEE41E386E49AE8A409A51AB7194BA11A254E037A848D0C4A9E5 +ENV APACHE_SPARK_CHECKSUM 2426a20c548bdfc07df288cd1d18d1da6b3189d0b78dee76fa034c52a4e02895f0ad460720c526f163ba63a17efae4764c46a1cd8f9b04c60f9937a554db85d2 ENV APACHE_SPARK_FILENAME spark-${APACHE_SPARK_VERSION}-bin-hadoop${HADOOP_VERSION}.tgz ENV APACHE_SPARK_REMOTE_PATH spark-${APACHE_SPARK_VERSION}/${APACHE_SPARK_FILENAME} diff --git a/version.sbt b/version.sbt index 338b0ba29..1b5f9da59 100644 --- a/version.sbt +++ b/version.sbt @@ -1 +1 @@ -version in ThisBuild := "0.9.0-SNAPSHOT" +version in ThisBuild := "0.9.0"