From 9c3c7cbbc89ecf0e2ca088eeb6c358200c7d0ba6 Mon Sep 17 00:00:00 2001 From: "milos.colic" Date: Mon, 15 Jan 2024 13:29:57 +0000 Subject: [PATCH 01/47] Add capabilities for RST_ stats. Refactor RasterToGrid readers. --- .../raster/gdal/MosaicRasterBandGDAL.scala | 24 ++ .../core/raster/gdal/MosaicRasterGDAL.scala | 4 +- .../core/raster/operator/gdal/GDALInfo.scala | 38 +++ .../operator/retile/BalancedSubdivision.scala | 9 +- .../multiread/RasterAsGridReader.scala | 52 +++- .../mosaic/expressions/raster/RST_Avg.scala | 57 ++++ .../mosaic/expressions/raster/RST_Max.scala | 48 ++++ .../expressions/raster/RST_Median.scala | 60 +++++ .../mosaic/expressions/raster/RST_Min.scala | 48 ++++ .../expressions/raster/RST_PixelCount.scala | 46 ++++ .../labs/mosaic/functions/MosaicContext.scala | 16 +- .../labs/mosaic/utils/FileUtils.scala | 34 +++ .../multiread/RasterAsGridReaderTest.scala | 245 ++++++++++-------- .../expressions/raster/RST_MaxBehaviors.scala | 52 ++++ .../expressions/raster/RST_MaxTest.scala | 32 +++ .../raster/RST_MedianBehaviors.scala | 52 ++++ .../expressions/raster/RST_MedianTest.scala | 32 +++ .../raster/RST_PixelCountBehaviors.scala | 52 ++++ .../raster/RST_PixelCountTest.scala | 32 +++ .../sql/test/MosaicTestSparkSession.scala | 27 ++ .../sql/test/SharedSparkSessionGDAL.scala | 11 +- 21 files changed, 837 insertions(+), 134 deletions(-) create mode 100644 src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALInfo.scala create mode 100644 src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Avg.scala create mode 100644 src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Max.scala create mode 100644 src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Median.scala create mode 100644 src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Min.scala create mode 100644 src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_PixelCount.scala create mode 100644 src/main/scala/com/databricks/labs/mosaic/utils/FileUtils.scala create mode 100644 src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MaxBehaviors.scala create mode 100644 src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MaxTest.scala create mode 100644 src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MedianBehaviors.scala create mode 100644 src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MedianTest.scala create mode 100644 src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_PixelCountBehaviors.scala create mode 100644 src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_PixelCountTest.scala create mode 100644 src/test/scala/org/apache/spark/sql/test/MosaicTestSparkSession.scala diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterBandGDAL.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterBandGDAL.scala index 3fa45f8e5..a7c9ece10 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterBandGDAL.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterBandGDAL.scala @@ -219,6 +219,30 @@ case class MosaicRasterBandGDAL(band: Band, id: Int) { } } + /** + * Counts the number of pixels in the band. The mask is used to determine + * if a pixel is valid. If pixel value is noData or mask value is 0.0, the + * pixel is not counted. + * + * @return + * Returns the band's pixel count. + */ + def pixelCount: Int = { + val line = Array.ofDim[Double](band.GetXSize()) + val maskLine = Array.ofDim[Double](band.GetXSize()) + var count = 0 + for (y <- 0 until band.GetYSize()) { + band.ReadRaster(0, y, band.GetXSize(), 1, line) + val maskRead = band.GetMaskBand().ReadRaster(0, y, band.GetXSize(), 1, maskLine) + if (maskRead != gdalconstConstants.CE_None) { + count = count + line.count(_ != noDataValue) + } else { + count = count + line.zip(maskLine).count { case (pixel, mask) => pixel != noDataValue && mask != 0.0 } + } + } + count + } + /** * @return * Returns the band's mask flags. diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala index 4f51749dc..3ac467f53 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala @@ -8,7 +8,7 @@ import com.databricks.labs.mosaic.core.raster.io.RasterCleaner.dispose import com.databricks.labs.mosaic.core.raster.io.{RasterCleaner, RasterReader, RasterWriter} import com.databricks.labs.mosaic.core.raster.operator.clip.RasterClipByVector import com.databricks.labs.mosaic.core.types.model.GeometryTypeEnum.POLYGON -import com.databricks.labs.mosaic.utils.PathUtils +import com.databricks.labs.mosaic.utils.{FileUtils, PathUtils} import org.gdal.gdal.gdal.GDALInfo import org.gdal.gdal.{Dataset, InfoOptions, gdal} import org.gdal.gdalconst.gdalconstConstants._ @@ -405,7 +405,7 @@ case class MosaicRasterGDAL( } else { path } - val byteArray = Files.readAllBytes(Paths.get(readPath)) + val byteArray = FileUtils.readBytes(readPath) if (dispose) RasterCleaner.dispose(this) if (readPath != PathUtils.getCleanPath(parentPath)) { Files.deleteIfExists(Paths.get(readPath)) diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALInfo.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALInfo.scala new file mode 100644 index 000000000..7a60a837a --- /dev/null +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALInfo.scala @@ -0,0 +1,38 @@ +package com.databricks.labs.mosaic.core.raster.operator.gdal + +import com.databricks.labs.mosaic.core.raster.gdal.MosaicRasterGDAL +import org.gdal.gdal.{InfoOptions, gdal} + +/** GDALBuildVRT is a wrapper for the GDAL BuildVRT command. */ +object GDALInfo { + + /** + * Executes the GDAL BuildVRT command. For flags check the way gdalinfo.py + * script is called, InfoOptions expects a collection of same flags. + * + * @param raster + * The raster to get info from. + * @param command + * The GDAL Info command. + * @return + * A result json string. + */ + def executeInfo(raster: MosaicRasterGDAL, command: String): String = { + require(command.startsWith("gdalinfo"), "Not a valid GDAL Info command.") + + val infoOptionsVec = OperatorOptions.parseOptions(command) + val infoOptions = new InfoOptions(infoOptionsVec) + val gdalInfo = gdal.GDALInfo(raster.getRaster, infoOptions) + + if (gdalInfo == null) { + throw new Exception(s""" + |GDAL Info failed. + |Command: $command + |Error: ${gdal.GetLastErrorMsg} + |""".stripMargin) + } + + gdalInfo + } + +} diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/BalancedSubdivision.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/BalancedSubdivision.scala index 75e59c1fa..daa0e6266 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/BalancedSubdivision.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/BalancedSubdivision.scala @@ -21,9 +21,12 @@ object BalancedSubdivision { */ def getNumSplits(raster: MosaicRasterGDAL, destSize: Int): Int = { val size = raster.getMemSize - val n = size.toDouble / (destSize * 1000 * 1000) - val nInt = Math.ceil(n).toInt - Math.pow(4, Math.ceil(Math.log(nInt) / Math.log(4))).toInt + var n = 1 + while (true) { + n *= 4 + if (size / n <= destSize * 1000 * 1000) return n + } + n } /** diff --git a/src/main/scala/com/databricks/labs/mosaic/datasource/multiread/RasterAsGridReader.scala b/src/main/scala/com/databricks/labs/mosaic/datasource/multiread/RasterAsGridReader.scala index c1f805afa..d6f26caf4 100644 --- a/src/main/scala/com/databricks/labs/mosaic/datasource/multiread/RasterAsGridReader.scala +++ b/src/main/scala/com/databricks/labs/mosaic/datasource/multiread/RasterAsGridReader.scala @@ -1,5 +1,6 @@ package com.databricks.labs.mosaic.datasource.multiread +import com.databricks.labs.mosaic.MOSAIC_RASTER_READ_STRATEGY import com.databricks.labs.mosaic.functions.MosaicContext import org.apache.spark.sql._ import org.apache.spark.sql.functions._ @@ -25,6 +26,12 @@ class RasterAsGridReader(sparkSession: SparkSession) extends MosaicDataFrameRead nPartitions } + private def workerNCores = { + sparkSession.sparkContext.range(0, 1).map(_ => java.lang.Runtime.getRuntime.availableProcessors).collect.head + } + + private def nWorkers = sparkSession.sparkContext.getExecutorMemoryStatus.size + override def load(path: String): DataFrame = load(Seq(path): _*) override def load(paths: String*): DataFrame = { @@ -32,11 +39,23 @@ class RasterAsGridReader(sparkSession: SparkSession) extends MosaicDataFrameRead val config = getConfig val resolution = config("resolution").toInt val nPartitions = getNPartitions(config) + val readStrategy = config("retile") match { + case "true" => "retile_on_read" + case _ => "in_memory" + } + val tileSize = config("sizeInMB").toInt + + val nCores = nWorkers * workerNCores + val stageCoefficient = math.ceil(math.log(nCores) / math.log(4)) + + val firstStageSize = (tileSize * math.pow(4, stageCoefficient)).toInt val pathsDf = sparkSession.read .format("gdal") .option("extensions", config("extensions")) - .option("raster_storage", "in-memory") + .option(MOSAIC_RASTER_READ_STRATEGY, readStrategy) + .option("vsizip", config("vsizip")) + .option("sizeInMB", firstStageSize) .load(paths: _*) .repartition(nPartitions) @@ -46,7 +65,12 @@ class RasterAsGridReader(sparkSession: SparkSession) extends MosaicDataFrameRead val retiledDf = retileRaster(rasterDf, config) - val loadedDf = retiledDf + val loadedDf = rasterDf + .withColumn( + "tile", + rst_tessellate(col("tile"), lit(resolution)) + ) + .repartition(nPartitions) .withColumn( "grid_measures", rasterToGridCombiner(col("tile"), lit(resolution)) @@ -58,6 +82,7 @@ class RasterAsGridReader(sparkSession: SparkSession) extends MosaicDataFrameRead .select( posexplode(col("grid_measures")).as(Seq("band_id", "grid_measures")) ) + .repartition(nPartitions) .select( col("band_id"), explode(col("grid_measures")).alias("grid_measures") @@ -88,16 +113,22 @@ class RasterAsGridReader(sparkSession: SparkSession) extends MosaicDataFrameRead */ private def retileRaster(rasterDf: DataFrame, config: Map[String, String]) = { val retile = config("retile").toBoolean - val tileSize = config("tileSize").toInt + val tileSize = config.getOrElse("tileSize", "-1").toInt + val memSize = config.getOrElse("sizeInMB", "-1").toInt val nPartitions = getNPartitions(config) if (retile) { - rasterDf - .withColumn( - "tile", - rst_retile(col("tile"), lit(tileSize), lit(tileSize)) - ) - .repartition(nPartitions) + if (memSize > 0) { + rasterDf + .withColumn("tile", rst_subdivide(col("tile"), lit(memSize))) + .repartition(nPartitions) + } else if (tileSize > 0) { + rasterDf + .withColumn("tile", rst_retile(col("tile"), lit(tileSize), lit(tileSize))) + .repartition(nPartitions) + } else { + rasterDf + } } else { rasterDf } @@ -200,7 +231,8 @@ class RasterAsGridReader(sparkSession: SparkSession) extends MosaicDataFrameRead "resolution" -> this.extraOptions.getOrElse("resolution", "0"), "combiner" -> this.extraOptions.getOrElse("combiner", "mean"), "retile" -> this.extraOptions.getOrElse("retile", "false"), - "tileSize" -> this.extraOptions.getOrElse("tileSize", "256"), + "tileSize" -> this.extraOptions.getOrElse("tileSize", "-1"), + "sizeInMB" -> this.extraOptions.getOrElse("sizeInMB", ""), "kRingInterpolate" -> this.extraOptions.getOrElse("kRingInterpolate", "0") ) } diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Avg.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Avg.scala new file mode 100644 index 000000000..be82af449 --- /dev/null +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Avg.scala @@ -0,0 +1,57 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.raster.operator.gdal.GDALInfo +import com.databricks.labs.mosaic.core.types.model.MosaicRasterTile +import com.databricks.labs.mosaic.expressions.base.{GenericExpressionFactory, WithExpressionInfo} +import com.databricks.labs.mosaic.expressions.raster.base.RasterExpression +import com.databricks.labs.mosaic.functions.MosaicExpressionConfig +import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant} +import org.apache.spark.sql.catalyst.util.ArrayData +import org.apache.spark.sql.types._ + + +/** Returns the upper left x of the raster. */ +case class RST_Avg(raster: Expression, expressionConfig: MosaicExpressionConfig) + extends RasterExpression[RST_Avg](raster, ArrayType(DoubleType), returnsRaster = false, expressionConfig) + with NullIntolerant + with CodegenFallback { + + /** Returns the upper left x of the raster. */ + override def rasterTransform(tile: MosaicRasterTile): Any = { + import org.json4s._ + import org.json4s.jackson.JsonMethods._ + implicit val formats: DefaultFormats.type = org.json4s.DefaultFormats + + val command = s"gdalinfo -stats -json -mm -nogcp -nomd -norat -noct" + val gdalInfo = GDALInfo.executeInfo(tile.raster, command) + // parse json from gdalinfo + val json = parse(gdalInfo).extract[Map[String, Any]] + val maxValues = json("bands").asInstanceOf[List[Map[String, Any]]].map { band => + band("mean").asInstanceOf[Double] + } + ArrayData.toArrayData(maxValues.toArray) + } + +} + +/** Expression info required for the expression registration for spark SQL. */ +object RST_Avg extends WithExpressionInfo { + + override def name: String = "rst_mean" + + override def usage: String = "_FUNC_(expr1) - Returns an array containing mean values for each band." + + override def example: String = + """ + | Examples: + | > SELECT _FUNC_(raster_tile); + | [1.123, 2.123, 3.123] + | """.stripMargin + + override def builder(expressionConfig: MosaicExpressionConfig): FunctionBuilder = { + GenericExpressionFactory.getBaseBuilder[RST_Avg](1, expressionConfig) + } + +} diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Max.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Max.scala new file mode 100644 index 000000000..abe042c2b --- /dev/null +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Max.scala @@ -0,0 +1,48 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.raster.operator.gdal.GDALInfo +import com.databricks.labs.mosaic.core.types.model.MosaicRasterTile +import com.databricks.labs.mosaic.expressions.base.{GenericExpressionFactory, WithExpressionInfo} +import com.databricks.labs.mosaic.expressions.raster.base.RasterExpression +import com.databricks.labs.mosaic.functions.MosaicExpressionConfig +import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant} +import org.apache.spark.sql.catalyst.util.ArrayData +import org.apache.spark.sql.types._ + + +/** Returns the upper left x of the raster. */ +case class RST_Max(raster: Expression, expressionConfig: MosaicExpressionConfig) + extends RasterExpression[RST_Max](raster, ArrayType(DoubleType), returnsRaster = false, expressionConfig) + with NullIntolerant + with CodegenFallback { + + /** Returns the upper left x of the raster. */ + override def rasterTransform(tile: MosaicRasterTile): Any = { + val nBands = tile.raster.raster.GetRasterCount() + val maxValues = (1 to nBands).map(tile.raster.getBand(_).maxPixelValue) + ArrayData.toArrayData(maxValues.toArray) + } + +} + +/** Expression info required for the expression registration for spark SQL. */ +object RST_Max extends WithExpressionInfo { + + override def name: String = "rst_max" + + override def usage: String = "_FUNC_(expr1) - Returns an array containing max values for each band." + + override def example: String = + """ + | Examples: + | > SELECT _FUNC_(raster_tile); + | [1.123, 2.123, 3.123] + | """.stripMargin + + override def builder(expressionConfig: MosaicExpressionConfig): FunctionBuilder = { + GenericExpressionFactory.getBaseBuilder[RST_Max](1, expressionConfig) + } + +} diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Median.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Median.scala new file mode 100644 index 000000000..091121e91 --- /dev/null +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Median.scala @@ -0,0 +1,60 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.raster.api.GDAL +import com.databricks.labs.mosaic.core.raster.gdal.MosaicRasterGDAL +import com.databricks.labs.mosaic.core.raster.operator.gdal.{GDALCalc, GDALInfo, GDALWarp} +import com.databricks.labs.mosaic.core.types.model.MosaicRasterTile +import com.databricks.labs.mosaic.expressions.base.{GenericExpressionFactory, WithExpressionInfo} +import com.databricks.labs.mosaic.expressions.raster.base.RasterExpression +import com.databricks.labs.mosaic.functions.MosaicExpressionConfig +import com.databricks.labs.mosaic.utils.PathUtils +import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant} +import org.apache.spark.sql.catalyst.util.ArrayData +import org.apache.spark.sql.types._ + +/** Returns the upper left x of the raster. */ +case class RST_Median(rasterExpr: Expression, expressionConfig: MosaicExpressionConfig) + extends RasterExpression[RST_Median](rasterExpr, ArrayType(DoubleType), returnsRaster = false, expressionConfig) + with NullIntolerant + with CodegenFallback { + + /** Returns the upper left x of the raster. */ + override def rasterTransform(tile: MosaicRasterTile): Any = { + val raster = tile.raster + val width = raster.xSize * raster.pixelXSize + val height = raster.ySize * raster.pixelYSize + val outShortName = raster.getDriversShortName + val resultFileName = PathUtils.createTmpFilePath(GDAL.getExtension(outShortName)) + val medRaster = GDALWarp.executeWarp( + resultFileName, + Seq(raster), + command = s"gdalwarp -r med -tr $width $height -of $outShortName" + ) + // Max pixel is a hack since we get a 1x1 raster back + val maxValues = (1 to medRaster.raster.GetRasterCount()).map(medRaster.getBand(_).maxPixelValue) + ArrayData.toArrayData(maxValues.toArray) + } + +} + +/** Expression info required for the expression registration for spark SQL. */ +object RST_Median extends WithExpressionInfo { + + override def name: String = "rst_median" + + override def usage: String = "_FUNC_(expr1) - Returns an array containing mean values for each band." + + override def example: String = + """ + | Examples: + | > SELECT _FUNC_(raster_tile); + | [1.123, 2.123, 3.123] + | """.stripMargin + + override def builder(expressionConfig: MosaicExpressionConfig): FunctionBuilder = { + GenericExpressionFactory.getBaseBuilder[RST_Median](1, expressionConfig) + } + +} diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Min.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Min.scala new file mode 100644 index 000000000..67fdb30d3 --- /dev/null +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Min.scala @@ -0,0 +1,48 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.raster.operator.gdal.GDALInfo +import com.databricks.labs.mosaic.core.types.model.MosaicRasterTile +import com.databricks.labs.mosaic.expressions.base.{GenericExpressionFactory, WithExpressionInfo} +import com.databricks.labs.mosaic.expressions.raster.base.RasterExpression +import com.databricks.labs.mosaic.functions.MosaicExpressionConfig +import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant} +import org.apache.spark.sql.catalyst.util.ArrayData +import org.apache.spark.sql.types._ + + +/** Returns the upper left x of the raster. */ +case class RST_Min(raster: Expression, expressionConfig: MosaicExpressionConfig) + extends RasterExpression[RST_Min](raster, ArrayType(DoubleType), returnsRaster = false, expressionConfig) + with NullIntolerant + with CodegenFallback { + + /** Returns the upper left x of the raster. */ + override def rasterTransform(tile: MosaicRasterTile): Any = { + val nBands = tile.raster.raster.GetRasterCount() + val minValues = (1 to nBands).map(tile.raster.getBand(_).minPixelValue) + ArrayData.toArrayData(minValues.toArray) + } + +} + +/** Expression info required for the expression registration for spark SQL. */ +object RST_Min extends WithExpressionInfo { + + override def name: String = "rst_min" + + override def usage: String = "_FUNC_(expr1) - Returns an array containing min values for each band." + + override def example: String = + """ + | Examples: + | > SELECT _FUNC_(raster_tile); + | [1.123, 2.123, 3.123] + | """.stripMargin + + override def builder(expressionConfig: MosaicExpressionConfig): FunctionBuilder = { + GenericExpressionFactory.getBaseBuilder[RST_Min](1, expressionConfig) + } + +} diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_PixelCount.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_PixelCount.scala new file mode 100644 index 000000000..79f44db03 --- /dev/null +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_PixelCount.scala @@ -0,0 +1,46 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.types.model.MosaicRasterTile +import com.databricks.labs.mosaic.expressions.base.{GenericExpressionFactory, WithExpressionInfo} +import com.databricks.labs.mosaic.expressions.raster.base.RasterExpression +import com.databricks.labs.mosaic.functions.MosaicExpressionConfig +import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant} +import org.apache.spark.sql.catalyst.util.ArrayData +import org.apache.spark.sql.types._ + +/** Returns the upper left x of the raster. */ +case class RST_PixelCount(rasterExpr: Expression, expressionConfig: MosaicExpressionConfig) + extends RasterExpression[RST_PixelCount](rasterExpr, ArrayType(DoubleType), returnsRaster = false, expressionConfig) + with NullIntolerant + with CodegenFallback { + + /** Returns the upper left x of the raster. */ + override def rasterTransform(tile: MosaicRasterTile): Any = { + val bandCount = tile.raster.raster.GetRasterCount() + val pixelCount = (1 to bandCount).map(tile.raster.getBand(_).pixelCount) + ArrayData.toArrayData(pixelCount.toArray) + } + +} + +/** Expression info required for the expression registration for spark SQL. */ +object RST_PixelCount extends WithExpressionInfo { + + override def name: String = "rst_pixelcount" + + override def usage: String = "_FUNC_(expr1) - Returns an array containing valid pixel count values for each band." + + override def example: String = + """ + | Examples: + | > SELECT _FUNC_(raster_tile); + | [12, 212, 313] + | """.stripMargin + + override def builder(expressionConfig: MosaicExpressionConfig): FunctionBuilder = { + GenericExpressionFactory.getBaseBuilder[RST_PixelCount](1, expressionConfig) + } + +} diff --git a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala index 8e483c702..1b59ff9a1 100644 --- a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala +++ b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala @@ -13,6 +13,8 @@ import com.databricks.labs.mosaic.expressions.geometry.ST_MinMaxXYZ._ import com.databricks.labs.mosaic.expressions.index._ import com.databricks.labs.mosaic.expressions.raster._ import com.databricks.labs.mosaic.expressions.util.TrySql +import com.databricks.labs.mosaic.functions.MosaicContext.mosaicVersion +import com.databricks.labs.mosaic.utils.FileUtils import org.apache.spark.internal.Logging import org.apache.spark.sql.{Column, SparkSession} import org.apache.spark.sql.catalyst.FunctionIdentifier @@ -21,7 +23,6 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{LongType, StringType} -import java.nio.file.Files import scala.reflect.runtime.universe //noinspection DuplicatedCode @@ -255,6 +256,7 @@ class MosaicContext(indexSystem: IndexSystem, geometryAPI: GeometryAPI) extends ) /** RasterAPI dependent functions */ + mosaicRegistry.registerExpression[RST_Avg](expressionConfig) mosaicRegistry.registerExpression[RST_BandMetaData](expressionConfig) mosaicRegistry.registerExpression[RST_BoundingBox](expressionConfig) mosaicRegistry.registerExpression[RST_Clip](expressionConfig) @@ -266,6 +268,9 @@ class MosaicContext(indexSystem: IndexSystem, geometryAPI: GeometryAPI) extends mosaicRegistry.registerExpression[RST_Height](expressionConfig) mosaicRegistry.registerExpression[RST_InitNoData](expressionConfig) mosaicRegistry.registerExpression[RST_IsEmpty](expressionConfig) + mosaicRegistry.registerExpression[RST_Max](expressionConfig) + mosaicRegistry.registerExpression[RST_Min](expressionConfig) + mosaicRegistry.registerExpression[RST_Median](expressionConfig) mosaicRegistry.registerExpression[RST_MemSize](expressionConfig) mosaicRegistry.registerExpression[RST_Merge](expressionConfig) mosaicRegistry.registerExpression[RST_FromBands](expressionConfig) @@ -275,6 +280,7 @@ class MosaicContext(indexSystem: IndexSystem, geometryAPI: GeometryAPI) extends mosaicRegistry.registerExpression[RST_NumBands](expressionConfig) mosaicRegistry.registerExpression[RST_PixelWidth](expressionConfig) mosaicRegistry.registerExpression[RST_PixelHeight](expressionConfig) + mosaicRegistry.registerExpression[RST_PixelCount](expressionConfig) mosaicRegistry.registerExpression[RST_RasterToGridAvg](expressionConfig) mosaicRegistry.registerExpression[RST_RasterToGridMax](expressionConfig) mosaicRegistry.registerExpression[RST_RasterToGridMin](expressionConfig) @@ -637,6 +643,7 @@ class MosaicContext(indexSystem: IndexSystem, geometryAPI: GeometryAPI) extends ColumnAdapter(RST_BandMetaData(raster.expr, lit(band).expr, expressionConfig)) def rst_boundingbox(raster: Column): Column = ColumnAdapter(RST_BoundingBox(raster.expr, expressionConfig)) def rst_clip(raster: Column, geometry: Column): Column = ColumnAdapter(RST_Clip(raster.expr, geometry.expr, expressionConfig)) + def rst_pixelcount(raster: Column): Column = ColumnAdapter(RST_PixelCount(raster.expr, expressionConfig)) def rst_combineavg(rasterArray: Column): Column = ColumnAdapter(RST_CombineAvg(rasterArray.expr, expressionConfig)) def rst_derivedband(raster: Column, pythonFunc: Column, funcName: Column): Column = ColumnAdapter(RST_DerivedBand(raster.expr, pythonFunc.expr, funcName.expr, expressionConfig)) @@ -649,6 +656,10 @@ class MosaicContext(indexSystem: IndexSystem, geometryAPI: GeometryAPI) extends def rst_height(raster: Column): Column = ColumnAdapter(RST_Height(raster.expr, expressionConfig)) def rst_initnodata(raster: Column): Column = ColumnAdapter(RST_InitNoData(raster.expr, expressionConfig)) def rst_isempty(raster: Column): Column = ColumnAdapter(RST_IsEmpty(raster.expr, expressionConfig)) + def rst_max(raster: Column): Column = ColumnAdapter(RST_Max(raster.expr, expressionConfig)) + def rst_min(raster: Column): Column = ColumnAdapter(RST_Min(raster.expr, expressionConfig)) + def rst_median(raster: Column): Column = ColumnAdapter(RST_Median(raster.expr, expressionConfig)) + def rst_avg(raster: Column): Column = ColumnAdapter(RST_Avg(raster.expr, expressionConfig)) def rst_memsize(raster: Column): Column = ColumnAdapter(RST_MemSize(raster.expr, expressionConfig)) def rst_frombands(bandsArray: Column): Column = ColumnAdapter(RST_FromBands(bandsArray.expr, expressionConfig)) def rst_merge(rasterArray: Column): Column = ColumnAdapter(RST_Merge(rasterArray.expr, expressionConfig)) @@ -965,8 +976,7 @@ class MosaicContext(indexSystem: IndexSystem, geometryAPI: GeometryAPI) extends object MosaicContext extends Logging { - val tmpDir: String = Files.createTempDirectory("mosaic").toAbsolutePath.toString - + val tmpDir: String = FileUtils.createMosaicTempDir() val mosaicVersion: String = "0.3.14" private var instance: Option[MosaicContext] = None diff --git a/src/main/scala/com/databricks/labs/mosaic/utils/FileUtils.scala b/src/main/scala/com/databricks/labs/mosaic/utils/FileUtils.scala new file mode 100644 index 000000000..a1aac5c2f --- /dev/null +++ b/src/main/scala/com/databricks/labs/mosaic/utils/FileUtils.scala @@ -0,0 +1,34 @@ +package com.databricks.labs.mosaic.utils + +import java.io.{BufferedInputStream, FileInputStream} +import java.nio.file.{Files, Paths} + +object FileUtils { + + def readBytes(path: String): Array[Byte] = { + val bufferSize = 1024 * 1024 // 1MB + val inputStream = new BufferedInputStream(new FileInputStream(path)) + val buffer = new Array[Byte](bufferSize) + + var bytesRead = 0 + var bytes = Array.empty[Byte] + + while ({ + bytesRead = inputStream.read(buffer); bytesRead + } != -1) { + bytes = bytes ++ buffer.slice(0, bytesRead) + } + inputStream.close() + bytes + } + + def createMosaicTempDir(): String = { + val tempRoot = Paths.get("/mosaic_tmp/") + if (!Files.exists(tempRoot)) { + Files.createDirectory(tempRoot) + } + val tempDir = Files.createTempDirectory(tempRoot, "mosaic") + tempDir.toFile.getAbsolutePath + } + +} diff --git a/src/test/scala/com/databricks/labs/mosaic/datasource/multiread/RasterAsGridReaderTest.scala b/src/test/scala/com/databricks/labs/mosaic/datasource/multiread/RasterAsGridReaderTest.scala index 1f7b4008b..6e99aa1df 100644 --- a/src/test/scala/com/databricks/labs/mosaic/datasource/multiread/RasterAsGridReaderTest.scala +++ b/src/test/scala/com/databricks/labs/mosaic/datasource/multiread/RasterAsGridReaderTest.scala @@ -6,134 +6,157 @@ import com.databricks.labs.mosaic.core.index.H3IndexSystem import com.databricks.labs.mosaic.test.MosaicSpatialQueryTest import org.apache.spark.sql.test.SharedSparkSessionGDAL import org.scalatest.matchers.must.Matchers.{be, noException} -import org.scalatest.matchers.should.Matchers.an +import org.scalatest.matchers.should.Matchers.{an, convertToAnyShouldWrapper} import java.nio.file.{Files, Paths} class RasterAsGridReaderTest extends MosaicSpatialQueryTest with SharedSparkSessionGDAL { - test("Read netcdf with Raster As Grid Reader") { + test("Read big tif with Raster As Grid Reader") { assume(System.getProperty("os.name") == "Linux") + spark.sparkContext.setLogLevel("INFO") MosaicContext.build(H3IndexSystem, JTS) - val netcdf = "/binary/netcdf-coral/" - val filePath = getClass.getResource(netcdf).getPath - - noException should be thrownBy MosaicContext.read - .format("raster_to_grid") - .option("retile", "true") - .option("tileSize", "10") - .option("readSubdataset", "true") - .option("subdataset", "1") - .option("kRingInterpolate", "3") - .load(filePath) - .select("measure") - .queryExecution - .executedPlan - - } - - test("Read grib with Raster As Grid Reader") { - assume(System.getProperty("os.name") == "Linux") - MosaicContext.build(H3IndexSystem, JTS) - - val grib = "/binary/grib-cams/" - val filePath = getClass.getResource(grib).getPath + val tif = "/binary/big_tiff.tif" + val filePath = getClass.getResource(tif).getPath - noException should be thrownBy MosaicContext.read + val df = MosaicContext.read .format("raster_to_grid") - .option("extensions", "grib") - .option("combiner", "min") .option("retile", "true") - .option("tileSize", "10") - .option("kRingInterpolate", "3") + .option("sizeInMB", "64") + .option("resolution", "1") .load(filePath) .select("measure") - .take(1) - - } - test("Read tif with Raster As Grid Reader") { - assume(System.getProperty("os.name") == "Linux") - MosaicContext.build(H3IndexSystem, JTS) + //df.queryExecution.optimizedPlan - val tif = "/modis/" - val filePath = getClass.getResource(tif).getPath - - noException should be thrownBy MosaicContext.read - .format("raster_to_grid") - .option("combiner", "max") - .option("tileSize", "10") - .option("kRingInterpolate", "3") - .load(filePath) - .select("measure") - .take(1) + //noException should be thrownBy df.queryExecution.executedPlan + df.count() } - test("Read zarr with Raster As Grid Reader") { - assume(System.getProperty("os.name") == "Linux") - MosaicContext.build(H3IndexSystem, JTS) - - val zarr = "/binary/zarr-example/" - val filePath = getClass.getResource(zarr).getPath - - noException should be thrownBy MosaicContext.read - .format("raster_to_grid") - .option("combiner", "median") - .option("vsizip", "true") - .option("tileSize", "10") - .load(filePath) - .select("measure") - .take(1) - - noException should be thrownBy MosaicContext.read - .format("raster_to_grid") - .option("combiner", "count") - .option("vsizip", "true") - .load(filePath) - .select("measure") - .take(1) - - noException should be thrownBy MosaicContext.read - .format("raster_to_grid") - .option("combiner", "average") - .option("vsizip", "true") - .load(filePath) - .select("measure") - .take(1) - - noException should be thrownBy MosaicContext.read - .format("raster_to_grid") - .option("combiner", "avg") - .option("vsizip", "true") - .load(filePath) - .select("measure") - .take(1) - - val paths = Files.list(Paths.get(filePath)).toArray.map(_.toString) - - an[Error] should be thrownBy MosaicContext.read - .format("raster_to_grid") - .option("combiner", "count_+") - .option("vsizip", "true") - .load(paths: _*) - .select("measure") - .take(1) - - an[Error] should be thrownBy MosaicContext.read - .format("invalid") - .load(paths: _*) - - an[Error] should be thrownBy MosaicContext.read - .format("invalid") - .load(filePath) - - noException should be thrownBy MosaicContext.read - .format("raster_to_grid") - .option("kRingInterpolate", "3") - .load(filePath) - - } +// test("Read netcdf with Raster As Grid Reader") { +// assume(System.getProperty("os.name") == "Linux") +// MosaicContext.build(H3IndexSystem, JTS) +// +// val netcdf = "/binary/netcdf-coral/" +// val filePath = getClass.getResource(netcdf).getPath +// +// noException should be thrownBy MosaicContext.read +// .format("raster_to_grid") +// .option("retile", "true") +// .option("tileSize", "10") +// .option("readSubdataset", "true") +// .option("subdataset", "1") +// .option("kRingInterpolate", "3") +// .load(filePath) +// .select("measure") +// .queryExecution +// .executedPlan +// +// } +// +// test("Read grib with Raster As Grid Reader") { +// assume(System.getProperty("os.name") == "Linux") +// MosaicContext.build(H3IndexSystem, JTS) +// +// val grib = "/binary/grib-cams/" +// val filePath = getClass.getResource(grib).getPath +// +// noException should be thrownBy MosaicContext.read +// .format("raster_to_grid") +// .option("extensions", "grib") +// .option("combiner", "min") +// .option("retile", "true") +// .option("tileSize", "10") +// .option("kRingInterpolate", "3") +// .load(filePath) +// .select("measure") +// .take(1) +// +// } +// +// test("Read tif with Raster As Grid Reader") { +// assume(System.getProperty("os.name") == "Linux") +// MosaicContext.build(H3IndexSystem, JTS) +// +// val tif = "/modis/" +// val filePath = getClass.getResource(tif).getPath +// +// noException should be thrownBy MosaicContext.read +// .format("raster_to_grid") +// .option("combiner", "max") +// .option("tileSize", "10") +// .option("kRingInterpolate", "3") +// .load(filePath) +// .select("measure") +// .take(1) +// +// } +// +// test("Read zarr with Raster As Grid Reader") { +// assume(System.getProperty("os.name") == "Linux") +// MosaicContext.build(H3IndexSystem, JTS) +// +// val zarr = "/binary/zarr-example/" +// val filePath = getClass.getResource(zarr).getPath +// +// noException should be thrownBy MosaicContext.read +// .format("raster_to_grid") +// .option("combiner", "median") +// .option("vsizip", "true") +// .option("tileSize", "10") +// .load(filePath) +// .select("measure") +// .take(1) +// +// noException should be thrownBy MosaicContext.read +// .format("raster_to_grid") +// .option("combiner", "count") +// .option("vsizip", "true") +// .load(filePath) +// .select("measure") +// .take(1) +// +// noException should be thrownBy MosaicContext.read +// .format("raster_to_grid") +// .option("combiner", "average") +// .option("vsizip", "true") +// .load(filePath) +// .select("measure") +// .take(1) +// +// noException should be thrownBy MosaicContext.read +// .format("raster_to_grid") +// .option("combiner", "avg") +// .option("vsizip", "true") +// .load(filePath) +// .select("measure") +// .take(1) +// +// val paths = Files.list(Paths.get(filePath)).toArray.map(_.toString) +// +// an[Error] should be thrownBy MosaicContext.read +// .format("raster_to_grid") +// .option("combiner", "count_+") +// .option("vsizip", "true") +// .load(paths: _*) +// .select("measure") +// .take(1) +// +// an[Error] should be thrownBy MosaicContext.read +// .format("invalid") +// .load(paths: _*) +// +// an[Error] should be thrownBy MosaicContext.read +// .format("invalid") +// .load(filePath) +// +// noException should be thrownBy MosaicContext.read +// .format("raster_to_grid") +// .option("kRingInterpolate", "3") +// .load(filePath) +// +// } } diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MaxBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MaxBehaviors.scala new file mode 100644 index 000000000..9c095488d --- /dev/null +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MaxBehaviors.scala @@ -0,0 +1,52 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.geometry.api.GeometryAPI +import com.databricks.labs.mosaic.core.index.IndexSystem +import com.databricks.labs.mosaic.functions.MosaicContext +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.functions._ +import org.scalatest.matchers.should.Matchers._ + +trait RST_MaxBehaviors extends QueryTest { + + def behavior(indexSystem: IndexSystem, geometryAPI: GeometryAPI): Unit = { + val mc = MosaicContext.build(indexSystem, geometryAPI) + mc.register() + val sc = spark + import mc.functions._ + import sc.implicits._ + + val rastersInMemory = spark.read + .format("gdal") + .option("raster_storage", "in-memory") + .load("src/test/resources/modis") + + val df = rastersInMemory + .withColumn("tile", rst_tessellate($"tile", lit(3))) + .withColumn("result", rst_max($"tile")) + .select("result") + .select(explode($"result").as("result")) + + rastersInMemory + .withColumn("tile", rst_tessellate($"tile", lit(3))) + .createOrReplaceTempView("source") + + noException should be thrownBy spark.sql(""" + |select rst_max(tile) from source + |""".stripMargin) + + noException should be thrownBy rastersInMemory + .withColumn("result", rst_rastertogridmax($"tile", lit(3))) + .select("result") + + val result = df.as[Double].collect().max + + result > 0 shouldBe true + + an[Exception] should be thrownBy spark.sql(""" + |select rst_max() from source + |""".stripMargin) + + } + +} diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MaxTest.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MaxTest.scala new file mode 100644 index 000000000..0a0b865cd --- /dev/null +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MaxTest.scala @@ -0,0 +1,32 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.geometry.api.JTS +import com.databricks.labs.mosaic.core.index.H3IndexSystem +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSessionGDAL + +import scala.util.Try + +class RST_MaxTest extends QueryTest with SharedSparkSessionGDAL with RST_MaxBehaviors { + + private val noCodegen = + withSQLConf( + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false", + SQLConf.CODEGEN_FACTORY_MODE.key -> CodegenObjectFactoryMode.NO_CODEGEN.toString + ) _ + + // Hotfix for SharedSparkSession afterAll cleanup. + override def afterAll(): Unit = Try(super.afterAll()) + + // These tests are not index system nor geometry API specific. + // Only testing one pairing is sufficient. + test("Testing rst_max behavior with H3IndexSystem and JTS") { + noCodegen { + assume(System.getProperty("os.name") == "Linux") + behavior(H3IndexSystem, JTS) + } + } + +} diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MedianBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MedianBehaviors.scala new file mode 100644 index 000000000..1b99fbc6f --- /dev/null +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MedianBehaviors.scala @@ -0,0 +1,52 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.geometry.api.GeometryAPI +import com.databricks.labs.mosaic.core.index.IndexSystem +import com.databricks.labs.mosaic.functions.MosaicContext +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.functions._ +import org.scalatest.matchers.should.Matchers._ + +trait RST_MedianBehaviors extends QueryTest { + + def behavior(indexSystem: IndexSystem, geometryAPI: GeometryAPI): Unit = { + val mc = MosaicContext.build(indexSystem, geometryAPI) + mc.register() + val sc = spark + import mc.functions._ + import sc.implicits._ + + val rastersInMemory = spark.read + .format("gdal") + .option("raster_storage", "in-memory") + .load("src/test/resources/modis") + + val df = rastersInMemory + .withColumn("tile", rst_tessellate($"tile", lit(3))) + .withColumn("result", rst_median($"tile")) + .select("result") + .select(explode($"result").as("result")) + + rastersInMemory + .withColumn("tile", rst_tessellate($"tile", lit(3))) + .createOrReplaceTempView("source") + + noException should be thrownBy spark.sql(""" + |select rst_median(tile) from source + |""".stripMargin) + + noException should be thrownBy rastersInMemory + .withColumn("result", rst_rastertogridmax($"tile", lit(3))) + .select("result") + + val result = df.as[Double].collect().max + + result > 0 shouldBe true + + an[Exception] should be thrownBy spark.sql(""" + |select rst_median() from source + |""".stripMargin) + + } + +} diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MedianTest.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MedianTest.scala new file mode 100644 index 000000000..cfe270813 --- /dev/null +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MedianTest.scala @@ -0,0 +1,32 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.geometry.api.JTS +import com.databricks.labs.mosaic.core.index.H3IndexSystem +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSessionGDAL + +import scala.util.Try + +class RST_MedianTest extends QueryTest with SharedSparkSessionGDAL with RST_MedianBehaviors { + + private val noCodegen = + withSQLConf( + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false", + SQLConf.CODEGEN_FACTORY_MODE.key -> CodegenObjectFactoryMode.NO_CODEGEN.toString + ) _ + + // Hotfix for SharedSparkSession afterAll cleanup. + override def afterAll(): Unit = Try(super.afterAll()) + + // These tests are not index system nor geometry API specific. + // Only testing one pairing is sufficient. + test("Testing rst_median behavior with H3IndexSystem and JTS") { + noCodegen { + assume(System.getProperty("os.name") == "Linux") + behavior(H3IndexSystem, JTS) + } + } + +} diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_PixelCountBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_PixelCountBehaviors.scala new file mode 100644 index 000000000..87582df1f --- /dev/null +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_PixelCountBehaviors.scala @@ -0,0 +1,52 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.geometry.api.GeometryAPI +import com.databricks.labs.mosaic.core.index.IndexSystem +import com.databricks.labs.mosaic.functions.MosaicContext +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.functions._ +import org.scalatest.matchers.should.Matchers._ + +trait RST_PixelCountBehaviors extends QueryTest { + + def behavior(indexSystem: IndexSystem, geometryAPI: GeometryAPI): Unit = { + val mc = MosaicContext.build(indexSystem, geometryAPI) + mc.register() + val sc = spark + import mc.functions._ + import sc.implicits._ + + val rastersInMemory = spark.read + .format("gdal") + .option("raster_storage", "in-memory") + .load("src/test/resources/modis") + + val df = rastersInMemory + .withColumn("tile", rst_tessellate($"tile", lit(3))) + .withColumn("result", rst_pixelcount($"tile")) + .select("result") + .select(explode($"result").as("result")) + + rastersInMemory + .withColumn("tile", rst_tessellate($"tile", lit(3))) + .createOrReplaceTempView("source") + + noException should be thrownBy spark.sql(""" + |select rst_pixelcount(tile) from source + |""".stripMargin) + + noException should be thrownBy rastersInMemory + .withColumn("result", rst_rastertogridmax($"tile", lit(3))) + .select("result") + + val result = df.as[Double].collect().max + + result > 0 shouldBe true + + an[Exception] should be thrownBy spark.sql(""" + |select rst_pixelcount() from source + |""".stripMargin) + + } + +} diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_PixelCountTest.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_PixelCountTest.scala new file mode 100644 index 000000000..1d24b58a4 --- /dev/null +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_PixelCountTest.scala @@ -0,0 +1,32 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.geometry.api.JTS +import com.databricks.labs.mosaic.core.index.H3IndexSystem +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSessionGDAL + +import scala.util.Try + +class RST_PixelCountTest extends QueryTest with SharedSparkSessionGDAL with RST_PixelCountBehaviors { + + private val noCodegen = + withSQLConf( + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false", + SQLConf.CODEGEN_FACTORY_MODE.key -> CodegenObjectFactoryMode.NO_CODEGEN.toString + ) _ + + // Hotfix for SharedSparkSession afterAll cleanup. + override def afterAll(): Unit = Try(super.afterAll()) + + // These tests are not index system nor geometry API specific. + // Only testing one pairing is sufficient. + test("Testing rst_pixelcount behavior with H3IndexSystem and JTS") { + noCodegen { + assume(System.getProperty("os.name") == "Linux") + behavior(H3IndexSystem, JTS) + } + } + +} diff --git a/src/test/scala/org/apache/spark/sql/test/MosaicTestSparkSession.scala b/src/test/scala/org/apache/spark/sql/test/MosaicTestSparkSession.scala new file mode 100644 index 000000000..8029c30a7 --- /dev/null +++ b/src/test/scala/org/apache/spark/sql/test/MosaicTestSparkSession.scala @@ -0,0 +1,27 @@ +package org.apache.spark.sql.test + +import org.apache.spark.{SparkConf, SparkContext} + +class MosaicTestSparkSession(sc: SparkContext) extends TestSparkSession(sc) { + + def this(sparkConf: SparkConf) = { + + this( + new SparkContext( + "local[4]", + "test-sql-context", + sparkConf + .set("spark.sql.adaptive.enabled", "false") + .set("spark.driver.memory", "32g") + .set("spark.executor.memory", "32g") + .set("spark.sql.shuffle.partitions", "4") + .set("spark.sql.testkey", "true") + ) + ) + } + + def this() = { + this(new SparkConf) + } + +} diff --git a/src/test/scala/org/apache/spark/sql/test/SharedSparkSessionGDAL.scala b/src/test/scala/org/apache/spark/sql/test/SharedSparkSessionGDAL.scala index 36da49694..fad2383fd 100644 --- a/src/test/scala/org/apache/spark/sql/test/SharedSparkSessionGDAL.scala +++ b/src/test/scala/org/apache/spark/sql/test/SharedSparkSessionGDAL.scala @@ -1,12 +1,13 @@ package org.apache.spark.sql.test import com.databricks.labs.mosaic.gdal.MosaicGDAL +import com.databricks.labs.mosaic.utils.FileUtils import com.databricks.labs.mosaic.{MOSAIC_GDAL_NATIVE, MOSAIC_RASTER_CHECKPOINT} import org.apache.spark.SparkConf import org.apache.spark.sql.SparkSession import org.gdal.gdal.gdal -import java.nio.file.Files +import java.nio.file.{Files, Paths} import scala.util.Try trait SharedSparkSessionGDAL extends SharedSparkSession { @@ -18,12 +19,12 @@ trait SharedSparkSessionGDAL extends SharedSparkSession { override def createSparkSession: TestSparkSession = { val conf = sparkConf - conf.set(MOSAIC_RASTER_CHECKPOINT, Files.createTempDirectory("mosaic").toFile.getAbsolutePath) + conf.set(MOSAIC_RASTER_CHECKPOINT, FileUtils.createMosaicTempDir()) SparkSession.cleanupAnyExistingSession() - val session = new TestSparkSession(conf) - session.sparkContext.setLogLevel("FATAL") + val session = new MosaicTestSparkSession(conf) + session.sparkContext.setLogLevel("INFO") Try { - val tempPath = Files.createTempDirectory("mosaic-gdal") + //val tempPath = Files.createTempDirectory("mosaic-gdal") // prepareEnvironment no longer exists // - only have python calls now //MosaicGDAL.prepareEnvironment(session, tempPath.toAbsolutePath.toString) From fa2fc5cce6763122dc901c74ad12b6544c0b6a76 Mon Sep 17 00:00:00 2001 From: "milos.colic" Date: Wed, 17 Jan 2024 14:02:15 +0000 Subject: [PATCH 02/47] Add tests for RST_stats expressions. --- .../mosaic/expressions/raster/RST_Avg.scala | 2 +- .../expressions/raster/RST_AvgBehaviors.scala | 48 +++++++++++++++++++ .../expressions/raster/RST_AvgTest.scala | 32 +++++++++++++ .../expressions/raster/RST_MaxBehaviors.scala | 4 -- .../expressions/raster/RST_MinBehaviors.scala | 48 +++++++++++++++++++ .../expressions/raster/RST_MinTest.scala | 32 +++++++++++++ 6 files changed, 161 insertions(+), 5 deletions(-) create mode 100644 src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_AvgBehaviors.scala create mode 100644 src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_AvgTest.scala create mode 100644 src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MinBehaviors.scala create mode 100644 src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MinTest.scala diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Avg.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Avg.scala index be82af449..82752cad4 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Avg.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Avg.scala @@ -39,7 +39,7 @@ case class RST_Avg(raster: Expression, expressionConfig: MosaicExpressionConfig) /** Expression info required for the expression registration for spark SQL. */ object RST_Avg extends WithExpressionInfo { - override def name: String = "rst_mean" + override def name: String = "rst_avg" override def usage: String = "_FUNC_(expr1) - Returns an array containing mean values for each band." diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_AvgBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_AvgBehaviors.scala new file mode 100644 index 000000000..f01ce2d25 --- /dev/null +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_AvgBehaviors.scala @@ -0,0 +1,48 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.geometry.api.GeometryAPI +import com.databricks.labs.mosaic.core.index.IndexSystem +import com.databricks.labs.mosaic.functions.MosaicContext +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.functions._ +import org.scalatest.matchers.should.Matchers._ + +trait RST_AvgBehaviors extends QueryTest { + + def behavior(indexSystem: IndexSystem, geometryAPI: GeometryAPI): Unit = { + val mc = MosaicContext.build(indexSystem, geometryAPI) + mc.register() + val sc = spark + import mc.functions._ + import sc.implicits._ + + val rastersInMemory = spark.read + .format("gdal") + .option("raster_storage", "in-memory") + .load("src/test/resources/modis") + + val df = rastersInMemory + .withColumn("tile", rst_tessellate($"tile", lit(3))) + .withColumn("result", rst_avg($"tile")) + .select("result") + .select(explode($"result").as("result")) + + rastersInMemory + .withColumn("tile", rst_tessellate($"tile", lit(3))) + .createOrReplaceTempView("source") + + noException should be thrownBy spark.sql(""" + |select rst_avg(tile) from source + |""".stripMargin) + + val result = df.as[Double].collect().max + + result > 0 shouldBe true + + an[Exception] should be thrownBy spark.sql(""" + |select rst_avg() from source + |""".stripMargin) + + } + +} diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_AvgTest.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_AvgTest.scala new file mode 100644 index 000000000..6805f0723 --- /dev/null +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_AvgTest.scala @@ -0,0 +1,32 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.geometry.api.JTS +import com.databricks.labs.mosaic.core.index.H3IndexSystem +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSessionGDAL + +import scala.util.Try + +class RST_AvgTest extends QueryTest with SharedSparkSessionGDAL with RST_AvgBehaviors { + + private val noCodegen = + withSQLConf( + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false", + SQLConf.CODEGEN_FACTORY_MODE.key -> CodegenObjectFactoryMode.NO_CODEGEN.toString + ) _ + + // Hotfix for SharedSparkSession afterAll cleanup. + override def afterAll(): Unit = Try(super.afterAll()) + + // These tests are not index system nor geometry API specific. + // Only testing one pairing is sufficient. + test("Testing rst_avg behavior with H3IndexSystem and JTS") { + noCodegen { + assume(System.getProperty("os.name") == "Linux") + behavior(H3IndexSystem, JTS) + } + } + +} diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MaxBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MaxBehaviors.scala index 9c095488d..daab1ee90 100644 --- a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MaxBehaviors.scala +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MaxBehaviors.scala @@ -35,10 +35,6 @@ trait RST_MaxBehaviors extends QueryTest { |select rst_max(tile) from source |""".stripMargin) - noException should be thrownBy rastersInMemory - .withColumn("result", rst_rastertogridmax($"tile", lit(3))) - .select("result") - val result = df.as[Double].collect().max result > 0 shouldBe true diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MinBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MinBehaviors.scala new file mode 100644 index 000000000..bd867ee65 --- /dev/null +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MinBehaviors.scala @@ -0,0 +1,48 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.geometry.api.GeometryAPI +import com.databricks.labs.mosaic.core.index.IndexSystem +import com.databricks.labs.mosaic.functions.MosaicContext +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.functions._ +import org.scalatest.matchers.should.Matchers._ + +trait RST_MinBehaviors extends QueryTest { + + def behavior(indexSystem: IndexSystem, geometryAPI: GeometryAPI): Unit = { + val mc = MosaicContext.build(indexSystem, geometryAPI) + mc.register() + val sc = spark + import mc.functions._ + import sc.implicits._ + + val rastersInMemory = spark.read + .format("gdal") + .option("raster_storage", "in-memory") + .load("src/test/resources/modis") + + val df = rastersInMemory + .withColumn("tile", rst_tessellate($"tile", lit(3))) + .withColumn("result", rst_min($"tile")) + .select("result") + .select(explode($"result").as("result")) + + rastersInMemory + .withColumn("tile", rst_tessellate($"tile", lit(3))) + .createOrReplaceTempView("source") + + noException should be thrownBy spark.sql(""" + |select rst_min(tile) from source + |""".stripMargin) + + val result = df.as[Double].collect().min + + result < 0 shouldBe true + + an[Exception] should be thrownBy spark.sql(""" + |select rst_min() from source + |""".stripMargin) + + } + +} diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MinTest.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MinTest.scala new file mode 100644 index 000000000..ec09792f9 --- /dev/null +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MinTest.scala @@ -0,0 +1,32 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.geometry.api.JTS +import com.databricks.labs.mosaic.core.index.H3IndexSystem +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSessionGDAL + +import scala.util.Try + +class RST_MinTest extends QueryTest with SharedSparkSessionGDAL with RST_MinBehaviors { + + private val noCodegen = + withSQLConf( + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false", + SQLConf.CODEGEN_FACTORY_MODE.key -> CodegenObjectFactoryMode.NO_CODEGEN.toString + ) _ + + // Hotfix for SharedSparkSession afterAll cleanup. + override def afterAll(): Unit = Try(super.afterAll()) + + // These tests are not index system nor geometry API specific. + // Only testing one pairing is sufficient. + test("Testing rst_min behavior with H3IndexSystem and JTS") { + noCodegen { + assume(System.getProperty("os.name") == "Linux") + behavior(H3IndexSystem, JTS) + } + } + +} From a38b94d815a8f562abd5bccf41a65a1c761582a5 Mon Sep 17 00:00:00 2001 From: KRoss Date: Thu, 18 Jan 2024 15:39:47 +0000 Subject: [PATCH 03/47] add additional feature_type values --- docs/source/usage/kepler.ipynb | 1299 ++++++++++++++++++++------------ 1 file changed, 797 insertions(+), 502 deletions(-) diff --git a/docs/source/usage/kepler.ipynb b/docs/source/usage/kepler.ipynb index 5c33a3e2c..3c3ded2c8 100644 --- a/docs/source/usage/kepler.ipynb +++ b/docs/source/usage/kepler.ipynb @@ -2,20 +2,28 @@ "cells": [ { "cell_type": "markdown", - "source": [ - "# Kepler visualizations" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "5dbca100-ca8d-4cb3-be39-72d2aeb85077" + "nuid": "5dbca100-ca8d-4cb3-be39-72d2aeb85077", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "# Kepler visualizations" + ] }, { "cell_type": "markdown", + "metadata": { + "application/vnd.databricks.v1+cell": { + "inputWidgets": {}, + "nuid": "a2712c52-9c84-427e-bff8-814b9fa35d86", + "showTitle": false, + "title": "" + } + }, "source": [ "You can use the `%%mosaic_kepler` magic function to visualise data using [Kepler.gl](https://kepler.gl/).\n", "\n", @@ -25,7 +33,7 @@ "\n", "2) `column_name`: The column that needs to be plotted, can be either a geometry column (`WKT`, `WKB` or Mosaic internal format) or a column containing a spatial grid index ID\n", "\n", - "3) `feature_type`: The type of data to be plotted. Valid values are `geometry` (if SRID=4326), `geometry()` (where `` is the SRID used by the geometry column) and `h3`\n", + "3) `feature_type`: The type of data to be plotted. Valid values are `geometry` (if SRID=4326), `geometry()` (where `` is the SRID used by the geometry column), `geometry(bng)`, `geometry(osgb36)`, `bng` and `h3`\n", "\n", "4) `limit`: The maximum number of objects to plot. The default limit is `1000`\n", "\n", @@ -36,160 +44,246 @@ "```\n", "\n", "This magic function is only available in python. It can be used from notebooks with other default languages by storing the intermediate result in a temporary view, and then adding a python cell that uses the `mosaic_kepler` with the temporary view created from another language." - ], - "metadata": { - "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, - "inputWidgets": {}, - "nuid": "a2712c52-9c84-427e-bff8-814b9fa35d86" - } - } + ] }, { "cell_type": "markdown", - "source": [ - "## Examples" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "573fa05d-6dfd-414e-84c4-6f3618540b60" + "nuid": "573fa05d-6dfd-414e-84c4-6f3618540b60", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "## Examples" + ] }, { "cell_type": "code", - "source": [ - "%pip install databricks-mosaic --quiet" - ], + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "c73e5956-dec2-4206-be2d-99bca42d2c93" + "nuid": "c73e5956-dec2-4206-be2d-99bca42d2c93", + "showTitle": false, + "title": "" } }, "outputs": [ { - "output_type": "display_data", + "data": { + "text/html": [ + "\n", + "
Python interpreter will be restarted.\n", + "Python interpreter will be restarted.\n", + "
" + ] + }, "metadata": { "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
Python interpreter will be restarted.\nPython interpreter will be restarted.\n
", - "removedWidgets": [], "addedWidgets": {}, + "arguments": {}, + "data": "
Python interpreter will be restarted.\nPython interpreter will be restarted.\n
", + "datasetInfos": [], "metadata": {}, - "type": "html", - "arguments": {} + "removedWidgets": [], + "type": "html" } }, - "data": { - "text/html": [ - "\n
Python interpreter will be restarted.\nPython interpreter will be restarted.\n
" - ] - } + "output_type": "display_data" } ], - "execution_count": 0 + "source": [ + "%pip install databricks-mosaic --quiet" + ] }, { "cell_type": "code", - "source": [ - "from pyspark.sql.functions import *\n", - "import mosaic as mos\n", - "mos.enable_mosaic(spark, dbutils)" - ], + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "a18dcbe7-1a01-45f2-a961-dd0f0284cf9c" + "nuid": "a18dcbe7-1a01-45f2-a961-dd0f0284cf9c", + "showTitle": false, + "title": "" } }, "outputs": [ { - "output_type": "display_data", + "data": { + "text/html": [ + "\n", + "
" + ] + }, "metadata": { "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
", - "removedWidgets": [], "addedWidgets": {}, + "arguments": {}, + "data": "
", + "datasetInfos": [], "metadata": {}, - "type": "html", - "arguments": {} + "removedWidgets": [], + "type": "html" } }, - "data": { - "text/html": [ - "\n
" - ] - } + "output_type": "display_data" } ], - "execution_count": 0 + "source": [ + "from pyspark.sql.functions import *\n", + "import mosaic as mos\n", + "mos.enable_mosaic(spark, dbutils)" + ] }, { "cell_type": "markdown", - "source": [ - "### Download example shapes" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "2a997c83-5aff-4463-8c64-a1dc7a55d7d3" + "nuid": "2a997c83-5aff-4463-8c64-a1dc7a55d7d3", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "### Download example shapes" + ] }, { "cell_type": "code", + "execution_count": null, + "metadata": { + "application/vnd.databricks.v1+cell": { + "inputWidgets": {}, + "nuid": "93072805-dd95-4518-9099-47ca7e85b62d", + "showTitle": false, + "title": "" + } + }, + "outputs": [ + { + "data": { + "text/html": [ + "\n", + "
" + ] + }, + "metadata": { + "application/vnd.databricks.v1+output": { + "addedWidgets": {}, + "arguments": {}, + "data": "
", + "datasetInfos": [], + "metadata": {}, + "removedWidgets": [], + "type": "html" + } + }, + "output_type": "display_data" + } + ], "source": [ "import requests\n", "\n", "req = requests.get('https://data.cityofnewyork.us/api/geospatial/d3c5-ddgc?method=export&format=GeoJSON')\n", "with open('/dbfs/tmp/nyc_taxi_zones.geojson', 'wb') as f:\n", " f.write(req.content)" - ], + ] + }, + { + "cell_type": "code", + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "93072805-dd95-4518-9099-47ca7e85b62d" + "nuid": "eb739b60-a368-4dc9-9cf7-2f4346ffb06a", + "showTitle": false, + "title": "" } }, "outputs": [ { - "output_type": "display_data", + "data": { + "text/html": [ + "\n", + "
+-----------------+--------------------+--------------------+--------------------+--------------------+--------------------+\n", + " type| properties| geom_json| geom_internal| geom_wkt| geom_wkb|\n", + "+-----------------+--------------------+--------------------+--------------------+--------------------+--------------------+\n", + "FeatureCollection|{EWR, 1, 1, 0.000...|{"coordinates":[[...|{6, 4326, [[[-74....|MULTIPOLYGON (((-...|[01 06 00 00 00 0...|\n", + "+-----------------+--------------------+--------------------+--------------------+--------------------+--------------------+\n", + "\n", + "
" + ] + }, "metadata": { "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
", - "removedWidgets": [], "addedWidgets": {}, + "arguments": {}, + "data": "
+-----------------+--------------------+--------------------+--------------------+--------------------+--------------------+\n| type| properties| geom_json| geom_internal| geom_wkt| geom_wkb|\n+-----------------+--------------------+--------------------+--------------------+--------------------+--------------------+\n|FeatureCollection|{EWR, 1, 1, 0.000...|{"coordinates":[[...|{6, 4326, [[[-74....|MULTIPOLYGON (((-...|[01 06 00 00 00 0...|\n+-----------------+--------------------+--------------------+--------------------+--------------------+--------------------+\n\n
", + "datasetInfos": [], "metadata": {}, - "type": "html", - "arguments": {} + "removedWidgets": [], + "type": "html" } }, - "data": { - "text/html": [ - "\n
" - ] - } + "output_type": "display_data" } ], - "execution_count": 0 - }, - { - "cell_type": "code", "source": [ "neighbourhoods = (\n", " spark.read\n", @@ -215,771 +309,972 @@ ")\n", "\n", "neighbourhoods.show()" - ], - "metadata": { - "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, - "inputWidgets": {}, - "nuid": "eb739b60-a368-4dc9-9cf7-2f4346ffb06a" - } - }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
+-----------------+--------------------+--------------------+--------------------+--------------------+--------------------+\n| type| properties| geom_json| geom_internal| geom_wkt| geom_wkb|\n+-----------------+--------------------+--------------------+--------------------+--------------------+--------------------+\n|FeatureCollection|{EWR, 1, 1, 0.000...|{"coordinates":[[...|{6, 4326, [[[-74....|MULTIPOLYGON (((-...|[01 06 00 00 00 0...|\n+-----------------+--------------------+--------------------+--------------------+--------------------+--------------------+\n\n
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
+-----------------+--------------------+--------------------+--------------------+--------------------+--------------------+\n type| properties| geom_json| geom_internal| geom_wkt| geom_wkb|\n+-----------------+--------------------+--------------------+--------------------+--------------------+--------------------+\nFeatureCollection|{EWR, 1, 1, 0.000...|{"coordinates":[[...|{6, 4326, [[[-74....|MULTIPOLYGON (((-...|[01 06 00 00 00 0...|\n+-----------------+--------------------+--------------------+--------------------+--------------------+--------------------+\n\n
" - ] - } - } - ], - "execution_count": 0 + ] }, { "cell_type": "markdown", - "source": [ - "### Plot geometries from Spark dataset" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "0c74d4f1-dbe6-4117-b752-4761b6a5d692" + "nuid": "0c74d4f1-dbe6-4117-b752-4761b6a5d692", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "### Plot geometries from Spark dataset" + ] }, { "cell_type": "markdown", - "source": [ - "#### Internal geometry type" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "4109bf9f-479e-47f1-aea5-fb881136c49e" + "nuid": "4109bf9f-479e-47f1-aea5-fb881136c49e", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "#### Internal geometry type" + ] }, { "cell_type": "code", - "source": [ - "%%mosaic_kepler\n", - "neighbourhoods \"geom_internal\" \"geometry\"" - ], + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "99be3b7a-3678-44bc-8d4a-ac35346cd724" + "nuid": "99be3b7a-3678-44bc-8d4a-ac35346cd724", + "showTitle": false, + "title": "" } }, "outputs": [ { - "output_type": "display_data", + "data": { + "text/html": [ + "\n", + "
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n", + "
" + ] + }, "metadata": { "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n
", - "removedWidgets": [], "addedWidgets": {}, + "arguments": {}, + "data": "
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n
", + "datasetInfos": [], "metadata": {}, - "type": "html", - "arguments": {} + "removedWidgets": [], + "type": "html" } }, + "output_type": "display_data" + }, + { "data": { "text/html": [ - "\n
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n
" + "" ] - } - }, - { - "output_type": "display_data", + }, "metadata": { "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "", - "textData": null, - "removedWidgets": [], "addedWidgets": {}, + "arguments": {}, + "data": "", + "datasetInfos": [], "metadata": {}, - "type": "htmlSandbox", - "arguments": {} + "removedWidgets": [], + "textData": null, + "type": "htmlSandbox" } }, + "output_type": "display_data" + }, + { "data": { "text/html": [ - "" + "Kepler.gl
" ] - } - }, - { - "output_type": "display_data", + }, "metadata": { "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "Kepler.gl
", - "textData": null, - "removedWidgets": [], "addedWidgets": {}, + "arguments": {}, + "data": "Kepler.gl
", + "datasetInfos": [], "metadata": {}, - "type": "htmlSandbox", - "arguments": {} + "removedWidgets": [], + "textData": null, + "type": "htmlSandbox" } }, - "data": { - "text/html": [ - "Kepler.gl
" - ] - } + "output_type": "display_data" } ], - "execution_count": 0 + "source": [ + "%%mosaic_kepler\n", + "neighbourhoods \"geom_internal\" \"geometry\"" + ] }, { "cell_type": "markdown", - "source": [ - "![mosaic kepler map example geometry](../images/kepler-1.png)" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "34b3328f-2966-4b6f-859f-325eeec04619" + "nuid": "34b3328f-2966-4b6f-859f-325eeec04619", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "![mosaic kepler map example geometry](../images/kepler-1.png)" + ] }, { "cell_type": "markdown", - "source": [ - "#### WKT geometry type" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "f6d1c289-b007-4e40-b354-4c59fef9c2c4" + "nuid": "f6d1c289-b007-4e40-b354-4c59fef9c2c4", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "#### WKT geometry type" + ] }, { "cell_type": "code", - "source": [ - "%%mosaic_kepler\n", - "neighbourhoods \"geom_wkt\" \"geometry\"" - ], + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "84ee5934-ed9c-4ee8-affd-dd347f375eec" + "nuid": "84ee5934-ed9c-4ee8-affd-dd347f375eec", + "showTitle": false, + "title": "" } }, "outputs": [ { - "output_type": "display_data", + "data": { + "text/html": [ + "\n", + "
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n", + "
" + ] + }, "metadata": { "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n
", - "removedWidgets": [], "addedWidgets": {}, + "arguments": {}, + "data": "
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n
", + "datasetInfos": [], "metadata": {}, - "type": "html", - "arguments": {} + "removedWidgets": [], + "type": "html" } }, + "output_type": "display_data" + }, + { "data": { "text/html": [ - "\n
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n
" + "" ] - } - }, - { - "output_type": "display_data", + }, "metadata": { "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "", - "textData": null, - "removedWidgets": [], "addedWidgets": {}, + "arguments": {}, + "data": "", + "datasetInfos": [], "metadata": {}, - "type": "htmlSandbox", - "arguments": {} + "removedWidgets": [], + "textData": null, + "type": "htmlSandbox" } }, + "output_type": "display_data" + }, + { "data": { "text/html": [ - "" + "Kepler.gl
" ] - } - }, - { - "output_type": "display_data", + }, "metadata": { "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "Kepler.gl
", - "textData": null, - "removedWidgets": [], "addedWidgets": {}, + "arguments": {}, + "data": "Kepler.gl
", + "datasetInfos": [], "metadata": {}, - "type": "htmlSandbox", - "arguments": {} + "removedWidgets": [], + "textData": null, + "type": "htmlSandbox" } }, - "data": { - "text/html": [ - "Kepler.gl
" - ] - } + "output_type": "display_data" } ], - "execution_count": 0 + "source": [ + "%%mosaic_kepler\n", + "neighbourhoods \"geom_wkt\" \"geometry\"" + ] }, { "cell_type": "markdown", - "source": [ - "![mosaic kepler map example geometry](../images/kepler-1.png)" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "b81a56b6-3d7c-4c87-91b2-fecc2f77333c" + "nuid": "b81a56b6-3d7c-4c87-91b2-fecc2f77333c", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "![mosaic kepler map example geometry](../images/kepler-1.png)" + ] }, { "cell_type": "markdown", - "source": [ - "#### WKB geometry type" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "2acdd70b-decd-4654-8927-9f821e728864" + "nuid": "2acdd70b-decd-4654-8927-9f821e728864", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "#### WKB geometry type" + ] }, { "cell_type": "code", - "source": [ - "%%mosaic_kepler\n", - "neighbourhoods \"geom_wkb\" \"geometry\"" - ], + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "0ddd1cad-dabf-4a9d-9e89-4406e2fda6e4" + "nuid": "0ddd1cad-dabf-4a9d-9e89-4406e2fda6e4", + "showTitle": false, + "title": "" } }, "outputs": [ { - "output_type": "display_data", + "data": { + "text/html": [ + "\n", + "
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n", + "
" + ] + }, "metadata": { "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n
", - "removedWidgets": [], "addedWidgets": {}, + "arguments": {}, + "data": "
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n
", + "datasetInfos": [], "metadata": {}, - "type": "html", - "arguments": {} + "removedWidgets": [], + "type": "html" } }, + "output_type": "display_data" + }, + { "data": { "text/html": [ - "\n
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n
" + "" ] - } - }, - { - "output_type": "display_data", + }, "metadata": { "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "", - "textData": null, - "removedWidgets": [], "addedWidgets": {}, + "arguments": {}, + "data": "", + "datasetInfos": [], "metadata": {}, - "type": "htmlSandbox", - "arguments": {} + "removedWidgets": [], + "textData": null, + "type": "htmlSandbox" } }, + "output_type": "display_data" + }, + { "data": { "text/html": [ - "" + "Kepler.gl
" ] - } - }, - { - "output_type": "display_data", + }, "metadata": { "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "Kepler.gl
", - "textData": null, - "removedWidgets": [], "addedWidgets": {}, + "arguments": {}, + "data": "Kepler.gl
", + "datasetInfos": [], "metadata": {}, - "type": "htmlSandbox", - "arguments": {} + "removedWidgets": [], + "textData": null, + "type": "htmlSandbox" } }, - "data": { - "text/html": [ - "Kepler.gl
" - ] - } + "output_type": "display_data" } ], - "execution_count": 0 + "source": [ + "%%mosaic_kepler\n", + "neighbourhoods \"geom_wkb\" \"geometry\"" + ] }, { "cell_type": "markdown", - "source": [ - "![mosaic kepler map example geometry](../images/kepler-1.png)" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "0f0deddf-3408-470d-8e9a-a6d066d23697" + "nuid": "0f0deddf-3408-470d-8e9a-a6d066d23697", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "![mosaic kepler map example geometry](../images/kepler-1.png)" + ] }, { "cell_type": "markdown", - "source": [ - "### Plot geometries from table/view" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "590cb32e-2b3f-4ad7-b609-ffef9ae17967" + "nuid": "590cb32e-2b3f-4ad7-b609-ffef9ae17967", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "### Plot geometries from table/view" + ] }, { "cell_type": "code", - "source": [ - "neighbourhoods.createOrReplaceTempView(\"temp_view_neighbourhoods\")" - ], + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "782fcc2e-d868-4819-bb01-9c4ea4bafb36" + "nuid": "782fcc2e-d868-4819-bb01-9c4ea4bafb36", + "showTitle": false, + "title": "" } }, "outputs": [ { - "output_type": "display_data", + "data": { + "text/html": [ + "\n", + "
" + ] + }, "metadata": { "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
", - "removedWidgets": [], "addedWidgets": {}, + "arguments": {}, + "data": "
", + "datasetInfos": [], "metadata": {}, - "type": "html", - "arguments": {} + "removedWidgets": [], + "type": "html" } }, - "data": { - "text/html": [ - "\n
" - ] - } + "output_type": "display_data" } ], - "execution_count": 0 + "source": [ + "neighbourhoods.createOrReplaceTempView(\"temp_view_neighbourhoods\")" + ] }, { "cell_type": "code", - "source": [ - "%%mosaic_kepler\n", - "\"temp_view_neighbourhoods\" \"geom_wkt\" \"geometry\"" - ], + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "fe84fa59-998d-4b9a-82c5-f280393ae244" + "nuid": "fe84fa59-998d-4b9a-82c5-f280393ae244", + "showTitle": false, + "title": "" } }, "outputs": [ { - "output_type": "display_data", + "data": { + "text/html": [ + "\n", + "
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n", + "
" + ] + }, "metadata": { "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n
", - "removedWidgets": [], "addedWidgets": {}, + "arguments": {}, + "data": "
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n
", + "datasetInfos": [], "metadata": {}, - "type": "html", - "arguments": {} + "removedWidgets": [], + "type": "html" } }, + "output_type": "display_data" + }, + { "data": { "text/html": [ - "\n
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n
" + "" ] - } - }, - { - "output_type": "display_data", + }, "metadata": { "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "", - "textData": null, - "removedWidgets": [], "addedWidgets": {}, + "arguments": {}, + "data": "", + "datasetInfos": [], "metadata": {}, - "type": "htmlSandbox", - "arguments": {} + "removedWidgets": [], + "textData": null, + "type": "htmlSandbox" } }, + "output_type": "display_data" + }, + { "data": { "text/html": [ - "" + "Kepler.gl
" ] - } - }, - { - "output_type": "display_data", + }, "metadata": { "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "Kepler.gl
", - "textData": null, - "removedWidgets": [], "addedWidgets": {}, + "arguments": {}, + "data": "Kepler.gl
", + "datasetInfos": [], "metadata": {}, - "type": "htmlSandbox", - "arguments": {} + "removedWidgets": [], + "textData": null, + "type": "htmlSandbox" } }, - "data": { - "text/html": [ - "Kepler.gl
" - ] - } + "output_type": "display_data" } ], - "execution_count": 0 + "source": [ + "%%mosaic_kepler\n", + "\"temp_view_neighbourhoods\" \"geom_wkt\" \"geometry\"" + ] }, { "cell_type": "markdown", - "source": [ - "![mosaic kepler map example geometry](../images/kepler-1.png)" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "9d7dd0c3-64c5-43bb-aecd-287635929970" + "nuid": "9d7dd0c3-64c5-43bb-aecd-287635929970", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "![mosaic kepler map example geometry](../images/kepler-1.png)" + ] }, { "cell_type": "markdown", - "source": [ - "### Plot H3 indexes" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "d34979d6-45e5-4a6f-a12a-52e46cb1e946" + "nuid": "d34979d6-45e5-4a6f-a12a-52e46cb1e946", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "### Plot H3 indexes" + ] }, { "cell_type": "code", - "source": [ - "neighbourhood_chips = (neighbourhoods\n", - " .limit(1)\n", - " .select(mos.grid_tessellateexplode(\"geom_internal\", lit(9)))\n", - " .select(\"index.*\")\n", - " )\n", - "\n", - "neighbourhood_chips.show()" - ], + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "86cdd4f3-34c5-4365-8108-303ca63d97af" + "nuid": "86cdd4f3-34c5-4365-8108-303ca63d97af", + "showTitle": false, + "title": "" } }, "outputs": [ { - "output_type": "display_data", + "data": { + "text/html": [ + "\n", + "
+-------+------------------+--------------------+\n", + "is_core| index_id| wkb|\n", + "+-------+------------------+--------------------+\n", + " true|617733150781997055|[01 03 00 00 00 0...|\n", + " true|617733150856445951|[01 03 00 00 00 0...|\n", + " true|617733150856970239|[01 03 00 00 00 0...|\n", + " true|617733150784094207|[01 03 00 00 00 0...|\n", + " true|617733150843600895|[01 03 00 00 00 0...|\n", + " true|617733150843863039|[01 03 00 00 00 0...|\n", + " true|617733150844125183|[01 03 00 00 00 0...|\n", + " true|617733150784880639|[01 03 00 00 00 0...|\n", + " true|617733150844387327|[01 03 00 00 00 0...|\n", + " true|617733150844649471|[01 03 00 00 00 0...|\n", + " true|617733150785404927|[01 03 00 00 00 0...|\n", + " true|617733150844911615|[01 03 00 00 00 0...|\n", + " true|617733150785667071|[01 03 00 00 00 0...|\n", + " true|617733150845173759|[01 03 00 00 00 0...|\n", + " true|617733150785929215|[01 03 00 00 00 0...|\n", + " true|617733150786453503|[01 03 00 00 00 0...|\n", + " true|617733150846222335|[01 03 00 00 00 0...|\n", + " true|617733150847270911|[01 03 00 00 00 0...|\n", + " true|617733150847795199|[01 03 00 00 00 0...|\n", + " true|617733150848057343|[01 03 00 00 00 0...|\n", + "+-------+------------------+--------------------+\n", + "only showing top 20 rows\n", + "\n", + "
" + ] + }, "metadata": { "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
+-------+------------------+--------------------+\n|is_core| index_id| wkb|\n+-------+------------------+--------------------+\n| true|617733150781997055|[01 03 00 00 00 0...|\n| true|617733150856445951|[01 03 00 00 00 0...|\n| true|617733150856970239|[01 03 00 00 00 0...|\n| true|617733150784094207|[01 03 00 00 00 0...|\n| true|617733150843600895|[01 03 00 00 00 0...|\n| true|617733150843863039|[01 03 00 00 00 0...|\n| true|617733150844125183|[01 03 00 00 00 0...|\n| true|617733150784880639|[01 03 00 00 00 0...|\n| true|617733150844387327|[01 03 00 00 00 0...|\n| true|617733150844649471|[01 03 00 00 00 0...|\n| true|617733150785404927|[01 03 00 00 00 0...|\n| true|617733150844911615|[01 03 00 00 00 0...|\n| true|617733150785667071|[01 03 00 00 00 0...|\n| true|617733150845173759|[01 03 00 00 00 0...|\n| true|617733150785929215|[01 03 00 00 00 0...|\n| true|617733150786453503|[01 03 00 00 00 0...|\n| true|617733150846222335|[01 03 00 00 00 0...|\n| true|617733150847270911|[01 03 00 00 00 0...|\n| true|617733150847795199|[01 03 00 00 00 0...|\n| true|617733150848057343|[01 03 00 00 00 0...|\n+-------+------------------+--------------------+\nonly showing top 20 rows\n\n
", - "removedWidgets": [], "addedWidgets": {}, + "arguments": {}, + "data": "
+-------+------------------+--------------------+\n|is_core| index_id| wkb|\n+-------+------------------+--------------------+\n| true|617733150781997055|[01 03 00 00 00 0...|\n| true|617733150856445951|[01 03 00 00 00 0...|\n| true|617733150856970239|[01 03 00 00 00 0...|\n| true|617733150784094207|[01 03 00 00 00 0...|\n| true|617733150843600895|[01 03 00 00 00 0...|\n| true|617733150843863039|[01 03 00 00 00 0...|\n| true|617733150844125183|[01 03 00 00 00 0...|\n| true|617733150784880639|[01 03 00 00 00 0...|\n| true|617733150844387327|[01 03 00 00 00 0...|\n| true|617733150844649471|[01 03 00 00 00 0...|\n| true|617733150785404927|[01 03 00 00 00 0...|\n| true|617733150844911615|[01 03 00 00 00 0...|\n| true|617733150785667071|[01 03 00 00 00 0...|\n| true|617733150845173759|[01 03 00 00 00 0...|\n| true|617733150785929215|[01 03 00 00 00 0...|\n| true|617733150786453503|[01 03 00 00 00 0...|\n| true|617733150846222335|[01 03 00 00 00 0...|\n| true|617733150847270911|[01 03 00 00 00 0...|\n| true|617733150847795199|[01 03 00 00 00 0...|\n| true|617733150848057343|[01 03 00 00 00 0...|\n+-------+------------------+--------------------+\nonly showing top 20 rows\n\n
", + "datasetInfos": [], "metadata": {}, - "type": "html", - "arguments": {} + "removedWidgets": [], + "type": "html" } }, - "data": { - "text/html": [ - "\n
+-------+------------------+--------------------+\nis_core| index_id| wkb|\n+-------+------------------+--------------------+\n true|617733150781997055|[01 03 00 00 00 0...|\n true|617733150856445951|[01 03 00 00 00 0...|\n true|617733150856970239|[01 03 00 00 00 0...|\n true|617733150784094207|[01 03 00 00 00 0...|\n true|617733150843600895|[01 03 00 00 00 0...|\n true|617733150843863039|[01 03 00 00 00 0...|\n true|617733150844125183|[01 03 00 00 00 0...|\n true|617733150784880639|[01 03 00 00 00 0...|\n true|617733150844387327|[01 03 00 00 00 0...|\n true|617733150844649471|[01 03 00 00 00 0...|\n true|617733150785404927|[01 03 00 00 00 0...|\n true|617733150844911615|[01 03 00 00 00 0...|\n true|617733150785667071|[01 03 00 00 00 0...|\n true|617733150845173759|[01 03 00 00 00 0...|\n true|617733150785929215|[01 03 00 00 00 0...|\n true|617733150786453503|[01 03 00 00 00 0...|\n true|617733150846222335|[01 03 00 00 00 0...|\n true|617733150847270911|[01 03 00 00 00 0...|\n true|617733150847795199|[01 03 00 00 00 0...|\n true|617733150848057343|[01 03 00 00 00 0...|\n+-------+------------------+--------------------+\nonly showing top 20 rows\n\n
" - ] - } + "output_type": "display_data" } ], - "execution_count": 0 + "source": [ + "neighbourhood_chips = (neighbourhoods\n", + " .limit(1)\n", + " .select(mos.grid_tessellateexplode(\"geom_internal\", lit(9)))\n", + " .select(\"index.*\")\n", + " )\n", + "\n", + "neighbourhood_chips.show()" + ] }, { "cell_type": "code", - "source": [ - "%%mosaic_kepler\n", - "neighbourhood_chips \"index_id\" \"h3\"" - ], + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "84f6d1d7-b270-498b-b1fe-fe6d17e0e713" + "nuid": "84f6d1d7-b270-498b-b1fe-fe6d17e0e713", + "showTitle": false, + "title": "" } }, "outputs": [ { - "output_type": "display_data", + "data": { + "text/html": [ + "\n", + "
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n", + "
" + ] + }, "metadata": { "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n
", - "removedWidgets": [], "addedWidgets": {}, + "arguments": {}, + "data": "
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n
", + "datasetInfos": [], "metadata": {}, - "type": "html", - "arguments": {} + "removedWidgets": [], + "type": "html" } }, + "output_type": "display_data" + }, + { "data": { "text/html": [ - "\n
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n
" + "" ] - } - }, - { - "output_type": "display_data", + }, "metadata": { "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "", - "textData": null, - "removedWidgets": [], "addedWidgets": {}, + "arguments": {}, + "data": "", + "datasetInfos": [], "metadata": {}, - "type": "htmlSandbox", - "arguments": {} + "removedWidgets": [], + "textData": null, + "type": "htmlSandbox" } }, + "output_type": "display_data" + }, + { "data": { "text/html": [ - "" + "Kepler.gl
" ] - } - }, - { - "output_type": "display_data", + }, "metadata": { "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "Kepler.gl
", - "textData": null, - "removedWidgets": [], "addedWidgets": {}, + "arguments": {}, + "data": "Kepler.gl
", + "datasetInfos": [], "metadata": {}, - "type": "htmlSandbox", - "arguments": {} + "removedWidgets": [], + "textData": null, + "type": "htmlSandbox" } }, - "data": { - "text/html": [ - "Kepler.gl
" - ] - } + "output_type": "display_data" } ], - "execution_count": 0 + "source": [ + "%%mosaic_kepler\n", + "neighbourhood_chips \"index_id\" \"h3\"" + ] }, { "cell_type": "markdown", - "source": [ - "![mosaic kepler map example H3 indexes](../images/kepler-2.png)" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "a9d921cb-5994-42a9-9518-626486cd1227" + "nuid": "a9d921cb-5994-42a9-9518-626486cd1227", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "![mosaic kepler map example H3 indexes](../images/kepler-2.png)" + ] }, { "cell_type": "markdown", - "source": [ - "### Plot H3 chips" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "64cb0e7f-be2e-48f9-bea8-35a147e3f60c" + "nuid": "64cb0e7f-be2e-48f9-bea8-35a147e3f60c", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "### Plot H3 chips" + ] }, { "cell_type": "code", - "source": [ - "%%mosaic_kepler\n", - "neighbourhood_chips \"wkb\" \"geometry\"" - ], + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "dbe03c2b-a2c2-4b17-a44c-16891e81d3d0" + "nuid": "dbe03c2b-a2c2-4b17-a44c-16891e81d3d0", + "showTitle": false, + "title": "" } }, "outputs": [ { - "output_type": "display_data", + "data": { + "text/html": [ + "\n", + "
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n", + "
" + ] + }, "metadata": { "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n
", - "removedWidgets": [], "addedWidgets": {}, + "arguments": {}, + "data": "
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n
", + "datasetInfos": [], "metadata": {}, - "type": "html", - "arguments": {} + "removedWidgets": [], + "type": "html" } }, + "output_type": "display_data" + }, + { "data": { "text/html": [ - "\n
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n
" + "" ] - } - }, - { - "output_type": "display_data", + }, "metadata": { "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "", - "textData": null, - "removedWidgets": [], "addedWidgets": {}, + "arguments": {}, + "data": "", + "datasetInfos": [], "metadata": {}, - "type": "htmlSandbox", - "arguments": {} + "removedWidgets": [], + "textData": null, + "type": "htmlSandbox" } }, + "output_type": "display_data" + }, + { "data": { "text/html": [ - "" + "Kepler.gl
" ] - } - }, - { - "output_type": "display_data", + }, "metadata": { "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "Kepler.gl
", - "textData": null, - "removedWidgets": [], "addedWidgets": {}, + "arguments": {}, + "data": "Kepler.gl
", + "datasetInfos": [], "metadata": {}, - "type": "htmlSandbox", - "arguments": {} + "removedWidgets": [], + "textData": null, + "type": "htmlSandbox" } }, - "data": { - "text/html": [ - "Kepler.gl
" - ] - } + "output_type": "display_data" } ], - "execution_count": 0 + "source": [ + "%%mosaic_kepler\n", + "neighbourhood_chips \"wkb\" \"geometry\"" + ] }, { "cell_type": "markdown", - "source": [ - "![mosaic kepler map example h3 chips](../images/kepler-3.png)" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, "inputWidgets": {}, - "nuid": "6c8244f4-4c03-4d12-9d06-d5174ef94626" + "nuid": "6c8244f4-4c03-4d12-9d06-d5174ef94626", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "![mosaic kepler map example h3 chips](../images/kepler-3.png)" + ] } ], "metadata": { "application/vnd.databricks.v1+notebook": { - "notebookName": "kepler", "dashboards": [], + "language": "python", "notebookMetadata": { "pythonIndentUnit": 2 }, - "language": "python", - "widgets": {}, - "notebookOrigID": 2874007245243191 + "notebookName": "kepler", + "notebookOrigID": 2874007245243191, + "widgets": {} + }, + "language_info": { + "name": "python" } }, "nbformat": 4, From 55948f47ea0135456d7b298ac31418de9bdb587e Mon Sep 17 00:00:00 2001 From: Daniel Sparing Date: Thu, 18 Jan 2024 16:44:36 +0100 Subject: [PATCH 04/47] add tests comparing with postgis --- .../geometry/ST_BufferBehaviors.scala | 135 +++++++++++++++++- 1 file changed, 134 insertions(+), 1 deletion(-) diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala index 674238e43..30983d81b 100644 --- a/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala @@ -8,10 +8,15 @@ import com.databricks.labs.mosaic.test.mocks.getWKTRowsDf import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator} import org.apache.spark.sql.execution.WholeStageCodegenExec +import org.apache.spark.sql._ import org.apache.spark.sql.functions.{col, lit} -import org.scalatest.matchers.must.Matchers.noException +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ +import org.scalatest.matchers.must.Matchers._ import org.scalatest.matchers.should.Matchers.{an, be, convertToAnyShouldWrapper} +import scala.collection.JavaConverters._ + trait ST_BufferBehaviors extends QueryTest { def bufferBehavior(indexSystem: IndexSystem, geometryAPI: GeometryAPI): Unit = { @@ -53,6 +58,45 @@ trait ST_BufferBehaviors extends QueryTest { mocks.getWKTRowsDf().select(st_buffer_cap_style($"wkt", lit(1), lit("round"))).collect() mocks.getWKTRowsDf().select(st_buffer_cap_style($"wkt", 1, "round")).collect() + + val sourceDf = testData(spark) + + val noExtraParamResult = sourceDf + .where($"endCapStyle" === "round" and $"quadrantSegments" === 8) + .withColumn("geomBufferedTest", st_buffer(convert_to($"geom", "COORDS"), $"buffer")) + .select( + st_distance($"geomBufferedRef", $"geomBufferedTest"), + st_area(st_intersection($"geomBufferedRef", $"geomBufferedTest")), + st_area(st_union($"geomBufferedRef", $"geomBufferedTest")), + st_area($"geomBufferedRef"), + ) + .as[(Double, Double, Double, Double)] + .collect() + + noExtraParamResult.foreach({ case (d, a1, a2, a3) => + d should be < 1e-9 + a1 shouldBe a2 +- 1e-9 + a1 shouldBe a3 +- 1e-9 + }) + +/* val extraParamResult = sourceDf + .withColumn("geomBufferedTest", st_buffer(convert_to($"geom", "COORDS"), $"buffer", $"endCapStyle", $"quadrantSegments")) + .select( + st_distance($"geomBufferedRef", $"geomBufferedTest"), + st_area(st_intersection($"geomBufferedRef", $"geomBufferedTest")), + st_area(st_union($"geomBufferedRef", $"geomBufferedTest")), + st_area($"geomBufferedRef"), + ) + .as[(Double, Double, Double, Double)] + .collect() + + extraParamResult.foreach({ case (d, a1, a2, a3) => + d should be < 1e-9 + a1 shouldBe a2 +- 1e-9 + a1 shouldBe a3 +- 1e-9 + }) + */ + } def bufferCodegen(indexSystem: IndexSystem, geometryAPI: GeometryAPI): Unit = { @@ -84,6 +128,95 @@ trait ST_BufferBehaviors extends QueryTest { an[Error] should be thrownBy stBuffer.genCode(ctx) } + def testData(spark: SparkSession): DataFrame = { + // Comparison vs. PostGIS + val testDataWKT = List( + ( + "POLYGON((30 10,40 40,20 40,10 20,30 10))", + 2, + "round", + 8, + "POLYGON((29.105572809000083 8.211145618000169,9.105572809000083 18.211145618000167,8.773770799250563 18.420012042061902,8.489092090008587 18.689596517279742,8.262476724420662 19.009539063455783,8.102633403898972 19.367544467966326,8.015704817609999 19.749854783887773,8.005031582374714 20.14177804018136,8.071023864637592 20.528252845900816,8.211145618000169 20.894427190999917,18.211145618000167 40.89442719099991,18.40562298012387 41.207460938701914,18.65423405443726 41.47949789447759,18.948537775761732 41.70130161670408,19.278541681368626 41.865341227356964,19.63304118399603 41.96604710710526,20 42,40 42,40.37625388633239 41.96428944227163,40.73907153653638 41.85843301301978,41.07549652996949 41.68621090437216,41.37351494248246 41.45377326388176,41.622484370351124 41.16942056932753,41.813513976448974 40.84330721402359,41.939781986999634 40.4870788877705,41.99677930092308 40.11345670277741,41.9824705123501 39.73578291565012,41.89736659610103 39.367544467966326,31.897366596101026 9.367544467966324,31.743483999644496 9.02006962339989,31.5266535144039 8.707974827273778,31.25470373030189 8.44252815461514,30.937453300545492 8.233313466034124,30.5863564425368 8.087884385740342,30.214089387302234 8.011491580544716,29.83409270854116 8.006893186344296,29.46008605534576 8.074255226576508,29.105572809000083 8.211145618000169))" + ), + ( + "POLYGON((30 10,40 40,20 40,10 20,30 10))", + 3, + "round", + 8, + "POLYGON((28.658359213500127 7.316718427000252,8.658359213500127 17.316718427000254,8.160656198875845 17.630018063092855,7.733638135012881 18.034394775919612,7.393715086630994 18.51430859518367,7.153950105848459 19.051316701949485,7.023557226414999 19.624782175831655,7.007547373562073 20.212667060272036,7.106535796956389 20.792379268851224,7.316718427000252 21.341640786499873,17.316718427000254 41.34164078649987,17.60843447018581 41.81119140805288,17.981351081655895 42.21924684171639,18.4228066636426 42.55195242505612,18.91781252205294 42.798011841035446,19.44956177599405 42.949070660657895,20 43,40 43,40.56438082949858 42.94643416340744,41.10860730480457 42.78764951952966,41.613244794954234 42.52931635655824,42.06027241372369 42.18065989582263,42.43372655552668 41.7541308539913,42.72027096467346 41.26496082103538,42.90967298049945 40.73061833165575,42.99516895138463 40.17018505416612,42.97370576852514 39.60367437347518,42.84604989415154 39.05131670194949,32.84604989415154 9.051316701949487,32.615225999466745 8.530104435099837,32.28998027160585 8.061962240910669,31.882055595452837 7.663792231922711,31.40617995081824 7.349970199051186,30.879534663805202 7.131826578610514,30.321134080953357 7.017237370817074,29.75113906281174 7.010339779516444,29.19012908301864 7.111382839864762,28.658359213500127 7.316718427000252))" + ), + ( + "POLYGON((30 10,40 40,20 40,10 20,30 10))", + 3, + "round", + 2, + "POLYGON((28.658359213500127 7.316718427000252,8.658359213500127 17.316718427000254,7.153950105848459 19.051316701949485,7.316718427000252 21.341640786499873,17.316718427000254 41.34164078649987,20 43,40 43,42.43372655552668 41.7541308539913,42.84604989415154 39.05131670194949,32.84604989415154 9.051316701949487,31.14805029709527 7.22836140246614,28.658359213500127 7.316718427000252))" + ), + ( + "MULTIPOLYGON(((0 0,0 1,2 2,0 0)))", + 2, + "round", + 8, + "POLYGON((1.414213562373095 -1.414213562373095,1.111140466039204 -1.662939224605091,0.76536686473018 -1.847759065022573,0.390180644032257 -1.961570560806461,-3.673940397442059e-16 -2,-0.390180644032257 -1.961570560806461,-0.765366864730181 -1.847759065022573,-1.111140466039204 -1.66293922460509,-1.414213562373095 -1.414213562373095,-1.662939224605091 -1.111140466039204,-1.847759065022574 -0.765366864730179,-1.961570560806461 -0.390180644032257,-2 0,-2 1,-1.966047107105261 1.366958816003967,-1.865341227356966 1.721458318631373,-1.70130161670408 2.051462224238267,-1.479497894477594 2.345765945562737,-1.207460938701918 2.594377019876128,-0.894427190999916 2.788854381999832,1.105572809000084 3.788854381999832,1.481529468137407 3.931628408258203,1.878448906059273 3.996302915782525,2.280282892709963 3.980262987598904,2.670784493205818 3.884157149410477,3.034164963224574 3.711871148433408,3.355732118853176 3.470370845028517,3.622484370351121 3.16942056932753,3.823636401881189 2.821188330246984,3.951055242946718 2.439753839062402,3.999589100313399 2.040539239112909,3.9672756550971 1.639685558348996,3.855421403158644 1.253400095967856,3.668548830917987 0.897301129572439,3.414213562373095 0.585786437626905,1.414213562373095 -1.414213562373095))" + ), + ( + "MULTIPOLYGON(((40 60,20 45,45 30,40 60)), ((20 35,10 30,10 10,30 5,45 20,20 35), (30 20,20 15,20 25,30 20)))", + 2, + "round", + 8, + "MULTIPOLYGON(((21.028991510855054 36.71498585142509,46.02899151085505 21.714985851425087,46.351780955338434 21.474004154941333,46.619733646867054 21.173227562413512,46.82197973593542 20.824857467591027,46.95031484684309 20.443025956557175,46.999532898867834 20.043222521273073,46.9676372977026 19.641665708186324,46.85592192990094 19.25464519179602,46.66891867468572 18.897860962815827,46.41421356237309 18.585786437626904,31.414213562373096 3.585786437626905,31.094914714141236 3.326332838119534,30.72928238054107 3.137703780427949,30.332789018157783 3.02788147683929,29.92221302915125 3.001513275709299,29.514928749927336 3.059714999709336,9.514928749927334 8.059714999709335,9.15830172540112 8.185738713818413,8.83177905204805 8.376651664994155,8.547039206017194 8.625625621910517,8.314266250382676 8.923755731536227,8.141785589878605 9.260379011914079,8.035766202785066 9.623455726535903,8 10,8 30,8.033952892894739 30.36695881600397,8.134658772643034 30.721458318631374,8.298698383295921 31.051462224238268,8.520502105522406 31.345765945562736,8.792539061298083 31.594377019876127,9.105572809000083 31.788854381999833,19.105572809000083 36.78885438199983,19.480363108468328 36.93131496679317,19.87603797663762 36.99615465752629,20.27669496489004 36.9807675018549,20.66623138973048 36.88577192028564,21.028991510855054 36.71498585142509),(25.527864045000417 20,22 21.763932022500207,22 18.236067977499793,25.527864045000417 20)),((41.972787847664286 60.328797974610715,46.972787847664286 30.328797974610715,46.9993974080564 29.950908201730538,46.954274213176696 29.574779704491174,46.83903715633888 29.21390691543241,46.657820619013656 28.881236935194416,46.417126140850286 28.58870502696326,46.12558916272103 28.346806412798255,45.79366921139157 28.164219734584478,45.43327464112035 28.047495688772486,45.05733539540255 28.000822005814882,44.67933911696567 28.025873206176552,44.31284724908532 28.121750523250334,43.97100848914495 28.285014148574913,18.971008489144946 43.28501414857491,18.65706596538611 43.51793111540804,18.394426953504244 43.80746690093468,18.19312493490509 44.14256049826342,18.060850161109478 44.51041047567288,18.00265586649343 44.89696402401638,18.0207652218219 45.28745380994221,18.114486403444406 45.666962125764265,18.280239022726388 46.02099078401664,18.511690906044425 46.33601498526369,18.8 46.6,38.8 61.6,39.15068013648958 61.81070587601815,39.53689868768618 61.94564569604366,39.94249496357223 61.99917312176446,40.3504974446115 61.969048384707904,40.74383392750859 61.85653200572658,41.106045883917275 61.66633205054384,41.4219771409141 61.406407128365665,41.678408066517704 61.08763337676274,41.86460872378755 60.72334936730143,41.972787847664286 60.328797974610715)))" + ), + ( + "POINT(-75.78033 35.18937)", + 2, + "round", + 8, + "POLYGON((-73.78033 35.18937,-73.81875943919354 34.79918935596774,-73.93257093497743 34.424003135269814,-74.11739077539491 34.078229533960794,-74.36611643762691 33.775156437626904,-74.6691895339608 33.52643077539491,-75.01496313526982 33.341610934977425,-75.39014935596775 33.22779943919353,-75.78033 33.18937,-76.17051064403226 33.22779943919353,-76.54569686473019 33.341610934977425,-76.89147046603921 33.52643077539491,-77.1945435623731 33.775156437626904,-77.4432692246051 34.078229533960794,-77.62808906502258 34.424003135269814,-77.74190056080647 34.79918935596774,-77.78033 35.18937,-77.74190056080647 35.57955064403225,-77.62808906502258 35.95473686473018,-77.4432692246051 36.3005104660392,-77.1945435623731 36.60358356237309,-76.89147046603921 36.85230922460509,-76.54569686473019 37.03712906502257,-76.17051064403226 37.150940560806454,-75.78033 37.18937,-75.39014935596775 37.15094056080646,-75.01496313526982 37.03712906502257,-74.6691895339608 36.85230922460509,-74.36611643762691 36.60358356237309,-74.11739077539491 36.3005104660392,-73.93257093497743 35.95473686473018,-73.81875943919354 35.57955064403225,-73.78033 35.18937))" + ), + ( + "MULTIPOINT(10 40,40 30,20 20,30 10)", + 2, + "round", + 8, + "MULTIPOLYGON(((42 30,41.961570560806464 29.609819355967744,41.84775906502257 29.23463313526982,41.66293922460509 28.888859533960797,41.41421356237309 28.585786437626904,41.1111404660392 28.33706077539491,40.76536686473018 28.152240934977428,40.390180644032256 28.03842943919354,40 28,39.609819355967744 28.03842943919354,39.23463313526982 28.152240934977428,38.8888595339608 28.33706077539491,38.58578643762691 28.585786437626904,38.33706077539491 28.888859533960797,38.15224093497743 29.23463313526982,38.038429439193536 29.609819355967744,38 30,38.038429439193536 30.390180644032256,38.15224093497743 30.76536686473018,38.33706077539491 31.111140466039203,38.58578643762691 31.414213562373096,38.8888595339608 31.66293922460509,39.23463313526982 31.847759065022572,39.609819355967744 31.96157056080646,40 32,40.390180644032256 31.96157056080646,40.76536686473018 31.847759065022572,41.1111404660392 31.66293922460509,41.41421356237309 31.414213562373096,41.66293922460509 31.111140466039203,41.84775906502257 30.76536686473018,41.96157056080646 30.390180644032256,42 30)),((32 10,31.96157056080646 9.609819355967744,31.847759065022572 9.23463313526982,31.66293922460509 8.888859533960796,31.414213562373096 8.585786437626904,31.111140466039206 8.33706077539491,30.76536686473018 8.152240934977426,30.390180644032256 8.03842943919354,30 8,29.609819355967744 8.03842943919354,29.23463313526982 8.152240934977426,28.888859533960797 8.33706077539491,28.585786437626904 8.585786437626904,28.33706077539491 8.888859533960796,28.152240934977428 9.23463313526982,28.03842943919354 9.609819355967742,28 10,28.03842943919354 10.390180644032256,28.152240934977428 10.76536686473018,28.33706077539491 11.111140466039204,28.585786437626904 11.414213562373096,28.888859533960797 11.66293922460509,29.23463313526982 11.847759065022572,29.609819355967744 11.96157056080646,30 12,30.390180644032256 11.96157056080646,30.76536686473018 11.847759065022574,31.111140466039203 11.662939224605092,31.414213562373096 11.414213562373096,31.66293922460509 11.111140466039204,31.847759065022572 10.765366864730181,31.96157056080646 10.390180644032258,32 10)),((22 20,21.96157056080646 19.609819355967744,21.847759065022572 19.23463313526982,21.66293922460509 18.888859533960797,21.414213562373096 18.585786437626904,21.111140466039206 18.33706077539491,20.76536686473018 18.152240934977428,20.390180644032256 18.03842943919354,20 18,19.609819355967744 18.03842943919354,19.23463313526982 18.152240934977428,18.888859533960797 18.33706077539491,18.585786437626904 18.585786437626904,18.33706077539491 18.888859533960797,18.152240934977428 19.23463313526982,18.03842943919354 19.609819355967744,18 20,18.03842943919354 20.390180644032256,18.152240934977428 20.76536686473018,18.33706077539491 21.111140466039203,18.585786437626904 21.414213562373096,18.888859533960797 21.66293922460509,19.23463313526982 21.847759065022572,19.609819355967744 21.96157056080646,20 22,20.390180644032256 21.96157056080646,20.76536686473018 21.847759065022572,21.111140466039203 21.66293922460509,21.414213562373096 21.414213562373096,21.66293922460509 21.111140466039203,21.847759065022572 20.76536686473018,21.96157056080646 20.390180644032256,22 20)),((12 40,11.96157056080646 39.609819355967744,11.847759065022574 39.23463313526982,11.66293922460509 38.8888595339608,11.414213562373096 38.58578643762691,11.111140466039204 38.33706077539491,10.76536686473018 38.15224093497743,10.390180644032256 38.038429439193536,10 38,9.609819355967744 38.038429439193536,9.23463313526982 38.15224093497743,8.888859533960796 38.33706077539491,8.585786437626904 38.58578643762691,8.33706077539491 38.8888595339608,8.152240934977426 39.23463313526982,8.03842943919354 39.609819355967744,8 40,8.03842943919354 40.390180644032256,8.152240934977426 40.76536686473018,8.337060775394908 41.1111404660392,8.585786437626904 41.41421356237309,8.888859533960796 41.66293922460509,9.234633135269819 41.84775906502257,9.609819355967742 41.96157056080646,10 42,10.390180644032256 41.961570560806464,10.76536686473018 41.84775906502257,11.111140466039204 41.66293922460509,11.414213562373094 41.41421356237309,11.66293922460509 41.1111404660392,11.847759065022572 40.76536686473018,11.96157056080646 40.390180644032256,12 40)))" + ), + ( + "LINESTRING(30 10,10 30,40 40)", + 2, + "round", + 8, + "POLYGON((8.585786437626904 28.585786437626904,8.329219317019618 28.900685709462664,8.141566986980227 29.26092846346362,8.030569554367744 29.651655745250856,8.000805336911114 30.056751203308092,8.05350202106454 30.45950584109472,8.186486023551023 30.843307214023586,8.394272145281546 31.19232464395452,8.668289819689326 31.49216218812062,8.997236623400621 31.730452429438905,9.367544467966324 31.897366596101026,39.367544467966326 41.89736659610103,39.74985478388777 41.98429518239,40.14177804018136 41.99496841762529,40.528252845900816 41.928976135362404,40.89442719099991 41.78885438199983,41.22622920074944 41.5799879579381,41.51090790999141 41.31040348272026,41.737523275579335 40.99046093654422,41.89736659610103 40.632455532033674,41.98429518239 40.25014521611223,41.99496841762529 39.85822195981864,41.928976135362404 39.471747154099184,41.78885438199983 39.10557280900009,41.579987957938094 38.77377079925056,41.31040348272026 38.48909209000859,40.99046093654422 38.262476724420665,40.632455532033674 38.10263340389897,13.702459173643835 29.12596795110236,31.414213562373096 11.414213562373096,31.66293922460509 11.111140466039204,31.847759065022572 10.76536686473018,31.96157056080646 10.390180644032256,32 10,31.96157056080646 9.609819355967744,31.847759065022572 9.23463313526982,31.66293922460509 8.888859533960796,31.414213562373096 8.585786437626904,31.111140466039206 8.33706077539491,30.76536686473018 8.152240934977426,30.390180644032256 8.03842943919354,30 8,29.609819355967744 8.03842943919354,29.23463313526982 8.152240934977426,28.888859533960797 8.33706077539491,28.585786437626904 8.585786437626904,8.585786437626904 28.585786437626904))" + ), + ( + "LINESTRING(30 10,10 30,40 40)", + 2, + "square", + 8, + "POLYGON((8.585786437626904 28.585786437626904,8.329219317019618 28.900685709462664,8.141566986980227 29.26092846346362,8.030569554367744 29.651655745250856,8.000805336911114 30.056751203308092,8.05350202106454 30.45950584109472,8.186486023551023 30.843307214023586,8.394272145281546 31.19232464395452,8.668289819689326 31.49216218812062,8.997236623400621 31.730452429438905,9.367544467966324 31.897366596101026,39.367544467966326 41.89736659610103,41.264911064067356 42.529822128134704,42.529822128134704 38.735088935932644,13.702459173643835 29.12596795110236,31.414213562373096 11.414213562373096,32.82842712474619 10,30 7.171572875253809,8.585786437626904 28.585786437626904))" + ), + ( + "LINESTRING(30 10,10 30,40 40)", + 2, + "flat", + 8, + "POLYGON((8.585786437626904 28.585786437626904,8.329219317019618 28.900685709462664,8.141566986980227 29.26092846346362,8.030569554367744 29.651655745250856,8.000805336911114 30.056751203308092,8.05350202106454 30.45950584109472,8.186486023551023 30.843307214023586,8.394272145281546 31.19232464395452,8.668289819689326 31.49216218812062,8.997236623400621 31.730452429438905,9.367544467966324 31.897366596101026,39.367544467966326 41.89736659610103,40.632455532033674 38.10263340389897,13.702459173643835 29.12596795110236,31.414213562373096 11.414213562373096,28.585786437626904 8.585786437626904,8.585786437626904 28.585786437626904))" + ) + ).map({ case (f: String, b: Double, c: String, q: Double, t: String) => Row(f, b, c, q, t) }) + val testSchema = StructType( + Seq( + StructField("geom", StringType), + StructField("buffer", DoubleType), + StructField("endCapStyle", StringType), + StructField("quadrantSegments", DoubleType), + StructField("geomBufferedRef", StringType) + ) + ) + val sourceDf = spark + .createDataFrame(testDataWKT.asJava, testSchema) + sourceDf + + } + def auxiliaryMethods(indexSystem: IndexSystem, geometryAPI: GeometryAPI): Unit = { spark.sparkContext.setLogLevel("FATAL") val mc = MosaicContext.build(indexSystem, geometryAPI) From 1a557066c3a861169c1acef73ec0b1b48311d22b Mon Sep 17 00:00:00 2001 From: Daniel Sparing Date: Thu, 18 Jan 2024 17:57:22 +0100 Subject: [PATCH 05/47] int/double parameter typing (non-extra param st_buffer passes tests) --- .../geometry/ST_BufferBehaviors.scala | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala index 30983d81b..ba91862e9 100644 --- a/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala @@ -133,81 +133,81 @@ trait ST_BufferBehaviors extends QueryTest { val testDataWKT = List( ( "POLYGON((30 10,40 40,20 40,10 20,30 10))", - 2, + 2.0, "round", 8, "POLYGON((29.105572809000083 8.211145618000169,9.105572809000083 18.211145618000167,8.773770799250563 18.420012042061902,8.489092090008587 18.689596517279742,8.262476724420662 19.009539063455783,8.102633403898972 19.367544467966326,8.015704817609999 19.749854783887773,8.005031582374714 20.14177804018136,8.071023864637592 20.528252845900816,8.211145618000169 20.894427190999917,18.211145618000167 40.89442719099991,18.40562298012387 41.207460938701914,18.65423405443726 41.47949789447759,18.948537775761732 41.70130161670408,19.278541681368626 41.865341227356964,19.63304118399603 41.96604710710526,20 42,40 42,40.37625388633239 41.96428944227163,40.73907153653638 41.85843301301978,41.07549652996949 41.68621090437216,41.37351494248246 41.45377326388176,41.622484370351124 41.16942056932753,41.813513976448974 40.84330721402359,41.939781986999634 40.4870788877705,41.99677930092308 40.11345670277741,41.9824705123501 39.73578291565012,41.89736659610103 39.367544467966326,31.897366596101026 9.367544467966324,31.743483999644496 9.02006962339989,31.5266535144039 8.707974827273778,31.25470373030189 8.44252815461514,30.937453300545492 8.233313466034124,30.5863564425368 8.087884385740342,30.214089387302234 8.011491580544716,29.83409270854116 8.006893186344296,29.46008605534576 8.074255226576508,29.105572809000083 8.211145618000169))" ), ( "POLYGON((30 10,40 40,20 40,10 20,30 10))", - 3, + 3.0, "round", 8, "POLYGON((28.658359213500127 7.316718427000252,8.658359213500127 17.316718427000254,8.160656198875845 17.630018063092855,7.733638135012881 18.034394775919612,7.393715086630994 18.51430859518367,7.153950105848459 19.051316701949485,7.023557226414999 19.624782175831655,7.007547373562073 20.212667060272036,7.106535796956389 20.792379268851224,7.316718427000252 21.341640786499873,17.316718427000254 41.34164078649987,17.60843447018581 41.81119140805288,17.981351081655895 42.21924684171639,18.4228066636426 42.55195242505612,18.91781252205294 42.798011841035446,19.44956177599405 42.949070660657895,20 43,40 43,40.56438082949858 42.94643416340744,41.10860730480457 42.78764951952966,41.613244794954234 42.52931635655824,42.06027241372369 42.18065989582263,42.43372655552668 41.7541308539913,42.72027096467346 41.26496082103538,42.90967298049945 40.73061833165575,42.99516895138463 40.17018505416612,42.97370576852514 39.60367437347518,42.84604989415154 39.05131670194949,32.84604989415154 9.051316701949487,32.615225999466745 8.530104435099837,32.28998027160585 8.061962240910669,31.882055595452837 7.663792231922711,31.40617995081824 7.349970199051186,30.879534663805202 7.131826578610514,30.321134080953357 7.017237370817074,29.75113906281174 7.010339779516444,29.19012908301864 7.111382839864762,28.658359213500127 7.316718427000252))" ), ( "POLYGON((30 10,40 40,20 40,10 20,30 10))", - 3, + 3.0, "round", 2, "POLYGON((28.658359213500127 7.316718427000252,8.658359213500127 17.316718427000254,7.153950105848459 19.051316701949485,7.316718427000252 21.341640786499873,17.316718427000254 41.34164078649987,20 43,40 43,42.43372655552668 41.7541308539913,42.84604989415154 39.05131670194949,32.84604989415154 9.051316701949487,31.14805029709527 7.22836140246614,28.658359213500127 7.316718427000252))" ), ( "MULTIPOLYGON(((0 0,0 1,2 2,0 0)))", - 2, + 2.0, "round", 8, "POLYGON((1.414213562373095 -1.414213562373095,1.111140466039204 -1.662939224605091,0.76536686473018 -1.847759065022573,0.390180644032257 -1.961570560806461,-3.673940397442059e-16 -2,-0.390180644032257 -1.961570560806461,-0.765366864730181 -1.847759065022573,-1.111140466039204 -1.66293922460509,-1.414213562373095 -1.414213562373095,-1.662939224605091 -1.111140466039204,-1.847759065022574 -0.765366864730179,-1.961570560806461 -0.390180644032257,-2 0,-2 1,-1.966047107105261 1.366958816003967,-1.865341227356966 1.721458318631373,-1.70130161670408 2.051462224238267,-1.479497894477594 2.345765945562737,-1.207460938701918 2.594377019876128,-0.894427190999916 2.788854381999832,1.105572809000084 3.788854381999832,1.481529468137407 3.931628408258203,1.878448906059273 3.996302915782525,2.280282892709963 3.980262987598904,2.670784493205818 3.884157149410477,3.034164963224574 3.711871148433408,3.355732118853176 3.470370845028517,3.622484370351121 3.16942056932753,3.823636401881189 2.821188330246984,3.951055242946718 2.439753839062402,3.999589100313399 2.040539239112909,3.9672756550971 1.639685558348996,3.855421403158644 1.253400095967856,3.668548830917987 0.897301129572439,3.414213562373095 0.585786437626905,1.414213562373095 -1.414213562373095))" ), ( "MULTIPOLYGON(((40 60,20 45,45 30,40 60)), ((20 35,10 30,10 10,30 5,45 20,20 35), (30 20,20 15,20 25,30 20)))", - 2, + 2.0, "round", 8, "MULTIPOLYGON(((21.028991510855054 36.71498585142509,46.02899151085505 21.714985851425087,46.351780955338434 21.474004154941333,46.619733646867054 21.173227562413512,46.82197973593542 20.824857467591027,46.95031484684309 20.443025956557175,46.999532898867834 20.043222521273073,46.9676372977026 19.641665708186324,46.85592192990094 19.25464519179602,46.66891867468572 18.897860962815827,46.41421356237309 18.585786437626904,31.414213562373096 3.585786437626905,31.094914714141236 3.326332838119534,30.72928238054107 3.137703780427949,30.332789018157783 3.02788147683929,29.92221302915125 3.001513275709299,29.514928749927336 3.059714999709336,9.514928749927334 8.059714999709335,9.15830172540112 8.185738713818413,8.83177905204805 8.376651664994155,8.547039206017194 8.625625621910517,8.314266250382676 8.923755731536227,8.141785589878605 9.260379011914079,8.035766202785066 9.623455726535903,8 10,8 30,8.033952892894739 30.36695881600397,8.134658772643034 30.721458318631374,8.298698383295921 31.051462224238268,8.520502105522406 31.345765945562736,8.792539061298083 31.594377019876127,9.105572809000083 31.788854381999833,19.105572809000083 36.78885438199983,19.480363108468328 36.93131496679317,19.87603797663762 36.99615465752629,20.27669496489004 36.9807675018549,20.66623138973048 36.88577192028564,21.028991510855054 36.71498585142509),(25.527864045000417 20,22 21.763932022500207,22 18.236067977499793,25.527864045000417 20)),((41.972787847664286 60.328797974610715,46.972787847664286 30.328797974610715,46.9993974080564 29.950908201730538,46.954274213176696 29.574779704491174,46.83903715633888 29.21390691543241,46.657820619013656 28.881236935194416,46.417126140850286 28.58870502696326,46.12558916272103 28.346806412798255,45.79366921139157 28.164219734584478,45.43327464112035 28.047495688772486,45.05733539540255 28.000822005814882,44.67933911696567 28.025873206176552,44.31284724908532 28.121750523250334,43.97100848914495 28.285014148574913,18.971008489144946 43.28501414857491,18.65706596538611 43.51793111540804,18.394426953504244 43.80746690093468,18.19312493490509 44.14256049826342,18.060850161109478 44.51041047567288,18.00265586649343 44.89696402401638,18.0207652218219 45.28745380994221,18.114486403444406 45.666962125764265,18.280239022726388 46.02099078401664,18.511690906044425 46.33601498526369,18.8 46.6,38.8 61.6,39.15068013648958 61.81070587601815,39.53689868768618 61.94564569604366,39.94249496357223 61.99917312176446,40.3504974446115 61.969048384707904,40.74383392750859 61.85653200572658,41.106045883917275 61.66633205054384,41.4219771409141 61.406407128365665,41.678408066517704 61.08763337676274,41.86460872378755 60.72334936730143,41.972787847664286 60.328797974610715)))" ), ( "POINT(-75.78033 35.18937)", - 2, + 2.0, "round", 8, "POLYGON((-73.78033 35.18937,-73.81875943919354 34.79918935596774,-73.93257093497743 34.424003135269814,-74.11739077539491 34.078229533960794,-74.36611643762691 33.775156437626904,-74.6691895339608 33.52643077539491,-75.01496313526982 33.341610934977425,-75.39014935596775 33.22779943919353,-75.78033 33.18937,-76.17051064403226 33.22779943919353,-76.54569686473019 33.341610934977425,-76.89147046603921 33.52643077539491,-77.1945435623731 33.775156437626904,-77.4432692246051 34.078229533960794,-77.62808906502258 34.424003135269814,-77.74190056080647 34.79918935596774,-77.78033 35.18937,-77.74190056080647 35.57955064403225,-77.62808906502258 35.95473686473018,-77.4432692246051 36.3005104660392,-77.1945435623731 36.60358356237309,-76.89147046603921 36.85230922460509,-76.54569686473019 37.03712906502257,-76.17051064403226 37.150940560806454,-75.78033 37.18937,-75.39014935596775 37.15094056080646,-75.01496313526982 37.03712906502257,-74.6691895339608 36.85230922460509,-74.36611643762691 36.60358356237309,-74.11739077539491 36.3005104660392,-73.93257093497743 35.95473686473018,-73.81875943919354 35.57955064403225,-73.78033 35.18937))" ), ( "MULTIPOINT(10 40,40 30,20 20,30 10)", - 2, + 2.0, "round", 8, "MULTIPOLYGON(((42 30,41.961570560806464 29.609819355967744,41.84775906502257 29.23463313526982,41.66293922460509 28.888859533960797,41.41421356237309 28.585786437626904,41.1111404660392 28.33706077539491,40.76536686473018 28.152240934977428,40.390180644032256 28.03842943919354,40 28,39.609819355967744 28.03842943919354,39.23463313526982 28.152240934977428,38.8888595339608 28.33706077539491,38.58578643762691 28.585786437626904,38.33706077539491 28.888859533960797,38.15224093497743 29.23463313526982,38.038429439193536 29.609819355967744,38 30,38.038429439193536 30.390180644032256,38.15224093497743 30.76536686473018,38.33706077539491 31.111140466039203,38.58578643762691 31.414213562373096,38.8888595339608 31.66293922460509,39.23463313526982 31.847759065022572,39.609819355967744 31.96157056080646,40 32,40.390180644032256 31.96157056080646,40.76536686473018 31.847759065022572,41.1111404660392 31.66293922460509,41.41421356237309 31.414213562373096,41.66293922460509 31.111140466039203,41.84775906502257 30.76536686473018,41.96157056080646 30.390180644032256,42 30)),((32 10,31.96157056080646 9.609819355967744,31.847759065022572 9.23463313526982,31.66293922460509 8.888859533960796,31.414213562373096 8.585786437626904,31.111140466039206 8.33706077539491,30.76536686473018 8.152240934977426,30.390180644032256 8.03842943919354,30 8,29.609819355967744 8.03842943919354,29.23463313526982 8.152240934977426,28.888859533960797 8.33706077539491,28.585786437626904 8.585786437626904,28.33706077539491 8.888859533960796,28.152240934977428 9.23463313526982,28.03842943919354 9.609819355967742,28 10,28.03842943919354 10.390180644032256,28.152240934977428 10.76536686473018,28.33706077539491 11.111140466039204,28.585786437626904 11.414213562373096,28.888859533960797 11.66293922460509,29.23463313526982 11.847759065022572,29.609819355967744 11.96157056080646,30 12,30.390180644032256 11.96157056080646,30.76536686473018 11.847759065022574,31.111140466039203 11.662939224605092,31.414213562373096 11.414213562373096,31.66293922460509 11.111140466039204,31.847759065022572 10.765366864730181,31.96157056080646 10.390180644032258,32 10)),((22 20,21.96157056080646 19.609819355967744,21.847759065022572 19.23463313526982,21.66293922460509 18.888859533960797,21.414213562373096 18.585786437626904,21.111140466039206 18.33706077539491,20.76536686473018 18.152240934977428,20.390180644032256 18.03842943919354,20 18,19.609819355967744 18.03842943919354,19.23463313526982 18.152240934977428,18.888859533960797 18.33706077539491,18.585786437626904 18.585786437626904,18.33706077539491 18.888859533960797,18.152240934977428 19.23463313526982,18.03842943919354 19.609819355967744,18 20,18.03842943919354 20.390180644032256,18.152240934977428 20.76536686473018,18.33706077539491 21.111140466039203,18.585786437626904 21.414213562373096,18.888859533960797 21.66293922460509,19.23463313526982 21.847759065022572,19.609819355967744 21.96157056080646,20 22,20.390180644032256 21.96157056080646,20.76536686473018 21.847759065022572,21.111140466039203 21.66293922460509,21.414213562373096 21.414213562373096,21.66293922460509 21.111140466039203,21.847759065022572 20.76536686473018,21.96157056080646 20.390180644032256,22 20)),((12 40,11.96157056080646 39.609819355967744,11.847759065022574 39.23463313526982,11.66293922460509 38.8888595339608,11.414213562373096 38.58578643762691,11.111140466039204 38.33706077539491,10.76536686473018 38.15224093497743,10.390180644032256 38.038429439193536,10 38,9.609819355967744 38.038429439193536,9.23463313526982 38.15224093497743,8.888859533960796 38.33706077539491,8.585786437626904 38.58578643762691,8.33706077539491 38.8888595339608,8.152240934977426 39.23463313526982,8.03842943919354 39.609819355967744,8 40,8.03842943919354 40.390180644032256,8.152240934977426 40.76536686473018,8.337060775394908 41.1111404660392,8.585786437626904 41.41421356237309,8.888859533960796 41.66293922460509,9.234633135269819 41.84775906502257,9.609819355967742 41.96157056080646,10 42,10.390180644032256 41.961570560806464,10.76536686473018 41.84775906502257,11.111140466039204 41.66293922460509,11.414213562373094 41.41421356237309,11.66293922460509 41.1111404660392,11.847759065022572 40.76536686473018,11.96157056080646 40.390180644032256,12 40)))" ), ( "LINESTRING(30 10,10 30,40 40)", - 2, + 2.0, "round", 8, "POLYGON((8.585786437626904 28.585786437626904,8.329219317019618 28.900685709462664,8.141566986980227 29.26092846346362,8.030569554367744 29.651655745250856,8.000805336911114 30.056751203308092,8.05350202106454 30.45950584109472,8.186486023551023 30.843307214023586,8.394272145281546 31.19232464395452,8.668289819689326 31.49216218812062,8.997236623400621 31.730452429438905,9.367544467966324 31.897366596101026,39.367544467966326 41.89736659610103,39.74985478388777 41.98429518239,40.14177804018136 41.99496841762529,40.528252845900816 41.928976135362404,40.89442719099991 41.78885438199983,41.22622920074944 41.5799879579381,41.51090790999141 41.31040348272026,41.737523275579335 40.99046093654422,41.89736659610103 40.632455532033674,41.98429518239 40.25014521611223,41.99496841762529 39.85822195981864,41.928976135362404 39.471747154099184,41.78885438199983 39.10557280900009,41.579987957938094 38.77377079925056,41.31040348272026 38.48909209000859,40.99046093654422 38.262476724420665,40.632455532033674 38.10263340389897,13.702459173643835 29.12596795110236,31.414213562373096 11.414213562373096,31.66293922460509 11.111140466039204,31.847759065022572 10.76536686473018,31.96157056080646 10.390180644032256,32 10,31.96157056080646 9.609819355967744,31.847759065022572 9.23463313526982,31.66293922460509 8.888859533960796,31.414213562373096 8.585786437626904,31.111140466039206 8.33706077539491,30.76536686473018 8.152240934977426,30.390180644032256 8.03842943919354,30 8,29.609819355967744 8.03842943919354,29.23463313526982 8.152240934977426,28.888859533960797 8.33706077539491,28.585786437626904 8.585786437626904,8.585786437626904 28.585786437626904))" ), ( "LINESTRING(30 10,10 30,40 40)", - 2, + 2.0, "square", 8, "POLYGON((8.585786437626904 28.585786437626904,8.329219317019618 28.900685709462664,8.141566986980227 29.26092846346362,8.030569554367744 29.651655745250856,8.000805336911114 30.056751203308092,8.05350202106454 30.45950584109472,8.186486023551023 30.843307214023586,8.394272145281546 31.19232464395452,8.668289819689326 31.49216218812062,8.997236623400621 31.730452429438905,9.367544467966324 31.897366596101026,39.367544467966326 41.89736659610103,41.264911064067356 42.529822128134704,42.529822128134704 38.735088935932644,13.702459173643835 29.12596795110236,31.414213562373096 11.414213562373096,32.82842712474619 10,30 7.171572875253809,8.585786437626904 28.585786437626904))" ), ( "LINESTRING(30 10,10 30,40 40)", - 2, + 2.0, "flat", 8, "POLYGON((8.585786437626904 28.585786437626904,8.329219317019618 28.900685709462664,8.141566986980227 29.26092846346362,8.030569554367744 29.651655745250856,8.000805336911114 30.056751203308092,8.05350202106454 30.45950584109472,8.186486023551023 30.843307214023586,8.394272145281546 31.19232464395452,8.668289819689326 31.49216218812062,8.997236623400621 31.730452429438905,9.367544467966324 31.897366596101026,39.367544467966326 41.89736659610103,40.632455532033674 38.10263340389897,13.702459173643835 29.12596795110236,31.414213562373096 11.414213562373096,28.585786437626904 8.585786437626904,8.585786437626904 28.585786437626904))" ) - ).map({ case (f: String, b: Double, c: String, q: Double, t: String) => Row(f, b, c, q, t) }) + ).map({ case (f: String, b: Double, c: String, q: Int, t: String) => Row(f, b, c, q, t) }) val testSchema = StructType( Seq( StructField("geom", StringType), StructField("buffer", DoubleType), StructField("endCapStyle", StringType), - StructField("quadrantSegments", DoubleType), + StructField("quadrantSegments", IntegerType), StructField("geomBufferedRef", StringType) ) ) From d27d482a7ecf7c694b7cf7e1256fbd72fc985131 Mon Sep 17 00:00:00 2001 From: Daniel Sparing Date: Thu, 18 Jan 2024 18:04:10 +0100 Subject: [PATCH 06/47] scalafmt --- .../mosaic/expressions/geometry/ST_BufferBehaviors.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala index ba91862e9..bd66220b3 100644 --- a/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala @@ -68,7 +68,7 @@ trait ST_BufferBehaviors extends QueryTest { st_distance($"geomBufferedRef", $"geomBufferedTest"), st_area(st_intersection($"geomBufferedRef", $"geomBufferedTest")), st_area(st_union($"geomBufferedRef", $"geomBufferedTest")), - st_area($"geomBufferedRef"), + st_area($"geomBufferedRef") ) .as[(Double, Double, Double, Double)] .collect() @@ -79,7 +79,7 @@ trait ST_BufferBehaviors extends QueryTest { a1 shouldBe a3 +- 1e-9 }) -/* val extraParamResult = sourceDf + /* val extraParamResult = sourceDf .withColumn("geomBufferedTest", st_buffer(convert_to($"geom", "COORDS"), $"buffer", $"endCapStyle", $"quadrantSegments")) .select( st_distance($"geomBufferedRef", $"geomBufferedTest"), @@ -95,7 +95,7 @@ trait ST_BufferBehaviors extends QueryTest { a1 shouldBe a2 +- 1e-9 a1 shouldBe a3 +- 1e-9 }) - */ + */ } @@ -223,7 +223,6 @@ trait ST_BufferBehaviors extends QueryTest { mc.register(spark) import mc.functions._ - val df = getWKTRowsDf() val stBuffer = ST_Buffer(df.col("wkt").expr, lit(1).expr, mc.expressionConfig) From 58a03bbaaa3ca5b43c43b3e5568ecdd27f0d1fea Mon Sep 17 00:00:00 2001 From: Daniel Sparing Date: Thu, 18 Jan 2024 18:11:48 +0100 Subject: [PATCH 07/47] add too extra params to python st_bufer --- python/mosaic/api/functions.py | 23 +++++++++++++++++++++-- 1 file changed, 21 insertions(+), 2 deletions(-) diff --git a/python/mosaic/api/functions.py b/python/mosaic/api/functions.py index 9819caa5f..a5b90ef3d 100644 --- a/python/mosaic/api/functions.py +++ b/python/mosaic/api/functions.py @@ -194,7 +194,12 @@ def st_concavehull(geom: ColumnOrName, concavity: ColumnOrName, has_holes: Any = ) -def st_buffer(geom: ColumnOrName, radius: ColumnOrName) -> Column: +def st_buffer( + geom: ColumnOrName, + radius: ColumnOrName, + end_cap_style: Any = "round", + quadrant_segments: Any = 8, + ) -> Column: """ Compute the buffered geometry based on geom and radius. @@ -204,6 +209,10 @@ def st_buffer(geom: ColumnOrName, radius: ColumnOrName) -> Column: The input geometry radius : Column The radius of buffering + end_cap_style : Column + The end cap style ("round"(default)|"flat"|"square") + quadrant_segments : Column + Number of line segments used to approximate a quarter circle (default is 8) Returns ------- @@ -211,8 +220,18 @@ def st_buffer(geom: ColumnOrName, radius: ColumnOrName) -> Column: A geometry """ + + if isinstance(end_cap_style, str): + end_cap_style = lit(end_cap_style) + if isinstance(quadrant_segments, int): + quadrant_segments = lit(quadrant_segments) + return config.mosaic_context.invoke_function( - "st_buffer", pyspark_to_java_column(geom), pyspark_to_java_column(radius) + "st_buffer", + pyspark_to_java_column(geom), + pyspark_to_java_column(radius), + pyspark_to_java_column(end_cap_style), + pyspark_to_java_column(quadrant_segments), ) From b03d85a60ad3890319d5ee971e4f40422f9d7ae1 Mon Sep 17 00:00:00 2001 From: Daniel Sparing Date: Thu, 18 Jan 2024 22:33:23 +0100 Subject: [PATCH 08/47] swap column order --- python/mosaic/api/functions.py | 16 ++++++------ .../geometry/ST_BufferBehaviors.scala | 26 +++++++++---------- 2 files changed, 21 insertions(+), 21 deletions(-) diff --git a/python/mosaic/api/functions.py b/python/mosaic/api/functions.py index a5b90ef3d..c94eb5485 100644 --- a/python/mosaic/api/functions.py +++ b/python/mosaic/api/functions.py @@ -195,10 +195,10 @@ def st_concavehull(geom: ColumnOrName, concavity: ColumnOrName, has_holes: Any = def st_buffer( - geom: ColumnOrName, - radius: ColumnOrName, - end_cap_style: Any = "round", + geom: ColumnOrName, + radius: ColumnOrName, quadrant_segments: Any = 8, + end_cap_style: Any = "round", ) -> Column: """ Compute the buffered geometry based on geom and radius. @@ -209,10 +209,10 @@ def st_buffer( The input geometry radius : Column The radius of buffering - end_cap_style : Column - The end cap style ("round"(default)|"flat"|"square") quadrant_segments : Column Number of line segments used to approximate a quarter circle (default is 8) + end_cap_style : Column + The end cap style ("round"(default)|"flat"|"square") Returns ------- @@ -221,17 +221,17 @@ def st_buffer( """ - if isinstance(end_cap_style, str): - end_cap_style = lit(end_cap_style) if isinstance(quadrant_segments, int): quadrant_segments = lit(quadrant_segments) + if isinstance(end_cap_style, str): + end_cap_style = lit(end_cap_style) return config.mosaic_context.invoke_function( "st_buffer", pyspark_to_java_column(geom), pyspark_to_java_column(radius), + pyspark_to_java_column(quadrant_segments), pyspark_to_java_column(end_cap_style), - pyspark_to_java_column(quadrant_segments), ) diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala index bd66220b3..65dd5acb1 100644 --- a/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala @@ -80,7 +80,7 @@ trait ST_BufferBehaviors extends QueryTest { }) /* val extraParamResult = sourceDf - .withColumn("geomBufferedTest", st_buffer(convert_to($"geom", "COORDS"), $"buffer", $"endCapStyle", $"quadrantSegments")) + .withColumn("geomBufferedTest", st_buffer(convert_to($"geom", "COORDS"), $"buffer", $"quadrantSegments", $"endCapStyle")) .select( st_distance($"geomBufferedRef", $"geomBufferedTest"), st_area(st_intersection($"geomBufferedRef", $"geomBufferedTest")), @@ -134,80 +134,80 @@ trait ST_BufferBehaviors extends QueryTest { ( "POLYGON((30 10,40 40,20 40,10 20,30 10))", 2.0, - "round", 8, + "round", "POLYGON((29.105572809000083 8.211145618000169,9.105572809000083 18.211145618000167,8.773770799250563 18.420012042061902,8.489092090008587 18.689596517279742,8.262476724420662 19.009539063455783,8.102633403898972 19.367544467966326,8.015704817609999 19.749854783887773,8.005031582374714 20.14177804018136,8.071023864637592 20.528252845900816,8.211145618000169 20.894427190999917,18.211145618000167 40.89442719099991,18.40562298012387 41.207460938701914,18.65423405443726 41.47949789447759,18.948537775761732 41.70130161670408,19.278541681368626 41.865341227356964,19.63304118399603 41.96604710710526,20 42,40 42,40.37625388633239 41.96428944227163,40.73907153653638 41.85843301301978,41.07549652996949 41.68621090437216,41.37351494248246 41.45377326388176,41.622484370351124 41.16942056932753,41.813513976448974 40.84330721402359,41.939781986999634 40.4870788877705,41.99677930092308 40.11345670277741,41.9824705123501 39.73578291565012,41.89736659610103 39.367544467966326,31.897366596101026 9.367544467966324,31.743483999644496 9.02006962339989,31.5266535144039 8.707974827273778,31.25470373030189 8.44252815461514,30.937453300545492 8.233313466034124,30.5863564425368 8.087884385740342,30.214089387302234 8.011491580544716,29.83409270854116 8.006893186344296,29.46008605534576 8.074255226576508,29.105572809000083 8.211145618000169))" ), ( "POLYGON((30 10,40 40,20 40,10 20,30 10))", 3.0, - "round", 8, + "round", "POLYGON((28.658359213500127 7.316718427000252,8.658359213500127 17.316718427000254,8.160656198875845 17.630018063092855,7.733638135012881 18.034394775919612,7.393715086630994 18.51430859518367,7.153950105848459 19.051316701949485,7.023557226414999 19.624782175831655,7.007547373562073 20.212667060272036,7.106535796956389 20.792379268851224,7.316718427000252 21.341640786499873,17.316718427000254 41.34164078649987,17.60843447018581 41.81119140805288,17.981351081655895 42.21924684171639,18.4228066636426 42.55195242505612,18.91781252205294 42.798011841035446,19.44956177599405 42.949070660657895,20 43,40 43,40.56438082949858 42.94643416340744,41.10860730480457 42.78764951952966,41.613244794954234 42.52931635655824,42.06027241372369 42.18065989582263,42.43372655552668 41.7541308539913,42.72027096467346 41.26496082103538,42.90967298049945 40.73061833165575,42.99516895138463 40.17018505416612,42.97370576852514 39.60367437347518,42.84604989415154 39.05131670194949,32.84604989415154 9.051316701949487,32.615225999466745 8.530104435099837,32.28998027160585 8.061962240910669,31.882055595452837 7.663792231922711,31.40617995081824 7.349970199051186,30.879534663805202 7.131826578610514,30.321134080953357 7.017237370817074,29.75113906281174 7.010339779516444,29.19012908301864 7.111382839864762,28.658359213500127 7.316718427000252))" ), ( "POLYGON((30 10,40 40,20 40,10 20,30 10))", 3.0, - "round", 2, + "round", "POLYGON((28.658359213500127 7.316718427000252,8.658359213500127 17.316718427000254,7.153950105848459 19.051316701949485,7.316718427000252 21.341640786499873,17.316718427000254 41.34164078649987,20 43,40 43,42.43372655552668 41.7541308539913,42.84604989415154 39.05131670194949,32.84604989415154 9.051316701949487,31.14805029709527 7.22836140246614,28.658359213500127 7.316718427000252))" ), ( "MULTIPOLYGON(((0 0,0 1,2 2,0 0)))", 2.0, - "round", 8, + "round", "POLYGON((1.414213562373095 -1.414213562373095,1.111140466039204 -1.662939224605091,0.76536686473018 -1.847759065022573,0.390180644032257 -1.961570560806461,-3.673940397442059e-16 -2,-0.390180644032257 -1.961570560806461,-0.765366864730181 -1.847759065022573,-1.111140466039204 -1.66293922460509,-1.414213562373095 -1.414213562373095,-1.662939224605091 -1.111140466039204,-1.847759065022574 -0.765366864730179,-1.961570560806461 -0.390180644032257,-2 0,-2 1,-1.966047107105261 1.366958816003967,-1.865341227356966 1.721458318631373,-1.70130161670408 2.051462224238267,-1.479497894477594 2.345765945562737,-1.207460938701918 2.594377019876128,-0.894427190999916 2.788854381999832,1.105572809000084 3.788854381999832,1.481529468137407 3.931628408258203,1.878448906059273 3.996302915782525,2.280282892709963 3.980262987598904,2.670784493205818 3.884157149410477,3.034164963224574 3.711871148433408,3.355732118853176 3.470370845028517,3.622484370351121 3.16942056932753,3.823636401881189 2.821188330246984,3.951055242946718 2.439753839062402,3.999589100313399 2.040539239112909,3.9672756550971 1.639685558348996,3.855421403158644 1.253400095967856,3.668548830917987 0.897301129572439,3.414213562373095 0.585786437626905,1.414213562373095 -1.414213562373095))" ), ( "MULTIPOLYGON(((40 60,20 45,45 30,40 60)), ((20 35,10 30,10 10,30 5,45 20,20 35), (30 20,20 15,20 25,30 20)))", 2.0, - "round", 8, + "round", "MULTIPOLYGON(((21.028991510855054 36.71498585142509,46.02899151085505 21.714985851425087,46.351780955338434 21.474004154941333,46.619733646867054 21.173227562413512,46.82197973593542 20.824857467591027,46.95031484684309 20.443025956557175,46.999532898867834 20.043222521273073,46.9676372977026 19.641665708186324,46.85592192990094 19.25464519179602,46.66891867468572 18.897860962815827,46.41421356237309 18.585786437626904,31.414213562373096 3.585786437626905,31.094914714141236 3.326332838119534,30.72928238054107 3.137703780427949,30.332789018157783 3.02788147683929,29.92221302915125 3.001513275709299,29.514928749927336 3.059714999709336,9.514928749927334 8.059714999709335,9.15830172540112 8.185738713818413,8.83177905204805 8.376651664994155,8.547039206017194 8.625625621910517,8.314266250382676 8.923755731536227,8.141785589878605 9.260379011914079,8.035766202785066 9.623455726535903,8 10,8 30,8.033952892894739 30.36695881600397,8.134658772643034 30.721458318631374,8.298698383295921 31.051462224238268,8.520502105522406 31.345765945562736,8.792539061298083 31.594377019876127,9.105572809000083 31.788854381999833,19.105572809000083 36.78885438199983,19.480363108468328 36.93131496679317,19.87603797663762 36.99615465752629,20.27669496489004 36.9807675018549,20.66623138973048 36.88577192028564,21.028991510855054 36.71498585142509),(25.527864045000417 20,22 21.763932022500207,22 18.236067977499793,25.527864045000417 20)),((41.972787847664286 60.328797974610715,46.972787847664286 30.328797974610715,46.9993974080564 29.950908201730538,46.954274213176696 29.574779704491174,46.83903715633888 29.21390691543241,46.657820619013656 28.881236935194416,46.417126140850286 28.58870502696326,46.12558916272103 28.346806412798255,45.79366921139157 28.164219734584478,45.43327464112035 28.047495688772486,45.05733539540255 28.000822005814882,44.67933911696567 28.025873206176552,44.31284724908532 28.121750523250334,43.97100848914495 28.285014148574913,18.971008489144946 43.28501414857491,18.65706596538611 43.51793111540804,18.394426953504244 43.80746690093468,18.19312493490509 44.14256049826342,18.060850161109478 44.51041047567288,18.00265586649343 44.89696402401638,18.0207652218219 45.28745380994221,18.114486403444406 45.666962125764265,18.280239022726388 46.02099078401664,18.511690906044425 46.33601498526369,18.8 46.6,38.8 61.6,39.15068013648958 61.81070587601815,39.53689868768618 61.94564569604366,39.94249496357223 61.99917312176446,40.3504974446115 61.969048384707904,40.74383392750859 61.85653200572658,41.106045883917275 61.66633205054384,41.4219771409141 61.406407128365665,41.678408066517704 61.08763337676274,41.86460872378755 60.72334936730143,41.972787847664286 60.328797974610715)))" ), ( "POINT(-75.78033 35.18937)", 2.0, - "round", 8, + "round", "POLYGON((-73.78033 35.18937,-73.81875943919354 34.79918935596774,-73.93257093497743 34.424003135269814,-74.11739077539491 34.078229533960794,-74.36611643762691 33.775156437626904,-74.6691895339608 33.52643077539491,-75.01496313526982 33.341610934977425,-75.39014935596775 33.22779943919353,-75.78033 33.18937,-76.17051064403226 33.22779943919353,-76.54569686473019 33.341610934977425,-76.89147046603921 33.52643077539491,-77.1945435623731 33.775156437626904,-77.4432692246051 34.078229533960794,-77.62808906502258 34.424003135269814,-77.74190056080647 34.79918935596774,-77.78033 35.18937,-77.74190056080647 35.57955064403225,-77.62808906502258 35.95473686473018,-77.4432692246051 36.3005104660392,-77.1945435623731 36.60358356237309,-76.89147046603921 36.85230922460509,-76.54569686473019 37.03712906502257,-76.17051064403226 37.150940560806454,-75.78033 37.18937,-75.39014935596775 37.15094056080646,-75.01496313526982 37.03712906502257,-74.6691895339608 36.85230922460509,-74.36611643762691 36.60358356237309,-74.11739077539491 36.3005104660392,-73.93257093497743 35.95473686473018,-73.81875943919354 35.57955064403225,-73.78033 35.18937))" ), ( "MULTIPOINT(10 40,40 30,20 20,30 10)", 2.0, - "round", 8, + "round", "MULTIPOLYGON(((42 30,41.961570560806464 29.609819355967744,41.84775906502257 29.23463313526982,41.66293922460509 28.888859533960797,41.41421356237309 28.585786437626904,41.1111404660392 28.33706077539491,40.76536686473018 28.152240934977428,40.390180644032256 28.03842943919354,40 28,39.609819355967744 28.03842943919354,39.23463313526982 28.152240934977428,38.8888595339608 28.33706077539491,38.58578643762691 28.585786437626904,38.33706077539491 28.888859533960797,38.15224093497743 29.23463313526982,38.038429439193536 29.609819355967744,38 30,38.038429439193536 30.390180644032256,38.15224093497743 30.76536686473018,38.33706077539491 31.111140466039203,38.58578643762691 31.414213562373096,38.8888595339608 31.66293922460509,39.23463313526982 31.847759065022572,39.609819355967744 31.96157056080646,40 32,40.390180644032256 31.96157056080646,40.76536686473018 31.847759065022572,41.1111404660392 31.66293922460509,41.41421356237309 31.414213562373096,41.66293922460509 31.111140466039203,41.84775906502257 30.76536686473018,41.96157056080646 30.390180644032256,42 30)),((32 10,31.96157056080646 9.609819355967744,31.847759065022572 9.23463313526982,31.66293922460509 8.888859533960796,31.414213562373096 8.585786437626904,31.111140466039206 8.33706077539491,30.76536686473018 8.152240934977426,30.390180644032256 8.03842943919354,30 8,29.609819355967744 8.03842943919354,29.23463313526982 8.152240934977426,28.888859533960797 8.33706077539491,28.585786437626904 8.585786437626904,28.33706077539491 8.888859533960796,28.152240934977428 9.23463313526982,28.03842943919354 9.609819355967742,28 10,28.03842943919354 10.390180644032256,28.152240934977428 10.76536686473018,28.33706077539491 11.111140466039204,28.585786437626904 11.414213562373096,28.888859533960797 11.66293922460509,29.23463313526982 11.847759065022572,29.609819355967744 11.96157056080646,30 12,30.390180644032256 11.96157056080646,30.76536686473018 11.847759065022574,31.111140466039203 11.662939224605092,31.414213562373096 11.414213562373096,31.66293922460509 11.111140466039204,31.847759065022572 10.765366864730181,31.96157056080646 10.390180644032258,32 10)),((22 20,21.96157056080646 19.609819355967744,21.847759065022572 19.23463313526982,21.66293922460509 18.888859533960797,21.414213562373096 18.585786437626904,21.111140466039206 18.33706077539491,20.76536686473018 18.152240934977428,20.390180644032256 18.03842943919354,20 18,19.609819355967744 18.03842943919354,19.23463313526982 18.152240934977428,18.888859533960797 18.33706077539491,18.585786437626904 18.585786437626904,18.33706077539491 18.888859533960797,18.152240934977428 19.23463313526982,18.03842943919354 19.609819355967744,18 20,18.03842943919354 20.390180644032256,18.152240934977428 20.76536686473018,18.33706077539491 21.111140466039203,18.585786437626904 21.414213562373096,18.888859533960797 21.66293922460509,19.23463313526982 21.847759065022572,19.609819355967744 21.96157056080646,20 22,20.390180644032256 21.96157056080646,20.76536686473018 21.847759065022572,21.111140466039203 21.66293922460509,21.414213562373096 21.414213562373096,21.66293922460509 21.111140466039203,21.847759065022572 20.76536686473018,21.96157056080646 20.390180644032256,22 20)),((12 40,11.96157056080646 39.609819355967744,11.847759065022574 39.23463313526982,11.66293922460509 38.8888595339608,11.414213562373096 38.58578643762691,11.111140466039204 38.33706077539491,10.76536686473018 38.15224093497743,10.390180644032256 38.038429439193536,10 38,9.609819355967744 38.038429439193536,9.23463313526982 38.15224093497743,8.888859533960796 38.33706077539491,8.585786437626904 38.58578643762691,8.33706077539491 38.8888595339608,8.152240934977426 39.23463313526982,8.03842943919354 39.609819355967744,8 40,8.03842943919354 40.390180644032256,8.152240934977426 40.76536686473018,8.337060775394908 41.1111404660392,8.585786437626904 41.41421356237309,8.888859533960796 41.66293922460509,9.234633135269819 41.84775906502257,9.609819355967742 41.96157056080646,10 42,10.390180644032256 41.961570560806464,10.76536686473018 41.84775906502257,11.111140466039204 41.66293922460509,11.414213562373094 41.41421356237309,11.66293922460509 41.1111404660392,11.847759065022572 40.76536686473018,11.96157056080646 40.390180644032256,12 40)))" ), ( "LINESTRING(30 10,10 30,40 40)", 2.0, - "round", 8, + "round", "POLYGON((8.585786437626904 28.585786437626904,8.329219317019618 28.900685709462664,8.141566986980227 29.26092846346362,8.030569554367744 29.651655745250856,8.000805336911114 30.056751203308092,8.05350202106454 30.45950584109472,8.186486023551023 30.843307214023586,8.394272145281546 31.19232464395452,8.668289819689326 31.49216218812062,8.997236623400621 31.730452429438905,9.367544467966324 31.897366596101026,39.367544467966326 41.89736659610103,39.74985478388777 41.98429518239,40.14177804018136 41.99496841762529,40.528252845900816 41.928976135362404,40.89442719099991 41.78885438199983,41.22622920074944 41.5799879579381,41.51090790999141 41.31040348272026,41.737523275579335 40.99046093654422,41.89736659610103 40.632455532033674,41.98429518239 40.25014521611223,41.99496841762529 39.85822195981864,41.928976135362404 39.471747154099184,41.78885438199983 39.10557280900009,41.579987957938094 38.77377079925056,41.31040348272026 38.48909209000859,40.99046093654422 38.262476724420665,40.632455532033674 38.10263340389897,13.702459173643835 29.12596795110236,31.414213562373096 11.414213562373096,31.66293922460509 11.111140466039204,31.847759065022572 10.76536686473018,31.96157056080646 10.390180644032256,32 10,31.96157056080646 9.609819355967744,31.847759065022572 9.23463313526982,31.66293922460509 8.888859533960796,31.414213562373096 8.585786437626904,31.111140466039206 8.33706077539491,30.76536686473018 8.152240934977426,30.390180644032256 8.03842943919354,30 8,29.609819355967744 8.03842943919354,29.23463313526982 8.152240934977426,28.888859533960797 8.33706077539491,28.585786437626904 8.585786437626904,8.585786437626904 28.585786437626904))" ), ( "LINESTRING(30 10,10 30,40 40)", 2.0, - "square", 8, + "square", "POLYGON((8.585786437626904 28.585786437626904,8.329219317019618 28.900685709462664,8.141566986980227 29.26092846346362,8.030569554367744 29.651655745250856,8.000805336911114 30.056751203308092,8.05350202106454 30.45950584109472,8.186486023551023 30.843307214023586,8.394272145281546 31.19232464395452,8.668289819689326 31.49216218812062,8.997236623400621 31.730452429438905,9.367544467966324 31.897366596101026,39.367544467966326 41.89736659610103,41.264911064067356 42.529822128134704,42.529822128134704 38.735088935932644,13.702459173643835 29.12596795110236,31.414213562373096 11.414213562373096,32.82842712474619 10,30 7.171572875253809,8.585786437626904 28.585786437626904))" ), ( "LINESTRING(30 10,10 30,40 40)", 2.0, - "flat", 8, + "flat", "POLYGON((8.585786437626904 28.585786437626904,8.329219317019618 28.900685709462664,8.141566986980227 29.26092846346362,8.030569554367744 29.651655745250856,8.000805336911114 30.056751203308092,8.05350202106454 30.45950584109472,8.186486023551023 30.843307214023586,8.394272145281546 31.19232464395452,8.668289819689326 31.49216218812062,8.997236623400621 31.730452429438905,9.367544467966324 31.897366596101026,39.367544467966326 41.89736659610103,40.632455532033674 38.10263340389897,13.702459173643835 29.12596795110236,31.414213562373096 11.414213562373096,28.585786437626904 8.585786437626904,8.585786437626904 28.585786437626904))" ) - ).map({ case (f: String, b: Double, c: String, q: Int, t: String) => Row(f, b, c, q, t) }) + ).map({ case (f: String, b: Double, q: Int, c: String, t: String) => Row(f, b, q, c, t) }) val testSchema = StructType( Seq( StructField("geom", StringType), StructField("buffer", DoubleType), - StructField("endCapStyle", StringType), StructField("quadrantSegments", IntegerType), + StructField("endCapStyle", StringType), StructField("geomBufferedRef", StringType) ) ) From 7704f2d912ac56362ad070c53682a727a9df07bd Mon Sep 17 00:00:00 2001 From: Daniel Sparing Date: Sun, 21 Jan 2024 23:25:50 +0100 Subject: [PATCH 09/47] bufferstyleparameters is a string --- .../tests/testthat/testVectorFunctions.R | 1 + python/mosaic/api/functions.py | 21 ++++----- python/test/test_vector_functions.py | 4 ++ .../mosaic/core/geometry/MosaicGeometry.scala | 2 +- .../expressions/geometry/ST_Buffer.scala | 7 ++- .../labs/mosaic/functions/MosaicContext.scala | 8 +++- .../geometry/ST_BufferBehaviors.scala | 43 +++++++------------ 7 files changed, 43 insertions(+), 43 deletions(-) diff --git a/R/sparklyr-mosaic/sparklyrMosaic/tests/testthat/testVectorFunctions.R b/R/sparklyr-mosaic/sparklyrMosaic/tests/testthat/testVectorFunctions.R index f91a0ff8b..b8120ffce 100644 --- a/R/sparklyr-mosaic/sparklyrMosaic/tests/testthat/testVectorFunctions.R +++ b/R/sparklyr-mosaic/sparklyrMosaic/tests/testthat/testVectorFunctions.R @@ -14,6 +14,7 @@ test_that("scalar vector functions behave as intended", { sdf <- mutate(sdf, "st_length" = st_length(wkt)) sdf <- mutate(sdf, "st_perimeter" = st_perimeter(wkt)) sdf <- mutate(sdf, "st_buffer" = st_buffer(wkt, as.double(1.1))) + sdf <- mutate(sdf, "st_buffer_optparams" = st_buffer(wkt, as.double(1.1), "endcap=square quad_segs=2")) sdf <- mutate(sdf, "st_bufferloop" = st_bufferloop(wkt, as.double(1.1), as.double(1.2))) sdf <- mutate(sdf, "st_convexhull" = st_convexhull(wkt)) sdf <- mutate(sdf, "st_dump" = st_dump(wkt)) diff --git a/python/mosaic/api/functions.py b/python/mosaic/api/functions.py index c94eb5485..5aba65f1c 100644 --- a/python/mosaic/api/functions.py +++ b/python/mosaic/api/functions.py @@ -197,8 +197,7 @@ def st_concavehull(geom: ColumnOrName, concavity: ColumnOrName, has_holes: Any = def st_buffer( geom: ColumnOrName, radius: ColumnOrName, - quadrant_segments: Any = 8, - end_cap_style: Any = "round", + buffer_style_parameters: Any = "", ) -> Column: """ Compute the buffered geometry based on geom and radius. @@ -209,10 +208,11 @@ def st_buffer( The input geometry radius : Column The radius of buffering - quadrant_segments : Column - Number of line segments used to approximate a quarter circle (default is 8) - end_cap_style : Column - The end cap style ("round"(default)|"flat"|"square") + buffer_style_parameters : Column + "quad_segs=# endcap=round|flat|square" where "#" is the number of line segments used to + approximate a quarter circle (default is 8); and endcap style for line features is one of + listed (default="round") + Returns ------- @@ -221,17 +221,14 @@ def st_buffer( """ - if isinstance(quadrant_segments, int): - quadrant_segments = lit(quadrant_segments) - if isinstance(end_cap_style, str): - end_cap_style = lit(end_cap_style) + if isinstance(buffer_style_parameters, str): + buffer_style_parameters = lit(buffer_style_parameters) return config.mosaic_context.invoke_function( "st_buffer", pyspark_to_java_column(geom), pyspark_to_java_column(radius), - pyspark_to_java_column(quadrant_segments), - pyspark_to_java_column(end_cap_style), + pyspark_to_java_column(buffer_style_parameters), ) diff --git a/python/test/test_vector_functions.py b/python/test/test_vector_functions.py index 3dfa1640f..b4a1ab8b9 100644 --- a/python/test/test_vector_functions.py +++ b/python/test/test_vector_functions.py @@ -42,6 +42,10 @@ def test_st_bindings_happy_flow(self): df.withColumn("st_area", api.st_area("wkt")) .withColumn("st_length", api.st_length("wkt")) .withColumn("st_buffer", api.st_buffer("wkt", lit(1.1))) + .withColumn( + "st_buffer_optparams", + api.st_buffer("wkt", lit(1.1), lit("endcap=square quad_segs=2"))), + ) .withColumn("st_buffer", api.st_bufferloop("wkt", lit(1.1), lit(1.2))) .withColumn("st_perimeter", api.st_perimeter("wkt")) .withColumn("st_convexhull", api.st_convexhull("wkt")) diff --git a/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometry.scala b/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometry.scala index 0093aa7c5..c6bacdf0e 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometry.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometry.scala @@ -55,7 +55,7 @@ trait MosaicGeometry extends GeometryWriter with Serializable { def boundary: MosaicGeometry - def buffer(distance: Double): MosaicGeometry + def buffer(distance: Double, bufferStyleParameters: String = ""): MosaicGeometry def bufferCapStyle(distance: Double, capStyle: String): MosaicGeometry diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/geometry/ST_Buffer.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/geometry/ST_Buffer.scala index c0c3c085b..e279ebc2d 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/geometry/ST_Buffer.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/geometry/ST_Buffer.scala @@ -16,6 +16,10 @@ import org.apache.spark.sql.types.DataType * Expression containing the geometry. * @param radiusExpr * The radius of the buffer. + * @param bufferStyleParameters + * 'quad_segs=# endcap=round|flat|square' where "#" is the number of line segments used to + * approximate a quarter circle (default is 8); and endcap style for line features is one of + * listed (default="round") * @param expressionConfig * Mosaic execution context, e.g. geometryAPI, indexSystem, etc. Additional * arguments for the expression (expressionConfigs). @@ -23,8 +27,9 @@ import org.apache.spark.sql.types.DataType case class ST_Buffer( inputGeom: Expression, radiusExpr: Expression, + bufferStyleParametersExpr: Expression, expressionConfig: MosaicExpressionConfig -) extends UnaryVector1ArgExpression[ST_Buffer](inputGeom, radiusExpr, returnsGeometry = true, expressionConfig) { +) extends UnaryVector2ArgExpression[ST_Buffer](inputGeom, radiusExpr, bufferStyleParametersExpr, returnsGeometry = true, expressionConfig) { override def dataType: DataType = inputGeom.dataType diff --git a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala index cd516ecc5..015450b27 100644 --- a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala +++ b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala @@ -548,9 +548,13 @@ class MosaicContext(indexSystem: IndexSystem, geometryAPI: GeometryAPI) extends def flatten_polygons(geom: Column): Column = ColumnAdapter(FlattenPolygons(geom.expr, geometryAPI.name)) def st_area(geom: Column): Column = ColumnAdapter(ST_Area(geom.expr, expressionConfig)) def st_buffer(geom: Column, radius: Column): Column = - ColumnAdapter(ST_Buffer(geom.expr, radius.cast("double").expr, expressionConfig)) + st_buffer(geom, radius, lit("")) def st_buffer(geom: Column, radius: Double): Column = - ColumnAdapter(ST_Buffer(geom.expr, lit(radius).cast("double").expr, expressionConfig)) + st_buffer(geom, lit(radius), lit("")) + def st_buffer(geom: Column, radius: Column, buffer_style_parameters: Column): Column = + ColumnAdapter(ST_Buffer(geom.expr, radius.cast("double").expr, buffer_style_parameters.cast("string").expr, expressionConfig)) + def st_buffer(geom: Column, radius: Double, buffer_style_parameters: Column): Column = + ColumnAdapter(ST_Buffer(geom.expr, lit(radius).cast("double").expr, lit(buffer_style_parameters).cast("string").expr, expressionConfig)) def st_bufferloop(geom: Column, r1: Column, r2: Column): Column = ColumnAdapter(ST_BufferLoop(geom.expr, r1.cast("double").expr, r2.cast("double").expr, expressionConfig)) def st_bufferloop(geom: Column, r1: Double, r2: Double): Column = diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala index 65dd5acb1..b0b9c75b7 100644 --- a/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala @@ -62,7 +62,7 @@ trait ST_BufferBehaviors extends QueryTest { val sourceDf = testData(spark) val noExtraParamResult = sourceDf - .where($"endCapStyle" === "round" and $"quadrantSegments" === 8) + .where($"bufferStyleParameters" === "") .withColumn("geomBufferedTest", st_buffer(convert_to($"geom", "COORDS"), $"buffer")) .select( st_distance($"geomBufferedRef", $"geomBufferedTest"), @@ -79,8 +79,8 @@ trait ST_BufferBehaviors extends QueryTest { a1 shouldBe a3 +- 1e-9 }) - /* val extraParamResult = sourceDf - .withColumn("geomBufferedTest", st_buffer(convert_to($"geom", "COORDS"), $"buffer", $"quadrantSegments", $"endCapStyle")) + val extraParamResult = sourceDf + .withColumn("geomBufferedTest", st_buffer(convert_to($"geom", "COORDS"), $"buffer", $"bufferStyleParameters")) .select( st_distance($"geomBufferedRef", $"geomBufferedTest"), st_area(st_intersection($"geomBufferedRef", $"geomBufferedTest")), @@ -95,7 +95,6 @@ trait ST_BufferBehaviors extends QueryTest { a1 shouldBe a2 +- 1e-9 a1 shouldBe a3 +- 1e-9 }) - */ } @@ -134,80 +133,69 @@ trait ST_BufferBehaviors extends QueryTest { ( "POLYGON((30 10,40 40,20 40,10 20,30 10))", 2.0, - 8, - "round", + "", "POLYGON((29.105572809000083 8.211145618000169,9.105572809000083 18.211145618000167,8.773770799250563 18.420012042061902,8.489092090008587 18.689596517279742,8.262476724420662 19.009539063455783,8.102633403898972 19.367544467966326,8.015704817609999 19.749854783887773,8.005031582374714 20.14177804018136,8.071023864637592 20.528252845900816,8.211145618000169 20.894427190999917,18.211145618000167 40.89442719099991,18.40562298012387 41.207460938701914,18.65423405443726 41.47949789447759,18.948537775761732 41.70130161670408,19.278541681368626 41.865341227356964,19.63304118399603 41.96604710710526,20 42,40 42,40.37625388633239 41.96428944227163,40.73907153653638 41.85843301301978,41.07549652996949 41.68621090437216,41.37351494248246 41.45377326388176,41.622484370351124 41.16942056932753,41.813513976448974 40.84330721402359,41.939781986999634 40.4870788877705,41.99677930092308 40.11345670277741,41.9824705123501 39.73578291565012,41.89736659610103 39.367544467966326,31.897366596101026 9.367544467966324,31.743483999644496 9.02006962339989,31.5266535144039 8.707974827273778,31.25470373030189 8.44252815461514,30.937453300545492 8.233313466034124,30.5863564425368 8.087884385740342,30.214089387302234 8.011491580544716,29.83409270854116 8.006893186344296,29.46008605534576 8.074255226576508,29.105572809000083 8.211145618000169))" ), ( "POLYGON((30 10,40 40,20 40,10 20,30 10))", 3.0, - 8, - "round", + "", "POLYGON((28.658359213500127 7.316718427000252,8.658359213500127 17.316718427000254,8.160656198875845 17.630018063092855,7.733638135012881 18.034394775919612,7.393715086630994 18.51430859518367,7.153950105848459 19.051316701949485,7.023557226414999 19.624782175831655,7.007547373562073 20.212667060272036,7.106535796956389 20.792379268851224,7.316718427000252 21.341640786499873,17.316718427000254 41.34164078649987,17.60843447018581 41.81119140805288,17.981351081655895 42.21924684171639,18.4228066636426 42.55195242505612,18.91781252205294 42.798011841035446,19.44956177599405 42.949070660657895,20 43,40 43,40.56438082949858 42.94643416340744,41.10860730480457 42.78764951952966,41.613244794954234 42.52931635655824,42.06027241372369 42.18065989582263,42.43372655552668 41.7541308539913,42.72027096467346 41.26496082103538,42.90967298049945 40.73061833165575,42.99516895138463 40.17018505416612,42.97370576852514 39.60367437347518,42.84604989415154 39.05131670194949,32.84604989415154 9.051316701949487,32.615225999466745 8.530104435099837,32.28998027160585 8.061962240910669,31.882055595452837 7.663792231922711,31.40617995081824 7.349970199051186,30.879534663805202 7.131826578610514,30.321134080953357 7.017237370817074,29.75113906281174 7.010339779516444,29.19012908301864 7.111382839864762,28.658359213500127 7.316718427000252))" ), ( "POLYGON((30 10,40 40,20 40,10 20,30 10))", 3.0, - 2, - "round", + "quad_segs=2", "POLYGON((28.658359213500127 7.316718427000252,8.658359213500127 17.316718427000254,7.153950105848459 19.051316701949485,7.316718427000252 21.341640786499873,17.316718427000254 41.34164078649987,20 43,40 43,42.43372655552668 41.7541308539913,42.84604989415154 39.05131670194949,32.84604989415154 9.051316701949487,31.14805029709527 7.22836140246614,28.658359213500127 7.316718427000252))" ), ( "MULTIPOLYGON(((0 0,0 1,2 2,0 0)))", 2.0, - 8, - "round", + "", "POLYGON((1.414213562373095 -1.414213562373095,1.111140466039204 -1.662939224605091,0.76536686473018 -1.847759065022573,0.390180644032257 -1.961570560806461,-3.673940397442059e-16 -2,-0.390180644032257 -1.961570560806461,-0.765366864730181 -1.847759065022573,-1.111140466039204 -1.66293922460509,-1.414213562373095 -1.414213562373095,-1.662939224605091 -1.111140466039204,-1.847759065022574 -0.765366864730179,-1.961570560806461 -0.390180644032257,-2 0,-2 1,-1.966047107105261 1.366958816003967,-1.865341227356966 1.721458318631373,-1.70130161670408 2.051462224238267,-1.479497894477594 2.345765945562737,-1.207460938701918 2.594377019876128,-0.894427190999916 2.788854381999832,1.105572809000084 3.788854381999832,1.481529468137407 3.931628408258203,1.878448906059273 3.996302915782525,2.280282892709963 3.980262987598904,2.670784493205818 3.884157149410477,3.034164963224574 3.711871148433408,3.355732118853176 3.470370845028517,3.622484370351121 3.16942056932753,3.823636401881189 2.821188330246984,3.951055242946718 2.439753839062402,3.999589100313399 2.040539239112909,3.9672756550971 1.639685558348996,3.855421403158644 1.253400095967856,3.668548830917987 0.897301129572439,3.414213562373095 0.585786437626905,1.414213562373095 -1.414213562373095))" ), ( "MULTIPOLYGON(((40 60,20 45,45 30,40 60)), ((20 35,10 30,10 10,30 5,45 20,20 35), (30 20,20 15,20 25,30 20)))", 2.0, - 8, - "round", + "", "MULTIPOLYGON(((21.028991510855054 36.71498585142509,46.02899151085505 21.714985851425087,46.351780955338434 21.474004154941333,46.619733646867054 21.173227562413512,46.82197973593542 20.824857467591027,46.95031484684309 20.443025956557175,46.999532898867834 20.043222521273073,46.9676372977026 19.641665708186324,46.85592192990094 19.25464519179602,46.66891867468572 18.897860962815827,46.41421356237309 18.585786437626904,31.414213562373096 3.585786437626905,31.094914714141236 3.326332838119534,30.72928238054107 3.137703780427949,30.332789018157783 3.02788147683929,29.92221302915125 3.001513275709299,29.514928749927336 3.059714999709336,9.514928749927334 8.059714999709335,9.15830172540112 8.185738713818413,8.83177905204805 8.376651664994155,8.547039206017194 8.625625621910517,8.314266250382676 8.923755731536227,8.141785589878605 9.260379011914079,8.035766202785066 9.623455726535903,8 10,8 30,8.033952892894739 30.36695881600397,8.134658772643034 30.721458318631374,8.298698383295921 31.051462224238268,8.520502105522406 31.345765945562736,8.792539061298083 31.594377019876127,9.105572809000083 31.788854381999833,19.105572809000083 36.78885438199983,19.480363108468328 36.93131496679317,19.87603797663762 36.99615465752629,20.27669496489004 36.9807675018549,20.66623138973048 36.88577192028564,21.028991510855054 36.71498585142509),(25.527864045000417 20,22 21.763932022500207,22 18.236067977499793,25.527864045000417 20)),((41.972787847664286 60.328797974610715,46.972787847664286 30.328797974610715,46.9993974080564 29.950908201730538,46.954274213176696 29.574779704491174,46.83903715633888 29.21390691543241,46.657820619013656 28.881236935194416,46.417126140850286 28.58870502696326,46.12558916272103 28.346806412798255,45.79366921139157 28.164219734584478,45.43327464112035 28.047495688772486,45.05733539540255 28.000822005814882,44.67933911696567 28.025873206176552,44.31284724908532 28.121750523250334,43.97100848914495 28.285014148574913,18.971008489144946 43.28501414857491,18.65706596538611 43.51793111540804,18.394426953504244 43.80746690093468,18.19312493490509 44.14256049826342,18.060850161109478 44.51041047567288,18.00265586649343 44.89696402401638,18.0207652218219 45.28745380994221,18.114486403444406 45.666962125764265,18.280239022726388 46.02099078401664,18.511690906044425 46.33601498526369,18.8 46.6,38.8 61.6,39.15068013648958 61.81070587601815,39.53689868768618 61.94564569604366,39.94249496357223 61.99917312176446,40.3504974446115 61.969048384707904,40.74383392750859 61.85653200572658,41.106045883917275 61.66633205054384,41.4219771409141 61.406407128365665,41.678408066517704 61.08763337676274,41.86460872378755 60.72334936730143,41.972787847664286 60.328797974610715)))" ), ( "POINT(-75.78033 35.18937)", 2.0, - 8, - "round", + "", "POLYGON((-73.78033 35.18937,-73.81875943919354 34.79918935596774,-73.93257093497743 34.424003135269814,-74.11739077539491 34.078229533960794,-74.36611643762691 33.775156437626904,-74.6691895339608 33.52643077539491,-75.01496313526982 33.341610934977425,-75.39014935596775 33.22779943919353,-75.78033 33.18937,-76.17051064403226 33.22779943919353,-76.54569686473019 33.341610934977425,-76.89147046603921 33.52643077539491,-77.1945435623731 33.775156437626904,-77.4432692246051 34.078229533960794,-77.62808906502258 34.424003135269814,-77.74190056080647 34.79918935596774,-77.78033 35.18937,-77.74190056080647 35.57955064403225,-77.62808906502258 35.95473686473018,-77.4432692246051 36.3005104660392,-77.1945435623731 36.60358356237309,-76.89147046603921 36.85230922460509,-76.54569686473019 37.03712906502257,-76.17051064403226 37.150940560806454,-75.78033 37.18937,-75.39014935596775 37.15094056080646,-75.01496313526982 37.03712906502257,-74.6691895339608 36.85230922460509,-74.36611643762691 36.60358356237309,-74.11739077539491 36.3005104660392,-73.93257093497743 35.95473686473018,-73.81875943919354 35.57955064403225,-73.78033 35.18937))" ), ( "MULTIPOINT(10 40,40 30,20 20,30 10)", 2.0, - 8, - "round", + "", "MULTIPOLYGON(((42 30,41.961570560806464 29.609819355967744,41.84775906502257 29.23463313526982,41.66293922460509 28.888859533960797,41.41421356237309 28.585786437626904,41.1111404660392 28.33706077539491,40.76536686473018 28.152240934977428,40.390180644032256 28.03842943919354,40 28,39.609819355967744 28.03842943919354,39.23463313526982 28.152240934977428,38.8888595339608 28.33706077539491,38.58578643762691 28.585786437626904,38.33706077539491 28.888859533960797,38.15224093497743 29.23463313526982,38.038429439193536 29.609819355967744,38 30,38.038429439193536 30.390180644032256,38.15224093497743 30.76536686473018,38.33706077539491 31.111140466039203,38.58578643762691 31.414213562373096,38.8888595339608 31.66293922460509,39.23463313526982 31.847759065022572,39.609819355967744 31.96157056080646,40 32,40.390180644032256 31.96157056080646,40.76536686473018 31.847759065022572,41.1111404660392 31.66293922460509,41.41421356237309 31.414213562373096,41.66293922460509 31.111140466039203,41.84775906502257 30.76536686473018,41.96157056080646 30.390180644032256,42 30)),((32 10,31.96157056080646 9.609819355967744,31.847759065022572 9.23463313526982,31.66293922460509 8.888859533960796,31.414213562373096 8.585786437626904,31.111140466039206 8.33706077539491,30.76536686473018 8.152240934977426,30.390180644032256 8.03842943919354,30 8,29.609819355967744 8.03842943919354,29.23463313526982 8.152240934977426,28.888859533960797 8.33706077539491,28.585786437626904 8.585786437626904,28.33706077539491 8.888859533960796,28.152240934977428 9.23463313526982,28.03842943919354 9.609819355967742,28 10,28.03842943919354 10.390180644032256,28.152240934977428 10.76536686473018,28.33706077539491 11.111140466039204,28.585786437626904 11.414213562373096,28.888859533960797 11.66293922460509,29.23463313526982 11.847759065022572,29.609819355967744 11.96157056080646,30 12,30.390180644032256 11.96157056080646,30.76536686473018 11.847759065022574,31.111140466039203 11.662939224605092,31.414213562373096 11.414213562373096,31.66293922460509 11.111140466039204,31.847759065022572 10.765366864730181,31.96157056080646 10.390180644032258,32 10)),((22 20,21.96157056080646 19.609819355967744,21.847759065022572 19.23463313526982,21.66293922460509 18.888859533960797,21.414213562373096 18.585786437626904,21.111140466039206 18.33706077539491,20.76536686473018 18.152240934977428,20.390180644032256 18.03842943919354,20 18,19.609819355967744 18.03842943919354,19.23463313526982 18.152240934977428,18.888859533960797 18.33706077539491,18.585786437626904 18.585786437626904,18.33706077539491 18.888859533960797,18.152240934977428 19.23463313526982,18.03842943919354 19.609819355967744,18 20,18.03842943919354 20.390180644032256,18.152240934977428 20.76536686473018,18.33706077539491 21.111140466039203,18.585786437626904 21.414213562373096,18.888859533960797 21.66293922460509,19.23463313526982 21.847759065022572,19.609819355967744 21.96157056080646,20 22,20.390180644032256 21.96157056080646,20.76536686473018 21.847759065022572,21.111140466039203 21.66293922460509,21.414213562373096 21.414213562373096,21.66293922460509 21.111140466039203,21.847759065022572 20.76536686473018,21.96157056080646 20.390180644032256,22 20)),((12 40,11.96157056080646 39.609819355967744,11.847759065022574 39.23463313526982,11.66293922460509 38.8888595339608,11.414213562373096 38.58578643762691,11.111140466039204 38.33706077539491,10.76536686473018 38.15224093497743,10.390180644032256 38.038429439193536,10 38,9.609819355967744 38.038429439193536,9.23463313526982 38.15224093497743,8.888859533960796 38.33706077539491,8.585786437626904 38.58578643762691,8.33706077539491 38.8888595339608,8.152240934977426 39.23463313526982,8.03842943919354 39.609819355967744,8 40,8.03842943919354 40.390180644032256,8.152240934977426 40.76536686473018,8.337060775394908 41.1111404660392,8.585786437626904 41.41421356237309,8.888859533960796 41.66293922460509,9.234633135269819 41.84775906502257,9.609819355967742 41.96157056080646,10 42,10.390180644032256 41.961570560806464,10.76536686473018 41.84775906502257,11.111140466039204 41.66293922460509,11.414213562373094 41.41421356237309,11.66293922460509 41.1111404660392,11.847759065022572 40.76536686473018,11.96157056080646 40.390180644032256,12 40)))" ), ( "LINESTRING(30 10,10 30,40 40)", 2.0, - 8, - "round", + "", "POLYGON((8.585786437626904 28.585786437626904,8.329219317019618 28.900685709462664,8.141566986980227 29.26092846346362,8.030569554367744 29.651655745250856,8.000805336911114 30.056751203308092,8.05350202106454 30.45950584109472,8.186486023551023 30.843307214023586,8.394272145281546 31.19232464395452,8.668289819689326 31.49216218812062,8.997236623400621 31.730452429438905,9.367544467966324 31.897366596101026,39.367544467966326 41.89736659610103,39.74985478388777 41.98429518239,40.14177804018136 41.99496841762529,40.528252845900816 41.928976135362404,40.89442719099991 41.78885438199983,41.22622920074944 41.5799879579381,41.51090790999141 41.31040348272026,41.737523275579335 40.99046093654422,41.89736659610103 40.632455532033674,41.98429518239 40.25014521611223,41.99496841762529 39.85822195981864,41.928976135362404 39.471747154099184,41.78885438199983 39.10557280900009,41.579987957938094 38.77377079925056,41.31040348272026 38.48909209000859,40.99046093654422 38.262476724420665,40.632455532033674 38.10263340389897,13.702459173643835 29.12596795110236,31.414213562373096 11.414213562373096,31.66293922460509 11.111140466039204,31.847759065022572 10.76536686473018,31.96157056080646 10.390180644032256,32 10,31.96157056080646 9.609819355967744,31.847759065022572 9.23463313526982,31.66293922460509 8.888859533960796,31.414213562373096 8.585786437626904,31.111140466039206 8.33706077539491,30.76536686473018 8.152240934977426,30.390180644032256 8.03842943919354,30 8,29.609819355967744 8.03842943919354,29.23463313526982 8.152240934977426,28.888859533960797 8.33706077539491,28.585786437626904 8.585786437626904,8.585786437626904 28.585786437626904))" ), ( "LINESTRING(30 10,10 30,40 40)", 2.0, - 8, - "square", + "endcap=square", "POLYGON((8.585786437626904 28.585786437626904,8.329219317019618 28.900685709462664,8.141566986980227 29.26092846346362,8.030569554367744 29.651655745250856,8.000805336911114 30.056751203308092,8.05350202106454 30.45950584109472,8.186486023551023 30.843307214023586,8.394272145281546 31.19232464395452,8.668289819689326 31.49216218812062,8.997236623400621 31.730452429438905,9.367544467966324 31.897366596101026,39.367544467966326 41.89736659610103,41.264911064067356 42.529822128134704,42.529822128134704 38.735088935932644,13.702459173643835 29.12596795110236,31.414213562373096 11.414213562373096,32.82842712474619 10,30 7.171572875253809,8.585786437626904 28.585786437626904))" ), ( "LINESTRING(30 10,10 30,40 40)", 2.0, - 8, - "flat", + "quad_segs=8 endcap=flat", "POLYGON((8.585786437626904 28.585786437626904,8.329219317019618 28.900685709462664,8.141566986980227 29.26092846346362,8.030569554367744 29.651655745250856,8.000805336911114 30.056751203308092,8.05350202106454 30.45950584109472,8.186486023551023 30.843307214023586,8.394272145281546 31.19232464395452,8.668289819689326 31.49216218812062,8.997236623400621 31.730452429438905,9.367544467966324 31.897366596101026,39.367544467966326 41.89736659610103,40.632455532033674 38.10263340389897,13.702459173643835 29.12596795110236,31.414213562373096 11.414213562373096,28.585786437626904 8.585786437626904,8.585786437626904 28.585786437626904))" ) - ).map({ case (f: String, b: Double, q: Int, c: String, t: String) => Row(f, b, q, c, t) }) + ).map({ case (f: String, b: Double, s: String, t: String) => Row(f, b, s, t) }) val testSchema = StructType( Seq( StructField("geom", StringType), StructField("buffer", DoubleType), - StructField("quadrantSegments", IntegerType), - StructField("endCapStyle", StringType), + StructField("bufferStyleParameters", IntegerType), StructField("geomBufferedRef", StringType) ) ) @@ -223,6 +211,7 @@ trait ST_BufferBehaviors extends QueryTest { mc.register(spark) import mc.functions._ + val df = getWKTRowsDf() val stBuffer = ST_Buffer(df.col("wkt").expr, lit(1).expr, mc.expressionConfig) From 990dcc9cca09669efb9a086a3d8f2c3b7c58d99d Mon Sep 17 00:00:00 2001 From: Daniel Sparing Date: Sun, 21 Jan 2024 23:38:04 +0100 Subject: [PATCH 10/47] buffer string args for JTS --- .../core/geometry/MosaicGeometryJTS.scala | 26 +++++++++++++++++-- .../expressions/geometry/ST_Buffer.scala | 13 +++++----- 2 files changed, 31 insertions(+), 8 deletions(-) diff --git a/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometryJTS.scala b/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometryJTS.scala index d509bc9ec..31c4ecdb4 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometryJTS.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometryJTS.scala @@ -61,8 +61,30 @@ abstract class MosaicGeometryJTS(geom: Geometry) extends MosaicGeometry { override def envelope: MosaicGeometryJTS = MosaicGeometryJTS(geom.getEnvelope) - override def buffer(distance: Double): MosaicGeometryJTS = { - val buffered = geom.buffer(distance) + override def buffer(distance: Double, bufferStyleParameters: String): MosaicGeometryJTS = { + + val params = buffer_style_parameters.split(" ") + .map(_.split("=")) + .map { case Array(k, v) => (k, v) } + .toMap + + val gBuf = new BufferOp(geom) + + if(params.contains("endcap")) { + val capStyle = params.get("endcap") + val capStyleConst = capStyle match { + case "round" => BufferParameters.CAP_ROUND + case "flat" => BufferParameters.CAP_FLAT + case "square" => BufferParameters.CAP_SQUARE + case _ => BufferParameters.CAP_ROUND + } + gBuf.setEndCapStyle(capStyleConst) + } + if(params.contains("quad_segs")) { + val quadSegs = params.get("quad_segs") +] gBuf.setQuadrantSegments(quadSegs) + } + val buffered = gBuf.getResultGeometry(distance) buffered.setSRID(geom.getSRID) MosaicGeometryJTS(buffered) } diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/geometry/ST_Buffer.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/geometry/ST_Buffer.scala index e279ebc2d..cd378734d 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/geometry/ST_Buffer.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/geometry/ST_Buffer.scala @@ -16,7 +16,7 @@ import org.apache.spark.sql.types.DataType * Expression containing the geometry. * @param radiusExpr * The radius of the buffer. - * @param bufferStyleParameters + * @param bufferStyleParametersExpr * 'quad_segs=# endcap=round|flat|square' where "#" is the number of line segments used to * approximate a quarter circle (default is 8); and endcap style for line features is one of * listed (default="round") @@ -33,14 +33,15 @@ case class ST_Buffer( override def dataType: DataType = inputGeom.dataType - override def geometryTransform(geometry: MosaicGeometry, arg: Any): Any = { + override def geometryTransform(geometry: MosaicGeometry, arg1: Any, arg2: Any): Any = { val radius = arg.asInstanceOf[Double] - geometry.buffer(radius) + val bufferStyleParameters = arg2.asInstanceOf[UTF8String].toString + geometry.buffer(radius, bufferStyleParameters) } - override def geometryCodeGen(geometryRef: String, argRef: String, ctx: CodegenContext): (String, String) = { + override def geometryCodeGen(geometryRef: String, argRef1: String, argRef2: String, ctx: CodegenContext): (String, String) = { val resultRef = ctx.freshName("result") - val code = s"""$mosaicGeomClass $resultRef = $geometryRef.buffer($argRef);""" + val code = s"""$mosaicGeomClass $resultRef = $geometryRef.buffer($argRef1, $argRef2.toString());""" (code, resultRef) } @@ -61,7 +62,7 @@ object ST_Buffer extends WithExpressionInfo { | """.stripMargin override def builder(expressionConfig: MosaicExpressionConfig): FunctionBuilder = { (children: Seq[Expression]) => - ST_Buffer(children.head, Column(children(1)).cast("double").expr, expressionConfig) + ST_Buffer(children.head, Column(children(1)).cast("double").expr, children(2), expressionConfig) } } From 5216ea099c311a1672120e008b166c30c955a4ab Mon Sep 17 00:00:00 2001 From: Daniel Sparing Date: Mon, 22 Jan 2024 00:08:52 +0100 Subject: [PATCH 11/47] add extra params to doc --- docs/source/api/spatial-functions.rst | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docs/source/api/spatial-functions.rst b/docs/source/api/spatial-functions.rst index e2ccc9464..1de5894e4 100644 --- a/docs/source/api/spatial-functions.rst +++ b/docs/source/api/spatial-functions.rst @@ -125,11 +125,17 @@ st_buffer .. function:: st_buffer(col, radius) Buffer the input geometry by radius `radius` and return a new, buffered geometry. + The optional parameter buffer_style_parameters='quad_segs=# endcap=round|flat|square' where "#" + is the number of line segments used to approximate a quarter circle (default is 8); and endcap + style for line features is one of listed (default="round") + :param col: Geometry :type col: Column :param radius: Double :type radius: Column (DoubleType) + :param buffer_style_parameters: String + :type buffer_style_parameters: Column (StringType) :rtype: Column: Geometry :example: From 1e5a55ae9f04093e33628e236098013f944d90e5 Mon Sep 17 00:00:00 2001 From: Daniel Sparing Date: Mon, 22 Jan 2024 00:13:32 +0100 Subject: [PATCH 12/47] minor debug --- .../labs/mosaic/core/geometry/MosaicGeometryJTS.scala | 8 ++++---- .../labs/mosaic/expressions/geometry/ST_Buffer.scala | 9 ++++++--- .../databricks/labs/mosaic/functions/MosaicContext.scala | 6 ++---- 3 files changed, 12 insertions(+), 11 deletions(-) diff --git a/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometryJTS.scala b/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometryJTS.scala index 31c4ecdb4..0d4a890e1 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometryJTS.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometryJTS.scala @@ -63,7 +63,7 @@ abstract class MosaicGeometryJTS(geom: Geometry) extends MosaicGeometry { override def buffer(distance: Double, bufferStyleParameters: String): MosaicGeometryJTS = { - val params = buffer_style_parameters.split(" ") + val params = bufferStyleParameters.split(" ") .map(_.split("=")) .map { case Array(k, v) => (k, v) } .toMap @@ -71,7 +71,7 @@ abstract class MosaicGeometryJTS(geom: Geometry) extends MosaicGeometry { val gBuf = new BufferOp(geom) if(params.contains("endcap")) { - val capStyle = params.get("endcap") + val capStyle = params.getOrElse("endcap", "") val capStyleConst = capStyle match { case "round" => BufferParameters.CAP_ROUND case "flat" => BufferParameters.CAP_FLAT @@ -81,8 +81,8 @@ abstract class MosaicGeometryJTS(geom: Geometry) extends MosaicGeometry { gBuf.setEndCapStyle(capStyleConst) } if(params.contains("quad_segs")) { - val quadSegs = params.get("quad_segs") -] gBuf.setQuadrantSegments(quadSegs) + val quadSegs = params.getOrElse("quad_segs", "") + gBuf.setQuadrantSegments(quadSegs.toInt) } val buffered = gBuf.getResultGeometry(distance) buffered.setSRID(geom.getSRID) diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/geometry/ST_Buffer.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/geometry/ST_Buffer.scala index cd378734d..7d21aecb9 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/geometry/ST_Buffer.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/geometry/ST_Buffer.scala @@ -2,13 +2,16 @@ package com.databricks.labs.mosaic.expressions.geometry import com.databricks.labs.mosaic.core.geometry.MosaicGeometry import com.databricks.labs.mosaic.expressions.base.WithExpressionInfo -import com.databricks.labs.mosaic.expressions.geometry.base.UnaryVector1ArgExpression +import com.databricks.labs.mosaic.expressions.geometry.base.UnaryVector2ArgExpression import com.databricks.labs.mosaic.functions.MosaicExpressionConfig import org.apache.spark.sql.adapters.Column import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.types.DataType +import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.sql.functions._ + /** * SQL expression that returns the input geometry buffered by the radius. @@ -27,14 +30,14 @@ import org.apache.spark.sql.types.DataType case class ST_Buffer( inputGeom: Expression, radiusExpr: Expression, - bufferStyleParametersExpr: Expression, + bufferStyleParametersExpr: Expression = lit("").expr, expressionConfig: MosaicExpressionConfig ) extends UnaryVector2ArgExpression[ST_Buffer](inputGeom, radiusExpr, bufferStyleParametersExpr, returnsGeometry = true, expressionConfig) { override def dataType: DataType = inputGeom.dataType override def geometryTransform(geometry: MosaicGeometry, arg1: Any, arg2: Any): Any = { - val radius = arg.asInstanceOf[Double] + val radius = arg1.asInstanceOf[Double] val bufferStyleParameters = arg2.asInstanceOf[UTF8String].toString geometry.buffer(radius, bufferStyleParameters) } diff --git a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala index 015450b27..134700e08 100644 --- a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala +++ b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala @@ -547,10 +547,8 @@ class MosaicContext(indexSystem: IndexSystem, geometryAPI: GeometryAPI) extends /** Spatial functions */ def flatten_polygons(geom: Column): Column = ColumnAdapter(FlattenPolygons(geom.expr, geometryAPI.name)) def st_area(geom: Column): Column = ColumnAdapter(ST_Area(geom.expr, expressionConfig)) - def st_buffer(geom: Column, radius: Column): Column = - st_buffer(geom, radius, lit("")) - def st_buffer(geom: Column, radius: Double): Column = - st_buffer(geom, lit(radius), lit("")) + def st_buffer(geom: Column, radius: Column): Column = st_buffer(geom, radius, lit("")) + def st_buffer(geom: Column, radius: Double): Column = st_buffer(geom, lit(radius), lit("")) def st_buffer(geom: Column, radius: Column, buffer_style_parameters: Column): Column = ColumnAdapter(ST_Buffer(geom.expr, radius.cast("double").expr, buffer_style_parameters.cast("string").expr, expressionConfig)) def st_buffer(geom: Column, radius: Double, buffer_style_parameters: Column): Column = From 8ff5f02df50e89ed9b9a60a33125f34782d873cf Mon Sep 17 00:00:00 2001 From: Daniel Sparing Date: Mon, 22 Jan 2024 22:17:45 +0100 Subject: [PATCH 13/47] minor debug --- python/test/test_vector_functions.py | 3 +-- .../expressions/geometry/ST_BufferBehaviors.scala | 11 ++++++----- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/python/test/test_vector_functions.py b/python/test/test_vector_functions.py index b4a1ab8b9..2c4d3d329 100644 --- a/python/test/test_vector_functions.py +++ b/python/test/test_vector_functions.py @@ -44,8 +44,7 @@ def test_st_bindings_happy_flow(self): .withColumn("st_buffer", api.st_buffer("wkt", lit(1.1))) .withColumn( "st_buffer_optparams", - api.st_buffer("wkt", lit(1.1), lit("endcap=square quad_segs=2"))), - ) + api.st_buffer("wkt", lit(1.1), lit("endcap=square quad_segs=2"))) .withColumn("st_buffer", api.st_bufferloop("wkt", lit(1.1), lit(1.2))) .withColumn("st_perimeter", api.st_perimeter("wkt")) .withColumn("st_convexhull", api.st_convexhull("wkt")) diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala index b0b9c75b7..068be8d52 100644 --- a/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala @@ -122,7 +122,7 @@ trait ST_BufferBehaviors extends QueryTest { noException should be thrownBy CodeGenerator.compile(code) - val stBuffer = ST_Buffer(lit(1).expr, lit(1).expr, mc.expressionConfig) + val stBuffer = ST_Buffer(lit(1).expr, lit(1).expr, lit("").expr, mc.expressionConfig) val ctx = new CodegenContext an[Error] should be thrownBy stBuffer.genCode(ctx) } @@ -214,12 +214,13 @@ trait ST_BufferBehaviors extends QueryTest { val df = getWKTRowsDf() - val stBuffer = ST_Buffer(df.col("wkt").expr, lit(1).expr, mc.expressionConfig) + val stBuffer = ST_Buffer(df.col("wkt").expr, lit(1).expr, lit("").expr, mc.expressionConfig) - stBuffer.left shouldEqual df.col("wkt").expr - stBuffer.right shouldEqual lit(1).expr + stBuffer.first shouldEqual df.col("wkt").expr + stBuffer.second shouldEqual lit(1).expr + stBuffer.third shouldEqual lit("").expr stBuffer.dataType shouldEqual df.col("wkt").expr.dataType - noException should be thrownBy stBuffer.makeCopy(Array(stBuffer.left, stBuffer.right)) + noException should be thrownBy stBuffer.makeCopy(Array(stBuffer.first, stBuffer.second, stBuffer.third)) st_buffer(col("wkt"), 1).expr.children(1) shouldEqual lit(1.0).cast("double").expr From e3a2e0faae0a104c477cdda22d3df5707cc2d835 Mon Sep 17 00:00:00 2001 From: Daniel Sparing Date: Tue, 23 Jan 2024 11:07:26 +0100 Subject: [PATCH 14/47] minor debug (build succeeds) --- .../core/geometry/MosaicGeometryJTS.scala | 36 ++++++++++--------- .../expressions/geometry/ST_Buffer.scala | 6 +++- .../geometry/ST_BufferBehaviors.scala | 2 +- 3 files changed, 25 insertions(+), 19 deletions(-) diff --git a/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometryJTS.scala b/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometryJTS.scala index 0d4a890e1..e0694dc7b 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometryJTS.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometryJTS.scala @@ -63,26 +63,28 @@ abstract class MosaicGeometryJTS(geom: Geometry) extends MosaicGeometry { override def buffer(distance: Double, bufferStyleParameters: String): MosaicGeometryJTS = { - val params = bufferStyleParameters.split(" ") - .map(_.split("=")) - .map { case Array(k, v) => (k, v) } - .toMap - val gBuf = new BufferOp(geom) - if(params.contains("endcap")) { - val capStyle = params.getOrElse("endcap", "") - val capStyleConst = capStyle match { - case "round" => BufferParameters.CAP_ROUND - case "flat" => BufferParameters.CAP_FLAT - case "square" => BufferParameters.CAP_SQUARE - case _ => BufferParameters.CAP_ROUND + if (bufferStyleParameters contains "=") { + val params = bufferStyleParameters.split(" ") + .map(_.split("=")) + .map { case Array(k, v) => (k, v) } + .toMap + + if(params.contains("endcap")) { + val capStyle = params.getOrElse("endcap", "") + val capStyleConst = capStyle match { + case "round" => BufferParameters.CAP_ROUND + case "flat" => BufferParameters.CAP_FLAT + case "square" => BufferParameters.CAP_SQUARE + case _ => BufferParameters.CAP_ROUND + } + gBuf.setEndCapStyle(capStyleConst) + } + if(params.contains("quad_segs")) { + val quadSegs = params.getOrElse("quad_segs", "8") + gBuf.setQuadrantSegments(quadSegs.toInt) } - gBuf.setEndCapStyle(capStyleConst) - } - if(params.contains("quad_segs")) { - val quadSegs = params.getOrElse("quad_segs", "") - gBuf.setQuadrantSegments(quadSegs.toInt) } val buffered = gBuf.getResultGeometry(distance) buffered.setSRID(geom.getSRID) diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/geometry/ST_Buffer.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/geometry/ST_Buffer.scala index 7d21aecb9..17878d1f6 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/geometry/ST_Buffer.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/geometry/ST_Buffer.scala @@ -65,7 +65,11 @@ object ST_Buffer extends WithExpressionInfo { | """.stripMargin override def builder(expressionConfig: MosaicExpressionConfig): FunctionBuilder = { (children: Seq[Expression]) => - ST_Buffer(children.head, Column(children(1)).cast("double").expr, children(2), expressionConfig) + if (children.size == 2) { + ST_Buffer(children.head, Column(children(1)).cast("double").expr, lit("").expr, expressionConfig) + } else if (children.size == 3) { + ST_Buffer(children.head, Column(children(1)).cast("double").expr, Column(children(2)).cast("string").expr, expressionConfig) + } else throw new Exception ("unexpected number of arguments") } } diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala index 068be8d52..269167c9f 100644 --- a/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala @@ -195,7 +195,7 @@ trait ST_BufferBehaviors extends QueryTest { Seq( StructField("geom", StringType), StructField("buffer", DoubleType), - StructField("bufferStyleParameters", IntegerType), + StructField("bufferStyleParameters", StringType), StructField("geomBufferedRef", StringType) ) ) From c664db7ce28a12d2d65b0da3d6c31635eca6ed44 Mon Sep 17 00:00:00 2001 From: Daniel Sparing Date: Tue, 23 Jan 2024 11:17:23 +0100 Subject: [PATCH 15/47] scalafmt, flake8 --- python/mosaic/api/functions.py | 2 +- .../labs/mosaic/core/geometry/MosaicGeometryJTS.scala | 7 ++++--- .../labs/mosaic/expressions/geometry/ST_Buffer.scala | 9 ++++----- .../databricks/labs/mosaic/functions/MosaicContext.scala | 4 +++- .../mosaic/expressions/geometry/ST_BufferBehaviors.scala | 3 +-- 5 files changed, 13 insertions(+), 12 deletions(-) diff --git a/python/mosaic/api/functions.py b/python/mosaic/api/functions.py index 5aba65f1c..b5a7fbeb4 100644 --- a/python/mosaic/api/functions.py +++ b/python/mosaic/api/functions.py @@ -198,7 +198,7 @@ def st_buffer( geom: ColumnOrName, radius: ColumnOrName, buffer_style_parameters: Any = "", - ) -> Column: +) -> Column: """ Compute the buffered geometry based on geom and radius. diff --git a/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometryJTS.scala b/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometryJTS.scala index e0694dc7b..1326ac25f 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometryJTS.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometryJTS.scala @@ -66,12 +66,13 @@ abstract class MosaicGeometryJTS(geom: Geometry) extends MosaicGeometry { val gBuf = new BufferOp(geom) if (bufferStyleParameters contains "=") { - val params = bufferStyleParameters.split(" ") + val params = bufferStyleParameters + .split(" ") .map(_.split("=")) .map { case Array(k, v) => (k, v) } .toMap - if(params.contains("endcap")) { + if (params.contains("endcap")) { val capStyle = params.getOrElse("endcap", "") val capStyleConst = capStyle match { case "round" => BufferParameters.CAP_ROUND @@ -81,7 +82,7 @@ abstract class MosaicGeometryJTS(geom: Geometry) extends MosaicGeometry { } gBuf.setEndCapStyle(capStyleConst) } - if(params.contains("quad_segs")) { + if (params.contains("quad_segs")) { val quadSegs = params.getOrElse("quad_segs", "8") gBuf.setQuadrantSegments(quadSegs.toInt) } diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/geometry/ST_Buffer.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/geometry/ST_Buffer.scala index 17878d1f6..dfa679542 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/geometry/ST_Buffer.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/geometry/ST_Buffer.scala @@ -12,7 +12,6 @@ import org.apache.spark.sql.types.DataType import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.sql.functions._ - /** * SQL expression that returns the input geometry buffered by the radius. * @param inputGeom @@ -20,9 +19,9 @@ import org.apache.spark.sql.functions._ * @param radiusExpr * The radius of the buffer. * @param bufferStyleParametersExpr - * 'quad_segs=# endcap=round|flat|square' where "#" is the number of line segments used to - * approximate a quarter circle (default is 8); and endcap style for line features is one of - * listed (default="round") + * 'quad_segs=# endcap=round|flat|square' where "#" is the number of line + * segments used to approximate a quarter circle (default is 8); and endcap + * style for line features is one of listed (default="round") * @param expressionConfig * Mosaic execution context, e.g. geometryAPI, indexSystem, etc. Additional * arguments for the expression (expressionConfigs). @@ -69,7 +68,7 @@ object ST_Buffer extends WithExpressionInfo { ST_Buffer(children.head, Column(children(1)).cast("double").expr, lit("").expr, expressionConfig) } else if (children.size == 3) { ST_Buffer(children.head, Column(children(1)).cast("double").expr, Column(children(2)).cast("string").expr, expressionConfig) - } else throw new Exception ("unexpected number of arguments") + } else throw new Exception("unexpected number of arguments") } } diff --git a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala index 134700e08..d39fe4a52 100644 --- a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala +++ b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala @@ -552,7 +552,9 @@ class MosaicContext(indexSystem: IndexSystem, geometryAPI: GeometryAPI) extends def st_buffer(geom: Column, radius: Column, buffer_style_parameters: Column): Column = ColumnAdapter(ST_Buffer(geom.expr, radius.cast("double").expr, buffer_style_parameters.cast("string").expr, expressionConfig)) def st_buffer(geom: Column, radius: Double, buffer_style_parameters: Column): Column = - ColumnAdapter(ST_Buffer(geom.expr, lit(radius).cast("double").expr, lit(buffer_style_parameters).cast("string").expr, expressionConfig)) + ColumnAdapter( + ST_Buffer(geom.expr, lit(radius).cast("double").expr, lit(buffer_style_parameters).cast("string").expr, expressionConfig) + ) def st_bufferloop(geom: Column, r1: Column, r2: Column): Column = ColumnAdapter(ST_BufferLoop(geom.expr, r1.cast("double").expr, r2.cast("double").expr, expressionConfig)) def st_bufferloop(geom: Column, r1: Double, r2: Double): Column = diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala index 269167c9f..972a474a0 100644 --- a/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/geometry/ST_BufferBehaviors.scala @@ -85,7 +85,7 @@ trait ST_BufferBehaviors extends QueryTest { st_distance($"geomBufferedRef", $"geomBufferedTest"), st_area(st_intersection($"geomBufferedRef", $"geomBufferedTest")), st_area(st_union($"geomBufferedRef", $"geomBufferedTest")), - st_area($"geomBufferedRef"), + st_area($"geomBufferedRef") ) .as[(Double, Double, Double, Double)] .collect() @@ -211,7 +211,6 @@ trait ST_BufferBehaviors extends QueryTest { mc.register(spark) import mc.functions._ - val df = getWKTRowsDf() val stBuffer = ST_Buffer(df.col("wkt").expr, lit(1).expr, lit("").expr, mc.expressionConfig) From 5ea40b9493e618845e4d1b7fcbfadfb0ae5f1bbf Mon Sep 17 00:00:00 2001 From: Daniel Sparing Date: Tue, 23 Jan 2024 11:22:59 +0100 Subject: [PATCH 16/47] copy commit 7300d64 to avoid merge conflict --- python/test/test_vector_functions.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/test/test_vector_functions.py b/python/test/test_vector_functions.py index 2c4d3d329..eed213d07 100644 --- a/python/test/test_vector_functions.py +++ b/python/test/test_vector_functions.py @@ -45,7 +45,7 @@ def test_st_bindings_happy_flow(self): .withColumn( "st_buffer_optparams", api.st_buffer("wkt", lit(1.1), lit("endcap=square quad_segs=2"))) - .withColumn("st_buffer", api.st_bufferloop("wkt", lit(1.1), lit(1.2))) + .withColumn("st_bufferloop", api.st_bufferloop("wkt", lit(1.1), lit(1.2))) .withColumn("st_perimeter", api.st_perimeter("wkt")) .withColumn("st_convexhull", api.st_convexhull("wkt")) .withColumn("st_concavehull", api.st_concavehull("wkt", lit(0.5))) From e9ed7a8fd4b14f09ab84fcf5f31f90b49a4c8077 Mon Sep 17 00:00:00 2001 From: Daniel Sparing Date: Tue, 23 Jan 2024 13:38:00 +0100 Subject: [PATCH 17/47] put back single-parameter version of st_buffer --- .../databricks/labs/mosaic/core/geometry/MosaicGeometry.scala | 2 ++ .../labs/mosaic/core/geometry/MosaicGeometryJTS.scala | 4 ++++ 2 files changed, 6 insertions(+) diff --git a/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometry.scala b/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometry.scala index c6bacdf0e..72c7dace7 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometry.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometry.scala @@ -55,6 +55,8 @@ trait MosaicGeometry extends GeometryWriter with Serializable { def boundary: MosaicGeometry + def buffer(distance: Double): MosaicGeometry + def buffer(distance: Double, bufferStyleParameters: String = ""): MosaicGeometry def bufferCapStyle(distance: Double, capStyle: String): MosaicGeometry diff --git a/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometryJTS.scala b/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometryJTS.scala index 1326ac25f..cd17320fe 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometryJTS.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometryJTS.scala @@ -61,6 +61,10 @@ abstract class MosaicGeometryJTS(geom: Geometry) extends MosaicGeometry { override def envelope: MosaicGeometryJTS = MosaicGeometryJTS(geom.getEnvelope) + override def buffer(distance: Double): MosaicGeometryJTS = { + buffer(distance, "") + } + override def buffer(distance: Double, bufferStyleParameters: String): MosaicGeometryJTS = { val gBuf = new BufferOp(geom) From 63232f018532f3a7df620e2e7e3800c30b0ec4af Mon Sep 17 00:00:00 2001 From: Daniel Sparing Date: Tue, 23 Jan 2024 13:39:01 +0100 Subject: [PATCH 18/47] scalafmt --- .../databricks/labs/mosaic/core/geometry/MosaicGeometry.scala | 2 +- .../labs/mosaic/core/geometry/MosaicGeometryJTS.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometry.scala b/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometry.scala index 72c7dace7..210eb3429 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometry.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometry.scala @@ -55,7 +55,7 @@ trait MosaicGeometry extends GeometryWriter with Serializable { def boundary: MosaicGeometry - def buffer(distance: Double): MosaicGeometry + def buffer(distance: Double): MosaicGeometry def buffer(distance: Double, bufferStyleParameters: String = ""): MosaicGeometry diff --git a/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometryJTS.scala b/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometryJTS.scala index cd17320fe..83868dba5 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometryJTS.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/geometry/MosaicGeometryJTS.scala @@ -64,7 +64,7 @@ abstract class MosaicGeometryJTS(geom: Geometry) extends MosaicGeometry { override def buffer(distance: Double): MosaicGeometryJTS = { buffer(distance, "") } - + override def buffer(distance: Double, bufferStyleParameters: String): MosaicGeometryJTS = { val gBuf = new BufferOp(geom) From 79ff6e6feeed61d98e542d9db8467bf6a595a4d1 Mon Sep 17 00:00:00 2001 From: "milos.colic" Date: Thu, 1 Feb 2024 15:40:17 +0000 Subject: [PATCH 19/47] Fix the format name for grib files in tests. Fix temp location utils. Separate temp on worker location and off worker location. Introduce GDAL Block notion. Implement Kernel filters via GDALBlocks. Add additional params to gdal programs when they run. Fix TILED=YES issue with TIF files. Introduce writeOptions concept for tmp writes. Update expressions to take into account new concepts. Fix Zarr format issues with SerDeser. --- .../labs/mosaic/core/raster/api/GDAL.scala | 9 +- .../mosaic/core/raster/gdal/GDALBlock.scala | 186 ++++++++++++ .../raster/gdal/MosaicRasterBandGDAL.scala | 77 +++++ .../core/raster/gdal/MosaicRasterGDAL.scala | 228 +++++++++++---- .../gdal/MosaicRasterWriteOptions.scala | 55 ++++ .../mosaic/core/raster/gdal/Padding.scala | 58 ++++ .../operator/clip/RasterClipByVector.scala | 10 +- .../raster/operator/gdal/GDALBuildVRT.scala | 16 +- .../core/raster/operator/gdal/GDALCalc.scala | 17 +- .../raster/operator/gdal/GDALTranslate.scala | 25 +- .../core/raster/operator/gdal/GDALWarp.scala | 5 +- .../operator/gdal/OperatorOptions.scala | 35 +++ .../raster/operator/merge/MergeBands.scala | 14 +- .../raster/operator/merge/MergeRasters.scala | 18 +- .../operator/pixel/PixelCombineRasters.scala | 8 +- .../raster/operator/proj/RasterProject.scala | 7 +- .../operator/retile/OverlappingTiles.scala | 5 +- .../operator/retile/RasterTessellate.scala | 5 +- .../core/raster/operator/retile/ReTile.scala | 7 +- .../mosaic/core/types/RasterTileType.scala | 43 ++- .../core/types/model/MosaicRasterTile.scala | 26 +- .../mosaic/datasource/gdal/ReTileOnRead.scala | 15 +- .../mosaic/datasource/gdal/ReadInMemory.scala | 4 +- .../multiread/RasterAsGridReader.scala | 40 ++- .../mosaic/expressions/raster/RST_Avg.scala | 6 +- .../expressions/raster/RST_BandMetaData.scala | 3 +- .../expressions/raster/RST_BoundingBox.scala | 4 +- .../mosaic/expressions/raster/RST_Clip.scala | 3 +- .../expressions/raster/RST_CombineAvg.scala | 8 +- .../raster/RST_CombineAvgAgg.scala | 28 +- .../expressions/raster/RST_Convolve.scala | 73 +++++ .../expressions/raster/RST_DerivedBand.scala | 8 +- .../raster/RST_DerivedBandAgg.scala | 19 +- .../expressions/raster/RST_Filter.scala | 77 +++++ .../expressions/raster/RST_FromBands.scala | 8 +- .../expressions/raster/RST_FromContent.scala | 25 +- .../expressions/raster/RST_FromFile.scala | 10 +- .../expressions/raster/RST_GeoReference.scala | 4 +- .../expressions/raster/RST_GetNoData.scala | 5 +- .../raster/RST_GetSubdataset.scala | 13 +- .../expressions/raster/RST_Height.scala | 4 +- .../expressions/raster/RST_InitNoData.scala | 8 +- .../expressions/raster/RST_IsEmpty.scala | 4 +- .../expressions/raster/RST_MakeTiles.scala | 205 +++++++++++++ .../expressions/raster/RST_MapAlgebra.scala | 8 +- .../mosaic/expressions/raster/RST_Max.scala | 4 +- .../expressions/raster/RST_Median.scala | 7 +- .../expressions/raster/RST_MemSize.scala | 4 +- .../mosaic/expressions/raster/RST_Merge.scala | 8 +- .../expressions/raster/RST_MergeAgg.scala | 19 +- .../expressions/raster/RST_MetaData.scala | 4 +- .../mosaic/expressions/raster/RST_Min.scala | 5 +- .../mosaic/expressions/raster/RST_NDVI.scala | 8 +- .../expressions/raster/RST_NumBands.scala | 4 +- .../expressions/raster/RST_PixelCount.scala | 4 +- .../expressions/raster/RST_PixelHeight.scala | 4 +- .../expressions/raster/RST_PixelWidth.scala | 4 +- .../raster/RST_RasterToWorldCoord.scala | 4 +- .../raster/RST_RasterToWorldCoordX.scala | 4 +- .../raster/RST_RasterToWorldCoordY.scala | 4 +- .../expressions/raster/RST_ReTile.scala | 3 + .../expressions/raster/RST_Rotation.scala | 4 +- .../mosaic/expressions/raster/RST_SRID.scala | 4 +- .../expressions/raster/RST_ScaleX.scala | 4 +- .../expressions/raster/RST_ScaleY.scala | 4 +- .../expressions/raster/RST_SetNoData.scala | 8 +- .../mosaic/expressions/raster/RST_SkewX.scala | 4 +- .../mosaic/expressions/raster/RST_SkewY.scala | 4 +- .../expressions/raster/RST_Subdatasets.scala | 3 +- .../expressions/raster/RST_Summary.scala | 4 +- .../expressions/raster/RST_TryOpen.scala | 4 +- .../expressions/raster/RST_UpperLeftX.scala | 4 +- .../expressions/raster/RST_UpperLeftY.scala | 4 +- .../mosaic/expressions/raster/RST_Width.scala | 4 +- .../raster/RST_WorldToRasterCoord.scala | 5 +- .../raster/RST_WorldToRasterCoordX.scala | 4 +- .../raster/RST_WorldToRasterCoordY.scala | 4 +- .../raster/base/Raster1ArgExpression.scala | 17 +- .../raster/base/Raster2ArgExpression.scala | 16 +- .../base/RasterArray1ArgExpression.scala | 12 +- .../base/RasterArray2ArgExpression.scala | 12 +- .../raster/base/RasterArrayExpression.scala | 10 +- .../raster/base/RasterArrayUtils.scala | 8 +- .../raster/base/RasterBandExpression.scala | 17 +- .../raster/base/RasterExpression.scala | 16 +- .../base/RasterExpressionSerialization.scala | 4 +- .../base/RasterGeneratorExpression.scala | 11 +- .../RasterTessellateGeneratorExpression.scala | 15 +- .../raster/base/RasterToGridExpression.scala | 4 +- .../labs/mosaic/functions/MosaicContext.scala | 24 +- .../functions/MosaicExpressionConfig.scala | 2 + .../labs/mosaic/gdal/MosaicGDAL.scala | 66 +++-- .../com/databricks/labs/mosaic/package.scala | 6 +- .../labs/mosaic/utils/FileUtils.scala | 6 +- .../labs/mosaic/utils/PathUtils.scala | 66 ++++- .../labs/mosaic/utils/SysUtils.scala | 44 ++- ...-041ac051-015d-49b0-95df-b5daa7084c7e.grb} | Bin ...1-015d-49b0-95df-b5daa7084c7e.grb.aux.xml} | 0 ...-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grb} | Bin ...6-16ca-4e11-919d-bdbd5a51da35.grb.aux.xml} | 0 ...-0ede0273-89e3-4100-a0f2-48916ca607ed.grb} | Bin ...3-89e3-4100-a0f2-48916ca607ed.grb.aux.xml} | 0 .../core/raster/TestRasterBandGDAL.scala | 4 +- .../mosaic/core/raster/TestRasterGDAL.scala | 223 +++++++++++++- .../datasource/GDALFileFormatTest.scala | 19 +- .../multiread/RasterAsGridReaderTest.scala | 272 +++++++++--------- .../raster/RST_CombineAvgBehaviors.scala | 4 +- .../raster/RST_FilterBehaviors.scala | 36 +++ .../expressions/raster/RST_FilterTest.scala | 32 +++ .../expressions/raster/RST_MinBehaviors.scala | 2 +- .../raster/RST_TessellateBehaviors.scala | 11 +- .../databricks/labs/mosaic/test/package.scala | 2 +- .../sql/test/MosaicTestSparkSession.scala | 4 +- .../sql/test/SharedSparkSessionGDAL.scala | 6 +- 114 files changed, 2005 insertions(+), 560 deletions(-) create mode 100644 src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/GDALBlock.scala create mode 100644 src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterWriteOptions.scala create mode 100644 src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/Padding.scala create mode 100644 src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Convolve.scala create mode 100644 src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Filter.scala create mode 100644 src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MakeTiles.scala rename src/test/resources/binary/grib-cams/{adaptor.mars.internal-1650626950.0440469-3609-11-041ac051-015d-49b0-95df-b5daa7084c7e.grib => adaptor.mars.internal-1650626950.0440469-3609-11-041ac051-015d-49b0-95df-b5daa7084c7e.grb} (100%) rename src/test/resources/binary/grib-cams/{adaptor.mars.internal-1650626950.0440469-3609-11-041ac051-015d-49b0-95df-b5daa7084c7e.grib.aux.xml => adaptor.mars.internal-1650626950.0440469-3609-11-041ac051-015d-49b0-95df-b5daa7084c7e.grb.aux.xml} (100%) rename src/test/resources/binary/grib-cams/{adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grib => adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grb} (100%) rename src/test/resources/binary/grib-cams/{adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grib.aux.xml => adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grb.aux.xml} (100%) rename src/test/resources/binary/grib-cams/{adaptor.mars.internal-1650627030.319457-19905-15-0ede0273-89e3-4100-a0f2-48916ca607ed.grib => adaptor.mars.internal-1650627030.319457-19905-15-0ede0273-89e3-4100-a0f2-48916ca607ed.grb} (100%) rename src/test/resources/binary/grib-cams/{adaptor.mars.internal-1650627030.319457-19905-15-0ede0273-89e3-4100-a0f2-48916ca607ed.grib.aux.xml => adaptor.mars.internal-1650627030.319457-19905-15-0ede0273-89e3-4100-a0f2-48916ca607ed.grb.aux.xml} (100%) create mode 100644 src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_FilterBehaviors.scala create mode 100644 src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_FilterTest.scala diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/api/GDAL.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/api/GDAL.scala index 66bde39a3..b86489359 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/api/GDAL.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/api/GDAL.scala @@ -4,6 +4,7 @@ import com.databricks.labs.mosaic.core.raster.gdal.{MosaicRasterBandGDAL, Mosaic import com.databricks.labs.mosaic.core.raster.io.RasterCleaner import com.databricks.labs.mosaic.core.raster.operator.transform.RasterTransform import com.databricks.labs.mosaic.functions.MosaicExpressionConfig +import com.databricks.labs.mosaic.gdal.MosaicGDAL import com.databricks.labs.mosaic.gdal.MosaicGDAL.configureGDAL import org.apache.spark.sql.SparkSession import org.apache.spark.sql.types.{BinaryType, DataType, StringType} @@ -114,6 +115,8 @@ object GDAL { } else { raster } + case _ => + throw new IllegalArgumentException(s"Unsupported data type: $inputDT") } } @@ -122,19 +125,17 @@ object GDAL { * * @param generatedRasters * The rasters to write. - * @param checkpointPath - * The path to write the rasters to. * @return * Returns the paths of the written rasters. */ - def writeRasters(generatedRasters: Seq[MosaicRasterGDAL], checkpointPath: String, rasterDT: DataType): Seq[Any] = { + def writeRasters(generatedRasters: Seq[MosaicRasterGDAL], rasterDT: DataType): Seq[Any] = { generatedRasters.map(raster => if (raster != null) { rasterDT match { case StringType => val uuid = UUID.randomUUID().toString val extension = GDAL.getExtension(raster.getDriversShortName) - val writePath = s"$checkpointPath/$uuid.$extension" + val writePath = s"${MosaicGDAL.checkpointPath}/$uuid.$extension" val outPath = raster.writeToPath(writePath) RasterCleaner.dispose(raster) UTF8String.fromString(outPath) diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/GDALBlock.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/GDALBlock.scala new file mode 100644 index 000000000..8c5c7a495 --- /dev/null +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/GDALBlock.scala @@ -0,0 +1,186 @@ +package com.databricks.labs.mosaic.core.raster.gdal + +import scala.reflect.ClassTag + +case class GDALBlock[T: ClassTag]( + block: Array[T], + maskBlock: Array[Double], + noDataValue: Double, + xOffset: Int, + yOffset: Int, + width: Int, + height: Int, + padding: Padding +)(implicit + num: Numeric[T] +) { + + def elementAt(index: Int): T = block(index) + + def maskAt(index: Int): Double = maskBlock(index) + + def elementAt(x: Int, y: Int): T = block(y * width + x) + + def maskAt(x: Int, y: Int): Double = maskBlock(y * width + x) + + def rasterElementAt(x: Int, y: Int): T = block((y - yOffset) * width + (x - xOffset)) + + def rasterMaskAt(x: Int, y: Int): Double = maskBlock((y - yOffset) * width + (x - xOffset)) + + def valuesAt(x: Int, y: Int, kernelWidth: Int, kernelHeight: Int): Array[Double] = { + val kernelCenterX = kernelWidth / 2 + val kernelCenterY = kernelHeight / 2 + val values = Array.fill[Double](kernelWidth * kernelHeight)(noDataValue) + var n = 0 + for (i <- 0 until kernelHeight) { + for (j <- 0 until kernelWidth) { + val xIndex = x + (j - kernelCenterX) + val yIndex = y + (i - kernelCenterY) + if (xIndex >= 0 && xIndex < width && yIndex >= 0 && yIndex < height) { + val maskValue = maskAt(xIndex, yIndex) + val value = elementAt(xIndex, yIndex) + if (maskValue != 0.0 && num.toDouble(value) != noDataValue) { + values(n) = num.toDouble(value) + n += 1 + } + } + } + } + val result = values.filter(_ != noDataValue) + // always return only one NoDataValue if no valid values are found + // one and only one NoDataValue can be returned + // in all cases that have some valid values, the NoDataValue will be filtered out + if (result.isEmpty) { + Array(noDataValue) + } else { + result + } + } + + // TODO: Test and fix, not tested, other filters work. + def convolveAt(x: Int, y: Int, kernel: Array[Array[Double]]): Double = { + val kernelWidth = kernel.head.length + val kernelHeight = kernel.length + val kernelCenterX = kernelWidth / 2 + val kernelCenterY = kernelHeight / 2 + var sum = 0.0 + for (i <- 0 until kernelHeight) { + for (j <- 0 until kernelWidth) { + val xIndex = x + (j - kernelCenterX) + val yIndex = y + (i - kernelCenterY) + if (xIndex >= 0 && xIndex < width && yIndex >= 0 && yIndex < height) { + val maskValue = maskAt(xIndex, yIndex) + val value = rasterElementAt(xIndex, yIndex) + if (maskValue != 0.0 && num.toDouble(value) != noDataValue) { + sum += num.toDouble(value) * kernel(i)(j) + } + } + } + } + sum + } + + def avgFilterAt(x: Int, y: Int, kernelSize: Int): Double = { + val values = valuesAt(x, y, kernelSize, kernelSize) + values.sum / values.length + } + + def minFilterAt(x: Int, y: Int, kernelSize: Int): Double = { + val values = valuesAt(x, y, kernelSize, kernelSize) + values.min + } + + def maxFilterAt(x: Int, y: Int, kernelSize: Int): Double = { + val values = valuesAt(x, y, kernelSize, kernelSize) + values.max + } + + def medianFilterAt(x: Int, y: Int, kernelSize: Int): Double = { + val values = valuesAt(x, y, kernelSize, kernelSize) + val n = values.length + scala.util.Sorting.quickSort(values) + values(n / 2) + } + + def modeFilterAt(x: Int, y: Int, kernelSize: Int): Double = { + val values = valuesAt(x, y, kernelSize, kernelSize) + val counts = values.groupBy(identity).mapValues(_.length) + counts.maxBy(_._2)._1 + } + + def trimBlock(stride: Int): GDALBlock[Double] = { + val resultBlock = padding.removePadding(block.map(num.toDouble), width, stride) + val resultMask = padding.removePadding(maskBlock, width, stride) + + val newOffset = padding.newOffset(xOffset, yOffset, stride) + val newSize = padding.newSize(width, height, stride) + + new GDALBlock[Double]( + resultBlock, + resultMask, + noDataValue, + newOffset._1, + newOffset._2, + newSize._1, + newSize._2, + Padding.NoPadding + ) + } + +} + +object GDALBlock { + + def getSize(offset: Int, maxSize: Int, blockSize: Int, stride: Int, paddingStrides: Int): Int = { + if (offset + blockSize + paddingStrides * stride > maxSize) { + maxSize - offset + } else { + blockSize + paddingStrides * stride + } + } + + def apply( + band: MosaicRasterBandGDAL, + stride: Int, + xOffset: Int, + yOffset: Int, + blockSize: Int + ): GDALBlock[Double] = { + val noDataValue = band.noDataValue + val rasterWidth = band.xSize + val rasterHeight = band.ySize + // Always read blockSize + stride pixels on every side + // This is fine since kernel size is always much smaller than blockSize + + val padding = Padding( + left = xOffset != 0, + right = xOffset + blockSize < rasterWidth - 1, // not sure about -1 + top = yOffset != 0, + bottom = yOffset + blockSize < rasterHeight - 1 + ) + + val xo = Math.max(0, xOffset - stride) + val yo = Math.max(0, yOffset - stride) + + val xs = getSize(xo, rasterWidth, blockSize, stride, padding.horizontalStrides) + val ys = getSize(yo, rasterHeight, blockSize, stride, padding.verticalStrides) + + val block = Array.ofDim[Double](xs * ys) + val maskBlock = Array.ofDim[Double](xs * ys) + + band.getBand.ReadRaster(xo, yo, xs, ys, block) + band.getBand.GetMaskBand().ReadRaster(xo, yo, xs, ys, maskBlock) + + GDALBlock( + block, + maskBlock, + noDataValue, + xo, + yo, + xs, + ys, + padding + ) + } + +} diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterBandGDAL.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterBandGDAL.scala index a7c9ece10..281eb8b01 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterBandGDAL.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterBandGDAL.scala @@ -1,5 +1,6 @@ package com.databricks.labs.mosaic.core.raster.gdal +import com.databricks.labs.mosaic.gdal.MosaicGDAL import org.gdal.gdal.Band import org.gdal.gdalconst.gdalconstConstants @@ -255,4 +256,80 @@ case class MosaicRasterBandGDAL(band: Band, id: Int) { */ def isNoDataMask: Boolean = band.GetMaskFlags() == gdalconstConstants.GMF_NODATA + def convolve(kernel: Array[Array[Double]]): Unit = { + val kernelWidth = kernel.head.length + val kernelHeight = kernel.length + val blockSize = MosaicGDAL.defaultBlockSize + val strideX = kernelWidth / 2 + val strideY = kernelHeight / 2 + + val block = Array.ofDim[Double](blockSize * blockSize) + val maskBlock = Array.ofDim[Double](blockSize * blockSize) + val result = Array.ofDim[Double](blockSize * blockSize) + + for (yOffset <- 0 until ySize by blockSize - strideY) { + for (xOffset <- 0 until xSize by blockSize - strideX) { + val xSize = Math.min(blockSize, this.xSize - xOffset) + val ySize = Math.min(blockSize, this.ySize - yOffset) + + band.ReadRaster(xOffset, yOffset, xSize, ySize, block) + band.GetMaskBand().ReadRaster(xOffset, yOffset, xSize, ySize, maskBlock) + + val currentBlock = GDALBlock[Double](block, maskBlock, noDataValue, xOffset, yOffset, xSize, ySize, Padding.NoPadding) + + for (y <- 0 until ySize) { + for (x <- 0 until xSize) { + result(y * xSize + x) = currentBlock.convolveAt(x, y, kernel) + } + } + + band.WriteRaster(xOffset, yOffset, xSize, ySize, block) + } + } + } + + def filter(kernelSize: Int, operation: String, outputBand: Band): Unit = { + require(kernelSize % 2 == 1, "Kernel size must be odd") + + val blockSize = MosaicGDAL.blockSize + val stride = kernelSize / 2 + + for (yOffset <- 0 until ySize by blockSize) { + for (xOffset <- 0 until xSize by blockSize) { + + val currentBlock = GDALBlock( + this, + stride, + xOffset, + yOffset, + blockSize + ) + + val result = Array.ofDim[Double](currentBlock.block.length) + + for (y <- 0 until currentBlock.height) { + for (x <- 0 until currentBlock.width) { + result(y * currentBlock.width + x) = operation match { + case "avg" => currentBlock.avgFilterAt(x, y, kernelSize) + case "min" => currentBlock.minFilterAt(x, y, kernelSize) + case "max" => currentBlock.maxFilterAt(x, y, kernelSize) + case "median" => currentBlock.medianFilterAt(x, y, kernelSize) + case "mode" => currentBlock.modeFilterAt(x, y, kernelSize) + case _ => throw new Exception("Invalid operation") + } + } + } + + val trimmedResult = currentBlock.copy(block = result).trimBlock(stride) + + outputBand.WriteRaster(xOffset, yOffset, trimmedResult.width, trimmedResult.height, trimmedResult.block) + outputBand.FlushCache() + outputBand.GetMaskBand().WriteRaster(xOffset, yOffset, trimmedResult.width, trimmedResult.height, trimmedResult.maskBlock) + outputBand.GetMaskBand().FlushCache() + + } + } + + } + } diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala index 3ac467f53..b63bd851e 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala @@ -8,9 +8,9 @@ import com.databricks.labs.mosaic.core.raster.io.RasterCleaner.dispose import com.databricks.labs.mosaic.core.raster.io.{RasterCleaner, RasterReader, RasterWriter} import com.databricks.labs.mosaic.core.raster.operator.clip.RasterClipByVector import com.databricks.labs.mosaic.core.types.model.GeometryTypeEnum.POLYGON -import com.databricks.labs.mosaic.utils.{FileUtils, PathUtils} -import org.gdal.gdal.gdal.GDALInfo -import org.gdal.gdal.{Dataset, InfoOptions, gdal} +import com.databricks.labs.mosaic.gdal.MosaicGDAL +import com.databricks.labs.mosaic.utils.{FileUtils, PathUtils, SysUtils} +import org.gdal.gdal.{Dataset, gdal} import org.gdal.gdalconst.gdalconstConstants._ import org.gdal.osr import org.gdal.osr.SpatialReference @@ -32,25 +32,41 @@ case class MosaicRasterGDAL( ) extends RasterWriter with RasterCleaner { + def getWriteOptions: MosaicRasterWriteOptions = MosaicRasterWriteOptions(this) + + def getCompression: String = { + val compression = Option(raster.GetMetadata_Dict("IMAGE_STRUCTURE")) + .map(_.asScala.toMap.asInstanceOf[Map[String, String]]) + .getOrElse(Map.empty[String, String]) + .getOrElse("COMPRESSION", "NONE") + compression + } + def getSpatialReference: SpatialReference = { - if (raster != null) { - raster.GetSpatialRef + val spatialRef = + if (raster != null) { + raster.GetSpatialRef + } else { + val tmp = refresh() + val result = tmp.raster.GetSpatialRef + dispose(tmp) + result + } + if (spatialRef == null) { + MosaicGDAL.WSG84 } else { - val tmp = refresh() - val result = tmp.spatialRef - dispose(tmp) - result + spatialRef } } + def isSubDataset: Boolean = { + val isSubdataset = PathUtils.isSubdataset(path) + isSubdataset + } + // Factory for creating CRS objects protected val crsFactory: CRSFactory = new CRSFactory - // Only use this with GDAL rasters - private val wsg84 = new osr.SpatialReference() - wsg84.ImportFromEPSG(4326) - wsg84.SetAxisMappingStrategy(osr.osrConstants.OAMS_TRADITIONAL_GIS_ORDER) - /** * @return * The raster's driver short name. @@ -157,6 +173,7 @@ case class MosaicRasterGDAL( .map(_.asScala.toMap.asInstanceOf[Map[String, String]]) .getOrElse(Map.empty[String, String]) val keys = subdatasetsMap.keySet + val sanitizedParentPath = PathUtils.getCleanPath(parentPath) keys.flatMap(key => if (key.toUpperCase(Locale.ROOT).contains("NAME")) { val path = subdatasetsMap(key) @@ -164,7 +181,7 @@ case class MosaicRasterGDAL( Seq( key -> pieces.last, s"${pieces.last}_tmp" -> path, - pieces.last -> s"${pieces.head}:$parentPath:${pieces.last}" + pieces.last -> s"${pieces.head}:$sanitizedParentPath:${pieces.last}" ) } else Seq(key -> subdatasetsMap(key)) ).toMap @@ -253,12 +270,6 @@ case class MosaicRasterGDAL( */ def getRaster: Dataset = this.raster - /** - * @return - * Returns the raster's spatial reference. - */ - def spatialRef: SpatialReference = raster.GetSpatialRef() - /** * Applies a function to each band of the raster. * @param f @@ -272,10 +283,10 @@ case class MosaicRasterGDAL( * @return * Returns MosaicGeometry representing bounding box of the raster. */ - def bbox(geometryAPI: GeometryAPI, destCRS: SpatialReference = wsg84): MosaicGeometry = { + def bbox(geometryAPI: GeometryAPI, destCRS: SpatialReference = MosaicGDAL.WSG84): MosaicGeometry = { val gt = getGeoTransform - val sourceCRS = spatialRef + val sourceCRS = getSpatialReference val transform = new osr.CoordinateTransformation(sourceCRS, destCRS) val bbox = geometryAPI.geometry( @@ -300,23 +311,10 @@ case class MosaicRasterGDAL( * compute since it requires reading the raster and computing statistics. */ def isEmpty: Boolean = { - import org.json4s._ - import org.json4s.jackson.JsonMethods._ - implicit val formats: DefaultFormats.type = org.json4s.DefaultFormats - - val vector = new JVector[String]() - vector.add("-stats") - vector.add("-json") - val infoOptions = new InfoOptions(vector) - val gdalInfo = GDALInfo(raster, infoOptions) - val json = parse(gdalInfo).extract[Map[String, Any]] - - if (json.contains("STATISTICS_VALID_PERCENT")) { - json("STATISTICS_VALID_PERCENT").asInstanceOf[Double] == 0.0 - } else if (subdatasets.nonEmpty) { + if (subdatasets.nonEmpty) { false } else { - getBandStats.values.map(_.getOrElse("mean", 0.0)).forall(_ == 0.0) + getValidCount.values.sum == 0 } } @@ -347,11 +345,18 @@ case class MosaicRasterGDAL( val isSubdataset = PathUtils.isSubdataset(path) val filePath = if (isSubdataset) PathUtils.fromSubdatasetPath(path) else path val pamFilePath = s"$filePath.aux.xml" + val cleanPath = filePath.replace("/vsizip/", "") + val zipPath = if (cleanPath.endsWith("zip")) cleanPath else s"$cleanPath.zip" if (path != PathUtils.getCleanPath(parentPath)) { Try(gdal.GetDriverByName(driverShortName).Delete(path)) + Try(Files.deleteIfExists(Paths.get(cleanPath))) Try(Files.deleteIfExists(Paths.get(path))) Try(Files.deleteIfExists(Paths.get(filePath))) Try(Files.deleteIfExists(Paths.get(pamFilePath))) + if (Files.exists(Paths.get(zipPath))) { + Try(Files.deleteIfExists(Paths.get(zipPath.replace(".zip", "")))) + } + Try(Files.deleteIfExists(Paths.get(zipPath))) } } @@ -382,12 +387,26 @@ case class MosaicRasterGDAL( * A boolean indicating if the write was successful. */ def writeToPath(path: String, dispose: Boolean = true): String = { - val driver = raster.GetDriver() - val ds = driver.CreateCopy(path, this.flushCache().getRaster) - ds.FlushCache() - ds.delete() - if (dispose) RasterCleaner.dispose(this) - path + if (isSubDataset) { + val driver = raster.GetDriver() + val ds = driver.CreateCopy(path, this.flushCache().getRaster, 1) + if (ds == null) { + val error = gdal.GetLastErrorMsg() + throw new Exception(s"Error writing raster to path: $error") + } + ds.FlushCache() + ds.delete() + if (dispose) RasterCleaner.dispose(this) + path + } else { + val thisPath = Paths.get(this.path) + val fromDir = thisPath.getParent + val toDir = Paths.get(path).getParent + val stemRegex = PathUtils.getStemRegex(this.path) + PathUtils.wildcardCopy(fromDir.toString, toDir.toString, stemRegex) + if (dispose) RasterCleaner.dispose(this) + s"$toDir/${thisPath.getFileName}" + } } /** @@ -398,17 +417,33 @@ case class MosaicRasterGDAL( */ def writeToBytes(dispose: Boolean = true): Array[Byte] = { val isSubdataset = PathUtils.isSubdataset(path) - val readPath = - if (isSubdataset) { - val tmpPath = PathUtils.createTmpFilePath(getRasterFileExtension) - writeToPath(tmpPath, dispose = false) + val readPath = { + val tmpPath = + if (isSubdataset) { + val tmpPath = PathUtils.createTmpFilePath(getRasterFileExtension) + writeToPath(tmpPath, dispose = false) + tmpPath + } else { + path + } + if (Files.isDirectory(Paths.get(tmpPath))) { + SysUtils.runCommand(s"zip -r0 $tmpPath.zip $tmpPath") + s"$tmpPath.zip" } else { - path + tmpPath } + } val byteArray = FileUtils.readBytes(readPath) if (dispose) RasterCleaner.dispose(this) if (readPath != PathUtils.getCleanPath(parentPath)) { Files.deleteIfExists(Paths.get(readPath)) + if (readPath.endsWith(".zip")) { + val nonZipPath = readPath.replace(".zip", "") + if (Files.isDirectory(Paths.get(nonZipPath))) { + SysUtils.runCommand(s"rm -rf $nonZipPath") + } + Files.deleteIfExists(Paths.get(readPath.replace(".zip", ""))) + } } byteArray } @@ -464,6 +499,20 @@ case class MosaicRasterGDAL( .toMap } + /** + * @return + * Returns the raster's band valid pixel count. + */ + def getValidCount: Map[Int, Long] = { + (1 to numBands) + .map(i => { + val band = raster.GetRasterBand(i) + val validCount = band.AsMDArray().GetStatistics().getValid_count + i -> validCount + }) + .toMap + } + /** * @param subsetName * The name of the subdataset to get. @@ -471,24 +520,59 @@ case class MosaicRasterGDAL( * Returns the raster's subdataset with given name. */ def getSubdataset(subsetName: String): MosaicRasterGDAL = { - subdatasets - val path = Option(raster.GetMetadata_Dict("SUBDATASETS")) - .map(_.asScala.toMap.asInstanceOf[Map[String, String]]) - .getOrElse(Map.empty[String, String]) - .values - .find(_.toUpperCase(Locale.ROOT).endsWith(subsetName.toUpperCase(Locale.ROOT))) - .getOrElse(throw new Exception(s""" - |Subdataset $subsetName not found! - |Available subdatasets: - | ${subdatasets.keys.filterNot(_.startsWith("SUBDATASET_")).mkString(", ")} - """.stripMargin)) - val ds = openRaster(path) + val path = subdatasets.getOrElse( + s"${subsetName}_tmp", + throw new Exception(s""" + |Subdataset $subsetName not found! + |Available subdatasets: + | ${subdatasets.keys.filterNot(_.startsWith("SUBDATASET_")).mkString(", ")} + | """.stripMargin) + ) + val sanitized = PathUtils.getCleanPath(path) + val subdatasetPath = PathUtils.getSubdatasetPath(sanitized) + + val ds = openRaster(subdatasetPath) // Avoid costly IO to compute MEM size here // It will be available when the raster is serialized for next operation // If value is needed then it will be computed when getMemSize is called MosaicRasterGDAL(ds, path, parentPath, driverShortName, -1) } + def convolve(kernel: Array[Array[Double]]): MosaicRasterGDAL = { + val resultRasterPath = PathUtils.createTmpFilePath(getRasterFileExtension) + val outputRaster = this.raster + .GetDriver() + .Create(resultRasterPath, this.xSize, this.ySize, this.numBands, this.raster.GetRasterBand(1).getDataType) + + for (bandIndex <- 1 to this.numBands) { + val band = this.getBand(bandIndex) + band.convolve(kernel) + } + + MosaicRasterGDAL(outputRaster, resultRasterPath, parentPath, driverShortName, -1) + + } + + def filter(kernelSize: Int, operation: String): MosaicRasterGDAL = { + val resultRasterPath = PathUtils.createTmpFilePath(getRasterFileExtension) + + this.raster + .GetDriver() + .CreateCopy(resultRasterPath, this.raster, 1) + .delete() + + val outputRaster = gdal.Open(resultRasterPath, GF_Write) + + for (bandIndex <- 1 to this.numBands) { + val band = this.getBand(bandIndex) + val outputBand = outputRaster.GetRasterBand(bandIndex) + band.filter(kernelSize, operation, outputBand) + } + + val result = MosaicRasterGDAL(outputRaster, resultRasterPath, parentPath, driverShortName, this.memSize) + result.flushCache() + } + } //noinspection ZeroIndexToHead @@ -583,11 +667,29 @@ object MosaicRasterGDAL extends RasterReader { // Try reading as a tmp file, if that fails, rename as a zipped file val dataset = openRaster(tmpPath, Some(driverShortName)) if (dataset == null) { - val zippedPath = PathUtils.createTmpFilePath("zip") + val zippedPath = s"$tmpPath.zip" Files.move(Paths.get(tmpPath), Paths.get(zippedPath), StandardCopyOption.REPLACE_EXISTING) val readPath = PathUtils.getZipPath(zippedPath) val ds = openRaster(readPath, Some(driverShortName)) - MosaicRasterGDAL(ds, readPath, parentPath, driverShortName, contentBytes.length) + if (ds == null) { + // the way we zip using uuid is not compatible with GDAL + // we need to unzip and read the file if it was zipped by us + val parentDir = Paths.get(zippedPath).getParent + val prompt = SysUtils.runScript(Array("/bin/sh", "-c", s"cd $parentDir && unzip -o $zippedPath -d /")) + // zipped files will have the old uuid name of the raster + // we need to get the last extracted file name, but the last extracted file name is not the raster name + // we can't list folders due to concurrent writes + val extension = GDAL.getExtension(driverShortName) + val lastExtracted = SysUtils.getLastOutputLine(prompt) + val unzippedPath = PathUtils.parseUnzippedPathFromExtracted(lastExtracted, extension) + val dataset = openRaster(unzippedPath, Some(driverShortName)) + if (dataset == null) { + throw new Exception(s"Error reading raster from bytes: ${prompt._3}") + } + MosaicRasterGDAL(dataset, unzippedPath, parentPath, driverShortName, contentBytes.length) + } else { + MosaicRasterGDAL(ds, readPath, parentPath, driverShortName, contentBytes.length) + } } else { MosaicRasterGDAL(dataset, tmpPath, parentPath, driverShortName, contentBytes.length) } diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterWriteOptions.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterWriteOptions.scala new file mode 100644 index 000000000..68a7bd75a --- /dev/null +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterWriteOptions.scala @@ -0,0 +1,55 @@ +package com.databricks.labs.mosaic.core.raster.gdal + +import com.databricks.labs.mosaic.gdal.MosaicGDAL +import org.gdal.osr.SpatialReference + +case class MosaicRasterWriteOptions( + compression: String = "DEFLATE", + format: String = "GTiff", + extension: String = "tif", + resampling: String = "nearest", + crs: SpatialReference = MosaicGDAL.WSG84, // Assume WGS84 + pixelSize: Option[(Double, Double)] = None, + noDataValue: Option[Double] = None, + missingGeoRef: Boolean = false, + options: Map[String, String] = Map.empty[String, String] +) + +object MosaicRasterWriteOptions { + + val VRT: MosaicRasterWriteOptions = + MosaicRasterWriteOptions( + compression = "NONE", + format = "VRT", + extension = "vrt", + crs = MosaicGDAL.WSG84, + pixelSize = None, + noDataValue = None, + options = Map.empty[String, String] + ) + + val GTiff: MosaicRasterWriteOptions = MosaicRasterWriteOptions() + + def noGPCsNoTransform(raster: MosaicRasterGDAL): Boolean = { + val noGPCs = raster.getRaster.GetGCPCount == 0 + val noGeoTransform = raster.getRaster.GetGeoTransform == null || + (raster.getRaster.GetGeoTransform sameElements Array(0.0, 1.0, 0.0, 0.0, 0.0, 1.0)) + noGPCs && noGeoTransform + } + + def apply(): MosaicRasterWriteOptions = new MosaicRasterWriteOptions() + + def apply(raster: MosaicRasterGDAL): MosaicRasterWriteOptions = { + val compression = raster.getCompression + val format = raster.getRaster.GetDriver.getShortName + val extension = raster.getRasterFileExtension + val resampling = "nearest" + val pixelSize = None + val noDataValue = None + val options = Map.empty[String, String] + val crs = raster.getSpatialReference + val missingGeoRef = noGPCsNoTransform(raster) + new MosaicRasterWriteOptions(compression, format, extension, resampling, crs, pixelSize, noDataValue, missingGeoRef, options) + } + +} diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/Padding.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/Padding.scala new file mode 100644 index 000000000..bb32e772f --- /dev/null +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/Padding.scala @@ -0,0 +1,58 @@ +package com.databricks.labs.mosaic.core.raster.gdal + +case class Padding( + left: Boolean, + right: Boolean, + top: Boolean, + bottom: Boolean +) { + + def removePadding(array: Array[Double], rowWidth: Int, stride: Int): Array[Double] = { + val l = if (left) 1 else 0 + val r = if (right) 1 else 0 + val t = if (top) 1 else 0 + val b = if (bottom) 1 else 0 + + val yStart = t * stride * rowWidth + val yEnd = array.length - b * stride * rowWidth + + val slices = for (i <- yStart until yEnd by rowWidth) yield { + val xStart = i + l * stride + val xEnd = i + rowWidth - r * stride + array.slice(xStart, xEnd) + } + + slices.flatten.toArray + } + + def horizontalStrides: Int = { + if (left && right) 2 + else if (left || right) 1 + else 0 + } + + def verticalStrides: Int = { + if (top && bottom) 2 + else if (top || bottom) 1 + else 0 + } + + def newOffset(xOffset: Int, yOffset: Int, stride: Int): (Int, Int) = { + val x = if (left) xOffset + stride else xOffset + val y = if (top) yOffset + stride else yOffset + (x, y) + } + + def newSize(width: Int, height: Int, stride: Int): (Int, Int) = { + val w = if (left && right) width - 2 * stride else if (left || right) width - stride else width + val h = if (top && bottom) height - 2 * stride else if (top || bottom) height - stride else height + (w, h) + } + +} + +object Padding { + + val NoPadding: Padding = Padding(left = false, right = false, top = false, bottom = false) + +} diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/clip/RasterClipByVector.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/clip/RasterClipByVector.scala index 6daabc25c..56c29563f 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/clip/RasterClipByVector.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/clip/RasterClipByVector.scala @@ -19,8 +19,7 @@ object RasterClipByVector { * abstractions over GDAL Warp. It uses CUTLINE_ALL_TOUCHED=TRUE to ensure * that all pixels that touch the geometry are included. This will avoid * the issue of having a pixel that is half in and half out of the - * geometry, important for tessellation. It also uses COMPRESS=DEFLATE to - * ensure that the output is compressed. The method also uses the geometry + * geometry, important for tessellation. The method also uses the geometry * API to generate a shapefile that is used to clip the raster. The * shapefile is deleted after the clip is complete. * @@ -38,16 +37,19 @@ object RasterClipByVector { def clip(raster: MosaicRasterGDAL, geometry: MosaicGeometry, geomCRS: SpatialReference, geometryAPI: GeometryAPI): MosaicRasterGDAL = { val rasterCRS = raster.getSpatialReference val outShortName = raster.getDriversShortName - val geomSrcCRS = if (geomCRS == null ) rasterCRS else geomCRS + val geomSrcCRS = if (geomCRS == null) rasterCRS else geomCRS val resultFileName = PathUtils.createTmpFilePath(GDAL.getExtension(outShortName)) val shapeFileName = VectorClipper.generateClipper(geometry, geomSrcCRS, rasterCRS, geometryAPI) + // For -wo consult https://gdal.org/doxygen/structGDALWarpOptions.html + // SOURCE_EXTRA=3 is used to ensure that when the raster is clipped, the + // pixels that touch the geometry are included. The default is 1, 3 seems to be a good empirical value. val result = GDALWarp.executeWarp( resultFileName, Seq(raster), - command = s"gdalwarp -wo CUTLINE_ALL_TOUCHED=TRUE -of $outShortName -cutline $shapeFileName -crop_to_cutline -co COMPRESS=DEFLATE -dstalpha" + command = s"gdalwarp -wo CUTLINE_ALL_TOUCHED=TRUE -wo SOURCE_EXTRA=3 -cutline $shapeFileName -crop_to_cutline" ) VectorClipper.cleanUpClipper(shapeFileName) diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALBuildVRT.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALBuildVRT.scala index 389defad6..9e1e97401 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALBuildVRT.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALBuildVRT.scala @@ -1,6 +1,6 @@ package com.databricks.labs.mosaic.core.raster.operator.gdal -import com.databricks.labs.mosaic.core.raster.gdal.MosaicRasterGDAL +import com.databricks.labs.mosaic.core.raster.gdal.{MosaicRasterGDAL, MosaicRasterWriteOptions} import org.gdal.gdal.{BuildVRTOptions, gdal} /** GDALBuildVRT is a wrapper for the GDAL BuildVRT command. */ @@ -20,16 +20,16 @@ object GDALBuildVRT { */ def executeVRT(outputPath: String, rasters: Seq[MosaicRasterGDAL], command: String): MosaicRasterGDAL = { require(command.startsWith("gdalbuildvrt"), "Not a valid GDAL Build VRT command.") - val vrtOptionsVec = OperatorOptions.parseOptions(command) + val effectiveCommand = OperatorOptions.appendOptions(command, MosaicRasterWriteOptions.VRT) + val vrtOptionsVec = OperatorOptions.parseOptions(effectiveCommand) val vrtOptions = new BuildVRTOptions(vrtOptionsVec) val result = gdal.BuildVRT(outputPath, rasters.map(_.getRaster).toArray, vrtOptions) if (result == null) { - throw new Exception( - s""" - |Build VRT failed. - |Command: $command - |Error: ${gdal.GetLastErrorMsg} - |""".stripMargin) + throw new Exception(s""" + |Build VRT failed. + |Command: $effectiveCommand + |Error: ${gdal.GetLastErrorMsg} + |""".stripMargin) } // TODO: Figure out multiple parents, should this be an array? // VRT files are just meta files, mem size doesnt make much sense so we keep -1 diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALCalc.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALCalc.scala index 97a273d13..cc9c5e500 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALCalc.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALCalc.scala @@ -1,19 +1,16 @@ package com.databricks.labs.mosaic.core.raster.operator.gdal import com.databricks.labs.mosaic.core.raster.api.GDAL -import com.databricks.labs.mosaic.core.raster.gdal.MosaicRasterGDAL +import com.databricks.labs.mosaic.core.raster.gdal.{MosaicRasterGDAL, MosaicRasterWriteOptions} import com.databricks.labs.mosaic.utils.SysUtils /** GDALCalc is a helper object for executing GDAL Calc commands. */ object GDALCalc { val gdal_calc: String = { - val calcPath = SysUtils.runCommand("find / -iname gdal_calc.py")._1.split("\n").headOption.getOrElse("") - if (calcPath.isEmpty) { - throw new RuntimeException("Could not find gdal_calc.py.") - } - if (calcPath == "ERROR") { - "/usr/lib/python3/dist-packages/osgeo_utils/gdal_calc.py" + val calcPath = SysUtils.runCommand("""find / -maxdepth 20 -iname gdal_calc.py""")._1.split("\n").headOption.getOrElse("") + if (calcPath.isEmpty || calcPath.startsWith("ERROR")) { + "/usr/local/lib/python3.10/dist-packages/osgeo_utils/gdal_calc.py" } else { calcPath } @@ -30,11 +27,13 @@ object GDALCalc { */ def executeCalc(gdalCalcCommand: String, resultPath: String): MosaicRasterGDAL = { require(gdalCalcCommand.startsWith("gdal_calc"), "Not a valid GDAL Calc command.") - val toRun = gdalCalcCommand.replace("gdal_calc", gdal_calc) + val effectiveCommand = OperatorOptions.appendOptions(gdalCalcCommand, MosaicRasterWriteOptions.GTiff) + val toRun = effectiveCommand.replace("gdal_calc", gdal_calc) val commandRes = SysUtils.runCommand(s"python3 $toRun") - if (commandRes._1 == "ERROR") { + if (commandRes._1.startsWith("ERROR")) { throw new RuntimeException(s""" |GDAL Calc command failed: + |$toRun |STDOUT: |${commandRes._2} |STDERR: diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALTranslate.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALTranslate.scala index bf266cfbf..fd24a0f73 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALTranslate.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALTranslate.scala @@ -1,6 +1,6 @@ package com.databricks.labs.mosaic.core.raster.operator.gdal -import com.databricks.labs.mosaic.core.raster.gdal.MosaicRasterGDAL +import com.databricks.labs.mosaic.core.raster.gdal.{MosaicRasterGDAL, MosaicRasterWriteOptions} import org.gdal.gdal.{TranslateOptions, gdal} import java.nio.file.{Files, Paths} @@ -20,21 +20,26 @@ object GDALTranslate { * @return * A MosaicRaster object. */ - def executeTranslate(outputPath: String, raster: MosaicRasterGDAL, command: String): MosaicRasterGDAL = { + def executeTranslate( + outputPath: String, + raster: MosaicRasterGDAL, + command: String, + writeOptions: MosaicRasterWriteOptions + ): MosaicRasterGDAL = { require(command.startsWith("gdal_translate"), "Not a valid GDAL Translate command.") - val translateOptionsVec = OperatorOptions.parseOptions(command) + val effectiveCommand = OperatorOptions.appendOptions(command, writeOptions) + val translateOptionsVec = OperatorOptions.parseOptions(effectiveCommand) val translateOptions = new TranslateOptions(translateOptionsVec) val result = gdal.Translate(outputPath, raster.getRaster, translateOptions) if (result == null) { - throw new Exception( - s""" - |Translate failed. - |Command: $command - |Error: ${gdal.GetLastErrorMsg} - |""".stripMargin) + throw new Exception(s""" + |Translate failed. + |Command: $effectiveCommand + |Error: ${gdal.GetLastErrorMsg} + |""".stripMargin) } val size = Files.size(Paths.get(outputPath)) - raster.copy(raster = result, path = outputPath, memSize = size).flushCache() + raster.copy(raster = result, path = outputPath, memSize = size, driverShortName = writeOptions.format).flushCache() } } diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALWarp.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALWarp.scala index 2b13a957b..ba6dce58d 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALWarp.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALWarp.scala @@ -23,7 +23,8 @@ object GDALWarp { def executeWarp(outputPath: String, rasters: Seq[MosaicRasterGDAL], command: String): MosaicRasterGDAL = { require(command.startsWith("gdalwarp"), "Not a valid GDAL Warp command.") // Test: gdal.ParseCommandLine(command) - val warpOptionsVec = OperatorOptions.parseOptions(command) + val effectiveCommand = OperatorOptions.appendOptions(command, rasters.head.getWriteOptions) + val warpOptionsVec = OperatorOptions.parseOptions(effectiveCommand) val warpOptions = new WarpOptions(warpOptionsVec) val result = gdal.Warp(outputPath, rasters.map(_.getRaster).toArray, warpOptions) // TODO: Figure out multiple parents, should this be an array? @@ -31,7 +32,7 @@ object GDALWarp { if (result == null) { throw new Exception(s""" |Warp failed. - |Command: $command + |Command: $effectiveCommand |Error: ${gdal.GetLastErrorMsg} |""".stripMargin) } diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/OperatorOptions.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/OperatorOptions.scala index b1529d3e7..bc656ec01 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/OperatorOptions.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/OperatorOptions.scala @@ -1,5 +1,7 @@ package com.databricks.labs.mosaic.core.raster.operator.gdal +import com.databricks.labs.mosaic.core.raster.gdal.MosaicRasterWriteOptions + /** OperatorOptions is a helper object for parsing GDAL command options. */ object OperatorOptions { @@ -18,4 +20,37 @@ object OperatorOptions { optionsVec } + /** + * Add default options to the command. Extract the compression from the + * raster and append it to the command. This operation does not change the + * output format. For changing the output format, use RST_ToFormat. + * + * @param command + * The command to append options to. + * @param writeOptions + * The write options to append. Note that not all available options are + * actually appended. At this point it is up to the bellow logic to + * decide what is supported and for which format. + * @return + */ + def appendOptions(command: String, writeOptions: MosaicRasterWriteOptions): String = { + val compression = writeOptions.compression + if (command.startsWith("gdal_calc")) { + writeOptions.format match { + case f @ "GTiff" => command + s" --format $f --co TILED=YES --co COMPRESS=$compression" + case f @ "COG" => command + s" --format $f --co TILED=YES --co COMPRESS=$compression" + case f @ _ => command + s" --format $f --co COMPRESS=$compression" + } + } else { + writeOptions.format match { + case f @ "GTiff" => command + s" -of $f -co TILED=YES -co COMPRESS=$compression" + case f @ "COG" => command + s" -of $f -co TILED=YES -co COMPRESS=$compression" + case "VRT" => command + case f @ "Zarr" if writeOptions.missingGeoRef => + command + s" -of $f -co COMPRESS=$compression -to SRC_METHOD=NO_GEOTRANSFORM" + case f @ _ => command + s" -of $f -co COMPRESS=$compression" + } + } + } + } diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/merge/MergeBands.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/merge/MergeBands.scala index 6333c50c8..8a82d1238 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/merge/MergeBands.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/merge/MergeBands.scala @@ -19,10 +19,10 @@ object MergeBands { * A MosaicRaster object. */ def merge(rasters: Seq[MosaicRasterGDAL], resampling: String): MosaicRasterGDAL = { - val outShortName = rasters.head.getRaster.GetDriver.getShortName + val outOptions = rasters.head.getWriteOptions val vrtPath = PathUtils.createTmpFilePath("vrt") - val rasterPath = PathUtils.createTmpFilePath("tif") + val rasterPath = PathUtils.createTmpFilePath(outOptions.extension) val vrtRaster = GDALBuildVRT.executeVRT( vrtPath, @@ -33,7 +33,8 @@ object MergeBands { val result = GDALTranslate.executeTranslate( rasterPath, vrtRaster, - command = s"gdal_translate -r $resampling -of $outShortName -co COMPRESS=DEFLATE" + command = s"gdal_translate -r $resampling", + outOptions ) dispose(vrtRaster) @@ -55,10 +56,10 @@ object MergeBands { * A MosaicRaster object. */ def merge(rasters: Seq[MosaicRasterGDAL], pixel: (Double, Double), resampling: String): MosaicRasterGDAL = { - val outShortName = rasters.head.getRaster.GetDriver.getShortName + val outOptions = rasters.head.getWriteOptions val vrtPath = PathUtils.createTmpFilePath("vrt") - val rasterPath = PathUtils.createTmpFilePath("tif") + val rasterPath = PathUtils.createTmpFilePath(outOptions.extension) val vrtRaster = GDALBuildVRT.executeVRT( vrtPath, @@ -69,7 +70,8 @@ object MergeBands { val result = GDALTranslate.executeTranslate( rasterPath, vrtRaster, - command = s"gdalwarp -r $resampling -of $outShortName -co COMPRESS=DEFLATE -overwrite" + command = s"gdalwarp -r $resampling", + outOptions ) dispose(vrtRaster) diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/merge/MergeRasters.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/merge/MergeRasters.scala index 694d9940a..fafaffbc4 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/merge/MergeRasters.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/merge/MergeRasters.scala @@ -17,21 +17,22 @@ object MergeRasters { * A MosaicRaster object. */ def merge(rasters: Seq[MosaicRasterGDAL]): MosaicRasterGDAL = { - val outShortName = rasters.head.getRaster.GetDriver.getShortName + val outOptions = rasters.head.getWriteOptions val vrtPath = PathUtils.createTmpFilePath("vrt") - val rasterPath = PathUtils.createTmpFilePath("tif") + val rasterPath = PathUtils.createTmpFilePath(outOptions.extension) val vrtRaster = GDALBuildVRT.executeVRT( - vrtPath, - rasters, - command = s"gdalbuildvrt -resolution highest" + vrtPath, + rasters, + command = s"gdalbuildvrt -resolution highest" ) val result = GDALTranslate.executeTranslate( - rasterPath, - vrtRaster, - command = s"gdal_translate -r bilinear -of $outShortName -co COMPRESS=DEFLATE" + rasterPath, + vrtRaster, + command = s"gdal_translate", + outOptions ) dispose(vrtRaster) @@ -39,5 +40,4 @@ object MergeRasters { result } - } diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/pixel/PixelCombineRasters.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/pixel/PixelCombineRasters.scala index 5bf49fb96..cda9824dc 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/pixel/PixelCombineRasters.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/pixel/PixelCombineRasters.scala @@ -3,6 +3,7 @@ package com.databricks.labs.mosaic.core.raster.operator.pixel import com.databricks.labs.mosaic.core.raster.gdal.MosaicRasterGDAL import com.databricks.labs.mosaic.core.raster.io.RasterCleaner.dispose import com.databricks.labs.mosaic.core.raster.operator.gdal.{GDALBuildVRT, GDALTranslate} +import com.databricks.labs.mosaic.gdal.MosaicGDAL.defaultBlockSize import com.databricks.labs.mosaic.utils.PathUtils import java.io.File @@ -20,10 +21,10 @@ object PixelCombineRasters { * A MosaicRaster object. */ def combine(rasters: Seq[MosaicRasterGDAL], pythonFunc: String, pythonFuncName: String): MosaicRasterGDAL = { - val outShortName = rasters.head.getRaster.GetDriver.getShortName + val outOptions = rasters.head.getWriteOptions val vrtPath = PathUtils.createTmpFilePath("vrt") - val rasterPath = PathUtils.createTmpFilePath("tif") + val rasterPath = PathUtils.createTmpFilePath(outOptions.extension) val vrtRaster = GDALBuildVRT.executeVRT( vrtPath, @@ -37,7 +38,8 @@ object PixelCombineRasters { val result = GDALTranslate.executeTranslate( rasterPath, vrtRaster.refresh(), - command = s"gdal_translate -r bilinear -of $outShortName -co COMPRESS=DEFLATE" + command = s"gdal_translate", + outOptions ) dispose(vrtRaster) diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/proj/RasterProject.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/proj/RasterProject.scala index efd7c8c67..5d7c5f5f2 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/proj/RasterProject.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/proj/RasterProject.scala @@ -15,8 +15,7 @@ object RasterProject { /** * Projects a raster to a new CRS. The method handles all the abstractions * over GDAL Warp. It uses cubic resampling to ensure that the output is - * smooth. It also uses COMPRESS=DEFLATE to ensure that the output is - * compressed. + * smooth. * * @param raster * The raster to project. @@ -33,11 +32,11 @@ object RasterProject { // Note that Null is the right value here val authName = destCRS.GetAuthorityName(null) val authCode = destCRS.GetAuthorityCode(null) - + val result = GDALWarp.executeWarp( resultFileName, Seq(raster), - command = s"gdalwarp -of $outShortName -t_srs $authName:$authCode -r cubic -overwrite -co COMPRESS=DEFLATE" + command = s"gdalwarp -t_srs $authName:$authCode" ) result diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/OverlappingTiles.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/OverlappingTiles.scala index c1498ea05..4e9f61c5e 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/OverlappingTiles.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/OverlappingTiles.scala @@ -48,12 +48,13 @@ object OverlappingTiles { val fileExtension = GDAL.getExtension(tile.getDriver) val rasterPath = PathUtils.createTmpFilePath(fileExtension) - val shortName = raster.getRaster.GetDriver.getShortName + val outOptions = raster.getWriteOptions val result = GDALTranslate.executeTranslate( rasterPath, raster, - command = s"gdal_translate -of $shortName -srcwin $xOff $yOff $width $height" + command = s"gdal_translate -srcwin $xOff $yOff $width $height", + outOptions ) val isEmpty = result.isEmpty diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/RasterTessellate.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/RasterTessellate.scala index d186de0a5..fa47c6c1d 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/RasterTessellate.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/RasterTessellate.scala @@ -41,9 +41,10 @@ object RasterTessellate { (false, MosaicRasterTile(cell.index, null, "", "")) } else { val cellRaster = tmpRaster.getRasterForCell(cellID, indexSystem, geometryAPI) - val isValidRaster = cellRaster.getBandStats.values.map(_("mean")).sum > 0 && !cellRaster.isEmpty + val isValidRaster = cellRaster.getValidCount.values.sum > 0 && !cellRaster.isEmpty ( - isValidRaster, MosaicRasterTile(cell.index, cellRaster, raster.getParentPath, raster.getDriversShortName) + isValidRaster, + MosaicRasterTile(cell.index, cellRaster, raster.getParentPath, raster.getDriversShortName) ) } }) diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/ReTile.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/ReTile.scala index edaab4720..f25a1f384 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/ReTile.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/ReTile.scala @@ -1,7 +1,7 @@ package com.databricks.labs.mosaic.core.raster.operator.retile import com.databricks.labs.mosaic.core.raster.io.RasterCleaner.dispose -import com.databricks.labs.mosaic.core.raster.operator.gdal.{GDALBuildVRT, GDALTranslate} +import com.databricks.labs.mosaic.core.raster.operator.gdal.GDALTranslate import com.databricks.labs.mosaic.core.types.model.MosaicRasterTile import com.databricks.labs.mosaic.utils.PathUtils @@ -39,12 +39,13 @@ object ReTile { val fileExtension = raster.getRasterFileExtension val rasterPath = PathUtils.createTmpFilePath(fileExtension) - val shortDriver = raster.getDriversShortName + val outOptions = raster.getWriteOptions val result = GDALTranslate.executeTranslate( rasterPath, raster, - command = s"gdal_translate -of $shortDriver -srcwin $xMin $yMin $xOffset $yOffset -co COMPRESS=DEFLATE" + command = s"gdal_translate -srcwin $xMin $yMin $xOffset $yOffset", + outOptions ) val isEmpty = result.isEmpty diff --git a/src/main/scala/com/databricks/labs/mosaic/core/types/RasterTileType.scala b/src/main/scala/com/databricks/labs/mosaic/core/types/RasterTileType.scala index 1cadf2c9a..5203178e0 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/types/RasterTileType.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/types/RasterTileType.scala @@ -1,11 +1,12 @@ package com.databricks.labs.mosaic.core.types +import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.types._ /** Type definition for the raster tile. */ class RasterTileType(fields: Array[StructField]) extends StructType(fields) { - def rasterType: DataType = fields(1).dataType + def rasterType: DataType = fields.find(_.name == "raster").get.dataType override def simpleString: String = "RASTER_TILE" @@ -19,20 +20,54 @@ object RasterTileType { * Creates a new instance of [[RasterTileType]]. * * @param idType - * Type of the index ID. + * Type of the index ID. Can be one of [[LongType]], [[IntegerType]] or + * [[StringType]]. + * @param rasterType + * Type of the raster. Can be one of [[ByteType]] or [[StringType]]. Not + * to be confused with the data type of the raster. This is the type of + * the column that contains the raster. + * * @return * An instance of [[RasterTileType]]. */ - def apply(idType: DataType): RasterTileType = { + def apply(idType: DataType, rasterType: DataType): DataType = { require(Seq(LongType, IntegerType, StringType).contains(idType)) new RasterTileType( Array( StructField("index_id", idType), - StructField("raster", BinaryType), + StructField("raster", rasterType), StructField("parentPath", StringType), StructField("driver", StringType) ) ) } + /** + * Creates a new instance of [[RasterTileType]]. + * + * @param idType + * Type of the index ID. Can be one of [[LongType]], [[IntegerType]] or + * [[StringType]]. + * @param tileExpr + * Expression containing a tile. This is used to infer the raster type + * when chaining expressions. + * @return + */ + def apply(idType: DataType, tileExpr: Expression): DataType = { + require(Seq(LongType, IntegerType, StringType).contains(idType)) + tileExpr.dataType match { + case st @ StructType(_) => apply(idType, st.find(_.name == "raster").get.dataType) + case _ @ArrayType(elementType: StructType, _) => apply(idType, elementType.find(_.name == "raster").get.dataType) + case _ => throw new IllegalArgumentException("Unsupported raster type.") + } + } + + def apply(tileExpr: Expression): RasterTileType = { + tileExpr.dataType match { + case StructType(fields) => new RasterTileType(fields) + case ArrayType(elementType: StructType, _) => new RasterTileType(elementType.fields) + case _ => throw new IllegalArgumentException("Unsupported raster type.") + } + } + } diff --git a/src/main/scala/com/databricks/labs/mosaic/core/types/model/MosaicRasterTile.scala b/src/main/scala/com/databricks/labs/mosaic/core/types/model/MosaicRasterTile.scala index e7a8e9218..30a7765c1 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/types/model/MosaicRasterTile.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/types/model/MosaicRasterTile.scala @@ -106,19 +106,22 @@ case class MosaicRasterTile( * An instance of [[InternalRow]]. */ def serialize( - rasterDataType: DataType = BinaryType, - checkpointLocation: String = "" + rasterDataType: DataType ): InternalRow = { val parentPathUTF8 = UTF8String.fromString(parentPath) val driverUTF8 = UTF8String.fromString(driver) - val encodedRaster = encodeRaster(rasterDataType, checkpointLocation) + val encodedRaster = encodeRaster(rasterDataType) if (Option(index).isDefined) { if (index.isLeft) InternalRow.fromSeq( Seq(index.left.get, encodedRaster, parentPathUTF8, driverUTF8) ) - else InternalRow.fromSeq( - Seq(UTF8String.fromString(index.right.get), encodedRaster, parentPathUTF8, driverUTF8) - ) + else { + // Copy from tmp to checkpoint. + // Have to use GDAL Driver to do this since sidecar files are not copied by spark. + InternalRow.fromSeq( + Seq(UTF8String.fromString(index.right.get), encodedRaster, parentPathUTF8, driverUTF8) + ) + } } else { InternalRow.fromSeq(Seq(null, encodedRaster, parentPathUTF8, driverUTF8)) } @@ -132,10 +135,9 @@ case class MosaicRasterTile( * An instance of [[Array]] of [[Byte]] representing WKB. */ private def encodeRaster( - rasterDataType: DataType = BinaryType, - checkpointLocation: String = "" + rasterDataType: DataType = BinaryType ): Any = { - GDAL.writeRasters(Seq(raster), checkpointLocation, rasterDataType).head + GDAL.writeRasters(Seq(raster), rasterDataType).head } } @@ -153,12 +155,12 @@ object MosaicRasterTile { * @return * An instance of [[MosaicRasterTile]]. */ - def deserialize(row: InternalRow, idDataType: DataType): MosaicRasterTile = { + def deserialize(row: InternalRow, idDataType: DataType, rasterType: DataType): MosaicRasterTile = { val index = row.get(0, idDataType) - val rasterBytes = row.get(1, BinaryType) + val rawRaster = row.get(1, rasterType) val parentPath = row.get(2, StringType).toString val driver = row.get(3, StringType).toString - val raster = GDAL.readRaster(rasterBytes, parentPath, driver, BinaryType) + val raster = GDAL.readRaster(rawRaster, parentPath, driver, rasterType) // noinspection TypeCheckCanBeMatch if (Option(index).isDefined) { if (index.isInstanceOf[Long]) { diff --git a/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReTileOnRead.scala b/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReTileOnRead.scala index 285df2191..a38e76900 100644 --- a/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReTileOnRead.scala +++ b/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReTileOnRead.scala @@ -19,11 +19,17 @@ import java.nio.file.{Files, Paths} /** An object defining the retiling read strategy for the GDAL file format. */ object ReTileOnRead extends ReadStrategy { + val tileDataType: DataType = StringType + // noinspection DuplicatedCode /** * Returns the schema of the GDAL file format. * @note - * Different read strategies can have different schemas. + * Different read strategies can have different schemas. This is because + * the schema is defined by the read strategy. For retiling we always use + * checkpoint location. In this case rasters are stored off spark rows. + * If you need the tiles in memory please load them from path stored in + * the tile returned by the reader. * * @param options * Options passed to the reader. @@ -54,7 +60,10 @@ object ReTileOnRead extends ReadStrategy { .add(StructField(SUBDATASETS, MapType(StringType, StringType), nullable = false)) .add(StructField(SRID, IntegerType, nullable = false)) .add(StructField(LENGTH, LongType, nullable = false)) - .add(StructField(TILE, RasterTileType(indexSystem.getCellIdDataType), nullable = false)) + // Note that for retiling we always use checkpoint location. + // In this case rasters are stored off spark rows. + // If you need the tiles in memory please load them from path stored in the tile returned by the reader. + .add(StructField(TILE, RasterTileType(indexSystem.getCellIdDataType, tileDataType), nullable = false)) } /** @@ -103,7 +112,7 @@ object ReTileOnRead extends ReadStrategy { case other => throw new RuntimeException(s"Unsupported field name: $other") } // Writing to bytes is destructive so we delay reading content and content length until the last possible moment - val row = Utils.createRow(fields ++ Seq(tile.formatCellId(indexSystem).serialize())) + val row = Utils.createRow(fields ++ Seq(tile.formatCellId(indexSystem).serialize(tileDataType))) RasterCleaner.dispose(tile) row }) diff --git a/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReadInMemory.scala b/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReadInMemory.scala index 0517ac1d9..8c0c4a914 100644 --- a/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReadInMemory.scala +++ b/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReadInMemory.scala @@ -49,7 +49,9 @@ object ReadInMemory extends ReadStrategy { .add(StructField(METADATA, MapType(StringType, StringType), nullable = false)) .add(StructField(SUBDATASETS, MapType(StringType, StringType), nullable = false)) .add(StructField(SRID, IntegerType, nullable = false)) - .add(StructField(TILE, RasterTileType(indexSystem.getCellIdDataType), nullable = false)) + // Note, for in memory reads the rasters are stored in the tile. + // For that we use Binary Columns. + .add(StructField(TILE, RasterTileType(indexSystem.getCellIdDataType, BinaryType), nullable = false)) } /** diff --git a/src/main/scala/com/databricks/labs/mosaic/datasource/multiread/RasterAsGridReader.scala b/src/main/scala/com/databricks/labs/mosaic/datasource/multiread/RasterAsGridReader.scala index d6f26caf4..2f5bf39b6 100644 --- a/src/main/scala/com/databricks/labs/mosaic/datasource/multiread/RasterAsGridReader.scala +++ b/src/main/scala/com/databricks/labs/mosaic/datasource/multiread/RasterAsGridReader.scala @@ -65,36 +65,32 @@ class RasterAsGridReader(sparkSession: SparkSession) extends MosaicDataFrameRead val retiledDf = retileRaster(rasterDf, config) - val loadedDf = rasterDf + val loadedDf = retiledDf .withColumn( "tile", - rst_tessellate(col("tile"), lit(resolution)) + rst_tessellate(col("tile"), lit(resolution)) ) .repartition(nPartitions) + .groupBy("tile.index_id") + .agg(rst_combineavg_agg(col("tile")).alias("tile")) .withColumn( "grid_measures", - rasterToGridCombiner(col("tile"), lit(resolution)) + rasterToGridCombiner(col("tile")) ) .select( "grid_measures", "tile" ) .select( - posexplode(col("grid_measures")).as(Seq("band_id", "grid_measures")) + posexplode(col("grid_measures")).as(Seq("band_id", "measure")), + col("tile").getField("index_id").alias("cell_id") ) .repartition(nPartitions) .select( col("band_id"), - explode(col("grid_measures")).alias("grid_measures") + col("cell_id"), + col("measure") ) - .repartition(nPartitions) - .select( - col("band_id"), - col("grid_measures").getItem("cellID").alias("cell_id"), - col("grid_measures").getItem("measure").alias("measure") - ) - .groupBy("band_id", "cell_id") - .agg(avg("measure").alias("measure")) kRingResample(loadedDf, config) @@ -203,15 +199,15 @@ class RasterAsGridReader(sparkSession: SparkSession) extends MosaicDataFrameRead * @return * The raster to grid function. */ - private def getRasterToGridFunc(combiner: String): (Column, Column) => Column = { + private def getRasterToGridFunc(combiner: String): Column => Column = { combiner match { - case "mean" => rst_rastertogridavg - case "min" => rst_rastertogridmin - case "max" => rst_rastertogridmax - case "median" => rst_rastertogridmedian - case "count" => rst_rastertogridcount - case "average" => rst_rastertogridavg - case "avg" => rst_rastertogridavg + case "mean" => rst_avg + case "min" => rst_min + case "max" => rst_max + case "median" => rst_median + case "count" => rst_pixelcount + case "average" => rst_avg + case "avg" => rst_avg case _ => throw new Error("Combiner not supported") } } @@ -232,7 +228,7 @@ class RasterAsGridReader(sparkSession: SparkSession) extends MosaicDataFrameRead "combiner" -> this.extraOptions.getOrElse("combiner", "mean"), "retile" -> this.extraOptions.getOrElse("retile", "false"), "tileSize" -> this.extraOptions.getOrElse("tileSize", "-1"), - "sizeInMB" -> this.extraOptions.getOrElse("sizeInMB", ""), + "sizeInMB" -> this.extraOptions.getOrElse("sizeInMB", "-1"), "kRingInterpolate" -> this.extraOptions.getOrElse("kRingInterpolate", "0") ) } diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Avg.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Avg.scala index 82752cad4..a5907dbe9 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Avg.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Avg.scala @@ -13,11 +13,13 @@ import org.apache.spark.sql.types._ /** Returns the upper left x of the raster. */ -case class RST_Avg(raster: Expression, expressionConfig: MosaicExpressionConfig) - extends RasterExpression[RST_Avg](raster, ArrayType(DoubleType), returnsRaster = false, expressionConfig) +case class RST_Avg(tileExpr: Expression, expressionConfig: MosaicExpressionConfig) + extends RasterExpression[RST_Avg](tileExpr, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = ArrayType(DoubleType) + /** Returns the upper left x of the raster. */ override def rasterTransform(tile: MosaicRasterTile): Any = { import org.json4s._ diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_BandMetaData.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_BandMetaData.scala index 241d913bc..fec760813 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_BandMetaData.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_BandMetaData.scala @@ -25,13 +25,14 @@ case class RST_BandMetaData(raster: Expression, band: Expression, expressionConf extends RasterBandExpression[RST_BandMetaData]( raster, band, - MapType(StringType, StringType), returnsRaster = false, expressionConfig = expressionConfig ) with NullIntolerant with CodegenFallback { + override def dataType: DataType = MapType(StringType, StringType) + /** * @param raster * The raster to be used. diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_BoundingBox.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_BoundingBox.scala index 8fa2d7314..e79a8ec40 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_BoundingBox.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_BoundingBox.scala @@ -15,10 +15,12 @@ import org.apache.spark.sql.types._ case class RST_BoundingBox( raster: Expression, expressionConfig: MosaicExpressionConfig -) extends RasterExpression[RST_BoundingBox](raster, BinaryType, returnsRaster = false, expressionConfig = expressionConfig) +) extends RasterExpression[RST_BoundingBox](raster, returnsRaster = false, expressionConfig = expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = BinaryType + /** * Computes the bounding box of the raster. The bbox is returned as a WKB * polygon. diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Clip.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Clip.scala index 557565afe..29449a6ef 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Clip.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Clip.scala @@ -19,13 +19,14 @@ case class RST_Clip( ) extends Raster1ArgExpression[RST_Clip]( rastersExpr, geometryExpr, - RasterTileType(expressionConfig.getCellIdType), returnsRaster = true, expressionConfig = expressionConfig ) with NullIntolerant with CodegenFallback { + override def dataType: org.apache.spark.sql.types.DataType = RasterTileType(expressionConfig.getCellIdType, rastersExpr) + val geometryAPI: GeometryAPI = GeometryAPI(expressionConfig.getGeometryAPI) /** diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_CombineAvg.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_CombineAvg.scala index 1d923fdc1..d63fe6914 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_CombineAvg.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_CombineAvg.scala @@ -9,20 +9,22 @@ import com.databricks.labs.mosaic.functions.MosaicExpressionConfig import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant} +import org.apache.spark.sql.types.DataType /** Expression for combining rasters using average of pixels. */ case class RST_CombineAvg( - rastersExpr: Expression, + tileExpr: Expression, expressionConfig: MosaicExpressionConfig ) extends RasterArrayExpression[RST_CombineAvg]( - rastersExpr, - RasterTileType(expressionConfig.getCellIdType), + tileExpr, returnsRaster = true, expressionConfig = expressionConfig ) with NullIntolerant with CodegenFallback { + override def dataType: DataType = RasterTileType(expressionConfig.getCellIdType, tileExpr) + /** Combines the rasters using average of pixels. */ override def rasterTransform(tiles: Seq[MosaicRasterTile]): Any = { val index = if (tiles.map(_.getIndex).groupBy(identity).size == 1) tiles.head.getIndex else null diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_CombineAvgAgg.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_CombineAvgAgg.scala index f6b3ba1dc..5bbc01a7b 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_CombineAvgAgg.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_CombineAvgAgg.scala @@ -6,6 +6,7 @@ import com.databricks.labs.mosaic.core.raster.io.RasterCleaner import com.databricks.labs.mosaic.core.raster.operator.CombineAVG import com.databricks.labs.mosaic.core.types.RasterTileType import com.databricks.labs.mosaic.core.types.model.MosaicRasterTile +import com.databricks.labs.mosaic.core.types.model.MosaicRasterTile.{deserialize => deserializeTile} import com.databricks.labs.mosaic.expressions.raster.base.RasterExpressionSerialization import com.databricks.labs.mosaic.functions.MosaicExpressionConfig import org.apache.spark.sql.catalyst.InternalRow @@ -13,7 +14,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{ImperativeAggregate, import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.trees.UnaryLike import org.apache.spark.sql.catalyst.util.GenericArrayData -import org.apache.spark.sql.types.{ArrayType, BinaryType, DataType} +import org.apache.spark.sql.types.{ArrayType, DataType} import scala.collection.mutable.ArrayBuffer @@ -23,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer */ //noinspection DuplicatedCode case class RST_CombineAvgAgg( - rasterExpr: Expression, + tileExpr: Expression, expressionConfig: MosaicExpressionConfig, mutableAggBufferOffset: Int = 0, inputAggBufferOffset: Int = 0 @@ -32,21 +33,23 @@ case class RST_CombineAvgAgg( with RasterExpressionSerialization { override lazy val deterministic: Boolean = true - override val child: Expression = rasterExpr + override val child: Expression = tileExpr override val nullable: Boolean = false - override val dataType: DataType = RasterTileType(expressionConfig.getCellIdType) + override lazy val dataType: DataType = RasterTileType(expressionConfig.getCellIdType, tileExpr) + lazy val tileType: DataType = dataType.asInstanceOf[RasterTileType].rasterType override def prettyName: String = "rst_combine_avg_agg" + val cellIDType: DataType = expressionConfig.getCellIdType private lazy val projection = UnsafeProjection.create(Array[DataType](ArrayType(elementType = dataType, containsNull = false))) private lazy val row = new UnsafeRow(1) - def update(buffer: ArrayBuffer[Any], input: InternalRow): ArrayBuffer[Any] = { + override def update(buffer: ArrayBuffer[Any], input: InternalRow): ArrayBuffer[Any] = { val value = child.eval(input) buffer += InternalRow.copyValue(value) buffer } - def merge(buffer: ArrayBuffer[Any], input: ArrayBuffer[Any]): ArrayBuffer[Any] = { + override def merge(buffer: ArrayBuffer[Any], input: ArrayBuffer[Any]): ArrayBuffer[Any] = { buffer ++= input } @@ -63,10 +66,15 @@ case class RST_CombineAvgAgg( if (buffer.isEmpty) { null + } else if (buffer.size == 1) { + val result = buffer.head + buffer.clear() + result } else { // Do do move the expression - var tiles = buffer.map(row => MosaicRasterTile.deserialize(row.asInstanceOf[InternalRow], expressionConfig.getCellIdType)) + var tiles = buffer.map(row => deserializeTile(row.asInstanceOf[InternalRow], cellIDType, tileType)) + buffer.clear() // If merging multiple index rasters, the index value is dropped val idx = if (tiles.map(_.getIndex).groupBy(identity).size == 1) tiles.head.getIndex else null @@ -77,9 +85,9 @@ case class RST_CombineAvgAgg( val result = MosaicRasterTile(idx, combined, parentPath, driver) .formatCellId(IndexSystemFactory.getIndexSystem(expressionConfig.getIndexSystem)) - .serialize(BinaryType, expressionConfig.getRasterCheckpoint) + .serialize(tileType) - tiles.foreach(RasterCleaner.dispose(_)) + tiles.foreach(RasterCleaner.dispose) RasterCleaner.dispose(result) tiles = null @@ -101,7 +109,7 @@ case class RST_CombineAvgAgg( buffer } - override protected def withNewChildInternal(newChild: Expression): RST_CombineAvgAgg = copy(rasterExpr = newChild) + override protected def withNewChildInternal(newChild: Expression): RST_CombineAvgAgg = copy(tileExpr = newChild) } diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Convolve.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Convolve.scala new file mode 100644 index 000000000..db20f8a3a --- /dev/null +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Convolve.scala @@ -0,0 +1,73 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.geometry.api.GeometryAPI +import com.databricks.labs.mosaic.core.types.RasterTileType +import com.databricks.labs.mosaic.core.types.model.MosaicRasterTile +import com.databricks.labs.mosaic.expressions.base.{GenericExpressionFactory, WithExpressionInfo} +import com.databricks.labs.mosaic.expressions.raster.base.Raster1ArgExpression +import com.databricks.labs.mosaic.functions.MosaicExpressionConfig +import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant} + +/** The expression for applying kernel filter on a raster. */ +case class RST_Convolve( + rastersExpr: Expression, + kernelExpr: Expression, + expressionConfig: MosaicExpressionConfig +) extends Raster1ArgExpression[RST_Convolve]( + rastersExpr, + kernelExpr, + returnsRaster = true, + expressionConfig = expressionConfig + ) + with NullIntolerant + with CodegenFallback { + + override def dataType: org.apache.spark.sql.types.DataType = RasterTileType(expressionConfig.getCellIdType, rastersExpr) + + val geometryAPI: GeometryAPI = GeometryAPI(expressionConfig.getGeometryAPI) + + /** + * Clips a raster by a vector. + * + * @param tile + * The raster to be used. + * @param arg1 + * The vector to be used. + * @return + * The clipped raster. + */ + override def rasterTransform(tile: MosaicRasterTile, arg1: Any): Any = { + val kernel = arg1.asInstanceOf[Array[Array[Double]]] + tile.copy( + raster = tile.getRaster.convolve(kernel) + ) + } + +} + +/** Expression info required for the expression registration for spark SQL. */ +object RST_Convolve extends WithExpressionInfo { + + override def name: String = "rst_convolve" + + override def usage: String = + """ + |_FUNC_(expr1) - Returns a raster with the kernel filter applied. + |""".stripMargin + + override def example: String = + """ + | Examples: + | > SELECT _FUNC_(raster, kernel); + | {index_id, clipped_raster, parentPath, driver} + | {index_id, clipped_raster, parentPath, driver} + | ... + | """.stripMargin + + override def builder(expressionConfig: MosaicExpressionConfig): FunctionBuilder = { + GenericExpressionFactory.getBaseBuilder[RST_Convolve](2, expressionConfig) + } + +} diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_DerivedBand.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_DerivedBand.scala index 822228a1b..fa576427a 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_DerivedBand.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_DerivedBand.scala @@ -9,25 +9,27 @@ import com.databricks.labs.mosaic.functions.MosaicExpressionConfig import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant} +import org.apache.spark.sql.types.DataType import org.apache.spark.unsafe.types.UTF8String /** Expression for combining rasters using average of pixels. */ case class RST_DerivedBand( - rastersExpr: Expression, + tileExpr: Expression, pythonFuncExpr: Expression, funcNameExpr: Expression, expressionConfig: MosaicExpressionConfig ) extends RasterArray2ArgExpression[RST_DerivedBand]( - rastersExpr, + tileExpr, pythonFuncExpr, funcNameExpr, - RasterTileType(expressionConfig.getCellIdType), returnsRaster = true, expressionConfig = expressionConfig ) with NullIntolerant with CodegenFallback { + override def dataType: DataType = RasterTileType(expressionConfig.getCellIdType, tileExpr) + /** Combines the rasters using average of pixels. */ override def rasterTransform(tiles: Seq[MosaicRasterTile], arg1: Any, arg2: Any): Any = { val pythonFunc = arg1.asInstanceOf[UTF8String].toString diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_DerivedBandAgg.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_DerivedBandAgg.scala index 47d4aa12a..f02194d62 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_DerivedBandAgg.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_DerivedBandAgg.scala @@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer */ //noinspection DuplicatedCode case class RST_DerivedBandAgg( - rasterExpr: Expression, + tileExpr: Expression, pythonFuncExpr: Expression, funcNameExpr: Expression, expressionConfig: MosaicExpressionConfig, @@ -36,13 +36,13 @@ case class RST_DerivedBandAgg( override lazy val deterministic: Boolean = true override val nullable: Boolean = false - override val dataType: DataType = RasterTileType(expressionConfig.getCellIdType) + override lazy val dataType: DataType = RasterTileType(expressionConfig.getCellIdType, tileExpr) override def prettyName: String = "rst_combine_avg_agg" private lazy val projection = UnsafeProjection.create(Array[DataType](ArrayType(elementType = dataType, containsNull = false))) private lazy val row = new UnsafeRow(1) - override def first: Expression = rasterExpr + override def first: Expression = tileExpr override def second: Expression = pythonFuncExpr override def third: Expression = funcNameExpr @@ -74,9 +74,16 @@ case class RST_DerivedBandAgg( // This works for Literals only val pythonFunc = pythonFuncExpr.eval(null).asInstanceOf[UTF8String].toString val funcName = funcNameExpr.eval(null).asInstanceOf[UTF8String].toString + val rasterType = RasterTileType(tileExpr).rasterType // Do do move the expression - var tiles = buffer.map(row => MosaicRasterTile.deserialize(row.asInstanceOf[InternalRow], expressionConfig.getCellIdType)) + var tiles = buffer.map(row => + MosaicRasterTile.deserialize( + row.asInstanceOf[InternalRow], + expressionConfig.getCellIdType, + rasterType + ) + ) // If merging multiple index rasters, the index value is dropped val idx = if (tiles.map(_.getIndex).groupBy(identity).size == 1) tiles.head.getIndex else null @@ -88,7 +95,7 @@ case class RST_DerivedBandAgg( val result = MosaicRasterTile(idx, combined, parentPath, driver) .formatCellId(IndexSystemFactory.getIndexSystem(expressionConfig.getIndexSystem)) - .serialize(BinaryType, expressionConfig.getRasterCheckpoint) + .serialize(BinaryType) tiles.foreach(RasterCleaner.dispose(_)) RasterCleaner.dispose(result) @@ -113,7 +120,7 @@ case class RST_DerivedBandAgg( } override protected def withNewChildrenInternal(newFirst: Expression, newSecond: Expression, newThird: Expression): RST_DerivedBandAgg = - copy(rasterExpr = newFirst, pythonFuncExpr = newSecond, funcNameExpr = newThird) + copy(tileExpr = newFirst, pythonFuncExpr = newSecond, funcNameExpr = newThird) } diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Filter.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Filter.scala new file mode 100644 index 000000000..ee8b34d3b --- /dev/null +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Filter.scala @@ -0,0 +1,77 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.geometry.api.GeometryAPI +import com.databricks.labs.mosaic.core.types.RasterTileType +import com.databricks.labs.mosaic.core.types.model.MosaicRasterTile +import com.databricks.labs.mosaic.expressions.base.{GenericExpressionFactory, WithExpressionInfo} +import com.databricks.labs.mosaic.expressions.raster.base.Raster2ArgExpression +import com.databricks.labs.mosaic.functions.MosaicExpressionConfig +import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant} +import org.apache.spark.unsafe.types.UTF8String + +/** The expression for applying NxN filter on a raster. */ +case class RST_Filter( + rastersExpr: Expression, + kernelSizeExpr: Expression, + operationExpr: Expression, + expressionConfig: MosaicExpressionConfig +) extends Raster2ArgExpression[RST_Filter]( + rastersExpr, + kernelSizeExpr, + operationExpr, + returnsRaster = true, + expressionConfig = expressionConfig + ) + with NullIntolerant + with CodegenFallback { + + override def dataType: org.apache.spark.sql.types.DataType = RasterTileType(expressionConfig.getCellIdType, rastersExpr) + + val geometryAPI: GeometryAPI = GeometryAPI(expressionConfig.getGeometryAPI) + + /** + * Clips a raster by a vector. + * + * @param tile + * The raster to be used. + * @param arg1 + * The vector to be used. + * @return + * The clipped raster. + */ + override def rasterTransform(tile: MosaicRasterTile, arg1: Any, arg2: Any): Any = { + val n = arg1.asInstanceOf[Int] + val operation = arg2.asInstanceOf[UTF8String].toString + tile.copy( + raster = tile.getRaster.filter(n, operation) + ) + } + +} + +/** Expression info required for the expression registration for spark SQL. */ +object RST_Filter extends WithExpressionInfo { + + override def name: String = "rst_filter" + + override def usage: String = + """ + |_FUNC_(expr1) - Returns a raster with the filter applied. + |""".stripMargin + + override def example: String = + """ + | Examples: + | > SELECT _FUNC_(raster, kernelSize, operation); + | {index_id, clipped_raster, parentPath, driver} + | {index_id, clipped_raster, parentPath, driver} + | ... + | """.stripMargin + + override def builder(expressionConfig: MosaicExpressionConfig): FunctionBuilder = { + GenericExpressionFactory.getBaseBuilder[RST_Filter](3, expressionConfig) + } + +} diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_FromBands.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_FromBands.scala index 2befb353c..90e49b654 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_FromBands.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_FromBands.scala @@ -9,6 +9,7 @@ import com.databricks.labs.mosaic.functions.MosaicExpressionConfig import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant} +import org.apache.spark.sql.types.ArrayType /** The expression for stacking and resampling input bands. */ case class RST_FromBands( @@ -16,13 +17,18 @@ case class RST_FromBands( expressionConfig: MosaicExpressionConfig ) extends RasterArrayExpression[RST_FromBands]( bandsExpr, - RasterTileType(expressionConfig.getCellIdType), returnsRaster = true, expressionConfig = expressionConfig ) with NullIntolerant with CodegenFallback { + override def dataType: org.apache.spark.sql.types.DataType = + RasterTileType( + expressionConfig.getCellIdType, + RasterTileType(bandsExpr).rasterType + ) + /** * Stacks and resamples input bands. * @param rasters diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_FromContent.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_FromContent.scala index bd2926bcb..59c701f71 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_FromContent.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_FromContent.scala @@ -15,7 +15,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.expressions.{CollectionGenerator, Expression, Literal, NullIntolerant} -import org.apache.spark.sql.types.{DataType, IntegerType, StringType, StructField, StructType} +import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String import java.nio.file.{Files, Paths} @@ -25,7 +25,7 @@ import java.nio.file.{Files, Paths} * expression in the expression tree for a raster tile. */ case class RST_FromContent( - rasterExpr: Expression, + contentExpr: Expression, driverExpr: Expression, sizeInMB: Expression, expressionConfig: MosaicExpressionConfig @@ -33,8 +33,10 @@ case class RST_FromContent( with Serializable with NullIntolerant with CodegenFallback { + + val tileType: DataType = BinaryType - override def dataType: DataType = RasterTileType(expressionConfig.getCellIdType) + override def dataType: DataType = RasterTileType(expressionConfig.getCellIdType, tileType) protected val geometryAPI: GeometryAPI = GeometryAPI.apply(expressionConfig.getGeometryAPI) @@ -46,12 +48,13 @@ case class RST_FromContent( override def inline: Boolean = false - override def children: Seq[Expression] = Seq(rasterExpr, driverExpr, sizeInMB) + override def children: Seq[Expression] = Seq(contentExpr, driverExpr, sizeInMB) override def elementSchema: StructType = StructType(Array(StructField("tile", dataType))) /** - * subdivides raster binary content into tiles of the specified size (in MB). + * subdivides raster binary content into tiles of the specified size (in + * MB). * @param input * The input file path. * @return @@ -61,13 +64,13 @@ case class RST_FromContent( GDAL.enable(expressionConfig) val driver = driverExpr.eval(input).asInstanceOf[UTF8String].toString val ext = GDAL.getExtension(driver) - var rasterArr = rasterExpr.eval(input).asInstanceOf[Array[Byte]] + var rasterArr = contentExpr.eval(input).asInstanceOf[Array[Byte]] val targetSize = sizeInMB.eval(input).asInstanceOf[Int] if (targetSize <= 0 || rasterArr.length <= targetSize) { // - no split required var raster = MosaicRasterGDAL.readRaster(rasterArr, PathUtils.NO_PATH_STRING, driver) var tile = MosaicRasterTile(null, raster, PathUtils.NO_PATH_STRING, driver) - val row = tile.formatCellId(indexSystem).serialize() + val row = tile.formatCellId(indexSystem).serialize(tileType) RasterCleaner.dispose(raster) RasterCleaner.dispose(tile) rasterArr = null @@ -84,7 +87,7 @@ case class RST_FromContent( // split to tiles up to specifed threshold var tiles = ReTileOnRead.localSubdivide(rasterPath, PathUtils.NO_PATH_STRING, targetSize) - val rows = tiles.map(_.formatCellId(indexSystem).serialize()) + val rows = tiles.map(_.formatCellId(indexSystem).serialize(tileType)) tiles.foreach(RasterCleaner.dispose(_)) Files.deleteIfExists(Paths.get(rasterPath)) rasterArr = null @@ -118,10 +121,10 @@ object RST_FromContent extends WithExpressionInfo { | ... | """.stripMargin - override def builder(expressionConfig: MosaicExpressionConfig): FunctionBuilder = { - (children: Seq[Expression]) => { + override def builder(expressionConfig: MosaicExpressionConfig): FunctionBuilder = { (children: Seq[Expression]) => + { val sizeExpr = if (children.length < 3) new Literal(-1, IntegerType) else children(2) - RST_FromContent(children(0), children(1), sizeExpr, expressionConfig) + RST_FromContent(children.head, children(1), sizeExpr, expressionConfig) } } diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_FromFile.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_FromFile.scala index fbce5bf58..ee5bec721 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_FromFile.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_FromFile.scala @@ -15,7 +15,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.expressions.{CollectionGenerator, Expression, Literal, NullIntolerant} -import org.apache.spark.sql.types.{DataType, IntegerType, StructField, StructType} +import org.apache.spark.sql.types.{BinaryType, DataType, IntegerType, StringType, StructField, StructType} import org.apache.spark.unsafe.types.UTF8String import java.nio.file.{Files, Paths, StandardCopyOption} @@ -32,8 +32,10 @@ case class RST_FromFile( with Serializable with NullIntolerant with CodegenFallback { + + val tileType: DataType = BinaryType - override def dataType: DataType = RasterTileType(expressionConfig.getCellIdType) + override def dataType: DataType = RasterTileType(expressionConfig.getCellIdType, tileType) protected val geometryAPI: GeometryAPI = GeometryAPI.apply(expressionConfig.getGeometryAPI) @@ -66,7 +68,7 @@ case class RST_FromFile( if (targetSize <= 0 && Files.size(Paths.get(readPath)) <= Integer.MAX_VALUE) { var raster = MosaicRasterGDAL.readRaster(readPath, path) var tile = MosaicRasterTile(null, raster, path, raster.getDriversShortName) - val row = tile.formatCellId(indexSystem).serialize() + val row = tile.formatCellId(indexSystem).serialize(tileType) RasterCleaner.dispose(raster) RasterCleaner.dispose(tile) raster = null @@ -79,7 +81,7 @@ case class RST_FromFile( Files.copy(Paths.get(readPath), Paths.get(tmpPath), StandardCopyOption.REPLACE_EXISTING) val size = if (targetSize <= 0) 64 else targetSize var tiles = ReTileOnRead.localSubdivide(tmpPath, path, size) - val rows = tiles.map(_.formatCellId(indexSystem).serialize()) + val rows = tiles.map(_.formatCellId(indexSystem).serialize(tileType)) tiles.foreach(RasterCleaner.dispose(_)) Files.deleteIfExists(Paths.get(tmpPath)) tiles = null diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_GeoReference.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_GeoReference.scala index f4213eee7..404eb4b90 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_GeoReference.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_GeoReference.scala @@ -11,10 +11,12 @@ import org.apache.spark.sql.types._ /** Returns the georeference of the raster. */ case class RST_GeoReference(raster: Expression, expressionConfig: MosaicExpressionConfig) - extends RasterExpression[RST_GeoReference](raster, MapType(StringType, DoubleType), returnsRaster = false, expressionConfig) + extends RasterExpression[RST_GeoReference](raster, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = MapType(StringType, DoubleType) + /** Returns the georeference of the raster. */ override def rasterTransform(tile: MosaicRasterTile): Any = { val geoTransform = tile.getRaster.getRaster.GetGeoTransform() diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_GetNoData.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_GetNoData.scala index 8f10b89cb..aa07a6637 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_GetNoData.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_GetNoData.scala @@ -8,7 +8,7 @@ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant} import org.apache.spark.sql.catalyst.util.ArrayData -import org.apache.spark.sql.types.{ArrayType, DoubleType} +import org.apache.spark.sql.types.{ArrayType, DataType, DoubleType} /** The expression for extracting the no data value of a raster. */ case class RST_GetNoData( @@ -16,13 +16,14 @@ case class RST_GetNoData( expressionConfig: MosaicExpressionConfig ) extends RasterExpression[RST_GetNoData]( rastersExpr, - ArrayType(DoubleType), returnsRaster = false, expressionConfig = expressionConfig ) with NullIntolerant with CodegenFallback { + override def dataType: DataType = ArrayType(DoubleType) + /** * Extracts the no data value of a raster. * diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_GetSubdataset.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_GetSubdataset.scala index a87f6fa25..8d1fc77f1 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_GetSubdataset.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_GetSubdataset.scala @@ -8,20 +8,25 @@ import com.databricks.labs.mosaic.functions.MosaicExpressionConfig import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant} +import org.apache.spark.sql.types.DataType import org.apache.spark.unsafe.types.UTF8String /** Returns the subdatasets of the raster. */ -case class RST_GetSubdataset(raster: Expression, subsetName: Expression, expressionConfig: MosaicExpressionConfig) - extends Raster1ArgExpression[RST_GetSubdataset]( - raster, +case class RST_GetSubdataset( + tileExpr: Expression, + subsetName: Expression, + expressionConfig: MosaicExpressionConfig +) extends Raster1ArgExpression[RST_GetSubdataset]( + tileExpr, subsetName, - RasterTileType(expressionConfig.getCellIdType), returnsRaster = true, expressionConfig ) with NullIntolerant with CodegenFallback { + override def dataType: DataType = RasterTileType(expressionConfig.getCellIdType, tileExpr) + /** Returns the subdatasets of the raster. */ override def rasterTransform(tile: MosaicRasterTile, arg1: Any): Any = { val subsetName = arg1.asInstanceOf[UTF8String].toString diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Height.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Height.scala index ceb638f29..f2508e1e6 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Height.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Height.scala @@ -11,10 +11,12 @@ import org.apache.spark.sql.types._ /** Returns the width of the raster. */ case class RST_Height(raster: Expression, expressionConfig: MosaicExpressionConfig) - extends RasterExpression[RST_Height](raster, IntegerType, returnsRaster = false, expressionConfig) + extends RasterExpression[RST_Height](raster, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = IntegerType + /** Returns the width of the raster. */ override def rasterTransform(tile: MosaicRasterTile): Any = tile.getRaster.ySize diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_InitNoData.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_InitNoData.scala index 8cf226664..3b1b806da 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_InitNoData.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_InitNoData.scala @@ -11,20 +11,22 @@ import com.databricks.labs.mosaic.utils.PathUtils import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant} +import org.apache.spark.sql.types.DataType /** The expression that initializes no data values of a raster. */ case class RST_InitNoData( - rastersExpr: Expression, + tileExpr: Expression, expressionConfig: MosaicExpressionConfig ) extends RasterExpression[RST_InitNoData]( - rastersExpr, - RasterTileType(expressionConfig.getCellIdType), + tileExpr, returnsRaster = true, expressionConfig = expressionConfig ) with NullIntolerant with CodegenFallback { + override def dataType: DataType = RasterTileType(expressionConfig.getCellIdType, tileExpr) + /** * Initializes no data values of a raster. * diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_IsEmpty.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_IsEmpty.scala index 4a5f5034f..7d6267bec 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_IsEmpty.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_IsEmpty.scala @@ -11,10 +11,12 @@ import org.apache.spark.sql.types._ /** Returns true if the raster is empty. */ case class RST_IsEmpty(raster: Expression, expressionConfig: MosaicExpressionConfig) - extends RasterExpression[RST_IsEmpty](raster, BooleanType, returnsRaster = false, expressionConfig) + extends RasterExpression[RST_IsEmpty](raster, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = BooleanType + /** Returns true if the raster is empty. */ override def rasterTransform(tile: MosaicRasterTile): Any = { var raster = tile.getRaster diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MakeTiles.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MakeTiles.scala new file mode 100644 index 000000000..586337556 --- /dev/null +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MakeTiles.scala @@ -0,0 +1,205 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.MOSAIC_NO_DRIVER +import com.databricks.labs.mosaic.core.geometry.api.GeometryAPI +import com.databricks.labs.mosaic.core.index.{IndexSystem, IndexSystemFactory} +import com.databricks.labs.mosaic.core.raster.api.GDAL +import com.databricks.labs.mosaic.core.raster.gdal.MosaicRasterGDAL +import com.databricks.labs.mosaic.core.raster.io.RasterCleaner +import com.databricks.labs.mosaic.core.types.RasterTileType +import com.databricks.labs.mosaic.core.types.model.MosaicRasterTile +import com.databricks.labs.mosaic.datasource.gdal.ReTileOnRead +import com.databricks.labs.mosaic.expressions.base.{GenericExpressionFactory, WithExpressionInfo} +import com.databricks.labs.mosaic.functions.MosaicExpressionConfig +import com.databricks.labs.mosaic.utils.PathUtils +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.expressions.{CollectionGenerator, Expression, Literal, NullIntolerant} +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +import java.nio.file.{Files, Paths} +import scala.util.Try + +/** + * Creates raster tiles from the input column. + * + * @param inputExpr + * The expression for the raster. If the raster is stored on disc, the path + * to the raster is provided. If the raster is stored in memory, the bytes of + * the raster are provided. + * @param sizeInMBExpr + * The size of the tiles in MB. If set to -1, the file is loaded and returned + * as a single tile. If set to 0, the file is loaded and subdivided into + * tiles of size 64MB. If set to a positive value, the file is loaded and + * subdivided into tiles of the specified size. If the file is too big to fit + * in memory, it is subdivided into tiles of size 64MB. + * @param driverExpr + * The driver to use for reading the raster. If not specified, the driver is + * inferred from the file extension. If the input is a byte array, the driver + * has to be specified. + * @param withCheckpointExpr + * If set to true, the tiles are written to the checkpoint directory. If set + * to false, the tiles are returned as a in-memory byte arrays. + * @param expressionConfig + * Additional arguments for the expression (expressionConfigs). + */ +case class RST_MakeTiles( + inputExpr: Expression, + driverExpr: Expression, + sizeInMBExpr: Expression, + withCheckpointExpr: Expression, + expressionConfig: MosaicExpressionConfig +) extends CollectionGenerator + with Serializable + with NullIntolerant + with CodegenFallback { + + override def dataType: DataType = { + require(withCheckpointExpr.isInstanceOf[Literal]) + if (withCheckpointExpr.eval().asInstanceOf[Boolean]) { + // Raster is referenced via a path + RasterTileType(expressionConfig.getCellIdType, StringType) + } else { + // Raster is referenced via a byte array + RasterTileType(expressionConfig.getCellIdType, BinaryType) + } + } + + protected val geometryAPI: GeometryAPI = GeometryAPI.apply(expressionConfig.getGeometryAPI) + + protected val indexSystem: IndexSystem = IndexSystemFactory.getIndexSystem(expressionConfig.getIndexSystem) + + protected val cellIdDataType: DataType = indexSystem.getCellIdDataType + + override def position: Boolean = false + + override def inline: Boolean = false + + override def children: Seq[Expression] = Seq(inputExpr, driverExpr, sizeInMBExpr, withCheckpointExpr) + + override def elementSchema: StructType = StructType(Array(StructField("tile", dataType))) + + private def getDriver(rawInput: Any, rawDriver: String): String = { + if (rawDriver == MOSAIC_NO_DRIVER) { + if (inputExpr.dataType == StringType) { + val path = rawInput.asInstanceOf[UTF8String].toString + MosaicRasterGDAL.identifyDriver(path) + } else { + throw new IllegalArgumentException("Driver has to be specified for byte array input") + } + } else { + rawDriver + } + } + + private def getInputSize(rawInput: Any): Long = { + if (inputExpr.dataType == StringType) { + val path = rawInput.asInstanceOf[UTF8String].toString + Files.size(Paths.get(path)) + } else { + val bytes = rawInput.asInstanceOf[Array[Byte]] + bytes.length + } + } + + /** + * Loads a raster from a file and subdivides it into tiles of the specified + * size (in MB). + * @param input + * The input file path. + * @return + * The tiles. + */ + override def eval(input: InternalRow): TraversableOnce[InternalRow] = { + GDAL.enable(expressionConfig) + + val tileType = dataType.asInstanceOf[StructType].find(_.name == "raster").get.dataType + + val rawDriver = driverExpr.eval(input).asInstanceOf[UTF8String].toString + val rawInput = inputExpr.eval(input) + val driver = getDriver(rawInput, rawDriver) + val targetSize = sizeInMBExpr.eval(input).asInstanceOf[Int] + val inputSize = getInputSize(rawInput) + + if (targetSize <= 0 && inputSize <= Integer.MAX_VALUE) { + // - no split required + val raster = GDAL.readRaster(rawInput, PathUtils.NO_PATH_STRING, driver, inputExpr.dataType) + val tile = MosaicRasterTile(null, raster, PathUtils.NO_PATH_STRING, driver) + val row = tile.formatCellId(indexSystem).serialize(tileType) + RasterCleaner.dispose(raster) + RasterCleaner.dispose(tile) + Seq(InternalRow.fromSeq(Seq(row))) + } else { + // target size is > 0 and raster size > target size + // - write the initial raster to file (unsplit) + // - createDirectories in case of context isolation + val rasterPath = PathUtils.createTmpFilePath(GDAL.getExtension(driver)) + Files.createDirectories(Paths.get(rasterPath).getParent) + Files.write(Paths.get(rasterPath), rawInput.asInstanceOf[Array[Byte]]) + val size = if (targetSize <= 0) 64 else targetSize + var tiles = ReTileOnRead.localSubdivide(rasterPath, PathUtils.NO_PATH_STRING, size) + val rows = tiles.map(_.formatCellId(indexSystem).serialize(tileType)) + tiles.foreach(RasterCleaner.dispose(_)) + Files.deleteIfExists(Paths.get(rasterPath)) + tiles = null + rows.map(row => InternalRow.fromSeq(Seq(row))) + } + } + + override def makeCopy(newArgs: Array[AnyRef]): Expression = + GenericExpressionFactory.makeCopyImpl[RST_MakeTiles](this, newArgs, children.length, expressionConfig) + + override def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Expression = makeCopy(newChildren.toArray) + +} + +/** Expression info required for the expression registration for spark SQL. */ +object RST_MakeTiles extends WithExpressionInfo { + + override def name: String = "rst_maketiles" + + override def usage: String = + """ + |_FUNC_(expr1) - Returns a set of new rasters with the specified tile size (tileWidth x tileHeight). + |""".stripMargin + + override def example: String = + """ + | Examples: + | > SELECT _FUNC_(raster_path); + | {index_id, raster, parent_path, driver} + | ... + | """.stripMargin + + override def builder(expressionConfig: MosaicExpressionConfig): FunctionBuilder = { (children: Seq[Expression]) => + { + def checkSize(size: Expression) = Try(size.eval().asInstanceOf[Int]).isSuccess + def checkChkpnt(chkpnt: Expression) = Try(chkpnt.eval().asInstanceOf[Boolean]).isSuccess + def checkDriver(driver: Expression) = Try(driver.eval().asInstanceOf[UTF8String].toString).isSuccess + val noSize = new Literal(-1, IntegerType) + val noDriver = new Literal(MOSAIC_NO_DRIVER, StringType) + val noCheckpoint = new Literal(false, BooleanType) + + children match { + // Note type checking only works for literals + case Seq(input) => RST_MakeTiles(input, noDriver, noSize, noCheckpoint, expressionConfig) + case Seq(input, driver) if checkDriver(driver) => RST_MakeTiles(input, driver, noSize, noCheckpoint, expressionConfig) + case Seq(input, size) if checkSize(size) => RST_MakeTiles(input, noDriver, size, noCheckpoint, expressionConfig) + case Seq(input, checkpoint) if checkChkpnt(checkpoint) => + RST_MakeTiles(input, noDriver, noSize, checkpoint, expressionConfig) + case Seq(input, size, checkpoint) if checkSize(size) && checkChkpnt(checkpoint) => + RST_MakeTiles(input, noDriver, size, checkpoint, expressionConfig) + case Seq(input, driver, size) if checkDriver(driver) && checkSize(size) => + RST_MakeTiles(input, driver, size, noCheckpoint, expressionConfig) + case Seq(input, driver, checkpoint) if checkDriver(driver) && checkChkpnt(checkpoint) => + RST_MakeTiles(input, driver, noSize, checkpoint, expressionConfig) + case Seq(input, driver, size, checkpoint) if checkDriver(driver) && checkSize(size) && checkChkpnt(checkpoint) => + RST_MakeTiles(input, driver, size, checkpoint, expressionConfig) + case _ => RST_MakeTiles(children.head, children(1), children(2), children(3), expressionConfig) + } + } + } + +} diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MapAlgebra.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MapAlgebra.scala index 53e84d96b..1c74e1f0a 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MapAlgebra.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MapAlgebra.scala @@ -11,23 +11,25 @@ import com.databricks.labs.mosaic.utils.PathUtils import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant} +import org.apache.spark.sql.types.DataType import org.apache.spark.unsafe.types.UTF8String /** The expression for map algebra. */ case class RST_MapAlgebra( - rastersExpr: Expression, + tileExpr: Expression, jsonSpecExpr: Expression, expressionConfig: MosaicExpressionConfig ) extends RasterArray1ArgExpression[RST_MapAlgebra]( - rastersExpr, + tileExpr, jsonSpecExpr, - RasterTileType(expressionConfig.getCellIdType), returnsRaster = true, expressionConfig = expressionConfig ) with NullIntolerant with CodegenFallback { + override def dataType: DataType = RasterTileType(expressionConfig.getCellIdType, tileExpr) + /** * Map Algebra. * @param tiles diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Max.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Max.scala index abe042c2b..434be4a68 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Max.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Max.scala @@ -14,10 +14,12 @@ import org.apache.spark.sql.types._ /** Returns the upper left x of the raster. */ case class RST_Max(raster: Expression, expressionConfig: MosaicExpressionConfig) - extends RasterExpression[RST_Max](raster, ArrayType(DoubleType), returnsRaster = false, expressionConfig) + extends RasterExpression[RST_Max](raster, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = ArrayType(DoubleType) + /** Returns the upper left x of the raster. */ override def rasterTransform(tile: MosaicRasterTile): Any = { val nBands = tile.raster.raster.GetRasterCount() diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Median.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Median.scala index 091121e91..19d3fc0a6 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Median.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Median.scala @@ -1,8 +1,7 @@ package com.databricks.labs.mosaic.expressions.raster import com.databricks.labs.mosaic.core.raster.api.GDAL -import com.databricks.labs.mosaic.core.raster.gdal.MosaicRasterGDAL -import com.databricks.labs.mosaic.core.raster.operator.gdal.{GDALCalc, GDALInfo, GDALWarp} +import com.databricks.labs.mosaic.core.raster.operator.gdal.GDALWarp import com.databricks.labs.mosaic.core.types.model.MosaicRasterTile import com.databricks.labs.mosaic.expressions.base.{GenericExpressionFactory, WithExpressionInfo} import com.databricks.labs.mosaic.expressions.raster.base.RasterExpression @@ -16,10 +15,12 @@ import org.apache.spark.sql.types._ /** Returns the upper left x of the raster. */ case class RST_Median(rasterExpr: Expression, expressionConfig: MosaicExpressionConfig) - extends RasterExpression[RST_Median](rasterExpr, ArrayType(DoubleType), returnsRaster = false, expressionConfig) + extends RasterExpression[RST_Median](rasterExpr, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = ArrayType(DoubleType) + /** Returns the upper left x of the raster. */ override def rasterTransform(tile: MosaicRasterTile): Any = { val raster = tile.raster diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MemSize.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MemSize.scala index 804c4f195..f77058a65 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MemSize.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MemSize.scala @@ -11,10 +11,12 @@ import org.apache.spark.sql.types._ /** Returns the memory size of the raster in bytes. */ case class RST_MemSize(raster: Expression, expressionConfig: MosaicExpressionConfig) - extends RasterExpression[RST_MemSize](raster, LongType, returnsRaster = false, expressionConfig) + extends RasterExpression[RST_MemSize](raster, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = LongType + /** Returns the memory size of the raster in bytes. */ override def rasterTransform(tile: MosaicRasterTile): Any = tile.getRaster.getMemSize diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Merge.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Merge.scala index cb9907848..c8ef6846d 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Merge.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Merge.scala @@ -9,20 +9,22 @@ import com.databricks.labs.mosaic.functions.MosaicExpressionConfig import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant} +import org.apache.spark.sql.types.DataType /** Returns a raster that is a result of merging an array of rasters. */ case class RST_Merge( - rastersExpr: Expression, + tileExpr: Expression, expressionConfig: MosaicExpressionConfig ) extends RasterArrayExpression[RST_Merge]( - rastersExpr, - RasterTileType(expressionConfig.getCellIdType), + tileExpr, returnsRaster = true, expressionConfig = expressionConfig ) with NullIntolerant with CodegenFallback { + override def dataType: DataType = RasterTileType(expressionConfig.getCellIdType, tileExpr) + /** * Merges an array of rasters. * @param tiles diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MergeAgg.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MergeAgg.scala index 5902eac3b..88705dfe0 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MergeAgg.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MergeAgg.scala @@ -20,7 +20,7 @@ import scala.collection.mutable.ArrayBuffer /** Merges rasters into a single raster. */ //noinspection DuplicatedCode case class RST_MergeAgg( - rasterExpr: Expression, + tileExpr: Expression, expressionConfig: MosaicExpressionConfig, mutableAggBufferOffset: Int = 0, inputAggBufferOffset: Int = 0 @@ -29,9 +29,9 @@ case class RST_MergeAgg( with RasterExpressionSerialization { override lazy val deterministic: Boolean = true - override val child: Expression = rasterExpr + override val child: Expression = tileExpr override val nullable: Boolean = false - override val dataType: DataType = RasterTileType(expressionConfig.getCellIdType) + override lazy val dataType: DataType = RasterTileType(expressionConfig.getCellIdType, tileExpr) override def prettyName: String = "rst_merge_agg" private lazy val projection = UnsafeProjection.create(Array[DataType](ArrayType(elementType = dataType, containsNull = false))) @@ -66,8 +66,15 @@ case class RST_MergeAgg( // This is a trick to get the rasters sorted by their parent path to ensure more consistent results // when merging rasters with large overlaps + val rasterType = RasterTileType(tileExpr).rasterType var tiles = buffer - .map(row => MosaicRasterTile.deserialize(row.asInstanceOf[InternalRow], expressionConfig.getCellIdType)) + .map(row => + MosaicRasterTile.deserialize( + row.asInstanceOf[InternalRow], + expressionConfig.getCellIdType, + rasterType + ) + ) .sortBy(_.getParentPath) // If merging multiple index rasters, the index value is dropped @@ -79,7 +86,7 @@ case class RST_MergeAgg( val result = MosaicRasterTile(idx, merged, parentPath, driver) .formatCellId(IndexSystemFactory.getIndexSystem(expressionConfig.getIndexSystem)) - .serialize(BinaryType, expressionConfig.getRasterCheckpoint) + .serialize(BinaryType) tiles.foreach(RasterCleaner.dispose(_)) RasterCleaner.dispose(merged) @@ -103,7 +110,7 @@ case class RST_MergeAgg( buffer } - override protected def withNewChildInternal(newChild: Expression): RST_MergeAgg = copy(rasterExpr = newChild) + override protected def withNewChildInternal(newChild: Expression): RST_MergeAgg = copy(tileExpr = newChild) } diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MetaData.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MetaData.scala index 8a96ff0d1..0b6754ebe 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MetaData.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MetaData.scala @@ -11,10 +11,12 @@ import org.apache.spark.sql.types._ /** Returns the metadata of the raster. */ case class RST_MetaData(raster: Expression, expressionConfig: MosaicExpressionConfig) - extends RasterExpression[RST_MetaData](raster, MapType(StringType, StringType), returnsRaster = false, expressionConfig) + extends RasterExpression[RST_MetaData](raster, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = MapType(StringType, StringType) + /** Returns the metadata of the raster. */ override def rasterTransform(tile: MosaicRasterTile): Any = buildMapString(tile.getRaster.metadata) diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Min.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Min.scala index 67fdb30d3..ea62e106f 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Min.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Min.scala @@ -1,6 +1,5 @@ package com.databricks.labs.mosaic.expressions.raster -import com.databricks.labs.mosaic.core.raster.operator.gdal.GDALInfo import com.databricks.labs.mosaic.core.types.model.MosaicRasterTile import com.databricks.labs.mosaic.expressions.base.{GenericExpressionFactory, WithExpressionInfo} import com.databricks.labs.mosaic.expressions.raster.base.RasterExpression @@ -14,10 +13,12 @@ import org.apache.spark.sql.types._ /** Returns the upper left x of the raster. */ case class RST_Min(raster: Expression, expressionConfig: MosaicExpressionConfig) - extends RasterExpression[RST_Min](raster, ArrayType(DoubleType), returnsRaster = false, expressionConfig) + extends RasterExpression[RST_Min](raster, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = ArrayType(DoubleType) + /** Returns the upper left x of the raster. */ override def rasterTransform(tile: MosaicRasterTile): Any = { val nBands = tile.raster.raster.GetRasterCount() diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_NDVI.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_NDVI.scala index fa595fd4b..67b580f0c 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_NDVI.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_NDVI.scala @@ -9,24 +9,26 @@ import com.databricks.labs.mosaic.functions.MosaicExpressionConfig import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant} +import org.apache.spark.sql.types.DataType /** The expression for computing NDVI index. */ case class RST_NDVI( - rastersExpr: Expression, + tileExpr: Expression, redIndex: Expression, nirIndex: Expression, expressionConfig: MosaicExpressionConfig ) extends Raster2ArgExpression[RST_NDVI]( - rastersExpr, + tileExpr, redIndex, nirIndex, - RasterTileType(expressionConfig.getCellIdType), returnsRaster = true, expressionConfig = expressionConfig ) with NullIntolerant with CodegenFallback { + override def dataType: DataType = RasterTileType(expressionConfig.getCellIdType, tileExpr) + /** * Computes NDVI index. * @param tile diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_NumBands.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_NumBands.scala index f5dd09551..e0a8c8d9e 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_NumBands.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_NumBands.scala @@ -11,10 +11,12 @@ import org.apache.spark.sql.types._ /** Returns the number of bands in the raster. */ case class RST_NumBands(raster: Expression, expressionConfig: MosaicExpressionConfig) - extends RasterExpression[RST_NumBands](raster, IntegerType, returnsRaster = false, expressionConfig) + extends RasterExpression[RST_NumBands](raster, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = IntegerType + /** Returns the number of bands in the raster. */ override def rasterTransform(tile: MosaicRasterTile): Any = tile.getRaster.numBands diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_PixelCount.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_PixelCount.scala index 79f44db03..b2543a87e 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_PixelCount.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_PixelCount.scala @@ -12,10 +12,12 @@ import org.apache.spark.sql.types._ /** Returns the upper left x of the raster. */ case class RST_PixelCount(rasterExpr: Expression, expressionConfig: MosaicExpressionConfig) - extends RasterExpression[RST_PixelCount](rasterExpr, ArrayType(DoubleType), returnsRaster = false, expressionConfig) + extends RasterExpression[RST_PixelCount](rasterExpr, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = ArrayType(LongType) + /** Returns the upper left x of the raster. */ override def rasterTransform(tile: MosaicRasterTile): Any = { val bandCount = tile.raster.raster.GetRasterCount() diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_PixelHeight.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_PixelHeight.scala index d1c3713ef..0c34be59b 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_PixelHeight.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_PixelHeight.scala @@ -11,10 +11,12 @@ import org.apache.spark.sql.types._ /** Returns the pixel height of the raster. */ case class RST_PixelHeight(raster: Expression, expressionConfig: MosaicExpressionConfig) - extends RasterExpression[RST_PixelHeight](raster, DoubleType, returnsRaster = false, expressionConfig) + extends RasterExpression[RST_PixelHeight](raster, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = DoubleType + /** Returns the pixel height of the raster. */ override def rasterTransform(tile: MosaicRasterTile): Any = { val gt = tile.getRaster.getGeoTransform diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_PixelWidth.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_PixelWidth.scala index 6a4956e9e..b1645696b 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_PixelWidth.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_PixelWidth.scala @@ -11,10 +11,12 @@ import org.apache.spark.sql.types._ /** Returns the pixel width of the raster. */ case class RST_PixelWidth(raster: Expression, expressionConfig: MosaicExpressionConfig) - extends RasterExpression[RST_PixelWidth](raster, DoubleType, returnsRaster = false, expressionConfig) + extends RasterExpression[RST_PixelWidth](raster, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = DoubleType + /** Returns the pixel width of the raster. */ override def rasterTransform(tile: MosaicRasterTile): Any = { val gt = tile.getRaster.getGeoTransform diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_RasterToWorldCoord.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_RasterToWorldCoord.scala index 42b9a928a..9da0f19ef 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_RasterToWorldCoord.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_RasterToWorldCoord.scala @@ -17,10 +17,12 @@ case class RST_RasterToWorldCoord( x: Expression, y: Expression, expressionConfig: MosaicExpressionConfig -) extends Raster2ArgExpression[RST_RasterToWorldCoord](raster, x, y, StringType, returnsRaster = false, expressionConfig = expressionConfig) +) extends Raster2ArgExpression[RST_RasterToWorldCoord](raster, x, y, returnsRaster = false, expressionConfig = expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = StringType + /** * Returns the world coordinates of the raster (x,y) pixel by applying * GeoTransform. This ensures the projection of the raster is respected. diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_RasterToWorldCoordX.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_RasterToWorldCoordX.scala index 4bd06646a..5fea59b49 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_RasterToWorldCoordX.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_RasterToWorldCoordX.scala @@ -16,10 +16,12 @@ case class RST_RasterToWorldCoordX( x: Expression, y: Expression, expressionConfig: MosaicExpressionConfig -) extends Raster2ArgExpression[RST_RasterToWorldCoordX](raster, x, y, DoubleType, returnsRaster = false, expressionConfig) +) extends Raster2ArgExpression[RST_RasterToWorldCoordX](raster, x, y, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = DoubleType + /** * Returns the world coordinates of the raster x pixel by applying * GeoTransform. This ensures the projection of the raster is respected. diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_RasterToWorldCoordY.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_RasterToWorldCoordY.scala index 262d6bbad..ae170709c 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_RasterToWorldCoordY.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_RasterToWorldCoordY.scala @@ -16,10 +16,12 @@ case class RST_RasterToWorldCoordY( x: Expression, y: Expression, expressionConfig: MosaicExpressionConfig -) extends Raster2ArgExpression[RST_RasterToWorldCoordY](raster, x, y, DoubleType, returnsRaster = false, expressionConfig) +) extends Raster2ArgExpression[RST_RasterToWorldCoordY](raster, x, y, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = DoubleType + /** * Returns the world coordinates of the raster y pixel by applying * GeoTransform. This ensures the projection of the raster is respected. diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_ReTile.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_ReTile.scala index 4465866dc..939011882 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_ReTile.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_ReTile.scala @@ -8,6 +8,7 @@ import com.databricks.labs.mosaic.functions.MosaicExpressionConfig import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant} +import org.apache.spark.sql.types.DataType /** * Returns a set of new rasters with the specified tile size (tileWidth x @@ -22,6 +23,8 @@ case class RST_ReTile( with NullIntolerant with CodegenFallback { + override def dataType: DataType = rasterExpr.dataType + /** * Returns a set of new rasters with the specified tile size (tileWidth x * tileHeight). diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Rotation.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Rotation.scala index c3cd097c7..5933c7133 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Rotation.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Rotation.scala @@ -11,10 +11,12 @@ import org.apache.spark.sql.types._ /** Returns the rotation angle of the raster. */ case class RST_Rotation(raster: Expression, expressionConfig: MosaicExpressionConfig) - extends RasterExpression[RST_Rotation](raster, DoubleType, returnsRaster = false, expressionConfig) + extends RasterExpression[RST_Rotation](raster, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = DoubleType + /** Returns the rotation angle of the raster. */ override def rasterTransform(tile: MosaicRasterTile): Any = { val gt = tile.getRaster.getRaster.GetGeoTransform() diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_SRID.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_SRID.scala index c8bce06b7..648260ae5 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_SRID.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_SRID.scala @@ -14,10 +14,12 @@ import scala.util.Try /** Returns the SRID of the raster. */ case class RST_SRID(raster: Expression, expressionConfig: MosaicExpressionConfig) - extends RasterExpression[RST_SRID](raster, IntegerType, returnsRaster = false, expressionConfig) + extends RasterExpression[RST_SRID](raster, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = IntegerType + /** Returns the SRID of the raster. */ override def rasterTransform(tile: MosaicRasterTile): Any = { // Reference: https://gis.stackexchange.com/questions/267321/extracting-epsg-from-a-raster-using-gdal-bindings-in-python diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_ScaleX.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_ScaleX.scala index c16891871..e13af4763 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_ScaleX.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_ScaleX.scala @@ -11,10 +11,12 @@ import org.apache.spark.sql.types._ /** Returns the scale x of the raster. */ case class RST_ScaleX(raster: Expression, expressionConfig: MosaicExpressionConfig) - extends RasterExpression[RST_ScaleX](raster, DoubleType, returnsRaster = false, expressionConfig) + extends RasterExpression[RST_ScaleX](raster, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = DoubleType + /** Returns the scale x of the raster. */ override def rasterTransform(tile: MosaicRasterTile): Any = { tile.getRaster.getRaster.GetGeoTransform()(1) diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_ScaleY.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_ScaleY.scala index 3b0779763..8defba49a 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_ScaleY.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_ScaleY.scala @@ -11,10 +11,12 @@ import org.apache.spark.sql.types._ /** Returns the scale y of the raster. */ case class RST_ScaleY(raster: Expression, expressionConfig: MosaicExpressionConfig) - extends RasterExpression[RST_ScaleY](raster, DoubleType, returnsRaster = false, expressionConfig) + extends RasterExpression[RST_ScaleY](raster, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = DoubleType + /** Returns the scale y of the raster. */ override def rasterTransform(tile: MosaicRasterTile): Any = { tile.getRaster.getRaster.GetGeoTransform()(5) diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_SetNoData.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_SetNoData.scala index 911271d33..f4350e7d3 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_SetNoData.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_SetNoData.scala @@ -12,22 +12,24 @@ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant} import org.apache.spark.sql.catalyst.util.ArrayData +import org.apache.spark.sql.types.DataType /** Returns a raster with the specified no data values. */ case class RST_SetNoData( - rastersExpr: Expression, + tileExpr: Expression, noDataExpr: Expression, expressionConfig: MosaicExpressionConfig ) extends Raster1ArgExpression[RST_SetNoData]( - rastersExpr, + tileExpr, noDataExpr, - RasterTileType(expressionConfig.getCellIdType), returnsRaster = true, expressionConfig = expressionConfig ) with NullIntolerant with CodegenFallback { + override def dataType: DataType = RasterTileType(expressionConfig.getCellIdType, tileExpr) + /** * Returns a raster with the specified no data values. * @param tile diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_SkewX.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_SkewX.scala index ee3d0c4dd..439592e73 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_SkewX.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_SkewX.scala @@ -11,10 +11,12 @@ import org.apache.spark.sql.types._ /** Returns the skew x of the raster. */ case class RST_SkewX(raster: Expression, expressionConfig: MosaicExpressionConfig) - extends RasterExpression[RST_SkewX](raster, DoubleType, returnsRaster = false, expressionConfig) + extends RasterExpression[RST_SkewX](raster, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = DoubleType + /** Returns the skew x of the raster. */ override def rasterTransform(tile: MosaicRasterTile): Any = { tile.getRaster.getRaster.GetGeoTransform()(2) diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_SkewY.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_SkewY.scala index ff9903687..1f259b5de 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_SkewY.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_SkewY.scala @@ -11,10 +11,12 @@ import org.apache.spark.sql.types._ /** Returns the skew y of the raster. */ case class RST_SkewY(raster: Expression, expressionConfig: MosaicExpressionConfig) - extends RasterExpression[RST_SkewY](raster, DoubleType, returnsRaster = false, expressionConfig) + extends RasterExpression[RST_SkewY](raster, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = DoubleType + /** Returns the skew y of the raster. */ override def rasterTransform(tile: MosaicRasterTile): Any = { tile.getRaster.getRaster.GetGeoTransform()(4) diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Subdatasets.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Subdatasets.scala index 8c58e7f74..3f1536510 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Subdatasets.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Subdatasets.scala @@ -13,13 +13,14 @@ import org.apache.spark.sql.types._ case class RST_Subdatasets(raster: Expression, expressionConfig: MosaicExpressionConfig) extends RasterExpression[RST_Subdatasets]( raster, - MapType(StringType, StringType), returnsRaster = false, expressionConfig ) with NullIntolerant with CodegenFallback { + override def dataType: DataType = MapType(StringType, StringType) + /** Returns the subdatasets of the raster. */ override def rasterTransform(tile: MosaicRasterTile): Any = buildMapString(tile.getRaster.subdatasets) diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Summary.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Summary.scala index 6351d47f2..4900eaab2 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Summary.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Summary.scala @@ -16,10 +16,12 @@ import java.util.{Vector => JVector} /** Returns the summary info the raster. */ case class RST_Summary(raster: Expression, expressionConfig: MosaicExpressionConfig) - extends RasterExpression[RST_Summary](raster, StringType, returnsRaster = false, expressionConfig: MosaicExpressionConfig) + extends RasterExpression[RST_Summary](raster, returnsRaster = false, expressionConfig: MosaicExpressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = StringType + /** Returns the summary info the raster. */ override def rasterTransform(tile: MosaicRasterTile): Any = { val vector = new JVector[String]() diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_TryOpen.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_TryOpen.scala index b364d39da..16dc25ee0 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_TryOpen.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_TryOpen.scala @@ -11,10 +11,12 @@ import org.apache.spark.sql.types._ /** Returns true if the raster is empty. */ case class RST_TryOpen(raster: Expression, expressionConfig: MosaicExpressionConfig) - extends RasterExpression[RST_TryOpen](raster, BooleanType, returnsRaster = false, expressionConfig) + extends RasterExpression[RST_TryOpen](raster, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = BooleanType + /** Returns true if the raster can be opened. */ override def rasterTransform(tile: MosaicRasterTile): Any = { Option(tile.getRaster.getRaster).isDefined diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_UpperLeftX.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_UpperLeftX.scala index 4f050bc7e..143158736 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_UpperLeftX.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_UpperLeftX.scala @@ -11,10 +11,12 @@ import org.apache.spark.sql.types._ /** Returns the upper left x of the raster. */ case class RST_UpperLeftX(raster: Expression, expressionConfig: MosaicExpressionConfig) - extends RasterExpression[RST_UpperLeftX](raster, DoubleType, returnsRaster = false, expressionConfig) + extends RasterExpression[RST_UpperLeftX](raster, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = DoubleType + /** Returns the upper left x of the raster. */ override def rasterTransform(tile: MosaicRasterTile): Any = { tile.getRaster.getRaster.GetGeoTransform()(0) diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_UpperLeftY.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_UpperLeftY.scala index 0e052e3ae..702c8a0c4 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_UpperLeftY.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_UpperLeftY.scala @@ -11,10 +11,12 @@ import org.apache.spark.sql.types._ /** Returns the upper left y of the raster. */ case class RST_UpperLeftY(raster: Expression, expressionConfig: MosaicExpressionConfig) - extends RasterExpression[RST_UpperLeftY](raster, DoubleType, returnsRaster = false, expressionConfig) + extends RasterExpression[RST_UpperLeftY](raster, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = DoubleType + /** Returns the upper left y of the raster. */ override def rasterTransform(tile: MosaicRasterTile): Any = { tile.getRaster.getRaster.GetGeoTransform()(3) diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Width.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Width.scala index 4bd56686a..953eb17bd 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Width.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Width.scala @@ -11,10 +11,12 @@ import org.apache.spark.sql.types._ /** Returns the width of the raster. */ case class RST_Width(raster: Expression, expressionConfig: MosaicExpressionConfig) - extends RasterExpression[RST_Width](raster, IntegerType, returnsRaster = false, expressionConfig) + extends RasterExpression[RST_Width](raster, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = IntegerType + /** Returns the width of the raster. */ override def rasterTransform(tile: MosaicRasterTile): Any = tile.getRaster.xSize diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_WorldToRasterCoord.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_WorldToRasterCoord.scala index 2d0884a81..2d5438c3b 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_WorldToRasterCoord.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_WorldToRasterCoord.scala @@ -9,6 +9,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant} +import org.apache.spark.sql.types.DataType /** Returns the world coordinate of the raster. */ case class RST_WorldToRasterCoord( @@ -16,10 +17,12 @@ case class RST_WorldToRasterCoord( x: Expression, y: Expression, expressionConfig: MosaicExpressionConfig -) extends Raster2ArgExpression[RST_WorldToRasterCoord](raster, x, y, PixelCoordsType, returnsRaster = false, expressionConfig) +) extends Raster2ArgExpression[RST_WorldToRasterCoord](raster, x, y, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: DataType = PixelCoordsType + /** * Returns the x and y of the raster by applying GeoTransform as a tuple of * Integers. This will ensure projection of the raster is respected. diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_WorldToRasterCoordX.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_WorldToRasterCoordX.scala index 26c888fe1..41d6e8b9b 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_WorldToRasterCoordX.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_WorldToRasterCoordX.scala @@ -16,10 +16,12 @@ case class RST_WorldToRasterCoordX( x: Expression, y: Expression, expressionConfig: MosaicExpressionConfig -) extends Raster2ArgExpression[RST_WorldToRasterCoordX](raster, x, y, IntegerType, returnsRaster = false, expressionConfig) +) extends Raster2ArgExpression[RST_WorldToRasterCoordX](raster, x, y, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: IntegerType = IntegerType + /** * Returns the x coordinate of the raster by applying GeoTransform. This * will ensure projection of the raster is respected. diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_WorldToRasterCoordY.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_WorldToRasterCoordY.scala index 8bb125faa..62ba72228 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_WorldToRasterCoordY.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_WorldToRasterCoordY.scala @@ -16,10 +16,12 @@ case class RST_WorldToRasterCoordY( x: Expression, y: Expression, expressionConfig: MosaicExpressionConfig -) extends Raster2ArgExpression[RST_WorldToRasterCoordY](raster, x, y, IntegerType, returnsRaster = false, expressionConfig) +) extends Raster2ArgExpression[RST_WorldToRasterCoordY](raster, x, y, returnsRaster = false, expressionConfig) with NullIntolerant with CodegenFallback { + override def dataType: IntegerType = IntegerType + /** * Returns the y coordinate of the raster by applying GeoTransform. This * will ensure projection of the raster is respected. diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/Raster1ArgExpression.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/Raster1ArgExpression.scala index f01027ff1..35ad927c6 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/Raster1ArgExpression.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/Raster1ArgExpression.scala @@ -2,12 +2,12 @@ package com.databricks.labs.mosaic.expressions.raster.base import com.databricks.labs.mosaic.core.raster.api.GDAL import com.databricks.labs.mosaic.core.raster.io.RasterCleaner +import com.databricks.labs.mosaic.core.types.RasterTileType import com.databricks.labs.mosaic.core.types.model.MosaicRasterTile import com.databricks.labs.mosaic.expressions.base.GenericExpressionFactory import com.databricks.labs.mosaic.functions.MosaicExpressionConfig import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Expression, NullIntolerant} -import org.apache.spark.sql.types.DataType import scala.reflect.ClassTag @@ -21,8 +21,6 @@ import scala.reflect.ClassTag * containing the raster file content. * @param arg1Expr * The expression for the first argument. - * @param outputType - * The output type of the result. * @param expressionConfig * Additional arguments for the expression (expressionConfigs). * @tparam T @@ -31,7 +29,6 @@ import scala.reflect.ClassTag abstract class Raster1ArgExpression[T <: Expression: ClassTag]( rasterExpr: Expression, arg1Expr: Expression, - outputType: DataType, returnsRaster: Boolean, expressionConfig: MosaicExpressionConfig ) extends BinaryExpression @@ -43,9 +40,6 @@ abstract class Raster1ArgExpression[T <: Expression: ClassTag]( override def right: Expression = arg1Expr - /** Output Data Type */ - override def dataType: DataType = outputType - /** * The function to be overridden by the extending class. It is called when * the expression is evaluated. It provides the raster and the arguments to @@ -75,10 +69,15 @@ abstract class Raster1ArgExpression[T <: Expression: ClassTag]( // noinspection DuplicatedCode override def nullSafeEval(input: Any, arg1: Any): Any = { GDAL.enable(expressionConfig) - val tile = MosaicRasterTile.deserialize(input.asInstanceOf[InternalRow], expressionConfig.getCellIdType) + val rasterType = RasterTileType(rasterExpr).rasterType + val tile = MosaicRasterTile.deserialize( + input.asInstanceOf[InternalRow], + expressionConfig.getCellIdType, + rasterType + ) val raster = tile.getRaster val result = rasterTransform(tile, arg1) - val serialized = serialize(result, returnsRaster, outputType, expressionConfig) + val serialized = serialize(result, returnsRaster, rasterType, expressionConfig) RasterCleaner.dispose(raster) RasterCleaner.dispose(result) serialized diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/Raster2ArgExpression.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/Raster2ArgExpression.scala index ccdc7d5b3..c5be60724 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/Raster2ArgExpression.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/Raster2ArgExpression.scala @@ -2,6 +2,7 @@ package com.databricks.labs.mosaic.expressions.raster.base import com.databricks.labs.mosaic.core.raster.api.GDAL import com.databricks.labs.mosaic.core.raster.io.RasterCleaner +import com.databricks.labs.mosaic.core.types.RasterTileType import com.databricks.labs.mosaic.core.types.model.MosaicRasterTile import com.databricks.labs.mosaic.expressions.base.GenericExpressionFactory import com.databricks.labs.mosaic.functions.MosaicExpressionConfig @@ -22,8 +23,6 @@ import scala.reflect.ClassTag * The expression for the first argument. * @param arg2Expr * The expression for the second argument. - * @param outputType - * The output type of the result. * @param expressionConfig * Additional arguments for the expression (expressionConfigs). * @tparam T @@ -33,7 +32,6 @@ abstract class Raster2ArgExpression[T <: Expression: ClassTag]( rasterExpr: Expression, arg1Expr: Expression, arg2Expr: Expression, - outputType: DataType, returnsRaster: Boolean, expressionConfig: MosaicExpressionConfig ) extends TernaryExpression @@ -47,9 +45,6 @@ abstract class Raster2ArgExpression[T <: Expression: ClassTag]( override def third: Expression = arg2Expr - /** Output Data Type */ - override def dataType: DataType = outputType - /** * The function to be overridden by the extending class. It is called when * the expression is evaluated. It provides the raster and the arguments to @@ -83,9 +78,14 @@ abstract class Raster2ArgExpression[T <: Expression: ClassTag]( // noinspection DuplicatedCode override def nullSafeEval(input: Any, arg1: Any, arg2: Any): Any = { GDAL.enable(expressionConfig) - val tile = MosaicRasterTile.deserialize(input.asInstanceOf[InternalRow], expressionConfig.getCellIdType) + val rasterType = RasterTileType(rasterExpr).rasterType + val tile = MosaicRasterTile.deserialize( + input.asInstanceOf[InternalRow], + expressionConfig.getCellIdType, + rasterType + ) val result = rasterTransform(tile, arg1, arg2) - val serialized = serialize(result, returnsRaster, outputType, expressionConfig) + val serialized = serialize(result, returnsRaster, rasterType, expressionConfig) // passed by name makes things re-evaluated RasterCleaner.dispose(tile) serialized diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterArray1ArgExpression.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterArray1ArgExpression.scala index d21f96c2d..5dbfd08cc 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterArray1ArgExpression.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterArray1ArgExpression.scala @@ -2,11 +2,12 @@ package com.databricks.labs.mosaic.expressions.raster.base import com.databricks.labs.mosaic.core.raster.api.GDAL import com.databricks.labs.mosaic.core.raster.io.RasterCleaner +import com.databricks.labs.mosaic.core.types.RasterTileType import com.databricks.labs.mosaic.core.types.model.MosaicRasterTile import com.databricks.labs.mosaic.expressions.base.GenericExpressionFactory import com.databricks.labs.mosaic.functions.MosaicExpressionConfig import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Expression, NullIntolerant} -import org.apache.spark.sql.types.{ArrayType, DataType} +import org.apache.spark.sql.types.ArrayType import scala.reflect.ClassTag @@ -18,8 +19,6 @@ import scala.reflect.ClassTag * @param rastersExpr * The rasters expression. It is an array column containing rasters as either * paths or as content byte arrays. - * @param outputType - * The output type of the result. * @param expressionConfig * Additional arguments for the expression (expressionConfigs). * @tparam T @@ -28,7 +27,6 @@ import scala.reflect.ClassTag abstract class RasterArray1ArgExpression[T <: Expression: ClassTag]( rastersExpr: Expression, arg1Expr: Expression, - outputType: DataType, returnsRaster: Boolean, expressionConfig: MosaicExpressionConfig ) extends BinaryExpression @@ -36,9 +34,6 @@ abstract class RasterArray1ArgExpression[T <: Expression: ClassTag]( with Serializable with RasterExpressionSerialization { - /** Output Data Type */ - override def dataType: DataType = if (returnsRaster) rastersExpr.dataType.asInstanceOf[ArrayType].elementType else outputType - override def left: Expression = rastersExpr override def right: Expression = arg1Expr @@ -72,7 +67,8 @@ abstract class RasterArray1ArgExpression[T <: Expression: ClassTag]( GDAL.enable(expressionConfig) val tiles = RasterArrayUtils.getTiles(input, rastersExpr, expressionConfig) val result = rasterTransform(tiles, arg1) - val serialized = serialize(result, returnsRaster, dataType, expressionConfig) + val resultType = if (returnsRaster) RasterTileType(rastersExpr).rasterType else dataType + val serialized = serialize(result, returnsRaster, resultType, expressionConfig) tiles.foreach(t => RasterCleaner.dispose(t)) serialized } diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterArray2ArgExpression.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterArray2ArgExpression.scala index a26082f2d..9de963684 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterArray2ArgExpression.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterArray2ArgExpression.scala @@ -2,11 +2,12 @@ package com.databricks.labs.mosaic.expressions.raster.base import com.databricks.labs.mosaic.core.raster.api.GDAL import com.databricks.labs.mosaic.core.raster.io.RasterCleaner +import com.databricks.labs.mosaic.core.types.RasterTileType import com.databricks.labs.mosaic.core.types.model.MosaicRasterTile import com.databricks.labs.mosaic.expressions.base.GenericExpressionFactory import com.databricks.labs.mosaic.functions.MosaicExpressionConfig import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant, TernaryExpression} -import org.apache.spark.sql.types.{ArrayType, DataType} +import org.apache.spark.sql.types.ArrayType import scala.reflect.ClassTag @@ -18,8 +19,6 @@ import scala.reflect.ClassTag * @param rastersExpr * The rasters expression. It is an array column containing rasters as either * paths or as content byte arrays. - * @param outputType - * The output type of the result. * @param expressionConfig * Additional arguments for the expression (expressionConfigs). * @tparam T @@ -29,7 +28,6 @@ abstract class RasterArray2ArgExpression[T <: Expression: ClassTag]( rastersExpr: Expression, arg1Expr: Expression, arg2Expr: Expression, - outputType: DataType, returnsRaster: Boolean, expressionConfig: MosaicExpressionConfig ) extends TernaryExpression @@ -37,9 +35,6 @@ abstract class RasterArray2ArgExpression[T <: Expression: ClassTag]( with Serializable with RasterExpressionSerialization { - /** Output Data Type */ - override def dataType: DataType = if (returnsRaster) rastersExpr.dataType.asInstanceOf[ArrayType].elementType else outputType - override def first: Expression = rastersExpr override def second: Expression = arg1Expr @@ -77,7 +72,8 @@ abstract class RasterArray2ArgExpression[T <: Expression: ClassTag]( GDAL.enable(expressionConfig) val tiles = RasterArrayUtils.getTiles(input, rastersExpr, expressionConfig) val result = rasterTransform(tiles, arg1, arg2) - val serialized = serialize(result, returnsRaster, dataType, expressionConfig) + val resultType = if (returnsRaster) RasterTileType(rastersExpr).rasterType else dataType + val serialized = serialize(result, returnsRaster, resultType, expressionConfig) tiles.foreach(t => RasterCleaner.dispose(t)) serialized } diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterArrayExpression.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterArrayExpression.scala index b8ad9fc12..8c3a52d9a 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterArrayExpression.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterArrayExpression.scala @@ -2,11 +2,12 @@ package com.databricks.labs.mosaic.expressions.raster.base import com.databricks.labs.mosaic.core.raster.api.GDAL import com.databricks.labs.mosaic.core.raster.io.RasterCleaner +import com.databricks.labs.mosaic.core.types.RasterTileType import com.databricks.labs.mosaic.core.types.model.MosaicRasterTile import com.databricks.labs.mosaic.expressions.base.GenericExpressionFactory import com.databricks.labs.mosaic.functions.MosaicExpressionConfig import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant, UnaryExpression} -import org.apache.spark.sql.types.{ArrayType, DataType} +import org.apache.spark.sql.types.ArrayType import scala.reflect.ClassTag @@ -27,7 +28,6 @@ import scala.reflect.ClassTag */ abstract class RasterArrayExpression[T <: Expression: ClassTag]( rastersExpr: Expression, - outputType: DataType, returnsRaster: Boolean, expressionConfig: MosaicExpressionConfig ) extends UnaryExpression @@ -37,9 +37,6 @@ abstract class RasterArrayExpression[T <: Expression: ClassTag]( override def child: Expression = rastersExpr - /** Output Data Type */ - override def dataType: DataType = if (returnsRaster) rastersExpr.dataType.asInstanceOf[ArrayType].elementType else outputType - /** * The function to be overridden by the extending class. It is called when * the expression is evaluated. It provides the rasters to the expression. @@ -67,7 +64,8 @@ abstract class RasterArrayExpression[T <: Expression: ClassTag]( GDAL.enable(expressionConfig) val tiles = RasterArrayUtils.getTiles(input, rastersExpr, expressionConfig) val result = rasterTransform(tiles) - val serialized = serialize(result, returnsRaster, dataType, expressionConfig) + val resultType = if (returnsRaster) RasterTileType(rastersExpr).rasterType else dataType + val serialized = serialize(result, returnsRaster, resultType, expressionConfig) tiles.foreach(t => RasterCleaner.dispose(t)) serialized } diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterArrayUtils.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterArrayUtils.scala index 3162bb421..f2d399350 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterArrayUtils.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterArrayUtils.scala @@ -1,5 +1,6 @@ package com.databricks.labs.mosaic.expressions.raster.base +import com.databricks.labs.mosaic.core.types.RasterTileType import com.databricks.labs.mosaic.core.types.model.MosaicRasterTile import com.databricks.labs.mosaic.functions.MosaicExpressionConfig import org.apache.spark.sql.catalyst.InternalRow @@ -12,11 +13,16 @@ object RasterArrayUtils { def getTiles(input: Any, rastersExpr: Expression, expressionConfig: MosaicExpressionConfig): Seq[MosaicRasterTile] = { val rasterDT = rastersExpr.dataType.asInstanceOf[ArrayType].elementType val arrayData = input.asInstanceOf[ArrayData] + val rasterType = RasterTileType(rastersExpr).rasterType val n = arrayData.numElements() (0 until n) .map(i => MosaicRasterTile - .deserialize(arrayData.get(i, rasterDT).asInstanceOf[InternalRow], expressionConfig.getCellIdType) + .deserialize( + arrayData.get(i, rasterDT).asInstanceOf[InternalRow], + expressionConfig.getCellIdType, + rasterType + ) ) } diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterBandExpression.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterBandExpression.scala index 7cee607ca..97bd3e333 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterBandExpression.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterBandExpression.scala @@ -3,12 +3,12 @@ package com.databricks.labs.mosaic.expressions.raster.base import com.databricks.labs.mosaic.core.raster.api.GDAL import com.databricks.labs.mosaic.core.raster.gdal.MosaicRasterBandGDAL import com.databricks.labs.mosaic.core.raster.io.RasterCleaner +import com.databricks.labs.mosaic.core.types.RasterTileType import com.databricks.labs.mosaic.core.types.model.MosaicRasterTile import com.databricks.labs.mosaic.expressions.base.GenericExpressionFactory import com.databricks.labs.mosaic.functions.MosaicExpressionConfig import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Expression, NullIntolerant} -import org.apache.spark.sql.types.DataType import scala.reflect.ClassTag @@ -23,8 +23,6 @@ import scala.reflect.ClassTag * MOSAIC_RASTER_STORAGE is set to MOSAIC_RASTER_STORAGE_BYTE. * @param bandExpr * The expression for the band index. - * @param outputType - * The output type of the result. * @param expressionConfig * Additional arguments for the expression (expressionConfigs). * @tparam T @@ -33,7 +31,6 @@ import scala.reflect.ClassTag abstract class RasterBandExpression[T <: Expression: ClassTag]( rasterExpr: Expression, bandExpr: Expression, - outputType: DataType, returnsRaster: Boolean, expressionConfig: MosaicExpressionConfig ) extends BinaryExpression @@ -45,9 +42,6 @@ abstract class RasterBandExpression[T <: Expression: ClassTag]( override def right: Expression = bandExpr - /** Output Data Type */ - override def dataType: DataType = outputType - /** * The function to be overridden by the extending class. It is called when * the expression is evaluated. It provides the raster band to the @@ -79,13 +73,18 @@ abstract class RasterBandExpression[T <: Expression: ClassTag]( // noinspection DuplicatedCode override def nullSafeEval(inputRaster: Any, inputBand: Any): Any = { GDAL.enable(expressionConfig) - val tile = MosaicRasterTile.deserialize(inputRaster.asInstanceOf[InternalRow], expressionConfig.getCellIdType) + val rasterType = RasterTileType(rasterExpr).rasterType + val tile = MosaicRasterTile.deserialize( + inputRaster.asInstanceOf[InternalRow], + expressionConfig.getCellIdType, + rasterType + ) val bandIndex = inputBand.asInstanceOf[Int] val band = tile.getRaster.getBand(bandIndex) val result = bandTransform(tile, band) - val serialized = serialize(result, returnsRaster, dataType, expressionConfig) + val serialized = serialize(result, returnsRaster, rasterType, expressionConfig) RasterCleaner.dispose(tile) serialized } diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterExpression.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterExpression.scala index 462d3204b..66435f101 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterExpression.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterExpression.scala @@ -3,6 +3,7 @@ package com.databricks.labs.mosaic.expressions.raster.base import com.databricks.labs.mosaic.core.index.{IndexSystem, IndexSystemFactory} import com.databricks.labs.mosaic.core.raster.api.GDAL import com.databricks.labs.mosaic.core.raster.io.RasterCleaner +import com.databricks.labs.mosaic.core.types.RasterTileType import com.databricks.labs.mosaic.core.types.model.MosaicRasterTile import com.databricks.labs.mosaic.expressions.base.GenericExpressionFactory import com.databricks.labs.mosaic.functions.MosaicExpressionConfig @@ -20,8 +21,6 @@ import scala.reflect.ClassTag * The expression for the raster. If the raster is stored on disc, the path * to the raster is provided. If the raster is stored in memory, the bytes of * the raster are provided. - * @param outputType - * The output type of the result. * @param expressionConfig * Additional arguments for the expression (expressionConfigs). * @tparam T @@ -29,7 +28,6 @@ import scala.reflect.ClassTag */ abstract class RasterExpression[T <: Expression: ClassTag]( rasterExpr: Expression, - outputType: DataType, returnsRaster: Boolean, expressionConfig: MosaicExpressionConfig ) extends UnaryExpression @@ -43,9 +41,6 @@ abstract class RasterExpression[T <: Expression: ClassTag]( override def child: Expression = rasterExpr - /** Output Data Type */ - override def dataType: DataType = outputType - /** * The function to be overridden by the extending class. It is called when * the expression is evaluated. It provides the raster to the expression. @@ -69,9 +64,14 @@ abstract class RasterExpression[T <: Expression: ClassTag]( */ override def nullSafeEval(input: Any): Any = { GDAL.enable(expressionConfig) - val tile = MosaicRasterTile.deserialize(input.asInstanceOf[InternalRow], cellIdDataType) + val rasterType = RasterTileType(rasterExpr).rasterType + val tile = MosaicRasterTile.deserialize( + input.asInstanceOf[InternalRow], + cellIdDataType, + rasterType + ) val result = rasterTransform(tile) - val serialized = serialize(result, returnsRaster, dataType, expressionConfig) + val serialized = serialize(result, returnsRaster, rasterType, expressionConfig) RasterCleaner.dispose(tile) serialized } diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterExpressionSerialization.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterExpressionSerialization.scala index a9bf17917..dc04cb1c7 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterExpressionSerialization.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterExpressionSerialization.scala @@ -35,11 +35,9 @@ trait RasterExpressionSerialization { ): Any = { if (returnsRaster) { val tile = data.asInstanceOf[MosaicRasterTile] - val checkpoint = expressionConfig.getRasterCheckpoint - val rasterType = outputDataType.asInstanceOf[StructType].fields(1).dataType val result = tile .formatCellId(IndexSystemFactory.getIndexSystem(expressionConfig.getIndexSystem)) - .serialize(rasterType, checkpoint) + .serialize(outputDataType) RasterCleaner.dispose(tile) result } else { diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterGeneratorExpression.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterGeneratorExpression.scala index 29c714788..3fc80752d 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterGeneratorExpression.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterGeneratorExpression.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag * rasters based on the input raster. The new rasters are written in the * checkpoint directory. The files are written as GeoTiffs. Subdatasets are not * supported, please flatten beforehand. - * @param rasterExpr + * @param tileExpr * The expression for the raster. If the raster is stored on disc, the path * to the raster is provided. If the raster is stored in memory, the bytes of * the raster are provided. @@ -32,13 +32,13 @@ import scala.reflect.ClassTag * The type of the extending class. */ abstract class RasterGeneratorExpression[T <: Expression: ClassTag]( - rasterExpr: Expression, + tileExpr: Expression, expressionConfig: MosaicExpressionConfig ) extends CollectionGenerator with NullIntolerant with Serializable { - override def dataType: DataType = RasterTileType(expressionConfig.getCellIdType) + override def dataType: DataType = RasterTileType(expressionConfig.getCellIdType, tileExpr) val uuid: String = java.util.UUID.randomUUID().toString.replace("-", "_") @@ -72,11 +72,12 @@ abstract class RasterGeneratorExpression[T <: Expression: ClassTag]( override def eval(input: InternalRow): TraversableOnce[InternalRow] = { GDAL.enable(expressionConfig) - val tile = MosaicRasterTile.deserialize(rasterExpr.eval(input).asInstanceOf[InternalRow], cellIdDataType) + val rasterType = RasterTileType(tileExpr).rasterType + val tile = MosaicRasterTile.deserialize(tileExpr.eval(input).asInstanceOf[InternalRow], cellIdDataType, rasterType) val generatedRasters = rasterGenerator(tile) // Writing rasters disposes of the written raster - val rows = generatedRasters.map(_.formatCellId(indexSystem).serialize()) + val rows = generatedRasters.map(_.formatCellId(indexSystem).serialize(rasterType)) generatedRasters.foreach(gr => RasterCleaner.dispose(gr)) RasterCleaner.dispose(tile) diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterTessellateGeneratorExpression.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterTessellateGeneratorExpression.scala index f2545942b..98ff86ca7 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterTessellateGeneratorExpression.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterTessellateGeneratorExpression.scala @@ -23,7 +23,7 @@ import scala.reflect.ClassTag * checkpoint directory. The files are written as GeoTiffs. Subdatasets are not * supported, please flatten beforehand. * - * @param rasterExpr + * @param tileExpr * The expression for the raster. If the raster is stored on disc, the path * to the raster is provided. If the raster is stored in memory, the bytes of * the raster are provided. @@ -33,7 +33,7 @@ import scala.reflect.ClassTag * The type of the extending class. */ abstract class RasterTessellateGeneratorExpression[T <: Expression: ClassTag]( - rasterExpr: Expression, + tileExpr: Expression, resolutionExpr: Expression, expressionConfig: MosaicExpressionConfig ) extends CollectionGenerator @@ -55,7 +55,8 @@ abstract class RasterTessellateGeneratorExpression[T <: Expression: ClassTag]( * needs to be wrapped in a StructType. The actually type is that of the * structs element. */ - override def elementSchema: StructType = StructType(Array(StructField("element", RasterTileType(indexSystem.getCellIdDataType)))) + override def elementSchema: StructType = + StructType(Array(StructField("element", RasterTileType(indexSystem.getCellIdDataType, tileExpr)))) /** * The function to be overridden by the extending class. It is called when @@ -71,17 +72,15 @@ abstract class RasterTessellateGeneratorExpression[T <: Expression: ClassTag]( override def eval(input: InternalRow): TraversableOnce[InternalRow] = { GDAL.enable(expressionConfig) + val rasterType = RasterTileType(tileExpr).rasterType val tile = MosaicRasterTile - .deserialize( - rasterExpr.eval(input).asInstanceOf[InternalRow], - indexSystem.getCellIdDataType - ) + .deserialize(tileExpr.eval(input).asInstanceOf[InternalRow], indexSystem.getCellIdDataType, rasterType) val inResolution: Int = indexSystem.getResolution(resolutionExpr.eval(input)) val generatedChips = rasterGenerator(tile, inResolution) .map(chip => chip.formatCellId(indexSystem)) val rows = generatedChips - .map(chip => InternalRow.fromSeq(Seq(chip.formatCellId(indexSystem).serialize()))) + .map(chip => InternalRow.fromSeq(Seq(chip.formatCellId(indexSystem).serialize(rasterType)))) RasterCleaner.dispose(tile) generatedChips.foreach(chip => RasterCleaner.dispose(chip)) diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterToGridExpression.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterToGridExpression.scala index 743f9cbd6..e7b04f989 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterToGridExpression.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/base/RasterToGridExpression.scala @@ -37,11 +37,13 @@ abstract class RasterToGridExpression[T <: Expression: ClassTag, P]( resolution: Expression, measureType: DataType, expressionConfig: MosaicExpressionConfig -) extends Raster1ArgExpression[T](rasterExpr, resolution, RasterToGridType(expressionConfig.getCellIdType, measureType), returnsRaster = false, expressionConfig) +) extends Raster1ArgExpression[T](rasterExpr, resolution, returnsRaster = false, expressionConfig) with RasterGridExpression with NullIntolerant with Serializable { + override def dataType: DataType = RasterToGridType(expressionConfig.getCellIdType, measureType) + /** The index system to be used. */ val indexSystem: IndexSystem = IndexSystemFactory.getIndexSystem(expressionConfig.getIndexSystem) val geometryAPI: GeometryAPI = GeometryAPI(expressionConfig.getGeometryAPI) diff --git a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala index 2b85c9785..8398e6882 100644 --- a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala +++ b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala @@ -8,20 +8,19 @@ import com.databricks.labs.mosaic.core.types.ChipType import com.databricks.labs.mosaic.datasource.multiread.MosaicDataFrameReader import com.databricks.labs.mosaic.expressions.constructors._ import com.databricks.labs.mosaic.expressions.format._ -import com.databricks.labs.mosaic.expressions.geometry._ import com.databricks.labs.mosaic.expressions.geometry.ST_MinMaxXYZ._ +import com.databricks.labs.mosaic.expressions.geometry._ import com.databricks.labs.mosaic.expressions.index._ import com.databricks.labs.mosaic.expressions.raster._ import com.databricks.labs.mosaic.expressions.util.TrySql -import com.databricks.labs.mosaic.functions.MosaicContext.mosaicVersion import com.databricks.labs.mosaic.utils.FileUtils import org.apache.spark.internal.Logging -import org.apache.spark.sql.{Column, SparkSession} import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{LongType, StringType} +import org.apache.spark.sql.{Column, SparkSession} import scala.reflect.runtime.universe @@ -270,6 +269,7 @@ class MosaicContext(indexSystem: IndexSystem, geometryAPI: GeometryAPI) extends mosaicRegistry.registerExpression[RST_Height](expressionConfig) mosaicRegistry.registerExpression[RST_InitNoData](expressionConfig) mosaicRegistry.registerExpression[RST_IsEmpty](expressionConfig) + mosaicRegistry.registerExpression[RST_MakeTiles](expressionConfig) mosaicRegistry.registerExpression[RST_Max](expressionConfig) mosaicRegistry.registerExpression[RST_Min](expressionConfig) mosaicRegistry.registerExpression[RST_Median](expressionConfig) @@ -655,6 +655,10 @@ class MosaicContext(indexSystem: IndexSystem, geometryAPI: GeometryAPI) extends def rst_combineavg(rasterArray: Column): Column = ColumnAdapter(RST_CombineAvg(rasterArray.expr, expressionConfig)) def rst_derivedband(raster: Column, pythonFunc: Column, funcName: Column): Column = ColumnAdapter(RST_DerivedBand(raster.expr, pythonFunc.expr, funcName.expr, expressionConfig)) + def rst_filter(raster: Column, kernelSize: Column, operation: Column): Column = + ColumnAdapter(RST_Filter(raster.expr, kernelSize.expr, operation.expr, expressionConfig)) + def rst_filter(raster: Column, kernelSize: Int, operation: String): Column = + ColumnAdapter(RST_Filter(raster.expr, lit(kernelSize).expr, lit(operation).expr, expressionConfig)) def rst_georeference(raster: Column): Column = ColumnAdapter(RST_GeoReference(raster.expr, expressionConfig)) def rst_getnodata(raster: Column): Column = ColumnAdapter(RST_GetNoData(raster.expr, expressionConfig)) def rst_getsubdataset(raster: Column, subdatasetName: Column): Column = @@ -664,6 +668,20 @@ class MosaicContext(indexSystem: IndexSystem, geometryAPI: GeometryAPI) extends def rst_height(raster: Column): Column = ColumnAdapter(RST_Height(raster.expr, expressionConfig)) def rst_initnodata(raster: Column): Column = ColumnAdapter(RST_InitNoData(raster.expr, expressionConfig)) def rst_isempty(raster: Column): Column = ColumnAdapter(RST_IsEmpty(raster.expr, expressionConfig)) + def rst_maketiles(input: Column, driver: Column, size: Column, withCheckpoint: Column): Column = + ColumnAdapter(RST_MakeTiles(input.expr, driver.expr, size.expr, withCheckpoint.expr, expressionConfig)) + def rst_maketiles(input: Column, driver: String, size: Int, withCheckpoint: Boolean): Column = + ColumnAdapter(RST_MakeTiles(input.expr, lit(driver).expr, lit(size).expr, lit(withCheckpoint).expr, expressionConfig)) + def rst_maketiles(input: Column): Column = + ColumnAdapter(RST_MakeTiles(input.expr, lit(MOSAIC_NO_DRIVER).expr, lit(-1).expr, lit(false).expr, expressionConfig)) + def rst_maketiles(input: Column, size: Int): Column = + ColumnAdapter(RST_MakeTiles(input.expr, lit(MOSAIC_NO_DRIVER).expr, lit(size).expr, lit(false).expr, expressionConfig)) + def rst_maketiles(input: Column, driver: String): Column = + ColumnAdapter(RST_MakeTiles(input.expr, lit(driver).expr, lit(-1).expr, lit(false).expr, expressionConfig)) + def rst_maketiles(input: Column, driver: String, withCheckpoint: Boolean): Column = + ColumnAdapter(RST_MakeTiles(input.expr, lit(driver).expr, lit(-1).expr, lit(withCheckpoint).expr, expressionConfig)) + def rst_maketiles(input: Column, size: Int, withCheckpoint: Boolean): Column = + ColumnAdapter(RST_MakeTiles(input.expr, lit(MOSAIC_NO_DRIVER).expr, lit(size).expr, lit(withCheckpoint).expr, expressionConfig)) def rst_max(raster: Column): Column = ColumnAdapter(RST_Max(raster.expr, expressionConfig)) def rst_min(raster: Column): Column = ColumnAdapter(RST_Min(raster.expr, expressionConfig)) def rst_median(raster: Column): Column = ColumnAdapter(RST_Median(raster.expr, expressionConfig)) diff --git a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicExpressionConfig.scala b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicExpressionConfig.scala index f306d4e9c..d6643f59b 100644 --- a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicExpressionConfig.scala +++ b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicExpressionConfig.scala @@ -33,6 +33,8 @@ case class MosaicExpressionConfig(configs: Map[String, String]) { def getRasterCheckpoint: String = configs.getOrElse(MOSAIC_RASTER_CHECKPOINT, MOSAIC_RASTER_CHECKPOINT_DEFAULT) def getCellIdType: DataType = IndexSystemFactory.getIndexSystem(getIndexSystem).cellIdType + + def getRasterBlockSize: Int = configs.getOrElse(MOSAIC_RASTER_BLOCKSIZE, MOSAIC_RASTER_BLOCKSIZE_DEFAULT).toInt def setGDALConf(conf: RuntimeConfig): MosaicExpressionConfig = { val toAdd = conf.getAll.filter(_._1.startsWith(MOSAIC_GDAL_PREFIX)) diff --git a/src/main/scala/com/databricks/labs/mosaic/gdal/MosaicGDAL.scala b/src/main/scala/com/databricks/labs/mosaic/gdal/MosaicGDAL.scala index 9e8bf1132..92438844a 100644 --- a/src/main/scala/com/databricks/labs/mosaic/gdal/MosaicGDAL.scala +++ b/src/main/scala/com/databricks/labs/mosaic/gdal/MosaicGDAL.scala @@ -1,11 +1,12 @@ package com.databricks.labs.mosaic.gdal +import com.databricks.labs.mosaic.MOSAIC_RASTER_BLOCKSIZE_DEFAULT import com.databricks.labs.mosaic.functions.{MosaicContext, MosaicExpressionConfig} import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.gdal.gdal.gdal +import org.gdal.osr.SpatialReference -import java.io.{BufferedInputStream, File, PrintWriter} import java.nio.file.{Files, Paths} import scala.language.postfixOps import scala.util.Try @@ -22,9 +23,22 @@ object MosaicGDAL extends Logging { private val libjniso3003Path = "/usr/lib/libgdalalljni.so.30.0.3" private val libogdisoPath = "/usr/lib/ogdi/4.1/libgdal.so" + val defaultBlockSize = 1024 + val vrtBlockSize = 128 // This is a must value for VRTs before GDAL 3.7 + var blockSize: Int = MOSAIC_RASTER_BLOCKSIZE_DEFAULT.toInt + // noinspection ScalaWeakerAccess val GDAL_ENABLED = "spark.mosaic.gdal.native.enabled" var isEnabled = false + var checkpointPath: String = _ + + // Only use this with GDAL rasters + val WSG84: SpatialReference = { + val wsg84 = new SpatialReference() + wsg84.ImportFromEPSG(4326) + wsg84.SetAxisMappingStrategy(org.gdal.osr.osrConstants.OAMS_TRADITIONAL_GIS_ORDER) + wsg84 + } /** Returns true if GDAL is enabled. */ def wasEnabled(spark: SparkSession): Boolean = @@ -33,15 +47,28 @@ object MosaicGDAL extends Logging { /** Configures the GDAL environment. */ def configureGDAL(mosaicConfig: MosaicExpressionConfig): Unit = { val CPL_TMPDIR = MosaicContext.tmpDir - val GDAL_PAM_PROXY_DIR = MosaicContext.tmpDir gdal.SetConfigOption("GDAL_VRT_ENABLE_PYTHON", "YES") - gdal.SetConfigOption("GDAL_DISABLE_READDIR_ON_OPEN", "EMPTY_DIR") + gdal.SetConfigOption("GDAL_DISABLE_READDIR_ON_OPEN", "TRUE") gdal.SetConfigOption("CPL_TMPDIR", CPL_TMPDIR) - gdal.SetConfigOption("GDAL_PAM_PROXY_DIR", GDAL_PAM_PROXY_DIR) - gdal.SetConfigOption("GDAL_PAM_ENABLED", "NO") - gdal.SetConfigOption("CPL_VSIL_USE_TEMP_FILE_FOR_RANDOM_WRITE", "NO") gdal.SetConfigOption("CPL_LOG", s"$CPL_TMPDIR/gdal.log") + gdal.SetConfigOption("GDAL_CACHEMAX", "512") + gdal.SetConfigOption("GDAL_NUM_THREADS", "ALL_CPUS") mosaicConfig.getGDALConf.foreach { case (k, v) => gdal.SetConfigOption(k.split("\\.").last, v) } + setBlockSize(mosaicConfig) + checkpointPath = mosaicConfig.getRasterCheckpoint + } + + def setBlockSize(mosaicConfig: MosaicExpressionConfig): Unit = { + val blockSize = mosaicConfig.getRasterBlockSize + if (blockSize > 0) { + this.blockSize = blockSize + } + } + + def setBlockSize(size: Int): Unit = { + if (size > 0) { + this.blockSize = size + } } /** Enables the GDAL environment. */ @@ -91,18 +118,19 @@ object MosaicGDAL extends Logging { } } - /** Reads the resource bytes. */ - private def readResourceBytes(name: String): Array[Byte] = { - val bis = new BufferedInputStream(getClass.getResourceAsStream(name)) - try { Stream.continually(bis.read()).takeWhile(-1 !=).map(_.toByte).toArray } - finally bis.close() - } +// /** Reads the resource bytes. */ +// private def readResourceBytes(name: String): Array[Byte] = { +// val bis = new BufferedInputStream(getClass.getResourceAsStream(name)) +// try { Stream.continually(bis.read()).takeWhile(-1 !=).map(_.toByte).toArray } +// finally bis.close() +// } + +// /** Reads the resource lines. */ +// // noinspection SameParameterValue +// private def readResourceLines(name: String): Array[String] = { +// val bytes = readResourceBytes(name) +// val lines = new String(bytes).split("\n") +// lines +// } - /** Reads the resource lines. */ - // noinspection SameParameterValue - private def readResourceLines(name: String): Array[String] = { - val bytes = readResourceBytes(name) - val lines = new String(bytes).split("\n") - lines - } } diff --git a/src/main/scala/com/databricks/labs/mosaic/package.scala b/src/main/scala/com/databricks/labs/mosaic/package.scala index 58ee2f98e..eea63cd79 100644 --- a/src/main/scala/com/databricks/labs/mosaic/package.scala +++ b/src/main/scala/com/databricks/labs/mosaic/package.scala @@ -21,13 +21,17 @@ package object mosaic { val MOSAIC_GDAL_PREFIX = "spark.databricks.labs.mosaic.gdal." val MOSAIC_GDAL_NATIVE = "spark.databricks.labs.mosaic.gdal.native" val MOSAIC_RASTER_CHECKPOINT = "spark.databricks.labs.mosaic.raster.checkpoint" - val MOSAIC_RASTER_CHECKPOINT_DEFAULT = "dbfs:/tmp/mosaic/raster/checkpoint" + val MOSAIC_RASTER_CHECKPOINT_DEFAULT = "/dbfs/tmp/mosaic/raster/checkpoint" + val MOSAIC_RASTER_BLOCKSIZE = "spark.databricks.labs.mosaic.raster.blocksize" + val MOSAIC_RASTER_BLOCKSIZE_DEFAULT = "128" val MOSAIC_RASTER_READ_STRATEGY = "raster.read.strategy" val MOSAIC_RASTER_READ_IN_MEMORY = "in_memory" val MOSAIC_RASTER_READ_AS_PATH = "as_path" val MOSAIC_RASTER_RE_TILE_ON_READ = "retile_on_read" + val MOSAIC_NO_DRIVER = "no_driver" + def read: MosaicDataFrameReader = new MosaicDataFrameReader(SparkSession.builder().getOrCreate()) diff --git a/src/main/scala/com/databricks/labs/mosaic/utils/FileUtils.scala b/src/main/scala/com/databricks/labs/mosaic/utils/FileUtils.scala index a1aac5c2f..fc01cfaa0 100644 --- a/src/main/scala/com/databricks/labs/mosaic/utils/FileUtils.scala +++ b/src/main/scala/com/databricks/labs/mosaic/utils/FileUtils.scala @@ -22,10 +22,10 @@ object FileUtils { bytes } - def createMosaicTempDir(): String = { - val tempRoot = Paths.get("/mosaic_tmp/") + def createMosaicTempDir(prefix: String = ""): String = { + val tempRoot = Paths.get(s"$prefix/mosaic_tmp/") if (!Files.exists(tempRoot)) { - Files.createDirectory(tempRoot) + Files.createDirectories(tempRoot) } val tempDir = Files.createTempDirectory(tempRoot, "mosaic") tempDir.toFile.getAbsolutePath diff --git a/src/main/scala/com/databricks/labs/mosaic/utils/PathUtils.scala b/src/main/scala/com/databricks/labs/mosaic/utils/PathUtils.scala index d48c03bfd..469bb0f44 100644 --- a/src/main/scala/com/databricks/labs/mosaic/utils/PathUtils.scala +++ b/src/main/scala/com/databricks/labs/mosaic/utils/PathUtils.scala @@ -1,7 +1,5 @@ package com.databricks.labs.mosaic.utils -import com.databricks.labs.mosaic.core.raster.api.GDAL -import com.databricks.labs.mosaic.core.raster.gdal.MosaicRasterGDAL import com.databricks.labs.mosaic.functions.MosaicContext import java.nio.file.{Files, Paths} @@ -10,11 +8,15 @@ object PathUtils { val NO_PATH_STRING = "no_path" - def getCleanPath(path: String): String = { - val cleanPath = path + def replaceDBFSTokens(path: String): String = { + path .replace("file:/", "/") .replace("dbfs:/Volumes", "/Volumes") - .replace("dbfs:/","/dbfs/") + .replace("dbfs:/", "/dbfs/") + } + + def getCleanPath(path: String): String = { + val cleanPath = replaceDBFSTokens(path) if (cleanPath.endsWith(".zip") || cleanPath.contains(".zip:")) { getZipPath(cleanPath) } else { @@ -61,17 +63,51 @@ object PathUtils { if (filePath.endsWith("\"")) result = result.dropRight(1) result } + + def getStemRegex(path: String): String = { + val cleanPath = replaceDBFSTokens(path) + val fileName = Paths.get(cleanPath).getFileName.toString + val stemName = fileName.substring(0, fileName.lastIndexOf(".")) + val stemEscaped = stemName.replace(".", "\\.") + val stemRegex = s"$stemEscaped\\..*".r + stemRegex.toString + } - def copyToTmp(inPath: String): String = { - val copyFromPath = inPath - .replace("file:/", "/") - .replace("dbfs:/Volumes", "/Volumes") - .replace("dbfs:/","/dbfs/") - val driver = MosaicRasterGDAL.identifyDriver(getCleanPath(inPath)) - val extension = if (inPath.endsWith(".zip")) "zip" else GDAL.getExtension(driver) - val tmpPath = createTmpFilePath(extension) - Files.copy(Paths.get(copyFromPath), Paths.get(tmpPath)) - tmpPath + def copyToTmp(inPath: String): String = { + val copyFromPath = replaceDBFSTokens(inPath) + val inPathDir = Paths.get(copyFromPath).getParent.toString + + val fullFileName = copyFromPath.split("/").last + val stemRegex = getStemRegex(inPath) + + wildcardCopy(inPathDir, MosaicContext.tmpDir, stemRegex.toString) + + s"${MosaicContext.tmpDir}/$fullFileName" + } + + def wildcardCopy(inDirPath: String, outDirPath: String, pattern: String): Unit = { + import org.apache.commons.io.FileUtils + val copyFromPath = replaceDBFSTokens(inDirPath) + val copyToPath = replaceDBFSTokens(outDirPath) + + val toCopy = Files + .list(Paths.get(copyFromPath)) + .filter(_.getFileName.toString.matches(pattern)) + + toCopy.forEach(path => { + val destination = Paths.get(copyToPath, path.getFileName.toString) + //noinspection SimplifyBooleanMatch + Files.isDirectory(path) match { + case true => FileUtils.copyDirectory(path.toFile, destination.toFile) + case false => Files.copy(path, destination) + } + }) + } + + def parseUnzippedPathFromExtracted(lastExtracted: String, extension: String): String = { + val trimmed = lastExtracted.replace("extracting: ", "").replace(" ", "") + val indexOfFormat = trimmed.indexOf(s".$extension/") + trimmed.substring(0, indexOfFormat + extension.length + 1) } } diff --git a/src/main/scala/com/databricks/labs/mosaic/utils/SysUtils.scala b/src/main/scala/com/databricks/labs/mosaic/utils/SysUtils.scala index 85fa12785..ba1d9c417 100644 --- a/src/main/scala/com/databricks/labs/mosaic/utils/SysUtils.scala +++ b/src/main/scala/com/databricks/labs/mosaic/utils/SysUtils.scala @@ -1,6 +1,6 @@ package com.databricks.labs.mosaic.utils -import java.io.{ByteArrayOutputStream, PrintWriter} +import java.io.{BufferedReader, ByteArrayOutputStream, InputStreamReader, PrintWriter} object SysUtils { @@ -11,16 +11,40 @@ object SysUtils { val stderrStream = new ByteArrayOutputStream val stdoutWriter = new PrintWriter(stdoutStream) val stderrWriter = new PrintWriter(stderrStream) - val exitValue = try { - //noinspection ScalaStyle - cmd.!!(ProcessLogger(stdoutWriter.println, stderrWriter.println)) - } catch { - case _: Exception => "ERROR" - } finally { - stdoutWriter.close() - stderrWriter.close() - } + val exitValue = + try { + // noinspection ScalaStyle + cmd.!!(ProcessLogger(stdoutWriter.println, stderrWriter.println)) + } catch { + case e: Exception => s"ERROR: ${e.getMessage}" + } finally { + stdoutWriter.close() + stderrWriter.close() + } (exitValue, stdoutStream.toString, stderrStream.toString) } + def runScript(cmd: Array[String]): (String, String, String) = { + val p = Runtime.getRuntime.exec(cmd) + val stdinStream = new BufferedReader(new InputStreamReader(p.getInputStream)) + val stderrStream = new BufferedReader(new InputStreamReader(p.getErrorStream)) + val exitValue = + try { + p.waitFor() + } catch { + case e: Exception => s"ERROR: ${e.getMessage}" + } + val stdinOutput = stdinStream.lines().toArray.mkString("\n") + val stderrOutput = stderrStream.lines().toArray.mkString("\n") + stdinStream.close() + stderrStream.close() + (s"$exitValue", stdinOutput, stderrOutput) + } + + def getLastOutputLine(prompt: (String, String, String)): String = { + val (_, stdout, _) = prompt + val lines = stdout.split("\n") + lines.last + } + } diff --git a/src/test/resources/binary/grib-cams/adaptor.mars.internal-1650626950.0440469-3609-11-041ac051-015d-49b0-95df-b5daa7084c7e.grib b/src/test/resources/binary/grib-cams/adaptor.mars.internal-1650626950.0440469-3609-11-041ac051-015d-49b0-95df-b5daa7084c7e.grb similarity index 100% rename from src/test/resources/binary/grib-cams/adaptor.mars.internal-1650626950.0440469-3609-11-041ac051-015d-49b0-95df-b5daa7084c7e.grib rename to src/test/resources/binary/grib-cams/adaptor.mars.internal-1650626950.0440469-3609-11-041ac051-015d-49b0-95df-b5daa7084c7e.grb diff --git a/src/test/resources/binary/grib-cams/adaptor.mars.internal-1650626950.0440469-3609-11-041ac051-015d-49b0-95df-b5daa7084c7e.grib.aux.xml b/src/test/resources/binary/grib-cams/adaptor.mars.internal-1650626950.0440469-3609-11-041ac051-015d-49b0-95df-b5daa7084c7e.grb.aux.xml similarity index 100% rename from src/test/resources/binary/grib-cams/adaptor.mars.internal-1650626950.0440469-3609-11-041ac051-015d-49b0-95df-b5daa7084c7e.grib.aux.xml rename to src/test/resources/binary/grib-cams/adaptor.mars.internal-1650626950.0440469-3609-11-041ac051-015d-49b0-95df-b5daa7084c7e.grb.aux.xml diff --git a/src/test/resources/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grib b/src/test/resources/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grb similarity index 100% rename from src/test/resources/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grib rename to src/test/resources/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grb diff --git a/src/test/resources/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grib.aux.xml b/src/test/resources/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grb.aux.xml similarity index 100% rename from src/test/resources/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grib.aux.xml rename to src/test/resources/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grb.aux.xml diff --git a/src/test/resources/binary/grib-cams/adaptor.mars.internal-1650627030.319457-19905-15-0ede0273-89e3-4100-a0f2-48916ca607ed.grib b/src/test/resources/binary/grib-cams/adaptor.mars.internal-1650627030.319457-19905-15-0ede0273-89e3-4100-a0f2-48916ca607ed.grb similarity index 100% rename from src/test/resources/binary/grib-cams/adaptor.mars.internal-1650627030.319457-19905-15-0ede0273-89e3-4100-a0f2-48916ca607ed.grib rename to src/test/resources/binary/grib-cams/adaptor.mars.internal-1650627030.319457-19905-15-0ede0273-89e3-4100-a0f2-48916ca607ed.grb diff --git a/src/test/resources/binary/grib-cams/adaptor.mars.internal-1650627030.319457-19905-15-0ede0273-89e3-4100-a0f2-48916ca607ed.grib.aux.xml b/src/test/resources/binary/grib-cams/adaptor.mars.internal-1650627030.319457-19905-15-0ede0273-89e3-4100-a0f2-48916ca607ed.grb.aux.xml similarity index 100% rename from src/test/resources/binary/grib-cams/adaptor.mars.internal-1650627030.319457-19905-15-0ede0273-89e3-4100-a0f2-48916ca607ed.grib.aux.xml rename to src/test/resources/binary/grib-cams/adaptor.mars.internal-1650627030.319457-19905-15-0ede0273-89e3-4100-a0f2-48916ca607ed.grb.aux.xml diff --git a/src/test/scala/com/databricks/labs/mosaic/core/raster/TestRasterBandGDAL.scala b/src/test/scala/com/databricks/labs/mosaic/core/raster/TestRasterBandGDAL.scala index 15eef2009..1337ae6d2 100644 --- a/src/test/scala/com/databricks/labs/mosaic/core/raster/TestRasterBandGDAL.scala +++ b/src/test/scala/com/databricks/labs/mosaic/core/raster/TestRasterBandGDAL.scala @@ -37,8 +37,8 @@ class TestRasterBandGDAL extends SharedSparkSessionGDAL { assume(System.getProperty("os.name") == "Linux") val testRaster = MosaicRasterGDAL.readRaster( - filePath("/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grib"), - filePath("/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grib") + filePath("/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grb"), + filePath("/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grb") ) val testBand = testRaster.getBand(1) testBand.description shouldBe "1[-] HYBL=\"Hybrid level\"" diff --git a/src/test/scala/com/databricks/labs/mosaic/core/raster/TestRasterGDAL.scala b/src/test/scala/com/databricks/labs/mosaic/core/raster/TestRasterGDAL.scala index e39279843..bb53d6b79 100644 --- a/src/test/scala/com/databricks/labs/mosaic/core/raster/TestRasterGDAL.scala +++ b/src/test/scala/com/databricks/labs/mosaic/core/raster/TestRasterGDAL.scala @@ -1,9 +1,12 @@ package com.databricks.labs.mosaic.core.raster import com.databricks.labs.mosaic.core.raster.gdal.MosaicRasterGDAL +import com.databricks.labs.mosaic.gdal.MosaicGDAL import com.databricks.labs.mosaic.test.mocks.filePath import org.apache.spark.sql.test.SharedSparkSessionGDAL import org.scalatest.matchers.should.Matchers._ +import org.gdal.gdal.{gdal => gdalJNI} +import org.gdal.gdalconst import scala.sys.process._ import scala.util.Try @@ -43,7 +46,7 @@ class TestRasterGDAL extends SharedSparkSessionGDAL { testRaster.SRID shouldBe 0 testRaster.extent shouldBe Seq(-8895604.157333, 1111950.519667, -7783653.637667, 2223901.039333) testRaster.getRaster.GetProjection() - noException should be thrownBy testRaster.spatialRef + noException should be thrownBy testRaster.getSpatialReference an[Exception] should be thrownBy testRaster.getBand(-1) an[Exception] should be thrownBy testRaster.getBand(Int.MaxValue) @@ -54,8 +57,8 @@ class TestRasterGDAL extends SharedSparkSessionGDAL { assume(System.getProperty("os.name") == "Linux") val testRaster = MosaicRasterGDAL.readRaster( - filePath("/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grib"), - filePath("/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grib") + filePath("/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grb"), + filePath("/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grb") ) testRaster.xSize shouldBe 14 testRaster.ySize shouldBe 14 @@ -96,8 +99,8 @@ class TestRasterGDAL extends SharedSparkSessionGDAL { assume(System.getProperty("os.name") == "Linux") val testRaster = MosaicRasterGDAL.readRaster( - filePath("/modis/MCD43A4.A2018185.h10v07.006.2018194033728_B01.TIF"), - filePath("/modis/MCD43A4.A2018185.h10v07.006.2018194033728_B01.TIF") + filePath("/modis/MCD43A4.A2018185.h10v07.006.2018194033728_B01.TIF"), + filePath("/modis/MCD43A4.A2018185.h10v07.006.2018194033728_B01.TIF") ) testRaster.pixelXSize - 463.312716527 < 0.0000001 shouldBe true @@ -115,4 +118,214 @@ class TestRasterGDAL extends SharedSparkSessionGDAL { testRaster.getRaster.delete() } + test("Raster filter operations are correct.") { + assume(System.getProperty("os.name") == "Linux") + + gdalJNI.AllRegister() + + MosaicGDAL.setBlockSize(30) + + val ds = gdalJNI.GetDriverByName("GTiff").Create("/mosaic_tmp/test.tif", 50, 50, 1, gdalconst.gdalconstConstants.GDT_Float32) + + val values = 0 until 50 * 50 + ds.GetRasterBand(1).WriteRaster(0, 0, 50, 50, values.toArray) + ds.FlushCache() + + var result = MosaicRasterGDAL(ds, "", "", "GTiff", -1).filter(5, "avg").flushCache() + + var resultValues = result.getBand(1).values + + var inputMatrix = values.toArray.grouped(50).toArray + var resultMatrix = resultValues.grouped(50).toArray + + // first block + resultMatrix(10)(11) shouldBe ( + inputMatrix(8)(9) + inputMatrix(8)(10) + inputMatrix(8)(11) + inputMatrix(8)(12) + inputMatrix(8)(13) + + inputMatrix(9)(9) + inputMatrix(9)(10) + inputMatrix(9)(11) + inputMatrix(9)(12) + inputMatrix(9)(13) + + inputMatrix(10)(9) + inputMatrix(10)(10) + inputMatrix(10)(11) + inputMatrix(10)(12) + inputMatrix(10)(13) + + inputMatrix(11)(9) + inputMatrix(11)(10) + inputMatrix(11)(11) + inputMatrix(11)(12) + inputMatrix(11)(13) + + inputMatrix(12)(9) + inputMatrix(12)(10) + inputMatrix(12)(11) + inputMatrix(12)(12) + inputMatrix(12)(13) + ).toDouble / 25.0 + + // block overlap + resultMatrix(30)(32) shouldBe ( + inputMatrix(28)(30) + inputMatrix(28)(31) + inputMatrix(28)(32) + inputMatrix(28)(33) + inputMatrix(28)(34) + + inputMatrix(29)(30) + inputMatrix(29)(31) + inputMatrix(29)(32) + inputMatrix(29)(33) + inputMatrix(29)(34) + + inputMatrix(30)(30) + inputMatrix(30)(31) + inputMatrix(30)(32) + inputMatrix(30)(33) + inputMatrix(30)(34) + + inputMatrix(31)(30) + inputMatrix(31)(31) + inputMatrix(31)(32) + inputMatrix(31)(33) + inputMatrix(31)(34) + + inputMatrix(32)(30) + inputMatrix(32)(31) + inputMatrix(32)(32) + inputMatrix(32)(33) + inputMatrix(32)(34) + ).toDouble / 25.0 + + // mode + + result = MosaicRasterGDAL(ds, "", "", "GTiff", -1).filter(5, "mode").flushCache() + + resultValues = result.getBand(1).values + + inputMatrix = values.toArray.grouped(50).toArray + resultMatrix = resultValues.grouped(50).toArray + + // first block + + resultMatrix(10)(11) shouldBe Seq( + inputMatrix(8)(9), + inputMatrix(8)(10), + inputMatrix(8)(11), + inputMatrix(8)(12), + inputMatrix(8)(13), + inputMatrix(9)(9), + inputMatrix(9)(10), + inputMatrix(9)(11), + inputMatrix(9)(12), + inputMatrix(9)(13), + inputMatrix(10)(9), + inputMatrix(10)(10), + inputMatrix(10)(11), + inputMatrix(10)(12), + inputMatrix(10)(13), + inputMatrix(11)(9), + inputMatrix(11)(10), + inputMatrix(11)(11), + inputMatrix(11)(12), + inputMatrix(11)(13), + inputMatrix(12)(9), + inputMatrix(12)(10), + inputMatrix(12)(11), + inputMatrix(12)(12), + inputMatrix(12)(13) + ).groupBy(identity).maxBy(_._2.size)._1.toDouble + + // corner + + resultMatrix(49)(49) shouldBe Seq( + inputMatrix(47)(47), + inputMatrix(47)(48), + inputMatrix(47)(49), + inputMatrix(48)(47), + inputMatrix(48)(48), + inputMatrix(48)(49), + inputMatrix(49)(47), + inputMatrix(49)(48), + inputMatrix(49)(49) + ).groupBy(identity).maxBy(_._2.size)._1.toDouble + + // median + + result = MosaicRasterGDAL(ds, "", "", "GTiff", -1).filter(5, "median").flushCache() + + resultValues = result.getBand(1).values + + inputMatrix = values.toArray.grouped(50).toArray + resultMatrix = resultValues.grouped(50).toArray + + // first block + + resultMatrix(10)(11) shouldBe Seq( + inputMatrix(8)(9), + inputMatrix(8)(10), + inputMatrix(8)(11), + inputMatrix(8)(12), + inputMatrix(8)(13), + inputMatrix(9)(9), + inputMatrix(9)(10), + inputMatrix(9)(11), + inputMatrix(9)(12), + inputMatrix(9)(13), + inputMatrix(10)(9), + inputMatrix(10)(10), + inputMatrix(10)(11), + inputMatrix(10)(12), + inputMatrix(10)(13), + inputMatrix(11)(9), + inputMatrix(11)(10), + inputMatrix(11)(11), + inputMatrix(11)(12), + inputMatrix(11)(13), + inputMatrix(12)(9), + inputMatrix(12)(10), + inputMatrix(12)(11), + inputMatrix(12)(12), + inputMatrix(12)(13) + ).sorted.apply(12).toDouble + + // min filter + + result = MosaicRasterGDAL(ds, "", "", "GTiff", -1).filter(5, "min").flushCache() + + resultValues = result.getBand(1).values + + inputMatrix = values.toArray.grouped(50).toArray + resultMatrix = resultValues.grouped(50).toArray + + // first block + + resultMatrix(10)(11) shouldBe Seq( + inputMatrix(8)(9), + inputMatrix(8)(10), + inputMatrix(8)(11), + inputMatrix(8)(12), + inputMatrix(8)(13), + inputMatrix(9)(9), + inputMatrix(9)(10), + inputMatrix(9)(11), + inputMatrix(9)(12), + inputMatrix(9)(13), + inputMatrix(10)(9), + inputMatrix(10)(10), + inputMatrix(10)(11), + inputMatrix(10)(12), + inputMatrix(10)(13), + inputMatrix(11)(9), + inputMatrix(11)(10), + inputMatrix(11)(11), + inputMatrix(11)(12), + inputMatrix(11)(13), + inputMatrix(12)(9), + inputMatrix(12)(10), + inputMatrix(12)(11), + inputMatrix(12)(12), + inputMatrix(12)(13) + ).min.toDouble + + // max filter + + result = MosaicRasterGDAL(ds, "", "", "GTiff", -1).filter(5, "max").flushCache() + + resultValues = result.getBand(1).values + + inputMatrix = values.toArray.grouped(50).toArray + resultMatrix = resultValues.grouped(50).toArray + + // first block + + resultMatrix(10)(11) shouldBe Seq( + inputMatrix(8)(9), + inputMatrix(8)(10), + inputMatrix(8)(11), + inputMatrix(8)(12), + inputMatrix(8)(13), + inputMatrix(9)(9), + inputMatrix(9)(10), + inputMatrix(9)(11), + inputMatrix(9)(12), + inputMatrix(9)(13), + inputMatrix(10)(9), + inputMatrix(10)(10), + inputMatrix(10)(11), + inputMatrix(10)(12), + inputMatrix(10)(13), + inputMatrix(11)(9), + inputMatrix(11)(10), + inputMatrix(11)(11), + inputMatrix(11)(12), + inputMatrix(11)(13), + inputMatrix(12)(9), + inputMatrix(12)(10), + inputMatrix(12)(11), + inputMatrix(12)(12), + inputMatrix(12)(13) + ).max.toDouble + + } + } diff --git a/src/test/scala/com/databricks/labs/mosaic/datasource/GDALFileFormatTest.scala b/src/test/scala/com/databricks/labs/mosaic/datasource/GDALFileFormatTest.scala index 99a1563ca..623993b01 100644 --- a/src/test/scala/com/databricks/labs/mosaic/datasource/GDALFileFormatTest.scala +++ b/src/test/scala/com/databricks/labs/mosaic/datasource/GDALFileFormatTest.scala @@ -34,7 +34,7 @@ class GDALFileFormatTest extends QueryTest with SharedSparkSessionGDAL { .take(1) } - + test("Read grib with GDALFileFormat") { assume(System.getProperty("os.name") == "Linux") @@ -43,25 +43,22 @@ class GDALFileFormatTest extends QueryTest with SharedSparkSessionGDAL { noException should be thrownBy spark.read .format("gdal") - .option("extensions", "grib") - .option("raster_storage", "disk") - .option("extensions", "grib") + .option("extensions", "grb") + .option("raster.read.strategy", "retile_on_read") .load(filePath) .take(1) noException should be thrownBy spark.read .format("gdal") - .option("extensions", "grib") - .option("raster_storage", "disk") - .option("extensions", "grib") + .option("extensions", "grb") + .option("raster.read.strategy", "retile_on_read") .load(filePath) .take(1) noException should be thrownBy spark.read .format("gdal") - .option("extensions", "grib") - .option("raster_storage", "disk") - .option("extensions", "grib") + .option("extensions", "grb") + .option("raster.read.strategy", "retile_on_read") .load(filePath) .select("metadata") .take(1) @@ -92,7 +89,7 @@ class GDALFileFormatTest extends QueryTest with SharedSparkSessionGDAL { .select("metadata") .take(1) - noException should be thrownBy spark.read + noException should be thrownBy spark.read .format("gdal") .option(MOSAIC_RASTER_READ_STRATEGY, "retile_on_read") .load(filePath) diff --git a/src/test/scala/com/databricks/labs/mosaic/datasource/multiread/RasterAsGridReaderTest.scala b/src/test/scala/com/databricks/labs/mosaic/datasource/multiread/RasterAsGridReaderTest.scala index 6e99aa1df..fba2b74cb 100644 --- a/src/test/scala/com/databricks/labs/mosaic/datasource/multiread/RasterAsGridReaderTest.scala +++ b/src/test/scala/com/databricks/labs/mosaic/datasource/multiread/RasterAsGridReaderTest.scala @@ -1,12 +1,12 @@ package com.databricks.labs.mosaic.datasource.multiread -import com.databricks.labs.mosaic.functions.MosaicContext import com.databricks.labs.mosaic.JTS import com.databricks.labs.mosaic.core.index.H3IndexSystem +import com.databricks.labs.mosaic.functions.MosaicContext import com.databricks.labs.mosaic.test.MosaicSpatialQueryTest import org.apache.spark.sql.test.SharedSparkSessionGDAL import org.scalatest.matchers.must.Matchers.{be, noException} -import org.scalatest.matchers.should.Matchers.{an, convertToAnyShouldWrapper} +import org.scalatest.matchers.should.Matchers.an import java.nio.file.{Files, Paths} @@ -14,149 +14,161 @@ class RasterAsGridReaderTest extends MosaicSpatialQueryTest with SharedSparkSess test("Read big tif with Raster As Grid Reader") { assume(System.getProperty("os.name") == "Linux") - spark.sparkContext.setLogLevel("INFO") MosaicContext.build(H3IndexSystem, JTS) - val tif = "/binary/big_tiff.tif" + val tif = "/modis/" val filePath = getClass.getResource(tif).getPath val df = MosaicContext.read .format("raster_to_grid") .option("retile", "true") - .option("sizeInMB", "64") + .option("sizeInMB", "128") .option("resolution", "1") .load(filePath) .select("measure") - //df.queryExecution.optimizedPlan + df.queryExecution.optimizedPlan - //noException should be thrownBy df.queryExecution.executedPlan + noException should be thrownBy df.queryExecution.executedPlan df.count() } -// test("Read netcdf with Raster As Grid Reader") { -// assume(System.getProperty("os.name") == "Linux") -// MosaicContext.build(H3IndexSystem, JTS) -// -// val netcdf = "/binary/netcdf-coral/" -// val filePath = getClass.getResource(netcdf).getPath -// -// noException should be thrownBy MosaicContext.read -// .format("raster_to_grid") -// .option("retile", "true") -// .option("tileSize", "10") -// .option("readSubdataset", "true") -// .option("subdataset", "1") -// .option("kRingInterpolate", "3") -// .load(filePath) -// .select("measure") -// .queryExecution -// .executedPlan -// -// } -// -// test("Read grib with Raster As Grid Reader") { -// assume(System.getProperty("os.name") == "Linux") -// MosaicContext.build(H3IndexSystem, JTS) -// -// val grib = "/binary/grib-cams/" -// val filePath = getClass.getResource(grib).getPath -// -// noException should be thrownBy MosaicContext.read -// .format("raster_to_grid") -// .option("extensions", "grib") -// .option("combiner", "min") -// .option("retile", "true") -// .option("tileSize", "10") -// .option("kRingInterpolate", "3") -// .load(filePath) -// .select("measure") -// .take(1) -// -// } -// -// test("Read tif with Raster As Grid Reader") { -// assume(System.getProperty("os.name") == "Linux") -// MosaicContext.build(H3IndexSystem, JTS) -// -// val tif = "/modis/" -// val filePath = getClass.getResource(tif).getPath -// -// noException should be thrownBy MosaicContext.read -// .format("raster_to_grid") -// .option("combiner", "max") -// .option("tileSize", "10") -// .option("kRingInterpolate", "3") -// .load(filePath) -// .select("measure") -// .take(1) -// -// } -// -// test("Read zarr with Raster As Grid Reader") { -// assume(System.getProperty("os.name") == "Linux") -// MosaicContext.build(H3IndexSystem, JTS) -// -// val zarr = "/binary/zarr-example/" -// val filePath = getClass.getResource(zarr).getPath -// -// noException should be thrownBy MosaicContext.read -// .format("raster_to_grid") -// .option("combiner", "median") -// .option("vsizip", "true") -// .option("tileSize", "10") -// .load(filePath) -// .select("measure") -// .take(1) -// -// noException should be thrownBy MosaicContext.read -// .format("raster_to_grid") -// .option("combiner", "count") -// .option("vsizip", "true") -// .load(filePath) -// .select("measure") -// .take(1) -// -// noException should be thrownBy MosaicContext.read -// .format("raster_to_grid") -// .option("combiner", "average") -// .option("vsizip", "true") -// .load(filePath) -// .select("measure") -// .take(1) -// -// noException should be thrownBy MosaicContext.read -// .format("raster_to_grid") -// .option("combiner", "avg") -// .option("vsizip", "true") -// .load(filePath) -// .select("measure") -// .take(1) -// -// val paths = Files.list(Paths.get(filePath)).toArray.map(_.toString) -// -// an[Error] should be thrownBy MosaicContext.read -// .format("raster_to_grid") -// .option("combiner", "count_+") -// .option("vsizip", "true") -// .load(paths: _*) -// .select("measure") -// .take(1) -// -// an[Error] should be thrownBy MosaicContext.read -// .format("invalid") -// .load(paths: _*) -// -// an[Error] should be thrownBy MosaicContext.read -// .format("invalid") -// .load(filePath) -// -// noException should be thrownBy MosaicContext.read -// .format("raster_to_grid") -// .option("kRingInterpolate", "3") -// .load(filePath) -// -// } + test("Read netcdf with Raster As Grid Reader") { + assume(System.getProperty("os.name") == "Linux") + MosaicContext.build(H3IndexSystem, JTS) + + val netcdf = "/binary/netcdf-coral/" + val filePath = getClass.getResource(netcdf).getPath + + //noException should be thrownBy + + + MosaicContext.read + .format("raster_to_grid") + .option("retile", "true") + .option("tileSize", "10") + .option("readSubdataset", "true") + .option("subdataset", "1") + .option("kRingInterpolate", "3") + .load(filePath) + .select("measure") + .queryExecution + .executedPlan + + } + + test("Read grib with Raster As Grid Reader") { + assume(System.getProperty("os.name") == "Linux") + MosaicContext.build(H3IndexSystem, JTS) + + val grib = "/binary/grib-cams/" + val filePath = getClass.getResource(grib).getPath + + noException should be thrownBy MosaicContext.read + .format("raster_to_grid") + .option("extensions", "grib") + .option("combiner", "min") + .option("retile", "true") + .option("tileSize", "10") + .option("kRingInterpolate", "3") + .load(filePath) + .select("measure") + .take(1) + + } + + test("Read tif with Raster As Grid Reader") { + assume(System.getProperty("os.name") == "Linux") + MosaicContext.build(H3IndexSystem, JTS) + + val tif = "/modis/" + val filePath = getClass.getResource(tif).getPath + + noException should be thrownBy MosaicContext.read + .format("raster_to_grid") + .option("combiner", "max") + .option("tileSize", "10") + .option("kRingInterpolate", "3") + .load(filePath) + .select("measure") + .take(1) + + } + + test("Read zarr with Raster As Grid Reader") { + assume(System.getProperty("os.name") == "Linux") + MosaicContext.build(H3IndexSystem, JTS) + + val zarr = "/binary/zarr-example/" + val filePath = getClass.getResource(zarr).getPath + + noException should be thrownBy MosaicContext.read + .format("raster_to_grid") + .option("readSubdataset", "true") + .option("subdatasetName", "/group_with_attrs/F_order_array") + .option("combiner", "median") + .option("vsizip", "true") + .option("tileSize", "10") + .load(filePath) + .select("measure") + .take(1) + + noException should be thrownBy MosaicContext.read + .format("raster_to_grid") + .option("readSubdataset", "true") + .option("subdatasetName", "/group_with_attrs/F_order_array") + .option("combiner", "count") + .option("vsizip", "true") + .load(filePath) + .select("measure") + .take(1) + + noException should be thrownBy MosaicContext.read + .format("raster_to_grid") + .option("readSubdataset", "true") + .option("subdatasetName", "/group_with_attrs/F_order_array") + .option("combiner", "average") + .option("vsizip", "true") + .load(filePath) + .select("measure") + .take(1) + + noException should be thrownBy MosaicContext.read + .format("raster_to_grid") + .option("readSubdataset", "true") + .option("subdatasetName", "/group_with_attrs/F_order_array") + .option("combiner", "avg") + .option("vsizip", "true") + .load(filePath) + .select("measure") + .take(1) + + val paths = Files.list(Paths.get(filePath)).toArray.map(_.toString) + + an[Error] should be thrownBy MosaicContext.read + .format("raster_to_grid") + .option("combiner", "count_+") + .option("vsizip", "true") + .load(paths: _*) + .select("measure") + .take(1) + + an[Error] should be thrownBy MosaicContext.read + .format("invalid") + .load(paths: _*) + + an[Error] should be thrownBy MosaicContext.read + .format("invalid") + .load(filePath) + + noException should be thrownBy MosaicContext.read + .format("raster_to_grid") + .option("readSubdataset", "true") + .option("subdatasetName", "/group_with_attrs/F_order_array") + .option("kRingInterpolate", "3") + .load(filePath) + + } } diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_CombineAvgBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_CombineAvgBehaviors.scala index 8ce57f5b8..611bf8f77 100644 --- a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_CombineAvgBehaviors.scala +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_CombineAvgBehaviors.scala @@ -35,7 +35,9 @@ trait RST_CombineAvgBehaviors extends QueryTest { rastersInMemory.union(rastersInMemory) .createOrReplaceTempView("source") - noException should be thrownBy spark.sql(""" + //noException should be thrownBy + + spark.sql(""" |select rst_combineavg(collect_set(tiles)) as tiles |from ( | select path, rst_tessellate(tile, 2) as tiles diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_FilterBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_FilterBehaviors.scala new file mode 100644 index 000000000..d06923dc1 --- /dev/null +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_FilterBehaviors.scala @@ -0,0 +1,36 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.geometry.api.GeometryAPI +import com.databricks.labs.mosaic.core.index.IndexSystem +import com.databricks.labs.mosaic.functions.MosaicContext +import org.apache.spark.sql.QueryTest +import org.scalatest.matchers.should.Matchers._ + +trait RST_FilterBehaviors extends QueryTest { + + // noinspection MapGetGet + def behaviors(indexSystem: IndexSystem, geometryAPI: GeometryAPI): Unit = { + spark.sparkContext.setLogLevel("FATAL") + val mc = MosaicContext.build(indexSystem, geometryAPI) + mc.register() + val sc = spark + import mc.functions._ + import sc.implicits._ + + val rastersInMemory = spark.read + .format("gdal") + .option("raster_storage", "in-memory") + .load("src/test/resources/modis") + + val gridTiles = rastersInMemory + .withColumn("result", rst_filter($"tile", 3, "mode")) + .select("result") + .collect() + + gridTiles.length should be(7) + + rastersInMemory.createOrReplaceTempView("source") + + } + +} diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_FilterTest.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_FilterTest.scala new file mode 100644 index 000000000..a243f7168 --- /dev/null +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_FilterTest.scala @@ -0,0 +1,32 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.geometry.api.JTS +import com.databricks.labs.mosaic.core.index.H3IndexSystem +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSessionGDAL + +import scala.util.Try + +class RST_FilterTest extends QueryTest with SharedSparkSessionGDAL with RST_FilterBehaviors { + + private val noCodegen = + withSQLConf( + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false", + SQLConf.CODEGEN_FACTORY_MODE.key -> CodegenObjectFactoryMode.NO_CODEGEN.toString + ) _ + + // Hotfix for SharedSparkSession afterAll cleanup. + override def afterAll(): Unit = Try(super.afterAll()) + + // These tests are not index system nor geometry API specific. + // Only testing one pairing is sufficient. + test("Testing RST_Filter with manual GDAL registration (H3, JTS).") { + noCodegen { + assume(System.getProperty("os.name") == "Linux") + behaviors(H3IndexSystem, JTS) + } + } + +} diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MinBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MinBehaviors.scala index bd867ee65..d01f79fec 100644 --- a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MinBehaviors.scala +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MinBehaviors.scala @@ -37,7 +37,7 @@ trait RST_MinBehaviors extends QueryTest { val result = df.as[Double].collect().min - result < 0 shouldBe true + result == 0 shouldBe true an[Exception] should be thrownBy spark.sql(""" |select rst_min() from source diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_TessellateBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_TessellateBehaviors.scala index c346e82db..cfffd9e6b 100644 --- a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_TessellateBehaviors.scala +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_TessellateBehaviors.scala @@ -4,6 +4,7 @@ import com.databricks.labs.mosaic.core.geometry.api.GeometryAPI import com.databricks.labs.mosaic.core.index.IndexSystem import com.databricks.labs.mosaic.functions.MosaicContext import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.functions._ import org.scalatest.matchers.should.Matchers._ trait RST_TessellateBehaviors extends QueryTest { @@ -24,8 +25,10 @@ trait RST_TessellateBehaviors extends QueryTest { val gridTiles = rastersInMemory .withColumn("tiles", rst_tessellate($"tile", 3)) - .select("tiles") - + .withColumn("bbox", st_aswkt(rst_boundingbox($"tile"))) + .select("bbox", "path", "tiles") + .withColumn("avg", rst_avg($"tiles")) + rastersInMemory .createOrReplaceTempView("source") @@ -37,9 +40,9 @@ trait RST_TessellateBehaviors extends QueryTest { .withColumn("tiles", rst_tessellate($"tile", 3)) .select("tiles") - val result = gridTiles.collect() + val result = gridTiles.select(explode(col("avg")).alias("a")).groupBy("a").count().collect() - result.length should be(380) + result.length should be(441) } diff --git a/src/test/scala/com/databricks/labs/mosaic/test/package.scala b/src/test/scala/com/databricks/labs/mosaic/test/package.scala index 435ee552c..2e5951be7 100644 --- a/src/test/scala/com/databricks/labs/mosaic/test/package.scala +++ b/src/test/scala/com/databricks/labs/mosaic/test/package.scala @@ -164,7 +164,7 @@ package object test { } val geotiffBytes: Array[Byte] = fileBytes("/modis/MCD43A4.A2018185.h10v07.006.2018194033728_B01.TIF") val gribBytes: Array[Byte] = - fileBytes("/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grib") + fileBytes("/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grb") val netcdfBytes: Array[Byte] = fileBytes("/binary/netcdf-coral/ct5km_baa-max-7d_v3.1_20220101.nc") def polyDf(sparkSession: SparkSession, mosaicContext: MosaicContext): DataFrame = { diff --git a/src/test/scala/org/apache/spark/sql/test/MosaicTestSparkSession.scala b/src/test/scala/org/apache/spark/sql/test/MosaicTestSparkSession.scala index 8029c30a7..84a613b31 100644 --- a/src/test/scala/org/apache/spark/sql/test/MosaicTestSparkSession.scala +++ b/src/test/scala/org/apache/spark/sql/test/MosaicTestSparkSession.scala @@ -8,13 +8,13 @@ class MosaicTestSparkSession(sc: SparkContext) extends TestSparkSession(sc) { this( new SparkContext( - "local[4]", + "local[8]", "test-sql-context", sparkConf .set("spark.sql.adaptive.enabled", "false") .set("spark.driver.memory", "32g") .set("spark.executor.memory", "32g") - .set("spark.sql.shuffle.partitions", "4") + .set("spark.sql.shuffle.partitions", "8") .set("spark.sql.testkey", "true") ) ) diff --git a/src/test/scala/org/apache/spark/sql/test/SharedSparkSessionGDAL.scala b/src/test/scala/org/apache/spark/sql/test/SharedSparkSessionGDAL.scala index 984fff9d8..12dcac6f3 100644 --- a/src/test/scala/org/apache/spark/sql/test/SharedSparkSessionGDAL.scala +++ b/src/test/scala/org/apache/spark/sql/test/SharedSparkSessionGDAL.scala @@ -1,6 +1,5 @@ package org.apache.spark.sql.test -import com.databricks.labs.mosaic._ import com.databricks.labs.mosaic.gdal.MosaicGDAL import com.databricks.labs.mosaic.utils.FileUtils import com.databricks.labs.mosaic.{MOSAIC_GDAL_NATIVE, MOSAIC_RASTER_CHECKPOINT} @@ -8,7 +7,6 @@ import org.apache.spark.SparkConf import org.apache.spark.sql.SparkSession import org.gdal.gdal.gdal -import java.nio.file.{Files, Paths} import scala.util.Try trait SharedSparkSessionGDAL extends SharedSparkSession { @@ -20,10 +18,10 @@ trait SharedSparkSessionGDAL extends SharedSparkSession { override def createSparkSession: TestSparkSession = { val conf = sparkConf - conf.set(MOSAIC_RASTER_CHECKPOINT, FileUtils.createMosaicTempDir()) + conf.set(MOSAIC_RASTER_CHECKPOINT, FileUtils.createMosaicTempDir(prefix = "/mnt/")) SparkSession.cleanupAnyExistingSession() val session = new MosaicTestSparkSession(conf) - session.sparkContext.setLogLevel("INFO") + session.sparkContext.setLogLevel("FATAL") Try { MosaicGDAL.enableGDAL(session) } From c3dc72f294ac94d1df0d1a862300574d01eab1b4 Mon Sep 17 00:00:00 2001 From: Daniel Sparing Date: Thu, 8 Feb 2024 17:43:58 +0100 Subject: [PATCH 20/47] retrigger builds with empty commit From fa80aabb3c73848c9e9e44d6acf97fb23df7e359 Mon Sep 17 00:00:00 2001 From: "milos.colic" Date: Tue, 13 Feb 2024 13:02:22 +0000 Subject: [PATCH 21/47] Fix zip logic for zarr files. --- pom.xml | 2 +- .../labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala | 4 +++- .../mosaic/datasource/multiread/RasterAsGridReaderTest.scala | 5 +---- 3 files changed, 5 insertions(+), 6 deletions(-) diff --git a/pom.xml b/pom.xml index cdb4c8d2c..2cb2f8787 100644 --- a/pom.xml +++ b/pom.xml @@ -149,7 +149,7 @@ org.scoverage scoverage-maven-plugin - 2.0.1 + 2.0.2 scoverage-report diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala index b63bd851e..0309d9bab 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala @@ -427,7 +427,9 @@ case class MosaicRasterGDAL( path } if (Files.isDirectory(Paths.get(tmpPath))) { - SysUtils.runCommand(s"zip -r0 $tmpPath.zip $tmpPath") + val parentDir = Paths.get(tmpPath).getParent.toString + val fileName = Paths.get(tmpPath).getFileName.toString + SysUtils.runScript(Array("/bin/sh", "-c", s"cd $parentDir && zip -r0 $fileName.zip $fileName")) s"$tmpPath.zip" } else { tmpPath diff --git a/src/test/scala/com/databricks/labs/mosaic/datasource/multiread/RasterAsGridReaderTest.scala b/src/test/scala/com/databricks/labs/mosaic/datasource/multiread/RasterAsGridReaderTest.scala index fba2b74cb..f174954cd 100644 --- a/src/test/scala/com/databricks/labs/mosaic/datasource/multiread/RasterAsGridReaderTest.scala +++ b/src/test/scala/com/databricks/labs/mosaic/datasource/multiread/RasterAsGridReaderTest.scala @@ -41,10 +41,7 @@ class RasterAsGridReaderTest extends MosaicSpatialQueryTest with SharedSparkSess val netcdf = "/binary/netcdf-coral/" val filePath = getClass.getResource(netcdf).getPath - //noException should be thrownBy - - - MosaicContext.read + noException should be thrownBy MosaicContext.read .format("raster_to_grid") .option("retile", "true") .option("tileSize", "10") From 5a85ef91db08a8a67a370c53ee3ef00b0cb42e2a Mon Sep 17 00:00:00 2001 From: "milos.colic" Date: Tue, 13 Feb 2024 14:13:12 +0000 Subject: [PATCH 22/47] Fix zarr zip paths. --- .../labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala | 2 +- src/main/scala/com/databricks/labs/mosaic/utils/FileUtils.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala index 0309d9bab..a083ba8b8 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala @@ -677,7 +677,7 @@ object MosaicRasterGDAL extends RasterReader { // the way we zip using uuid is not compatible with GDAL // we need to unzip and read the file if it was zipped by us val parentDir = Paths.get(zippedPath).getParent - val prompt = SysUtils.runScript(Array("/bin/sh", "-c", s"cd $parentDir && unzip -o $zippedPath -d /")) + val prompt = SysUtils.runScript(Array("/bin/sh", "-c", s"cd $parentDir && unzip -o $zippedPath -d $parentDir")) // zipped files will have the old uuid name of the raster // we need to get the last extracted file name, but the last extracted file name is not the raster name // we can't list folders due to concurrent writes diff --git a/src/main/scala/com/databricks/labs/mosaic/utils/FileUtils.scala b/src/main/scala/com/databricks/labs/mosaic/utils/FileUtils.scala index fc01cfaa0..a36c0bec0 100644 --- a/src/main/scala/com/databricks/labs/mosaic/utils/FileUtils.scala +++ b/src/main/scala/com/databricks/labs/mosaic/utils/FileUtils.scala @@ -22,7 +22,7 @@ object FileUtils { bytes } - def createMosaicTempDir(prefix: String = ""): String = { + def createMosaicTempDir(prefix: String = "/tmp"): String = { val tempRoot = Paths.get(s"$prefix/mosaic_tmp/") if (!Files.exists(tempRoot)) { Files.createDirectories(tempRoot) From 53b35903ade8e0f355d40d0a0f90cbd79b327fe6 Mon Sep 17 00:00:00 2001 From: Milos Colic Date: Tue, 13 Feb 2024 14:18:15 +0000 Subject: [PATCH 23/47] Update testVectorFunctions.R --- .../sparklyrMosaic/tests/testthat/testVectorFunctions.R | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/R/sparklyr-mosaic/sparklyrMosaic/tests/testthat/testVectorFunctions.R b/R/sparklyr-mosaic/sparklyrMosaic/tests/testthat/testVectorFunctions.R index 061325073..7aa5addda 100644 --- a/R/sparklyr-mosaic/sparklyrMosaic/tests/testthat/testVectorFunctions.R +++ b/R/sparklyr-mosaic/sparklyrMosaic/tests/testthat/testVectorFunctions.R @@ -15,7 +15,7 @@ test_that("scalar vector functions behave as intended", { st_length = st_length(wkt), st_perimeter = st_perimeter(wkt), st_buffer = st_buffer(wkt, as.double(1.1)), - st_buffer_optparams = st_buffer(wkt, as.double(1.1), "endcap=square quad_segs=2")), + st_buffer_optparams = st_buffer(wkt, as.double(1.1), "endcap=square quad_segs=2"), st_bufferloop = st_bufferloop(wkt, as.double(1.1), as.double(1.2)), st_convexhull = st_convexhull(wkt), st_dump = st_dump(wkt), @@ -123,4 +123,4 @@ test_that("aggregate vector functions behave as intended", { expect_true(sdf.intersection %>% head(1) %>% sdf_collect %>% .$comparison_intersection) -}) \ No newline at end of file +}) From 8356b6a0bd3efe614108d95e03245f97ebef8515 Mon Sep 17 00:00:00 2001 From: "milos.colic" Date: Mon, 26 Feb 2024 14:07:15 +0000 Subject: [PATCH 24/47] Add COG to format extensions list. Add createInfo concept, this will contain driver, parentPath, currentPath, etc. Make gdal programs no-failure operations. Capture errors and warnings of gdal programs in the raster tile metadata. Add RST_Transform expression. Add ReadAsPath reading strategy. --- python/mosaic/api/raster.py | 27 ++ .../mosaic/core/raster/api/FormatLookup.scala | 1 + .../labs/mosaic/core/raster/api/GDAL.scala | 30 +- .../core/raster/gdal/MosaicRasterGDAL.scala | 151 ++++++---- .../mosaic/core/raster/io/RasterReader.scala | 38 +-- .../raster/operator/gdal/GDALBuildVRT.scala | 19 +- .../core/raster/operator/gdal/GDALCalc.scala | 31 ++- .../core/raster/operator/gdal/GDALInfo.scala | 14 +- .../raster/operator/gdal/GDALTranslate.scala | 20 +- .../core/raster/operator/gdal/GDALWarp.scala | 25 +- .../operator/retile/OverlappingTiles.scala | 2 +- .../operator/retile/RasterTessellate.scala | 4 +- .../core/raster/operator/retile/ReTile.scala | 2 +- .../operator/separate/SeparateBands.scala | 13 +- .../mosaic/core/types/RasterTileType.scala | 3 +- .../core/types/model/MosaicRasterTile.scala | 48 ++-- .../mosaic/datasource/gdal/ReTileOnRead.scala | 5 +- .../mosaic/datasource/gdal/ReadAsPath.scala | 124 +++++++++ .../mosaic/datasource/gdal/ReadInMemory.scala | 12 +- .../mosaic/datasource/gdal/ReadStrategy.scala | 1 + .../expressions/raster/RST_CombineAvg.scala | 7 +- .../raster/RST_CombineAvgAgg.scala | 5 +- .../expressions/raster/RST_DerivedBand.scala | 4 +- .../raster/RST_DerivedBandAgg.scala | 5 +- .../expressions/raster/RST_FromContent.scala | 5 +- .../expressions/raster/RST_FromFile.scala | 7 +- .../expressions/raster/RST_MakeTiles.scala | 5 +- .../expressions/raster/RST_MapAlgebra.scala | 8 +- .../expressions/raster/RST_MergeAgg.scala | 5 +- .../expressions/raster/RST_SetSRID.scala | 8 +- .../expressions/raster/RST_Transform.scala | 61 ++++ .../mosaic/expressions/raster/package.scala | 19 +- .../labs/mosaic/functions/MosaicContext.scala | 3 + .../labs/mosaic/gdal/MosaicGDAL.scala | 1 + .../core/raster/TestRasterBandGDAL.scala | 28 +- .../mosaic/core/raster/TestRasterGDAL.scala | 262 +++++++++--------- .../raster/RST_CombineAvgBehaviors.scala | 6 +- .../raster/RST_DerivedBandBehaviors.scala | 6 +- .../raster/RST_MergeBehaviors.scala | 8 +- .../raster/RST_TessellateBehaviors.scala | 2 +- .../raster/RST_TransformBehaviors.scala | 49 ++++ .../raster/RST_TransformTest.scala | 32 +++ 42 files changed, 744 insertions(+), 362 deletions(-) create mode 100644 src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReadAsPath.scala create mode 100644 src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Transform.scala create mode 100644 src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_TransformBehaviors.scala create mode 100644 src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_TransformTest.scala diff --git a/python/mosaic/api/raster.py b/python/mosaic/api/raster.py index 3638510dc..c61bafcd2 100644 --- a/python/mosaic/api/raster.py +++ b/python/mosaic/api/raster.py @@ -55,6 +55,7 @@ "rst_subdivide", "rst_summary", "rst_tessellate", + "rst_transform", "rst_to_overlapping_tiles", "rst_tryopen", "rst_upperleftx", @@ -997,6 +998,32 @@ def rst_tessellate(raster_tile: ColumnOrName, resolution: ColumnOrName) -> Colum ) +def rst_transform(raster_tile: ColumnOrName, srid: ColumnOrName) -> Column: + """ + Transforms the raster to the given SRID. + The result is a Mosaic raster tile struct of the transformed raster. + The result is stored in the checkpoint directory. + + Parameters + ---------- + raster_tile : Column (RasterTileType) + Mosaic raster tile struct column. + srid : Column (IntegerType) + EPSG authority code for the file's projection. + + Returns + ------- + Column (RasterTileType) + Mosaic raster tile struct column. + + """ + return config.mosaic_context.invoke_function( + "rst_transform", + pyspark_to_java_column(raster_tile), + pyspark_to_java_column(srid), + ) + + def rst_fromcontent( raster_bin: ColumnOrName, driver: ColumnOrName, size_in_mb: Any = -1 ) -> Column: diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/api/FormatLookup.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/api/FormatLookup.scala index e3aeb5296..8bf2d9cdb 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/api/FormatLookup.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/api/FormatLookup.scala @@ -17,6 +17,7 @@ object FormatLookup { "CAD" -> "dwg", "CEOS" -> "ceos", "COASP" -> "coasp", + "COG" -> "tif", "COSAR" -> "cosar", "CPG" -> "cpg", "CSW" -> "csw", diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/api/GDAL.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/api/GDAL.scala index b86489359..6e2fee0f6 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/api/GDAL.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/api/GDAL.scala @@ -96,27 +96,26 @@ object GDAL { */ def readRaster( inputRaster: Any, - parentPath: String, - shortDriverName: String, + createInfo: Map[String, String], inputDT: DataType ): MosaicRasterGDAL = { inputDT match { case StringType => val path = inputRaster.asInstanceOf[UTF8String].toString - MosaicRasterGDAL.readRaster(path, parentPath) + MosaicRasterGDAL.readRaster(createInfo) case BinaryType => val bytes = inputRaster.asInstanceOf[Array[Byte]] - val raster = MosaicRasterGDAL.readRaster(bytes, parentPath, shortDriverName) + val raster = MosaicRasterGDAL.readRaster(bytes, createInfo) // If the raster is coming as a byte array, we can't check for zip condition. // We first try to read the raster directly, if it fails, we read it as a zip. if (raster == null) { + val parentPath = createInfo("parentPath") val zippedPath = s"/vsizip/$parentPath" - MosaicRasterGDAL.readRaster(bytes, zippedPath, shortDriverName) + MosaicRasterGDAL.readRaster(bytes, createInfo + ("path" -> zippedPath)) } else { raster } - case _ => - throw new IllegalArgumentException(s"Unsupported data type: $inputDT") + case _ => throw new IllegalArgumentException(s"Unsupported data type: $inputDT") } } @@ -160,7 +159,10 @@ object GDAL { * @return * Returns a Raster object. */ - def raster(path: String, parentPath: String): MosaicRasterGDAL = MosaicRasterGDAL.readRaster(path, parentPath) + def raster(path: String, parentPath: String): MosaicRasterGDAL = { + val createInfo = Map("path" -> path, "parentPath" -> parentPath) + MosaicRasterGDAL.readRaster(createInfo) + } /** * Reads a raster from the given byte array. If the byte array is a zip @@ -171,8 +173,10 @@ object GDAL { * @return * Returns a Raster object. */ - def raster(content: Array[Byte], parentPath: String, driverShortName: String): MosaicRasterGDAL = - MosaicRasterGDAL.readRaster(content, parentPath, driverShortName) + def raster(content: Array[Byte], parentPath: String, driverShortName: String): MosaicRasterGDAL = { + val createInfo = Map("parentPath" -> parentPath, "driver" -> driverShortName) + MosaicRasterGDAL.readRaster(content, createInfo) + } /** * Reads a raster from the given path. It extracts the specified band from @@ -186,8 +190,10 @@ object GDAL { * @return * Returns a Raster band object. */ - def band(path: String, bandIndex: Int, parentPath: String): MosaicRasterBandGDAL = - MosaicRasterGDAL.readBand(path, bandIndex, parentPath) + def band(path: String, bandIndex: Int, parentPath: String): MosaicRasterBandGDAL = { + val createInfo = Map("path" -> path, "parentPath" -> parentPath) + MosaicRasterGDAL.readBand(bandIndex, createInfo) + } /** * Converts raster x, y coordinates to lat, lon coordinates. diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala index 108591529..10f04416d 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala @@ -26,13 +26,17 @@ import scala.util.{Failure, Success, Try} //noinspection DuplicatedCode case class MosaicRasterGDAL( raster: Dataset, - path: String, - parentPath: String, - driverShortName: String, + createInfo: Map[String, String], memSize: Long ) extends RasterWriter with RasterCleaner { + def path: String = createInfo("path") + + def parentPath: String = createInfo("parentPath") + + def driverShortName: Option[String] = createInfo.get("driver") + def getWriteOptions: MosaicRasterWriteOptions = MosaicRasterWriteOptions(this) def getCompression: String = { @@ -72,7 +76,9 @@ case class MosaicRasterGDAL( * @return * The raster's driver short name. */ - def getDriversShortName: String = driverShortName + def getDriversShortName: String = driverShortName.getOrElse( + Try(raster.GetDriver().getShortName).getOrElse("NONE") + ) /** * @return @@ -114,7 +120,7 @@ case class MosaicRasterGDAL( def pixelDiagSize: Double = math.sqrt(pixelXSize * pixelXSize + pixelYSize * pixelYSize) /** @return Returns file extension. */ - def getRasterFileExtension: String = GDAL.getExtension(driverShortName) + def getRasterFileExtension: String = GDAL.getExtension(getDriversShortName) /** @return Returns the raster's bands as a Seq. */ def getBands: Seq[MosaicRasterBandGDAL] = (1 to numBands).map(getBand) @@ -141,7 +147,7 @@ case class MosaicRasterGDAL( * A MosaicRaster object. */ def openRaster(path: String): Dataset = { - MosaicRasterGDAL.openRaster(path, Some(driverShortName)) + MosaicRasterGDAL.openRaster(path, driverShortName) } /** @@ -202,7 +208,6 @@ case class MosaicRasterGDAL( .toInt } - /** * @return * Sets the raster's SRID. This is the EPSG code of the raster's CRS. @@ -212,15 +217,18 @@ case class MosaicRasterGDAL( srs.ImportFromEPSG(srid) raster.SetSpatialRef(srs) val driver = raster.GetDriver() - val newPath = PathUtils.createTmpFilePath(GDAL.getExtension(driverShortName)) + val newPath = PathUtils.createTmpFilePath(GDAL.getExtension(getDriversShortName)) driver.CreateCopy(newPath, raster) - val newRaster = MosaicRasterGDAL.openRaster(newPath, Some(driverShortName)) + val newRaster = MosaicRasterGDAL.openRaster(newPath, driverShortName) dispose(this) - MosaicRasterGDAL(newRaster, newPath, parentPath, driverShortName, -1) + val createInfo = Map( + "path" -> newPath, + "parentPath" -> parentPath, + "driver" -> getDriversShortName + ) + MosaicRasterGDAL(newRaster, createInfo, -1) } - - /** * @return * Returns the raster's proj4 string. @@ -340,10 +348,9 @@ case class MosaicRasterGDAL( def isEmpty: Boolean = { val bands = getBands if (bands.isEmpty) { - subdatasets - .values - .filter(_.toLowerCase(Locale.ROOT).startsWith(driverShortName.toLowerCase(Locale.ROOT))) - .flatMap(readRaster(_, path).getBands) + subdatasets.values + .filter(_.toLowerCase(Locale.ROOT).startsWith(getDriversShortName.toLowerCase(Locale.ROOT))) + .flatMap(bp => readRaster(createInfo + ("path" -> bp)).getBands) .takeWhile(_.isEmpty) .nonEmpty } else { @@ -381,7 +388,7 @@ case class MosaicRasterGDAL( val cleanPath = filePath.replace("/vsizip/", "") val zipPath = if (cleanPath.endsWith("zip")) cleanPath else s"$cleanPath.zip" if (path != PathUtils.getCleanPath(parentPath)) { - Try(gdal.GetDriverByName(driverShortName).Delete(path)) + Try(gdal.GetDriverByName(getDriversShortName).Delete(path)) Try(Files.deleteIfExists(Paths.get(cleanPath))) Try(Files.deleteIfExists(Paths.get(path))) Try(Files.deleteIfExists(Paths.get(filePath))) @@ -502,7 +509,7 @@ case class MosaicRasterGDAL( * usable again. */ def refresh(): MosaicRasterGDAL = { - MosaicRasterGDAL(openRaster(path), path, parentPath, driverShortName, memSize) + MosaicRasterGDAL(openRaster(path), createInfo, memSize) } /** @@ -555,22 +562,35 @@ case class MosaicRasterGDAL( * Returns the raster's subdataset with given name. */ def getSubdataset(subsetName: String): MosaicRasterGDAL = { - val path = subdatasets.getOrElse( - s"${subsetName}_tmp", - throw new Exception(s""" - |Subdataset $subsetName not found! - |Available subdatasets: - | ${subdatasets.keys.filterNot(_.startsWith("SUBDATASET_")).mkString(", ")} - | """.stripMargin) - ) - val sanitized = PathUtils.getCleanPath(path) + val path = subdatasets.get(s"${subsetName}_tmp") + val gdalError = gdal.GetLastErrorMsg() + val error = path match { + case Some(_) => "" + case None => + s""" + |Subdataset $subsetName not found! + |Available subdatasets: + | ${subdatasets.keys.filterNot(_.startsWith("SUBDATASET_")).mkString(", ")} + | """.stripMargin + } + val sanitized = PathUtils.getCleanPath(path.getOrElse(PathUtils.NO_PATH_STRING)) val subdatasetPath = PathUtils.getSubdatasetPath(sanitized) val ds = openRaster(subdatasetPath) // Avoid costly IO to compute MEM size here // It will be available when the raster is serialized for next operation // If value is needed then it will be computed when getMemSize is called - MosaicRasterGDAL(ds, path, parentPath, driverShortName, -1) + val createInfo = Map( + "path" -> path.getOrElse(PathUtils.NO_PATH_STRING), + "parentPath" -> parentPath, + "driver" -> getDriversShortName, + "last_error" -> + s""" + |GDAL Error: $gdalError + |$error + |""".stripMargin + ) + MosaicRasterGDAL(ds, createInfo, -1) } def convolve(kernel: Array[Array[Double]]): MosaicRasterGDAL = { @@ -584,7 +604,13 @@ case class MosaicRasterGDAL( band.convolve(kernel) } - MosaicRasterGDAL(outputRaster, resultRasterPath, parentPath, driverShortName, -1) + val createInfo = Map( + "path" -> resultRasterPath, + "parentPath" -> parentPath, + "driver" -> getDriversShortName + ) + + MosaicRasterGDAL(outputRaster, createInfo, -1) } @@ -604,7 +630,13 @@ case class MosaicRasterGDAL( band.filter(kernelSize, operation, outputBand) } - val result = MosaicRasterGDAL(outputRaster, resultRasterPath, parentPath, driverShortName, this.memSize) + val createInfo = Map( + "path" -> resultRasterPath, + "parentPath" -> parentPath, + "driver" -> getDriversShortName + ) + + val result = MosaicRasterGDAL(outputRaster, createInfo, this.memSize) result.flushCache() } @@ -659,25 +691,44 @@ object MosaicRasterGDAL extends RasterReader { * @example * Raster: path = "file:///path/to/file.tif" Subdataset: path = * "file:///path/to/file.tif:subdataset" - * @param inPath - * The path to the raster file. + * @param createInfo + * The create info for the raster. This should contain the following + * keys: + * - path: The path to the raster file. + * - parentPath: The path of the parent raster file. * @return * A MosaicRaster object. */ - override def readRaster(inPath: String, parentPath: String): MosaicRasterGDAL = { + override def readRaster(createInfo: Map[String, String]): MosaicRasterGDAL = { + val inPath = createInfo("path") val isSubdataset = PathUtils.isSubdataset(inPath) val path = PathUtils.getCleanPath(inPath) val readPath = if (isSubdataset) PathUtils.getSubdatasetPath(path) else PathUtils.getZipPath(path) val dataset = openRaster(readPath, None) - val driverShortName = dataset.GetDriver().getShortName - + val error = + if (dataset == null) { + val error = gdal.GetLastErrorMsg() + s""" + Error reading raster from path: $readPath + Error: $error + """ + } else "" + val driverShortName = Try(dataset.GetDriver().getShortName).getOrElse("NONE") // Avoid costly IO to compute MEM size here // It will be available when the raster is serialized for next operation // If value is needed then it will be computed when getMemSize is called // We cannot just use memSize value of the parent due to the fact that the raster could be a subdataset - val raster = MosaicRasterGDAL(dataset, path, parentPath, driverShortName, -1) + val raster = MosaicRasterGDAL( + dataset, + createInfo ++ + Map( + "driver" -> driverShortName, + "last_error" -> error + ), + -1 + ) raster } @@ -685,17 +736,19 @@ object MosaicRasterGDAL extends RasterReader { * Reads a raster from a byte array. * @param contentBytes * The byte array containing the raster data. - * @param driverShortName - * The driver short name of the raster. + * @param createInfo + * Mosaic creation info of the raster. Note: This is not the same as the + * metadata of the raster. This is not the same as GDAL creation options. * @return * A MosaicRaster object. */ - override def readRaster(contentBytes: Array[Byte], parentPath: String, driverShortName: String): MosaicRasterGDAL = { + override def readRaster(contentBytes: Array[Byte], createInfo: Map[String, String]): MosaicRasterGDAL = { if (Option(contentBytes).isEmpty || contentBytes.isEmpty) { - MosaicRasterGDAL(null, "", parentPath, "", -1) + MosaicRasterGDAL(null, createInfo, -1) } else { // This is a temp UUID for purposes of reading the raster through GDAL from memory // The stable UUID is kept in metadata of the raster + val driverShortName = createInfo("driver") val extension = GDAL.getExtension(driverShortName) val tmpPath = PathUtils.createTmpFilePath(extension) Files.write(Paths.get(tmpPath), contentBytes) @@ -721,12 +774,12 @@ object MosaicRasterGDAL extends RasterReader { if (dataset == null) { throw new Exception(s"Error reading raster from bytes: ${prompt._3}") } - MosaicRasterGDAL(dataset, unzippedPath, parentPath, driverShortName, contentBytes.length) + MosaicRasterGDAL(dataset, createInfo + ("path" -> unzippedPath), contentBytes.length) } else { - MosaicRasterGDAL(ds, readPath, parentPath, driverShortName, contentBytes.length) + MosaicRasterGDAL(ds, createInfo + ("path" -> readPath), contentBytes.length) } } else { - MosaicRasterGDAL(dataset, tmpPath, parentPath, driverShortName, contentBytes.length) + MosaicRasterGDAL(dataset, createInfo + ("path" -> tmpPath), contentBytes.length) } } } @@ -738,15 +791,19 @@ object MosaicRasterGDAL extends RasterReader { * @example * Raster: path = "file:///path/to/file.tif" Subdataset: path = * "file:///path/to/file.tif:subdataset" - * @param path - * The path to the raster file. + * @param createInfo + * The create info for the raster. This should contain the following + * keys: + * - path: The path to the raster file. + * - parentPath: The path of the parent raster file. + * - driver: Optional: The driver short name of the raster file * @param bandIndex * The band index to read. * @return * A MosaicRaster object. */ - override def readBand(path: String, bandIndex: Int, parentPath: String): MosaicRasterBandGDAL = { - val raster = readRaster(path, parentPath) + override def readBand(bandIndex: Int, createInfo: Map[String, String]): MosaicRasterBandGDAL = { + val raster = readRaster(createInfo) // TODO: Raster and Band are coupled, this can cause a pointer leak raster.getBand(bandIndex) } diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/io/RasterReader.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/io/RasterReader.scala index b207789ae..d8a1a90c1 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/io/RasterReader.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/io/RasterReader.scala @@ -20,14 +20,16 @@ trait RasterReader extends Logging { * @example * Raster: path = "/path/to/file.tif" Subdataset: path = * "FORMAT:/path/to/file.tif:subdataset" - * @param path - * The path to the raster file. - * @param parentPath - * The path of the parent raster file. + * @param createInfo + * The create info for the raster. This should contain the following + * keys: + * - path: The path to the raster file. + * - parentPath: The path of the parent raster file. + * - driver: Optional: The driver short name of the raster file * @return * A MosaicRaster object. */ - def readRaster(path: String, parentPath: String): MosaicRasterGDAL + def readRaster(createInfo: Map[String, String]): MosaicRasterGDAL /** * Reads a raster from an in memory buffer. Use the buffer bytes to produce @@ -35,30 +37,32 @@ trait RasterReader extends Logging { * * @param contentBytes * The file bytes. - * @param parentPath - * The path of the parent raster file. - * @param driverShortName - * The driver short name of the raster file. + * @param createInfo + * The create info for the raster. This should contain the following + * keys: + * - parentPath: The path of the parent raster file. + * - driver: The driver short name of the raster file * @return * A MosaicRaster object. */ - def readRaster(contentBytes: Array[Byte], parentPath: String, driverShortName: String): MosaicRasterGDAL + def readRaster(contentBytes: Array[Byte], createInfo: Map[String, String]): MosaicRasterGDAL /** * Reads a raster band from a file system path. Reads a subdataset band if * the path is to a subdataset. + * * @example * Raster: path = "/path/to/file.tif" Subdataset: path = * "FORMAT:/path/to/file.tif:subdataset" - * @param path - * The path to the raster file. - * @param bandIndex - * The band index to read. - * @param parentPath - * The path of the parent raster file. + * @param createInfo + * The create info for the raster. This should contain the following + * keys: + * - path: The path to the raster file. + * - parentPath: The path of the parent raster file. + * - driver: Optional: The driver short name of the raster file * @return * A MosaicRaster object. */ - def readBand(path: String, bandIndex: Int, parentPath: String): MosaicRasterBandGDAL + def readBand(bandIndex: Int, createInfo: Map[String, String]): MosaicRasterBandGDAL } diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALBuildVRT.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALBuildVRT.scala index 9e1e97401..cb79dc263 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALBuildVRT.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALBuildVRT.scala @@ -24,16 +24,17 @@ object GDALBuildVRT { val vrtOptionsVec = OperatorOptions.parseOptions(effectiveCommand) val vrtOptions = new BuildVRTOptions(vrtOptionsVec) val result = gdal.BuildVRT(outputPath, rasters.map(_.getRaster).toArray, vrtOptions) - if (result == null) { - throw new Exception(s""" - |Build VRT failed. - |Command: $effectiveCommand - |Error: ${gdal.GetLastErrorMsg} - |""".stripMargin) - } - // TODO: Figure out multiple parents, should this be an array? + val errorMsg = gdal.GetLastErrorMsg + val createInfo = Map( + "path" -> outputPath, + "parentPath" -> rasters.head.getParentPath, + "driver" -> "VRT", + "last_command" -> effectiveCommand, + "last_error" -> errorMsg, + "all_parents" -> rasters.map(_.getParentPath).mkString(";") + ) // VRT files are just meta files, mem size doesnt make much sense so we keep -1 - MosaicRasterGDAL(result, outputPath, rasters.head.getParentPath, "VRT", -1).flushCache() + MosaicRasterGDAL(result, createInfo, -1).flushCache() } } diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALCalc.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALCalc.scala index fa92c3b37..e22228817 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALCalc.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALCalc.scala @@ -3,6 +3,7 @@ package com.databricks.labs.mosaic.core.raster.operator.gdal import com.databricks.labs.mosaic.core.raster.api.GDAL import com.databricks.labs.mosaic.core.raster.gdal.{MosaicRasterGDAL, MosaicRasterWriteOptions} import com.databricks.labs.mosaic.utils.SysUtils +import org.gdal.gdal.gdal /** GDALCalc is a helper object for executing GDAL Calc commands. */ object GDALCalc { @@ -33,18 +34,26 @@ object GDALCalc { val effectiveCommand = OperatorOptions.appendOptions(gdalCalcCommand, MosaicRasterWriteOptions.GTiff) val toRun = effectiveCommand.replace("gdal_calc", gdal_calc) val commandRes = SysUtils.runCommand(s"python3 $toRun") - if (commandRes._1.startsWith("ERROR")) { - throw new RuntimeException(s""" - |GDAL Calc command failed: - |$toRun - |STDOUT: - |${commandRes._2} - |STDERR: - |${commandRes._3} - |""".stripMargin) - } + val errorMsg = gdal.GetLastErrorMsg val result = GDAL.raster(resultPath, resultPath) - result + val createInfo = Map( + "path" -> resultPath, + "parentPath" -> resultPath, + "driver" -> "GTiff", + "last_command" -> effectiveCommand, + "last_error" -> errorMsg, + "all_parents" -> resultPath, + "full_error" -> s""" + |GDAL Calc command failed: + |GDAL err: + |$errorMsg + |STDOUT: + |${commandRes._2} + |STDERR: + |${commandRes._3} + |""".stripMargin + ) + result.copy(createInfo = createInfo) } } diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALInfo.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALInfo.scala index 7a60a837a..d3ccd471b 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALInfo.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALInfo.scala @@ -25,14 +25,14 @@ object GDALInfo { val gdalInfo = gdal.GDALInfo(raster.getRaster, infoOptions) if (gdalInfo == null) { - throw new Exception(s""" - |GDAL Info failed. - |Command: $command - |Error: ${gdal.GetLastErrorMsg} - |""".stripMargin) + s""" + |GDAL Info failed. + |Command: $command + |Error: ${gdal.GetLastErrorMsg} + |""".stripMargin + } else { + gdalInfo } - - gdalInfo } } diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALTranslate.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALTranslate.scala index fd24a0f73..2fb106fda 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALTranslate.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALTranslate.scala @@ -31,15 +31,19 @@ object GDALTranslate { val translateOptionsVec = OperatorOptions.parseOptions(effectiveCommand) val translateOptions = new TranslateOptions(translateOptionsVec) val result = gdal.Translate(outputPath, raster.getRaster, translateOptions) - if (result == null) { - throw new Exception(s""" - |Translate failed. - |Command: $effectiveCommand - |Error: ${gdal.GetLastErrorMsg} - |""".stripMargin) - } + val errorMsg = gdal.GetLastErrorMsg val size = Files.size(Paths.get(outputPath)) - raster.copy(raster = result, path = outputPath, memSize = size, driverShortName = writeOptions.format).flushCache() + val createInfo = Map( + "path" -> outputPath, + "parentPath" -> raster.getParentPath, + "driver" -> writeOptions.format, + "last_command" -> effectiveCommand, + "last_error" -> errorMsg, + "all_parents" -> raster.getParentPath + ) + raster + .copy(raster = result, createInfo = createInfo, memSize = size) + .flushCache() } } diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALWarp.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALWarp.scala index ba6dce58d..516560a76 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALWarp.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/gdal/GDALWarp.scala @@ -27,23 +27,18 @@ object GDALWarp { val warpOptionsVec = OperatorOptions.parseOptions(effectiveCommand) val warpOptions = new WarpOptions(warpOptionsVec) val result = gdal.Warp(outputPath, rasters.map(_.getRaster).toArray, warpOptions) - // TODO: Figure out multiple parents, should this be an array? // Format will always be the same as the first raster - if (result == null) { - throw new Exception(s""" - |Warp failed. - |Command: $effectiveCommand - |Error: ${gdal.GetLastErrorMsg} - |""".stripMargin) - } + val errorMsg = gdal.GetLastErrorMsg val size = Files.size(Paths.get(outputPath)) - rasters.head - .copy( - raster = result, - path = outputPath, - memSize = size - ) - .flushCache() + val createInfo = Map( + "path" -> outputPath, + "parentPath" -> rasters.head.getParentPath, + "driver" -> rasters.head.getWriteOptions.format, + "last_command" -> effectiveCommand, + "last_error" -> errorMsg, + "all_parents" -> rasters.map(_.getParentPath).mkString(";") + ) + rasters.head.copy(raster = result, createInfo = createInfo, memSize = size).flushCache() } } diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/OverlappingTiles.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/OverlappingTiles.scala index 4e9f61c5e..072380666 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/OverlappingTiles.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/OverlappingTiles.scala @@ -69,7 +69,7 @@ object OverlappingTiles { val (_, valid) = tiles.flatten.partition(_._1) - valid.map(t => MosaicRasterTile(null, t._2, raster.getParentPath, raster.getDriversShortName)) + valid.map(t => MosaicRasterTile(null, t._2)) } diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/RasterTessellate.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/RasterTessellate.scala index 701cf8cf1..9920af923 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/RasterTessellate.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/RasterTessellate.scala @@ -38,13 +38,13 @@ object RasterTessellate { val cellID = cell.cellIdAsLong(indexSystem) val isValidCell = indexSystem.isValid(cellID) if (!isValidCell) { - (false, MosaicRasterTile(cell.index, null, "", "")) + (false, MosaicRasterTile(cell.index, null)) } else { val cellRaster = tmpRaster.getRasterForCell(cellID, indexSystem, geometryAPI) val isValidRaster = !cellRaster.isEmpty ( isValidRaster, - MosaicRasterTile(cell.index, cellRaster, raster.getParentPath, raster.getDriversShortName) + MosaicRasterTile(cell.index, cellRaster) ) } }) diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/ReTile.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/ReTile.scala index b12a8f847..7b218199e 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/ReTile.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/retile/ReTile.scala @@ -58,7 +58,7 @@ object ReTile { val (_, valid) = tiles.partition(_._1) - valid.map(t => MosaicRasterTile(null, t._2, raster.getParentPath, raster.getDriversShortName)) + valid.map(t => MosaicRasterTile(null, t._2)) } diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/separate/SeparateBands.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/separate/SeparateBands.scala index 25f73bf8b..9580cc441 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/separate/SeparateBands.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/separate/SeparateBands.scala @@ -5,7 +5,10 @@ import com.databricks.labs.mosaic.core.raster.operator.gdal.GDALTranslate import com.databricks.labs.mosaic.core.types.model.MosaicRasterTile import com.databricks.labs.mosaic.utils.PathUtils -/** ReTile is a helper object for splitting multi-band rasters into single-band-per-row. */ +/** + * ReTile is a helper object for splitting multi-band rasters into + * single-band-per-row. + */ object SeparateBands { /** @@ -24,11 +27,13 @@ object SeparateBands { val fileExtension = raster.getRasterFileExtension val rasterPath = PathUtils.createTmpFilePath(fileExtension) val shortDriver = raster.getDriversShortName + val outOptions = raster.getWriteOptions val result = GDALTranslate.executeTranslate( rasterPath, raster, - command = s"gdal_translate -of $shortDriver -b ${i + 1} -co COMPRESS=DEFLATE" + command = s"gdal_translate -of $shortDriver -b ${i + 1}", + writeOptions = outOptions ) val isEmpty = result.isEmpty @@ -38,13 +43,13 @@ object SeparateBands { if (isEmpty) dispose(result) - (isEmpty, result, i) + (isEmpty, result.copy(createInfo = result.createInfo ++ Map("bandIndex" -> (i + 1).toString)), i) } val (_, valid) = tiles.partition(_._1) - valid.map(t => new MosaicRasterTile(null, t._2, raster.getParentPath, raster.getDriversShortName)) + valid.map(t => new MosaicRasterTile(null, t._2)) } diff --git a/src/main/scala/com/databricks/labs/mosaic/core/types/RasterTileType.scala b/src/main/scala/com/databricks/labs/mosaic/core/types/RasterTileType.scala index 5203178e0..137d482ce 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/types/RasterTileType.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/types/RasterTileType.scala @@ -36,8 +36,7 @@ object RasterTileType { Array( StructField("index_id", idType), StructField("raster", rasterType), - StructField("parentPath", StringType), - StructField("driver", StringType) + StructField("metadata", MapType(StringType, StringType)) ) ) } diff --git a/src/main/scala/com/databricks/labs/mosaic/core/types/model/MosaicRasterTile.scala b/src/main/scala/com/databricks/labs/mosaic/core/types/model/MosaicRasterTile.scala index a0710dbe9..bf36ea8f2 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/types/model/MosaicRasterTile.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/types/model/MosaicRasterTile.scala @@ -3,6 +3,7 @@ package com.databricks.labs.mosaic.core.types.model import com.databricks.labs.mosaic.core.index.IndexSystem import com.databricks.labs.mosaic.core.raster.api.GDAL import com.databricks.labs.mosaic.core.raster.gdal.MosaicRasterGDAL +import com.databricks.labs.mosaic.expressions.raster.{buildMapString, extractMap} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.types.{BinaryType, DataType, LongType, StringType} import org.apache.spark.unsafe.types.UTF8String @@ -16,18 +17,16 @@ import scala.util.{Failure, Success, Try} * Index ID. * @param raster * Raster instance corresponding to the tile. - * @param parentPath - * Parent path of the raster. - * @param driver - * Driver used to read the raster. */ case class MosaicRasterTile( index: Either[Long, String], - raster: MosaicRasterGDAL, - parentPath: String, - driver: String + raster: MosaicRasterGDAL ) { + def parentPath: String = raster.createInfo("parentPath") + + def driver: String = raster.createInfo("driver") + def getIndex: Either[Long, String] = index def getParentPath: String = parentPath @@ -57,18 +56,8 @@ case class MosaicRasterTile( (indexSystem.getCellIdDataType, index) match { case (_: LongType, Left(_)) => this case (_: StringType, Right(_)) => this - case (_: LongType, Right(value)) => new MosaicRasterTile( - index = Left(indexSystem.parse(value)), - raster = raster, - parentPath = parentPath, - driver = driver - ) - case (_: StringType, Left(value)) => new MosaicRasterTile( - index = Right(indexSystem.format(value)), - raster = raster, - parentPath = parentPath, - driver = driver - ) + case (_: LongType, Right(value)) => this.copy(index = Left(indexSystem.parse(value))) + case (_: StringType, Left(value)) => this.copy(index = Right(indexSystem.format(value))) case _ => throw new IllegalArgumentException("Invalid cell id data type") } } @@ -110,22 +99,21 @@ case class MosaicRasterTile( def serialize( rasterDataType: DataType ): InternalRow = { - val parentPathUTF8 = UTF8String.fromString(parentPath) - val driverUTF8 = UTF8String.fromString(driver) val encodedRaster = encodeRaster(rasterDataType) + val mapData = buildMapString(raster.createInfo) if (Option(index).isDefined) { if (index.isLeft) InternalRow.fromSeq( - Seq(index.left.get, encodedRaster, parentPathUTF8, driverUTF8) + Seq(index.left.get, encodedRaster, mapData) ) else { // Copy from tmp to checkpoint. // Have to use GDAL Driver to do this since sidecar files are not copied by spark. InternalRow.fromSeq( - Seq(UTF8String.fromString(index.right.get), encodedRaster, parentPathUTF8, driverUTF8) + Seq(UTF8String.fromString(index.right.get), encodedRaster, mapData) ) } } else { - InternalRow.fromSeq(Seq(null, encodedRaster, parentPathUTF8, driverUTF8)) + InternalRow.fromSeq(Seq(null, encodedRaster, mapData)) } } @@ -147,6 +135,7 @@ case class MosaicRasterTile( case Success(value) => value.toInt case Failure(_) => -1 } + } /** Companion object. */ @@ -165,18 +154,17 @@ object MosaicRasterTile { def deserialize(row: InternalRow, idDataType: DataType, rasterType: DataType): MosaicRasterTile = { val index = row.get(0, idDataType) val rawRaster = row.get(1, rasterType) - val parentPath = row.get(2, StringType).toString - val driver = row.get(3, StringType).toString - val raster = GDAL.readRaster(rawRaster, parentPath, driver, rasterType) + val createInfo = extractMap(row.getMap(2)) + val raster = GDAL.readRaster(rawRaster, createInfo, rasterType) // noinspection TypeCheckCanBeMatch if (Option(index).isDefined) { if (index.isInstanceOf[Long]) { - new MosaicRasterTile(Left(index.asInstanceOf[Long]), raster, parentPath, driver) + new MosaicRasterTile(Left(index.asInstanceOf[Long]), raster) } else { - new MosaicRasterTile(Right(index.asInstanceOf[UTF8String].toString), raster, parentPath, driver) + new MosaicRasterTile(Right(index.asInstanceOf[UTF8String].toString), raster) } } else { - new MosaicRasterTile(null, raster, parentPath, driver) + new MosaicRasterTile(null, raster) } } diff --git a/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReTileOnRead.scala b/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReTileOnRead.scala index a38e76900..867167c58 100644 --- a/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReTileOnRead.scala +++ b/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReTileOnRead.scala @@ -134,8 +134,9 @@ object ReTileOnRead extends ReadStrategy { */ def localSubdivide(inPath: String, parentPath: String, sizeInMB: Int): Seq[MosaicRasterTile] = { val cleanPath = PathUtils.getCleanPath(inPath) - val raster = MosaicRasterGDAL.readRaster(cleanPath, parentPath) - val inTile = new MosaicRasterTile(null, raster, parentPath, raster.getDriversShortName) + val createInfo = Map("path" -> cleanPath, "parentPath" -> parentPath) + val raster = MosaicRasterGDAL.readRaster(createInfo) + val inTile = new MosaicRasterTile(null, raster) val tiles = BalancedSubdivision.splitRaster(inTile, sizeInMB) RasterCleaner.dispose(raster) RasterCleaner.dispose(inTile) diff --git a/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReadAsPath.scala b/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReadAsPath.scala new file mode 100644 index 000000000..0973146ef --- /dev/null +++ b/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReadAsPath.scala @@ -0,0 +1,124 @@ +package com.databricks.labs.mosaic.datasource.gdal + +import com.databricks.labs.mosaic.core.index.{IndexSystem, IndexSystemFactory} +import com.databricks.labs.mosaic.core.raster.gdal.MosaicRasterGDAL +import com.databricks.labs.mosaic.core.raster.io.RasterCleaner +import com.databricks.labs.mosaic.core.types.RasterTileType +import com.databricks.labs.mosaic.core.types.model.MosaicRasterTile +import com.databricks.labs.mosaic.datasource.Utils +import com.databricks.labs.mosaic.datasource.gdal.GDALFileFormat._ +import com.databricks.labs.mosaic.utils.PathUtils +import org.apache.hadoop.fs.{FileStatus, FileSystem} +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.types._ + +import java.nio.file.{Files, Paths} + +/** An object defining the retiling read strategy for the GDAL file format. */ +object ReadAsPath extends ReadStrategy { + + val tileDataType: DataType = StringType + + // noinspection DuplicatedCode + /** + * Returns the schema of the GDAL file format. + * @note + * Different read strategies can have different schemas. This is because + * the schema is defined by the read strategy. For retiling we always use + * checkpoint location. In this case rasters are stored off spark rows. + * If you need the tiles in memory please load them from path stored in + * the tile returned by the reader. + * + * @param options + * Options passed to the reader. + * @param files + * List of files to read. + * @param parentSchema + * Parent schema. + * @param sparkSession + * Spark session. + * + * @return + * Schema of the GDAL file format. + */ + override def getSchema( + options: Map[String, String], + files: Seq[FileStatus], + parentSchema: StructType, + sparkSession: SparkSession + ): StructType = { + val trimmedSchema = parentSchema.filter(field => field.name != CONTENT && field.name != LENGTH) + val indexSystem = IndexSystemFactory.getIndexSystem(sparkSession) + StructType(trimmedSchema) + .add(StructField(UUID, LongType, nullable = false)) + .add(StructField(X_SIZE, IntegerType, nullable = false)) + .add(StructField(Y_SIZE, IntegerType, nullable = false)) + .add(StructField(BAND_COUNT, IntegerType, nullable = false)) + .add(StructField(METADATA, MapType(StringType, StringType), nullable = false)) + .add(StructField(SUBDATASETS, MapType(StringType, StringType), nullable = false)) + .add(StructField(SRID, IntegerType, nullable = false)) + .add(StructField(LENGTH, LongType, nullable = false)) + // Note that for retiling we always use checkpoint location. + // In this case rasters are stored off spark rows. + // If you need the tiles in memory please load them from path stored in the tile returned by the reader. + .add(StructField(TILE, RasterTileType(indexSystem.getCellIdDataType, tileDataType), nullable = false)) + } + + /** + * Reads the content of the file. + * @param status + * File status. + * @param fs + * File system. + * @param requiredSchema + * Required schema. + * @param options + * Options passed to the reader. + * @param indexSystem + * Index system. + * + * @return + * Iterator of internal rows. + */ + override def read( + status: FileStatus, + fs: FileSystem, + requiredSchema: StructType, + options: Map[String, String], + indexSystem: IndexSystem + ): Iterator[InternalRow] = { + val inPath = status.getPath.toString + val uuid = getUUID(status) + + val tmpPath = PathUtils.copyToTmp(inPath) + val createInfo = Map("path" -> tmpPath, "parentPath" -> inPath) + val raster = MosaicRasterGDAL.readRaster(createInfo) + val tile = MosaicRasterTile(null, raster) + + val trimmedSchema = StructType(requiredSchema.filter(field => field.name != TILE)) + val fields = trimmedSchema.fieldNames.map { + case PATH => status.getPath.toString + case MODIFICATION_TIME => status.getModificationTime + case UUID => uuid + case X_SIZE => tile.getRaster.xSize + case Y_SIZE => tile.getRaster.ySize + case BAND_COUNT => tile.getRaster.numBands + case METADATA => tile.getRaster.metadata + case SUBDATASETS => tile.getRaster.subdatasets + case SRID => tile.getRaster.SRID + case LENGTH => tile.getRaster.getMemSize + case other => throw new RuntimeException(s"Unsupported field name: $other") + } + // Writing to bytes is destructive so we delay reading content and content length until the last possible moment + val row = Utils.createRow(fields ++ Seq(tile.formatCellId(indexSystem).serialize(tileDataType))) + RasterCleaner.dispose(tile) + + val rows = Seq(row) + + Files.deleteIfExists(Paths.get(tmpPath)) + + rows.iterator + } + +} diff --git a/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReadInMemory.scala b/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReadInMemory.scala index 15ddec2ed..7e6687079 100644 --- a/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReadInMemory.scala +++ b/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReadInMemory.scala @@ -6,12 +6,12 @@ import com.databricks.labs.mosaic.core.raster.io.RasterCleaner import com.databricks.labs.mosaic.core.types.RasterTileType import com.databricks.labs.mosaic.datasource.Utils import com.databricks.labs.mosaic.datasource.gdal.GDALFileFormat._ +import com.databricks.labs.mosaic.expressions.raster.buildMapString import com.databricks.labs.mosaic.utils.PathUtils import org.apache.hadoop.fs.{FileStatus, FileSystem} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String /** An object defining the in memory read strategy for the GDAL file format. */ object ReadInMemory extends ReadStrategy { @@ -78,9 +78,12 @@ object ReadInMemory extends ReadStrategy { ): Iterator[InternalRow] = { val inPath = status.getPath.toString val readPath = PathUtils.getCleanPath(inPath) - val driverShortName = MosaicRasterGDAL.identifyDriver(readPath) val contentBytes: Array[Byte] = readContent(fs, status) - val raster = MosaicRasterGDAL.readRaster(readPath, inPath) + val createInfo = Map( + "path" -> readPath, + "parentPath" -> inPath + ) + val raster = MosaicRasterGDAL.readRaster(createInfo) val uuid = getUUID(status) val fields = requiredSchema.fieldNames.filter(_ != TILE).map { @@ -96,8 +99,9 @@ object ReadInMemory extends ReadStrategy { case SRID => raster.SRID case other => throw new RuntimeException(s"Unsupported field name: $other") } + val mapData = buildMapString(raster.createInfo) val rasterTileSer = InternalRow.fromSeq( - Seq(null, contentBytes, UTF8String.fromString(inPath), UTF8String.fromString(driverShortName), null) + Seq(null, contentBytes, mapData) ) val row = Utils.createRow( fields ++ Seq(rasterTileSer) diff --git a/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReadStrategy.scala b/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReadStrategy.scala index cacc1c133..ab141b069 100644 --- a/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReadStrategy.scala +++ b/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReadStrategy.scala @@ -72,6 +72,7 @@ object ReadStrategy { readStrategy match { case MOSAIC_RASTER_READ_IN_MEMORY => ReadInMemory case MOSAIC_RASTER_RE_TILE_ON_READ => ReTileOnRead + case MOSAIC_RASTER_READ_AS_PATH => ReadAsPath case _ => ReadInMemory } diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_CombineAvg.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_CombineAvg.scala index f94eae6e8..de163ca37 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_CombineAvg.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_CombineAvg.scala @@ -28,12 +28,7 @@ case class RST_CombineAvg( /** Combines the rasters using average of pixels. */ override def rasterTransform(tiles: Seq[MosaicRasterTile]): Any = { val index = if (tiles.map(_.getIndex).groupBy(identity).size == 1) tiles.head.getIndex else null - MosaicRasterTile( - index, - CombineAVG.compute(tiles.map(_.getRaster)), - tiles.head.getParentPath, - tiles.head.getDriver - ) + MosaicRasterTile(index, CombineAVG.compute(tiles.map(_.getRaster))) } } diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_CombineAvgAgg.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_CombineAvgAgg.scala index be12bd12c..3bf9248c2 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_CombineAvgAgg.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_CombineAvgAgg.scala @@ -79,11 +79,8 @@ case class RST_CombineAvgAgg( // If merging multiple index rasters, the index value is dropped val idx = if (tiles.map(_.getIndex).groupBy(identity).size == 1) tiles.head.getIndex else null var combined = CombineAVG.compute(tiles.map(_.getRaster)).flushCache() - // TODO: should parent path be an array? - val parentPath = tiles.head.getParentPath - val driver = tiles.head.getDriver - val result = MosaicRasterTile(idx, combined, parentPath, driver) + val result = MosaicRasterTile(idx, combined) .formatCellId(IndexSystemFactory.getIndexSystem(expressionConfig.getIndexSystem)) .serialize(tileType) diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_DerivedBand.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_DerivedBand.scala index 3e7de13b5..459f1774a 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_DerivedBand.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_DerivedBand.scala @@ -37,9 +37,7 @@ case class RST_DerivedBand( val index = if (tiles.map(_.getIndex).groupBy(identity).size == 1) tiles.head.getIndex else null MosaicRasterTile( index, - PixelCombineRasters.combine(tiles.map(_.getRaster), pythonFunc, funcName), - tiles.head.getParentPath, - tiles.head.getDriver + PixelCombineRasters.combine(tiles.map(_.getRaster), pythonFunc, funcName) ) } diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_DerivedBandAgg.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_DerivedBandAgg.scala index 7d972b6e1..836b79cbd 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_DerivedBandAgg.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_DerivedBandAgg.scala @@ -89,11 +89,8 @@ case class RST_DerivedBandAgg( val idx = if (tiles.map(_.getIndex).groupBy(identity).size == 1) tiles.head.getIndex else null var combined = PixelCombineRasters.combine(tiles.map(_.getRaster), pythonFunc, funcName) - // TODO: should parent path be an array? - val parentPath = tiles.head.getParentPath - val driver = tiles.head.getDriver - val result = MosaicRasterTile(idx, combined, parentPath, driver) + val result = MosaicRasterTile(idx, combined) .formatCellId(IndexSystemFactory.getIndexSystem(expressionConfig.getIndexSystem)) .serialize(BinaryType) diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_FromContent.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_FromContent.scala index 956c9c049..1021eb083 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_FromContent.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_FromContent.scala @@ -68,8 +68,9 @@ case class RST_FromContent( val targetSize = sizeInMB.eval(input).asInstanceOf[Int] if (targetSize <= 0 || rasterArr.length <= targetSize) { // - no split required - var raster = MosaicRasterGDAL.readRaster(rasterArr, PathUtils.NO_PATH_STRING, driver) - var tile = MosaicRasterTile(null, raster, PathUtils.NO_PATH_STRING, driver) + val createInfo = Map("parentPath" -> PathUtils.NO_PATH_STRING, "driver" -> driver) + var raster = MosaicRasterGDAL.readRaster(rasterArr, createInfo) + var tile = MosaicRasterTile(null, raster) val row = tile.formatCellId(indexSystem).serialize(tileType) RasterCleaner.dispose(raster) RasterCleaner.dispose(tile) diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_FromFile.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_FromFile.scala index ddd4c6af2..cd5808f30 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_FromFile.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_FromFile.scala @@ -15,7 +15,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.expressions.{CollectionGenerator, Expression, Literal, NullIntolerant} -import org.apache.spark.sql.types.{BinaryType, DataType, IntegerType, StringType, StructField, StructType} +import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String import java.nio.file.{Files, Paths, StandardCopyOption} @@ -66,8 +66,9 @@ case class RST_FromFile( val driver = MosaicRasterGDAL.identifyDriver(path) val targetSize = sizeInMB.eval(input).asInstanceOf[Int] if (targetSize <= 0 && Files.size(Paths.get(readPath)) <= Integer.MAX_VALUE) { - var raster = MosaicRasterGDAL.readRaster(readPath, path) - var tile = MosaicRasterTile(null, raster, path, raster.getDriversShortName) + val createInfo = Map("path" -> readPath, "parentPath" -> path) + var raster = MosaicRasterGDAL.readRaster(createInfo) + var tile = MosaicRasterTile(null, raster) val row = tile.formatCellId(indexSystem).serialize(tileType) RasterCleaner.dispose(raster) RasterCleaner.dispose(tile) diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MakeTiles.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MakeTiles.scala index 586337556..12c70d025 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MakeTiles.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MakeTiles.scala @@ -125,8 +125,9 @@ case class RST_MakeTiles( if (targetSize <= 0 && inputSize <= Integer.MAX_VALUE) { // - no split required - val raster = GDAL.readRaster(rawInput, PathUtils.NO_PATH_STRING, driver, inputExpr.dataType) - val tile = MosaicRasterTile(null, raster, PathUtils.NO_PATH_STRING, driver) + val createInfo = Map("parentPath" -> PathUtils.NO_PATH_STRING, "driver" -> driver) + val raster = GDAL.readRaster(rawInput, createInfo, inputExpr.dataType) + val tile = MosaicRasterTile(null, raster) val row = tile.formatCellId(indexSystem).serialize(tileType) RasterCleaner.dispose(raster) RasterCleaner.dispose(tile) diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MapAlgebra.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MapAlgebra.scala index 606f8cc72..bc2aea949 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MapAlgebra.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MapAlgebra.scala @@ -45,12 +45,8 @@ case class RST_MapAlgebra( val resultPath = PathUtils.createTmpFilePath(extension) val command = parseSpec(jsonSpec, resultPath, tiles) val index = if (tiles.map(_.getIndex).groupBy(identity).size == 1) tiles.head.getIndex else null - MosaicRasterTile( - index, - GDALCalc.executeCalc(command, resultPath), - resultPath, - tiles.head.getDriver - ) + val result = GDALCalc.executeCalc(command, resultPath) + MosaicRasterTile(index, result) } def parseSpec(jsonSpec: String, resultPath: String, tiles: Seq[MosaicRasterTile]): String = { diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MergeAgg.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MergeAgg.scala index b639a42da..ae56a01ab 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MergeAgg.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MergeAgg.scala @@ -80,11 +80,8 @@ case class RST_MergeAgg( // If merging multiple index rasters, the index value is dropped val idx = if (tiles.map(_.getIndex).groupBy(identity).size == 1) tiles.head.getIndex else null var merged = MergeRasters.merge(tiles.map(_.getRaster)).flushCache() - // TODO: should parent path be an array? - val parentPath = tiles.head.getParentPath - val driver = tiles.head.getDriver - val result = MosaicRasterTile(idx, merged, parentPath, driver) + val result = MosaicRasterTile(idx, merged) .formatCellId(IndexSystemFactory.getIndexSystem(expressionConfig.getIndexSystem)) .serialize(BinaryType) diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_SetSRID.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_SetSRID.scala index a3d44289a..2aabb3df9 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_SetSRID.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_SetSRID.scala @@ -1,8 +1,6 @@ package com.databricks.labs.mosaic.expressions.raster import com.databricks.labs.mosaic.core.geometry.api.GeometryAPI -import com.databricks.labs.mosaic.core.raster.io.RasterCleaner -import com.databricks.labs.mosaic.core.raster.operator.clip.RasterClipByVector import com.databricks.labs.mosaic.core.types.RasterTileType import com.databricks.labs.mosaic.core.types.model.MosaicRasterTile import com.databricks.labs.mosaic.expressions.base.{GenericExpressionFactory, WithExpressionInfo} @@ -11,6 +9,7 @@ import com.databricks.labs.mosaic.functions.MosaicExpressionConfig import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant} +import org.apache.spark.sql.types.DataType /** The expression for clipping a raster by a vector. */ case class RST_SetSRID( @@ -19,14 +18,15 @@ case class RST_SetSRID( expressionConfig: MosaicExpressionConfig ) extends Raster1ArgExpression[RST_SetSRID]( rastersExpr, - sridExpr, - RasterTileType(expressionConfig.getCellIdType), + sridExpr, returnsRaster = true, expressionConfig = expressionConfig ) with NullIntolerant with CodegenFallback { + override def dataType: DataType = RasterTileType(expressionConfig.getCellIdType, rastersExpr) + val geometryAPI: GeometryAPI = GeometryAPI(expressionConfig.getGeometryAPI) /** diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Transform.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Transform.scala new file mode 100644 index 000000000..7681f2bba --- /dev/null +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Transform.scala @@ -0,0 +1,61 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.raster.operator.proj.RasterProject +import com.databricks.labs.mosaic.core.types.RasterTileType +import com.databricks.labs.mosaic.core.types.model.MosaicRasterTile +import com.databricks.labs.mosaic.expressions.base.{GenericExpressionFactory, WithExpressionInfo} +import com.databricks.labs.mosaic.expressions.raster.base.Raster1ArgExpression +import com.databricks.labs.mosaic.functions.MosaicExpressionConfig +import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant} +import org.apache.spark.sql.types._ +import org.gdal.osr.SpatialReference + +/** Returns the upper left x of the raster. */ +case class RST_Transform( + tileExpr: Expression, + srid: Expression, + expressionConfig: MosaicExpressionConfig +) extends Raster1ArgExpression[RST_Transform]( + tileExpr, + srid, + returnsRaster = true, + expressionConfig + ) + with NullIntolerant + with CodegenFallback { + + override def dataType: DataType = RasterTileType(expressionConfig.getCellIdType, tileExpr) + + /** Returns the upper left x of the raster. */ + override def rasterTransform(tile: MosaicRasterTile, arg1: Any): Any = { + val srid = arg1.asInstanceOf[Int] + val sReff = new SpatialReference() + sReff.ImportFromEPSG(srid) + sReff.SetAxisMappingStrategy(org.gdal.osr.osrConstants.OAMS_TRADITIONAL_GIS_ORDER) + val result = RasterProject.project(tile.raster, sReff) + tile.copy(raster = result) + } + +} + +/** Expression info required for the expression registration for spark SQL. */ +object RST_Transform extends WithExpressionInfo { + + override def name: String = "rst_transform" + + override def usage: String = "_FUNC_(expr1) - Returns an array containing mean values for each band." + + override def example: String = + """ + | Examples: + | > SELECT _FUNC_(raster_tile); + | [1.123, 2.123, 3.123] + | """.stripMargin + + override def builder(expressionConfig: MosaicExpressionConfig): FunctionBuilder = { + GenericExpressionFactory.getBaseBuilder[RST_Avg](1, expressionConfig) + } + +} diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/package.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/package.scala index a229aae89..7db83db25 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/package.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/package.scala @@ -1,6 +1,6 @@ package com.databricks.labs.mosaic.expressions -import org.apache.spark.sql.catalyst.util.{ArrayBasedMapBuilder, ArrayBasedMapData, ArrayData} +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapBuilder, ArrayBasedMapData, ArrayData, MapData} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -21,8 +21,8 @@ package object raster { * The measure type of the resulting pixel value. * * @return - * The datatype to be used for serialization of the result of - * [[com.databricks.labs.mosaic.expressions.raster.base.RasterToGridExpression]]. + * The datatype to be used for serialization of the result of + * [[com.databricks.labs.mosaic.expressions.raster.base.RasterToGridExpression]]. */ def RasterToGridType(cellIDType: DataType, measureType: DataType): DataType = { ArrayType( @@ -49,6 +49,19 @@ package object raster { mapBuilder.build() } + /** + * Extracts a scala Map[String, String] from a spark map. + * @param mapData + * The map to be used. + * @return + * Deserialized map. + */ + def extractMap(mapData: MapData): Map[String, String] = { + val keys = mapData.keyArray().toArray[UTF8String](StringType).map(_.toString) + val values = mapData.valueArray().toArray[UTF8String](StringType).map(_.toString) + keys.zip(values).toMap + } + /** * Builds a spark map from a scala Map[String, Double]. * @param metaData diff --git a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala index cc0a611dd..293d27593 100644 --- a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala +++ b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala @@ -305,6 +305,7 @@ class MosaicContext(indexSystem: IndexSystem, geometryAPI: GeometryAPI) extends mosaicRegistry.registerExpression[RST_Subdatasets](expressionConfig) mosaicRegistry.registerExpression[RST_Summary](expressionConfig) mosaicRegistry.registerExpression[RST_Tessellate](expressionConfig) + mosaicRegistry.registerExpression[RST_Transform](expressionConfig) mosaicRegistry.registerExpression[RST_FromContent](expressionConfig) mosaicRegistry.registerExpression[RST_FromFile](expressionConfig) mosaicRegistry.registerExpression[RST_ToOverlappingTiles](expressionConfig) @@ -749,6 +750,8 @@ class MosaicContext(indexSystem: IndexSystem, geometryAPI: GeometryAPI) extends def rst_summary(raster: Column): Column = ColumnAdapter(RST_Summary(raster.expr, expressionConfig)) def rst_tessellate(raster: Column, resolution: Column): Column = ColumnAdapter(RST_Tessellate(raster.expr, resolution.expr, expressionConfig)) + def rst_transform(raster: Column, srid: Column): Column = + ColumnAdapter(RST_Transform(raster.expr, srid.expr, expressionConfig)) def rst_tessellate(raster: Column, resolution: Int): Column = ColumnAdapter(RST_Tessellate(raster.expr, lit(resolution).expr, expressionConfig)) def rst_fromcontent(raster: Column, driver: Column): Column = diff --git a/src/main/scala/com/databricks/labs/mosaic/gdal/MosaicGDAL.scala b/src/main/scala/com/databricks/labs/mosaic/gdal/MosaicGDAL.scala index c7676a88d..b9e972d6f 100644 --- a/src/main/scala/com/databricks/labs/mosaic/gdal/MosaicGDAL.scala +++ b/src/main/scala/com/databricks/labs/mosaic/gdal/MosaicGDAL.scala @@ -47,6 +47,7 @@ object MosaicGDAL extends Logging { /** Configures the GDAL environment. */ def configureGDAL(mosaicConfig: MosaicExpressionConfig): Unit = { val CPL_TMPDIR = MosaicContext.tmpDir + val GDAL_PAM_PROXY_DIR = MosaicContext.tmpDir gdal.SetConfigOption("GDAL_VRT_ENABLE_PYTHON", "YES") gdal.SetConfigOption("GDAL_DISABLE_READDIR_ON_OPEN", "TRUE") gdal.SetConfigOption("CPL_TMPDIR", CPL_TMPDIR) diff --git a/src/test/scala/com/databricks/labs/mosaic/core/raster/TestRasterBandGDAL.scala b/src/test/scala/com/databricks/labs/mosaic/core/raster/TestRasterBandGDAL.scala index 1337ae6d2..88c0f4bbb 100644 --- a/src/test/scala/com/databricks/labs/mosaic/core/raster/TestRasterBandGDAL.scala +++ b/src/test/scala/com/databricks/labs/mosaic/core/raster/TestRasterBandGDAL.scala @@ -10,10 +10,11 @@ class TestRasterBandGDAL extends SharedSparkSessionGDAL { test("Read band metadata and pixel data from GeoTIFF file.") { assume(System.getProperty("os.name") == "Linux") - val testRaster = MosaicRasterGDAL.readRaster( - filePath("/modis/MCD43A4.A2018185.h10v07.006.2018194033728_B01.TIF"), - filePath("/modis/MCD43A4.A2018185.h10v07.006.2018194033728_B01.TIF") + val createInfo = Map( + "path" -> filePath("/modis/MCD43A4.A2018185.h10v07.006.2018194033728_B01.TIF"), + "parentPath" -> filePath("/modis/MCD43A4.A2018185.h10v07.006.2018194033728_B01.TIF") ) + val testRaster = MosaicRasterGDAL.readRaster(createInfo) val testBand = testRaster.getBand(1) testBand.getBand testBand.index shouldBe 1 @@ -36,10 +37,11 @@ class TestRasterBandGDAL extends SharedSparkSessionGDAL { test("Read band metadata and pixel data from a GRIdded Binary file.") { assume(System.getProperty("os.name") == "Linux") - val testRaster = MosaicRasterGDAL.readRaster( - filePath("/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grb"), - filePath("/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grb") + val createInfo = Map( + "path" -> filePath("/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grb"), + "parentPath" -> filePath("/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grb") ) + val testRaster = MosaicRasterGDAL.readRaster(createInfo) val testBand = testRaster.getBand(1) testBand.description shouldBe "1[-] HYBL=\"Hybrid level\"" testBand.dataType shouldBe 7 @@ -55,15 +57,17 @@ class TestRasterBandGDAL extends SharedSparkSessionGDAL { test("Read band metadata and pixel data from a NetCDF file.") { assume(System.getProperty("os.name") == "Linux") - val superRaster = MosaicRasterGDAL.readRaster( - filePath("/binary/netcdf-coral/ct5km_baa-max-7d_v3.1_20220101.nc"), - filePath("/binary/netcdf-coral/ct5km_baa-max-7d_v3.1_20220101.nc") + val createInfo = Map( + "path" -> filePath("/binary/netcdf-coral/ct5km_baa-max-7d_v3.1_20220101.nc"), + "parentPath" -> filePath("/binary/netcdf-coral/ct5km_baa-max-7d_v3.1_20220101.nc") ) + val superRaster = MosaicRasterGDAL.readRaster(createInfo) val subdatasetPath = superRaster.subdatasets("bleaching_alert_area") - val testRaster = MosaicRasterGDAL.readRaster( - subdatasetPath, - subdatasetPath + val sdCreate = Map( + "path" -> subdatasetPath, + "parentPath" -> subdatasetPath ) + val testRaster = MosaicRasterGDAL.readRaster(sdCreate) val testBand = testRaster.getBand(1) testBand.dataType shouldBe 1 diff --git a/src/test/scala/com/databricks/labs/mosaic/core/raster/TestRasterGDAL.scala b/src/test/scala/com/databricks/labs/mosaic/core/raster/TestRasterGDAL.scala index bb53d6b79..8bbe4b1f3 100644 --- a/src/test/scala/com/databricks/labs/mosaic/core/raster/TestRasterGDAL.scala +++ b/src/test/scala/com/databricks/labs/mosaic/core/raster/TestRasterGDAL.scala @@ -34,11 +34,12 @@ class TestRasterGDAL extends SharedSparkSessionGDAL { test("Read raster metadata from GeoTIFF file.") { assume(System.getProperty("os.name") == "Linux") - - val testRaster = MosaicRasterGDAL.readRaster( - filePath("/modis/MCD43A4.A2018185.h10v07.006.2018194033728_B01.TIF"), - filePath("/modis/MCD43A4.A2018185.h10v07.006.2018194033728_B01.TIF") + + val createInfo = Map( + "path" -> filePath("/modis/MCD43A4.A2018185.h10v07.006.2018194033728_B01.TIF"), + "parentPath" -> filePath("/modis/MCD43A4.A2018185.h10v07.006.2018194033728_B01.TIF") ) + val testRaster = MosaicRasterGDAL.readRaster(createInfo) testRaster.xSize shouldBe 2400 testRaster.ySize shouldBe 2400 testRaster.numBands shouldBe 1 @@ -56,10 +57,11 @@ class TestRasterGDAL extends SharedSparkSessionGDAL { test("Read raster metadata from a GRIdded Binary file.") { assume(System.getProperty("os.name") == "Linux") - val testRaster = MosaicRasterGDAL.readRaster( - filePath("/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grb"), - filePath("/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grb") + val createInfo = Map( + "path" -> filePath("/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grb"), + "parentPath" -> filePath("/binary/grib-cams/adaptor.mars.internal-1650626995.380916-11651-14-ca8e7236-16ca-4e11-919d-bdbd5a51da35.grb") ) + val testRaster = MosaicRasterGDAL.readRaster(createInfo) testRaster.xSize shouldBe 14 testRaster.ySize shouldBe 14 testRaster.numBands shouldBe 14 @@ -72,17 +74,19 @@ class TestRasterGDAL extends SharedSparkSessionGDAL { test("Read raster metadata from a NetCDF file.") { assume(System.getProperty("os.name") == "Linux") - - val superRaster = MosaicRasterGDAL.readRaster( - filePath("/binary/netcdf-coral/ct5km_baa-max-7d_v3.1_20220101.nc"), - filePath("/binary/netcdf-coral/ct5km_baa-max-7d_v3.1_20220101.nc") + + val createInfo = Map( + "path" -> filePath("/binary/netcdf-coral/ct5km_baa-max-7d_v3.1_20220101.nc"), + "parentPath" -> filePath("/binary/netcdf-coral/ct5km_baa-max-7d_v3.1_20220101.nc") ) + val superRaster = MosaicRasterGDAL.readRaster(createInfo) val subdatasetPath = superRaster.subdatasets("bleaching_alert_area") - val testRaster = MosaicRasterGDAL.readRaster( - subdatasetPath, - subdatasetPath + val sdCreateInfo = Map( + "path" -> subdatasetPath, + "parentPath" -> subdatasetPath ) + val testRaster = MosaicRasterGDAL.readRaster(sdCreateInfo) testRaster.xSize shouldBe 7200 testRaster.ySize shouldBe 3600 @@ -98,10 +102,11 @@ class TestRasterGDAL extends SharedSparkSessionGDAL { test("Raster pixel and extent sizes are correct.") { assume(System.getProperty("os.name") == "Linux") - val testRaster = MosaicRasterGDAL.readRaster( - filePath("/modis/MCD43A4.A2018185.h10v07.006.2018194033728_B01.TIF"), - filePath("/modis/MCD43A4.A2018185.h10v07.006.2018194033728_B01.TIF") + val createInfo = Map( + "path" -> filePath("/modis/MCD43A4.A2018185.h10v07.006.2018194033728_B01.TIF"), + "parentPath" -> filePath("/modis/MCD43A4.A2018185.h10v07.006.2018194033728_B01.TIF") ) + val testRaster = MosaicRasterGDAL.readRaster(createInfo) testRaster.pixelXSize - 463.312716527 < 0.0000001 shouldBe true testRaster.pixelYSize - -463.312716527 < 0.0000001 shouldBe true @@ -125,13 +130,18 @@ class TestRasterGDAL extends SharedSparkSessionGDAL { MosaicGDAL.setBlockSize(30) - val ds = gdalJNI.GetDriverByName("GTiff").Create("/mosaic_tmp/test.tif", 50, 50, 1, gdalconst.gdalconstConstants.GDT_Float32) + val ds = gdalJNI.GetDriverByName("GTiff").Create("/tmp/mosaic_tmp/test.tif", 50, 50, 1, gdalconst.gdalconstConstants.GDT_Float32) val values = 0 until 50 * 50 ds.GetRasterBand(1).WriteRaster(0, 0, 50, 50, values.toArray) ds.FlushCache() - var result = MosaicRasterGDAL(ds, "", "", "GTiff", -1).filter(5, "avg").flushCache() + val createInfo = Map( + "path" -> "", + "parentPath" -> "", + "driver" -> "GTiff" + ) + var result = MosaicRasterGDAL(ds, createInfo, -1).filter(5, "avg").flushCache() var resultValues = result.getBand(1).values @@ -158,7 +168,7 @@ class TestRasterGDAL extends SharedSparkSessionGDAL { // mode - result = MosaicRasterGDAL(ds, "", "", "GTiff", -1).filter(5, "mode").flushCache() + result = MosaicRasterGDAL(ds, createInfo, -1).filter(5, "mode").flushCache() resultValues = result.getBand(1).values @@ -194,136 +204,136 @@ class TestRasterGDAL extends SharedSparkSessionGDAL { inputMatrix(12)(12), inputMatrix(12)(13) ).groupBy(identity).maxBy(_._2.size)._1.toDouble - + // corner resultMatrix(49)(49) shouldBe Seq( - inputMatrix(47)(47), - inputMatrix(47)(48), - inputMatrix(47)(49), - inputMatrix(48)(47), - inputMatrix(48)(48), - inputMatrix(48)(49), - inputMatrix(49)(47), - inputMatrix(49)(48), - inputMatrix(49)(49) + inputMatrix(47)(47), + inputMatrix(47)(48), + inputMatrix(47)(49), + inputMatrix(48)(47), + inputMatrix(48)(48), + inputMatrix(48)(49), + inputMatrix(49)(47), + inputMatrix(49)(48), + inputMatrix(49)(49) ).groupBy(identity).maxBy(_._2.size)._1.toDouble - + // median - - result = MosaicRasterGDAL(ds, "", "", "GTiff", -1).filter(5, "median").flushCache() - + + result = MosaicRasterGDAL(ds, createInfo, -1).filter(5, "median").flushCache() + resultValues = result.getBand(1).values - + inputMatrix = values.toArray.grouped(50).toArray resultMatrix = resultValues.grouped(50).toArray - + // first block - + resultMatrix(10)(11) shouldBe Seq( - inputMatrix(8)(9), - inputMatrix(8)(10), - inputMatrix(8)(11), - inputMatrix(8)(12), - inputMatrix(8)(13), - inputMatrix(9)(9), - inputMatrix(9)(10), - inputMatrix(9)(11), - inputMatrix(9)(12), - inputMatrix(9)(13), - inputMatrix(10)(9), - inputMatrix(10)(10), - inputMatrix(10)(11), - inputMatrix(10)(12), - inputMatrix(10)(13), - inputMatrix(11)(9), - inputMatrix(11)(10), - inputMatrix(11)(11), - inputMatrix(11)(12), - inputMatrix(11)(13), - inputMatrix(12)(9), - inputMatrix(12)(10), - inputMatrix(12)(11), - inputMatrix(12)(12), - inputMatrix(12)(13) + inputMatrix(8)(9), + inputMatrix(8)(10), + inputMatrix(8)(11), + inputMatrix(8)(12), + inputMatrix(8)(13), + inputMatrix(9)(9), + inputMatrix(9)(10), + inputMatrix(9)(11), + inputMatrix(9)(12), + inputMatrix(9)(13), + inputMatrix(10)(9), + inputMatrix(10)(10), + inputMatrix(10)(11), + inputMatrix(10)(12), + inputMatrix(10)(13), + inputMatrix(11)(9), + inputMatrix(11)(10), + inputMatrix(11)(11), + inputMatrix(11)(12), + inputMatrix(11)(13), + inputMatrix(12)(9), + inputMatrix(12)(10), + inputMatrix(12)(11), + inputMatrix(12)(12), + inputMatrix(12)(13) ).sorted.apply(12).toDouble - + // min filter - - result = MosaicRasterGDAL(ds, "", "", "GTiff", -1).filter(5, "min").flushCache() - + + result = MosaicRasterGDAL(ds, createInfo, -1).filter(5, "min").flushCache() + resultValues = result.getBand(1).values - + inputMatrix = values.toArray.grouped(50).toArray resultMatrix = resultValues.grouped(50).toArray - + // first block - + resultMatrix(10)(11) shouldBe Seq( - inputMatrix(8)(9), - inputMatrix(8)(10), - inputMatrix(8)(11), - inputMatrix(8)(12), - inputMatrix(8)(13), - inputMatrix(9)(9), - inputMatrix(9)(10), - inputMatrix(9)(11), - inputMatrix(9)(12), - inputMatrix(9)(13), - inputMatrix(10)(9), - inputMatrix(10)(10), - inputMatrix(10)(11), - inputMatrix(10)(12), - inputMatrix(10)(13), - inputMatrix(11)(9), - inputMatrix(11)(10), - inputMatrix(11)(11), - inputMatrix(11)(12), - inputMatrix(11)(13), - inputMatrix(12)(9), - inputMatrix(12)(10), - inputMatrix(12)(11), - inputMatrix(12)(12), - inputMatrix(12)(13) + inputMatrix(8)(9), + inputMatrix(8)(10), + inputMatrix(8)(11), + inputMatrix(8)(12), + inputMatrix(8)(13), + inputMatrix(9)(9), + inputMatrix(9)(10), + inputMatrix(9)(11), + inputMatrix(9)(12), + inputMatrix(9)(13), + inputMatrix(10)(9), + inputMatrix(10)(10), + inputMatrix(10)(11), + inputMatrix(10)(12), + inputMatrix(10)(13), + inputMatrix(11)(9), + inputMatrix(11)(10), + inputMatrix(11)(11), + inputMatrix(11)(12), + inputMatrix(11)(13), + inputMatrix(12)(9), + inputMatrix(12)(10), + inputMatrix(12)(11), + inputMatrix(12)(12), + inputMatrix(12)(13) ).min.toDouble - + // max filter - - result = MosaicRasterGDAL(ds, "", "", "GTiff", -1).filter(5, "max").flushCache() - + + result = MosaicRasterGDAL(ds, createInfo, -1).filter(5, "max").flushCache() + resultValues = result.getBand(1).values - + inputMatrix = values.toArray.grouped(50).toArray resultMatrix = resultValues.grouped(50).toArray - + // first block - + resultMatrix(10)(11) shouldBe Seq( - inputMatrix(8)(9), - inputMatrix(8)(10), - inputMatrix(8)(11), - inputMatrix(8)(12), - inputMatrix(8)(13), - inputMatrix(9)(9), - inputMatrix(9)(10), - inputMatrix(9)(11), - inputMatrix(9)(12), - inputMatrix(9)(13), - inputMatrix(10)(9), - inputMatrix(10)(10), - inputMatrix(10)(11), - inputMatrix(10)(12), - inputMatrix(10)(13), - inputMatrix(11)(9), - inputMatrix(11)(10), - inputMatrix(11)(11), - inputMatrix(11)(12), - inputMatrix(11)(13), - inputMatrix(12)(9), - inputMatrix(12)(10), - inputMatrix(12)(11), - inputMatrix(12)(12), - inputMatrix(12)(13) + inputMatrix(8)(9), + inputMatrix(8)(10), + inputMatrix(8)(11), + inputMatrix(8)(12), + inputMatrix(8)(13), + inputMatrix(9)(9), + inputMatrix(9)(10), + inputMatrix(9)(11), + inputMatrix(9)(12), + inputMatrix(9)(13), + inputMatrix(10)(9), + inputMatrix(10)(10), + inputMatrix(10)(11), + inputMatrix(10)(12), + inputMatrix(10)(13), + inputMatrix(11)(9), + inputMatrix(11)(10), + inputMatrix(11)(11), + inputMatrix(11)(12), + inputMatrix(11)(13), + inputMatrix(12)(9), + inputMatrix(12)(10), + inputMatrix(12)(11), + inputMatrix(12)(12), + inputMatrix(12)(13) ).max.toDouble } diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_CombineAvgBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_CombineAvgBehaviors.scala index 611bf8f77..4b7943d8a 100644 --- a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_CombineAvgBehaviors.scala +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_CombineAvgBehaviors.scala @@ -4,7 +4,7 @@ import com.databricks.labs.mosaic.core.geometry.api.GeometryAPI import com.databricks.labs.mosaic.core.index.IndexSystem import com.databricks.labs.mosaic.functions.MosaicContext import org.apache.spark.sql.QueryTest -import org.apache.spark.sql.functions.collect_set +import org.apache.spark.sql.functions.{collect_list, collect_set} import org.scalatest.matchers.should.Matchers._ trait RST_CombineAvgBehaviors extends QueryTest { @@ -28,7 +28,7 @@ trait RST_CombineAvgBehaviors extends QueryTest { .select("path", "tiles") .groupBy("path") .agg( - rst_combineavg(collect_set($"tiles")).as("tiles") + rst_combineavg(collect_list($"tiles")).as("tiles") ) .select("tiles") @@ -38,7 +38,7 @@ trait RST_CombineAvgBehaviors extends QueryTest { //noException should be thrownBy spark.sql(""" - |select rst_combineavg(collect_set(tiles)) as tiles + |select rst_combineavg(collect_list(tiles)) as tiles |from ( | select path, rst_tessellate(tile, 2) as tiles | from source diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_DerivedBandBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_DerivedBandBehaviors.scala index ef6466a88..d883fc5cc 100644 --- a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_DerivedBandBehaviors.scala +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_DerivedBandBehaviors.scala @@ -4,7 +4,7 @@ import com.databricks.labs.mosaic.core.geometry.api.GeometryAPI import com.databricks.labs.mosaic.core.index.IndexSystem import com.databricks.labs.mosaic.functions.MosaicContext import org.apache.spark.sql.QueryTest -import org.apache.spark.sql.functions.{collect_set, lit} +import org.apache.spark.sql.functions.{collect_list, lit} import org.scalatest.matchers.should.Matchers._ trait RST_DerivedBandBehaviors extends QueryTest { @@ -40,7 +40,7 @@ trait RST_DerivedBandBehaviors extends QueryTest { .select("path", "tiles") .groupBy("path") .agg( - rst_derivedband(collect_set($"tiles"), lit(pyFuncCode), lit(funcName)).as("tiles") + rst_derivedband(collect_list($"tiles"), lit(pyFuncCode), lit(funcName)).as("tiles") ) .select("tiles") @@ -52,7 +52,7 @@ trait RST_DerivedBandBehaviors extends QueryTest { noException should be thrownBy spark.sql( """ |select rst_derivedband( - | collect_set(tiles), + | collect_list(tiles), |" |import numpy as np |def multiply(in_ar, out_ar, xoff, yoff, xsize, ysize, raster_xsize,raster_ysize, buf_radius, gt, **kwargs): diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MergeBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MergeBehaviors.scala index 893d6bdf4..330345f20 100644 --- a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MergeBehaviors.scala +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MergeBehaviors.scala @@ -4,7 +4,7 @@ import com.databricks.labs.mosaic.core.geometry.api.GeometryAPI import com.databricks.labs.mosaic.core.index.IndexSystem import com.databricks.labs.mosaic.functions.MosaicContext import org.apache.spark.sql.QueryTest -import org.apache.spark.sql.functions.collect_set +import org.apache.spark.sql.functions.collect_list import org.scalatest.matchers.should.Matchers._ trait RST_MergeBehaviors extends QueryTest { @@ -29,7 +29,7 @@ trait RST_MergeBehaviors extends QueryTest { .select("path", "tile") .groupBy("path") .agg( - collect_set("tile").as("tiles") + collect_list("tile").as("tiles") ) .select( rst_merge($"tiles").as("tile") @@ -41,7 +41,7 @@ trait RST_MergeBehaviors extends QueryTest { spark.sql(""" |select rst_merge(tiles) as tile |from ( - | select collect_set(tile) as tiles + | select collect_list(tile) as tiles | from ( | select path, rst_tessellate(tile, 3) as tile | from source @@ -55,7 +55,7 @@ trait RST_MergeBehaviors extends QueryTest { .select("path", "tile") .groupBy("path") .agg( - collect_set("tile").as("tiles") + collect_list("tile").as("tiles") ) .select( rst_merge($"tiles").as("tile") diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_TessellateBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_TessellateBehaviors.scala index 050e5cb4d..38d3fc778 100644 --- a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_TessellateBehaviors.scala +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_TessellateBehaviors.scala @@ -42,7 +42,7 @@ trait RST_TessellateBehaviors extends QueryTest { val result = gridTiles.select(explode(col("avg")).alias("a")).groupBy("a").count().collect() - result.length should be(462) + result.length should be(441) val netcdf = spark.read .format("gdal") diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_TransformBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_TransformBehaviors.scala new file mode 100644 index 000000000..9ce449b13 --- /dev/null +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_TransformBehaviors.scala @@ -0,0 +1,49 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.geometry.api.GeometryAPI +import com.databricks.labs.mosaic.core.index.IndexSystem +import com.databricks.labs.mosaic.functions.MosaicContext +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.functions._ +import org.scalatest.matchers.should.Matchers._ + +trait RST_TransformBehaviors extends QueryTest { + + // noinspection MapGetGet + def behavior(indexSystem: IndexSystem, geometryAPI: GeometryAPI): Unit = { + spark.sparkContext.setLogLevel("FATAL") + val mc = MosaicContext.build(indexSystem, geometryAPI) + mc.register() + val sc = spark + import mc.functions._ + import sc.implicits._ + + val rastersInMemory = spark.read + .format("gdal") + .option("raster_storage", "in-memory") + .load("src/test/resources/modis") + + val gridTiles = rastersInMemory + .withColumn("tile", rst_transform($"tile", lit(27700))) + .withColumn("bbox", st_aswkt(rst_boundingbox($"tile"))) + .select("bbox", "path", "tile") + .withColumn("avg", rst_avg($"tile")) + + rastersInMemory + .createOrReplaceTempView("source") + + noException should be thrownBy spark.sql(""" + |select rst_transform(tile, 27700) from source + |""".stripMargin) + + noException should be thrownBy rastersInMemory + .withColumn("tile", rst_transform($"tile", lit(27700))) + .select("tile") + + val result = gridTiles.select(explode(col("avg")).alias("a")).groupBy("a").count().collect() + + result.length should be(7) + + } + +} diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_TransformTest.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_TransformTest.scala new file mode 100644 index 000000000..b7c10e548 --- /dev/null +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_TransformTest.scala @@ -0,0 +1,32 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.geometry.api.JTS +import com.databricks.labs.mosaic.core.index.H3IndexSystem +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSessionGDAL + +import scala.util.Try + +class RST_TransformTest extends QueryTest with SharedSparkSessionGDAL with RST_TransformBehaviors { + + private val noCodegen = + withSQLConf( + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false", + SQLConf.CODEGEN_FACTORY_MODE.key -> CodegenObjectFactoryMode.NO_CODEGEN.toString + ) _ + + // Hotfix for SharedSparkSession afterAll cleanup. + override def afterAll(): Unit = Try(super.afterAll()) + + // These tests are not index system nor geometry API specific. + // Only testing one pairing is sufficient. + test("Testing RST_Transform with manual GDAL registration (H3, JTS).") { + noCodegen { + assume(System.getProperty("os.name") == "Linux") + behavior(H3IndexSystem, JTS) + } + } + +} From 591e1fda0cff3d68fbbf39e36b190ba13a3489f9 Mon Sep 17 00:00:00 2001 From: "milos.colic" Date: Mon, 26 Feb 2024 14:35:47 +0000 Subject: [PATCH 25/47] Remove large file test, the test can only be run locally since the file was 2GB. --- .../multiread/RasterAsGridReaderTest.scala | 22 ------------------- 1 file changed, 22 deletions(-) diff --git a/src/test/scala/com/databricks/labs/mosaic/datasource/multiread/RasterAsGridReaderTest.scala b/src/test/scala/com/databricks/labs/mosaic/datasource/multiread/RasterAsGridReaderTest.scala index f174954cd..721201eaa 100644 --- a/src/test/scala/com/databricks/labs/mosaic/datasource/multiread/RasterAsGridReaderTest.scala +++ b/src/test/scala/com/databricks/labs/mosaic/datasource/multiread/RasterAsGridReaderTest.scala @@ -12,28 +12,6 @@ import java.nio.file.{Files, Paths} class RasterAsGridReaderTest extends MosaicSpatialQueryTest with SharedSparkSessionGDAL { - test("Read big tif with Raster As Grid Reader") { - assume(System.getProperty("os.name") == "Linux") - MosaicContext.build(H3IndexSystem, JTS) - - val tif = "/modis/" - val filePath = getClass.getResource(tif).getPath - - val df = MosaicContext.read - .format("raster_to_grid") - .option("retile", "true") - .option("sizeInMB", "128") - .option("resolution", "1") - .load(filePath) - .select("measure") - - df.queryExecution.optimizedPlan - - noException should be thrownBy df.queryExecution.executedPlan - - df.count() - } - test("Read netcdf with Raster As Grid Reader") { assume(System.getProperty("os.name") == "Linux") MosaicContext.build(H3IndexSystem, JTS) From ff07073a228b2900f1b7ed13fbadd8339aa99fa7 Mon Sep 17 00:00:00 2001 From: "milos.colic" Date: Mon, 26 Feb 2024 15:03:21 +0000 Subject: [PATCH 26/47] Fix python build. --- .github/actions/python_build/action.yml | 10 +++++----- .github/actions/scala_build/action.yml | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/.github/actions/python_build/action.yml b/.github/actions/python_build/action.yml index 17e0c53f6..3a8230af5 100644 --- a/.github/actions/python_build/action.yml +++ b/.github/actions/python_build/action.yml @@ -13,15 +13,15 @@ runs: # - install pip libs # note: gdal requires the extra args cd python - pip install build wheel pyspark==${{ matrix.spark }} numpy==${{ matrix.numpy }} - pip install --no-build-isolation --no-cache-dir --force-reinstall gdal==${{ matrix.gdal }} - pip install . + sudo pip install build wheel pyspark==${{ matrix.spark }} numpy==${{ matrix.numpy }} + sudo pip install --no-build-isolation --no-cache-dir --force-reinstall gdal==${{ matrix.gdal }} + sudo pip install . - name: Test and build python package shell: bash run: | cd python - python -m unittest - python -m build + sudo python -m unittest + sudo python -m build - name: Copy python artifacts to GH Actions run shell: bash run: cp python/dist/*.whl staging \ No newline at end of file diff --git a/.github/actions/scala_build/action.yml b/.github/actions/scala_build/action.yml index b33c1b453..e2c19c0a8 100644 --- a/.github/actions/scala_build/action.yml +++ b/.github/actions/scala_build/action.yml @@ -38,7 +38,7 @@ runs: - name: Test and build the scala JAR - skip tests is false if: inputs.skip_tests == 'false' shell: bash - run: sudo mvn -q clean install + run: sudo mvn -q clean install -DskipTests -Dscoverage.skip - name: Build the scala JAR - skip tests is true if: inputs.skip_tests == 'true' shell: bash From 6d3bce6ebeca951bfa22cd45f03effa4e7dbc9ea Mon Sep 17 00:00:00 2001 From: "milos.colic" Date: Mon, 26 Feb 2024 15:24:24 +0000 Subject: [PATCH 27/47] Fix python build. --- python/test/utils/spark_test_case.py | 1 + .../labs/mosaic/functions/MosaicContext.scala | 12 +++++++++++- .../mosaic/functions/MosaicExpressionConfig.scala | 7 +++++++ .../com/databricks/labs/mosaic/gdal/MosaicGDAL.scala | 4 ++-- .../scala/com/databricks/labs/mosaic/package.scala | 1 + .../com/databricks/labs/mosaic/utils/PathUtils.scala | 6 +++--- .../labs/mosaic/models/knn/SpatialKNNBehaviors.scala | 4 ++-- 7 files changed, 27 insertions(+), 8 deletions(-) diff --git a/python/test/utils/spark_test_case.py b/python/test/utils/spark_test_case.py index 6ae23b1b3..c9747fbef 100644 --- a/python/test/utils/spark_test_case.py +++ b/python/test/utils/spark_test_case.py @@ -33,6 +33,7 @@ def setUpClass(cls) -> None: .getOrCreate() ) cls.spark.conf.set("spark.databricks.labs.mosaic.jar.autoattach", "false") + cls.spark.conf.set("spark.databricks.labs.mosaic.raster.tmp.prefix", "/") cls.spark.sparkContext.setLogLevel("FATAL") @classmethod diff --git a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala index 293d27593..9cec6033e 100644 --- a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala +++ b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala @@ -1025,10 +1025,20 @@ class MosaicContext(indexSystem: IndexSystem, geometryAPI: GeometryAPI) extends object MosaicContext extends Logging { - val tmpDir: String = FileUtils.createMosaicTempDir() + var _tmpDir: String = "" val mosaicVersion: String = "0.4.0" private var instance: Option[MosaicContext] = None + + def tmpDir(mosaicConfig: MosaicExpressionConfig): String = { + if (_tmpDir == "" || mosaicConfig == null) { + val prefix = mosaicConfig.getTmpPrefix + _tmpDir = FileUtils.createMosaicTempDir(prefix) + _tmpDir + } else { + _tmpDir + } + } def build(indexSystem: IndexSystem, geometryAPI: GeometryAPI): MosaicContext = { instance = Some(new MosaicContext(indexSystem, geometryAPI)) diff --git a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicExpressionConfig.scala b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicExpressionConfig.scala index d6643f59b..b16b719cc 100644 --- a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicExpressionConfig.scala +++ b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicExpressionConfig.scala @@ -35,6 +35,8 @@ case class MosaicExpressionConfig(configs: Map[String, String]) { def getCellIdType: DataType = IndexSystemFactory.getIndexSystem(getIndexSystem).cellIdType def getRasterBlockSize: Int = configs.getOrElse(MOSAIC_RASTER_BLOCKSIZE, MOSAIC_RASTER_BLOCKSIZE_DEFAULT).toInt + + def getTmpPrefix: String = configs.getOrElse(MOSAIC_RASTER_TMP_PREFIX, "/tmp") def setGDALConf(conf: RuntimeConfig): MosaicExpressionConfig = { val toAdd = conf.getAll.filter(_._1.startsWith(MOSAIC_GDAL_PREFIX)) @@ -56,6 +58,10 @@ case class MosaicExpressionConfig(configs: Map[String, String]) { def setRasterCheckpoint(checkpoint: String): MosaicExpressionConfig = { MosaicExpressionConfig(configs + (MOSAIC_RASTER_CHECKPOINT -> checkpoint)) } + + def setTmpPrefix(prefix: String): MosaicExpressionConfig = { + MosaicExpressionConfig(configs + (MOSAIC_RASTER_TMP_PREFIX -> prefix)) + } def setConfig(key: String, value: String): MosaicExpressionConfig = { MosaicExpressionConfig(configs + (key -> value)) @@ -75,6 +81,7 @@ object MosaicExpressionConfig { .setGeometryAPI(spark.conf.get(MOSAIC_GEOMETRY_API, JTS.name)) .setIndexSystem(spark.conf.get(MOSAIC_INDEX_SYSTEM, H3.name)) .setRasterCheckpoint(spark.conf.get(MOSAIC_RASTER_CHECKPOINT, MOSAIC_RASTER_CHECKPOINT_DEFAULT)) + .setTmpPrefix(spark.conf.get(MOSAIC_RASTER_TMP_PREFIX, "/tmp")) .setGDALConf(spark.conf) } diff --git a/src/main/scala/com/databricks/labs/mosaic/gdal/MosaicGDAL.scala b/src/main/scala/com/databricks/labs/mosaic/gdal/MosaicGDAL.scala index b9e972d6f..6cc928edd 100644 --- a/src/main/scala/com/databricks/labs/mosaic/gdal/MosaicGDAL.scala +++ b/src/main/scala/com/databricks/labs/mosaic/gdal/MosaicGDAL.scala @@ -46,8 +46,8 @@ object MosaicGDAL extends Logging { /** Configures the GDAL environment. */ def configureGDAL(mosaicConfig: MosaicExpressionConfig): Unit = { - val CPL_TMPDIR = MosaicContext.tmpDir - val GDAL_PAM_PROXY_DIR = MosaicContext.tmpDir + val CPL_TMPDIR = MosaicContext.tmpDir(mosaicConfig) + val GDAL_PAM_PROXY_DIR = MosaicContext.tmpDir(mosaicConfig) gdal.SetConfigOption("GDAL_VRT_ENABLE_PYTHON", "YES") gdal.SetConfigOption("GDAL_DISABLE_READDIR_ON_OPEN", "TRUE") gdal.SetConfigOption("CPL_TMPDIR", CPL_TMPDIR) diff --git a/src/main/scala/com/databricks/labs/mosaic/package.scala b/src/main/scala/com/databricks/labs/mosaic/package.scala index eea63cd79..86bdbcec7 100644 --- a/src/main/scala/com/databricks/labs/mosaic/package.scala +++ b/src/main/scala/com/databricks/labs/mosaic/package.scala @@ -22,6 +22,7 @@ package object mosaic { val MOSAIC_GDAL_NATIVE = "spark.databricks.labs.mosaic.gdal.native" val MOSAIC_RASTER_CHECKPOINT = "spark.databricks.labs.mosaic.raster.checkpoint" val MOSAIC_RASTER_CHECKPOINT_DEFAULT = "/dbfs/tmp/mosaic/raster/checkpoint" + val MOSAIC_RASTER_TMP_PREFIX = "spark.databricks.labs.mosaic.raster.tmp.prefix" val MOSAIC_RASTER_BLOCKSIZE = "spark.databricks.labs.mosaic.raster.blocksize" val MOSAIC_RASTER_BLOCKSIZE_DEFAULT = "128" diff --git a/src/main/scala/com/databricks/labs/mosaic/utils/PathUtils.scala b/src/main/scala/com/databricks/labs/mosaic/utils/PathUtils.scala index 469bb0f44..aacd897f8 100644 --- a/src/main/scala/com/databricks/labs/mosaic/utils/PathUtils.scala +++ b/src/main/scala/com/databricks/labs/mosaic/utils/PathUtils.scala @@ -49,7 +49,7 @@ object PathUtils { } def createTmpFilePath(extension: String): String = { - val tmpDir = MosaicContext.tmpDir + val tmpDir = MosaicContext.tmpDir(null) val uuid = java.util.UUID.randomUUID.toString val outPath = s"$tmpDir/raster_${uuid.replace("-", "_")}.$extension" Files.createDirectories(Paths.get(outPath).getParent) @@ -80,9 +80,9 @@ object PathUtils { val fullFileName = copyFromPath.split("/").last val stemRegex = getStemRegex(inPath) - wildcardCopy(inPathDir, MosaicContext.tmpDir, stemRegex.toString) + wildcardCopy(inPathDir, MosaicContext.tmpDir(null), stemRegex.toString) - s"${MosaicContext.tmpDir}/$fullFileName" + s"${MosaicContext.tmpDir(null)}/$fullFileName" } def wildcardCopy(inDirPath: String, outDirPath: String, pattern: String): Unit = { diff --git a/src/test/scala/com/databricks/labs/mosaic/models/knn/SpatialKNNBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/models/knn/SpatialKNNBehaviors.scala index e9508c3c6..be9b5c402 100644 --- a/src/test/scala/com/databricks/labs/mosaic/models/knn/SpatialKNNBehaviors.scala +++ b/src/test/scala/com/databricks/labs/mosaic/models/knn/SpatialKNNBehaviors.scala @@ -29,7 +29,7 @@ trait SpatialKNNBehaviors { this: AnyFlatSpec => val boroughs: DataFrame = getBoroughs(mc) - val tempLocation = MosaicContext.tmpDir + val tempLocation = MosaicContext.tmpDir(null) spark.sparkContext.setCheckpointDir(tempLocation) spark.sparkContext.setLogLevel("ERROR") @@ -94,7 +94,7 @@ trait SpatialKNNBehaviors { this: AnyFlatSpec => val boroughs: DataFrame = getBoroughs(mc) - val tempLocation = MosaicContext.tmpDir + val tempLocation = MosaicContext.tmpDir(null) spark.sparkContext.setCheckpointDir(tempLocation) spark.sparkContext.setLogLevel("ERROR") From 398386b997bd742e1045d94aa9c0430929fe2ea7 Mon Sep 17 00:00:00 2001 From: "milos.colic" Date: Mon, 26 Feb 2024 15:31:33 +0000 Subject: [PATCH 28/47] Fix python build. --- .github/actions/python_build/action.yml | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/.github/actions/python_build/action.yml b/.github/actions/python_build/action.yml index 3a8230af5..17e0c53f6 100644 --- a/.github/actions/python_build/action.yml +++ b/.github/actions/python_build/action.yml @@ -13,15 +13,15 @@ runs: # - install pip libs # note: gdal requires the extra args cd python - sudo pip install build wheel pyspark==${{ matrix.spark }} numpy==${{ matrix.numpy }} - sudo pip install --no-build-isolation --no-cache-dir --force-reinstall gdal==${{ matrix.gdal }} - sudo pip install . + pip install build wheel pyspark==${{ matrix.spark }} numpy==${{ matrix.numpy }} + pip install --no-build-isolation --no-cache-dir --force-reinstall gdal==${{ matrix.gdal }} + pip install . - name: Test and build python package shell: bash run: | cd python - sudo python -m unittest - sudo python -m build + python -m unittest + python -m build - name: Copy python artifacts to GH Actions run shell: bash run: cp python/dist/*.whl staging \ No newline at end of file From 3609daab194c44a74e63b65b89f251a91dcdfdbe Mon Sep 17 00:00:00 2001 From: "milos.colic" Date: Mon, 26 Feb 2024 15:48:08 +0000 Subject: [PATCH 29/47] Fix python build. --- python/test/utils/spark_test_case.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/python/test/utils/spark_test_case.py b/python/test/utils/spark_test_case.py index c9747fbef..223f5d021 100644 --- a/python/test/utils/spark_test_case.py +++ b/python/test/utils/spark_test_case.py @@ -17,6 +17,8 @@ def setUpClass(cls) -> None: cls.library_location = f"{mosaic.__path__[0]}/lib/mosaic-{version('databricks-mosaic')}-jar-with-dependencies.jar" if not os.path.exists(cls.library_location): cls.library_location = f"{mosaic.__path__[0]}/lib/mosaic-{version('databricks-mosaic')}-SNAPSHOT-jar-with-dependencies.jar" + if not os.path.exists("/mosaic_test/"): + os.makedirs("/mosaic_test/") cls.spark = ( SparkSession.builder.master("local[*]") @@ -33,7 +35,7 @@ def setUpClass(cls) -> None: .getOrCreate() ) cls.spark.conf.set("spark.databricks.labs.mosaic.jar.autoattach", "false") - cls.spark.conf.set("spark.databricks.labs.mosaic.raster.tmp.prefix", "/") + cls.spark.conf.set("spark.databricks.labs.mosaic.raster.tmp.prefix", "/mosaic_test/") cls.spark.sparkContext.setLogLevel("FATAL") @classmethod From db50866b706e8dc7fa8c44ee821e8cf971e8771b Mon Sep 17 00:00:00 2001 From: "milos.colic" Date: Mon, 26 Feb 2024 16:00:30 +0000 Subject: [PATCH 30/47] Fix python build. --- python/test/utils/spark_test_case.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/python/test/utils/spark_test_case.py b/python/test/utils/spark_test_case.py index 223f5d021..42917f278 100644 --- a/python/test/utils/spark_test_case.py +++ b/python/test/utils/spark_test_case.py @@ -17,8 +17,11 @@ def setUpClass(cls) -> None: cls.library_location = f"{mosaic.__path__[0]}/lib/mosaic-{version('databricks-mosaic')}-jar-with-dependencies.jar" if not os.path.exists(cls.library_location): cls.library_location = f"{mosaic.__path__[0]}/lib/mosaic-{version('databricks-mosaic')}-SNAPSHOT-jar-with-dependencies.jar" - if not os.path.exists("/mosaic_test/"): - os.makedirs("/mosaic_test/") + + pwd_dir = os.getcwd() + tmp_dir = f"{pwd_dir}/mosaic_test/" + if not os.path.exists(tmp_dir): + os.makedirs(tmp_dir) cls.spark = ( SparkSession.builder.master("local[*]") @@ -35,7 +38,7 @@ def setUpClass(cls) -> None: .getOrCreate() ) cls.spark.conf.set("spark.databricks.labs.mosaic.jar.autoattach", "false") - cls.spark.conf.set("spark.databricks.labs.mosaic.raster.tmp.prefix", "/mosaic_test/") + cls.spark.conf.set("spark.databricks.labs.mosaic.raster.tmp.prefix", tmp_dir) cls.spark.sparkContext.setLogLevel("FATAL") @classmethod From f78629a07dd6ce7966c58efce0d93791860e9d1d Mon Sep 17 00:00:00 2001 From: "milos.colic" Date: Mon, 26 Feb 2024 16:18:20 +0000 Subject: [PATCH 31/47] Fix python build. --- .../com/databricks/labs/mosaic/functions/MosaicContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala index 9cec6033e..281dd0eac 100644 --- a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala +++ b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala @@ -1031,7 +1031,7 @@ object MosaicContext extends Logging { private var instance: Option[MosaicContext] = None def tmpDir(mosaicConfig: MosaicExpressionConfig): String = { - if (_tmpDir == "" || mosaicConfig == null) { + if (_tmpDir == "" || mosaicConfig != null) { val prefix = mosaicConfig.getTmpPrefix _tmpDir = FileUtils.createMosaicTempDir(prefix) _tmpDir From f824fba6849af49e3daafbcf7fe14ad76bd54805 Mon Sep 17 00:00:00 2001 From: Serge Smertin <259697+nfx@users.noreply.github.com> Date: Wed, 28 Feb 2024 12:26:06 +0100 Subject: [PATCH 32/47] Added tokei.rs badge [![lines of code](https://tokei.rs/b1/github/databrickslabs/mosaic)]([https://codecov.io/github/databrickslabs/mosaic](https://github.com/databrickslabs/mosaic)) --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index f2d74baf1..2a07a6294 100644 --- a/README.md +++ b/README.md @@ -10,6 +10,7 @@ An extension to the [Apache Spark](https://spark.apache.org/) framework that all [![docs](https://github.com/databrickslabs/mosaic/actions/workflows/docs.yml/badge.svg)](https://github.com/databrickslabs/mosaic/actions/workflows/docs.yml) [![Language grade: Python](https://img.shields.io/lgtm/grade/python/g/databrickslabs/mosaic.svg?logo=lgtm&logoWidth=18)](https://lgtm.com/projects/g/databrickslabs/mosaic/context:python) [![Code style: black](https://img.shields.io/badge/code%20style-black-000000.svg)](https://github.com/psf/black) +[![lines of code](https://tokei.rs/b1/github/databrickslabs/mosaic)]([https://codecov.io/github/databrickslabs/mosaic](https://github.com/databrickslabs/mosaic)) ## Why Mosaic? From 67b73850356dbd334349224a5507fe386032b7d6 Mon Sep 17 00:00:00 2001 From: "milos.colic" Date: Fri, 1 Mar 2024 13:32:38 +0000 Subject: [PATCH 33/47] Fix python build. --- python/test/utils/spark_test_case.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/python/test/utils/spark_test_case.py b/python/test/utils/spark_test_case.py index 42917f278..640713ba7 100644 --- a/python/test/utils/spark_test_case.py +++ b/python/test/utils/spark_test_case.py @@ -20,8 +20,11 @@ def setUpClass(cls) -> None: pwd_dir = os.getcwd() tmp_dir = f"{pwd_dir}/mosaic_test/" + check_dir = f"{pwd_dir}/checkpoint" if not os.path.exists(tmp_dir): os.makedirs(tmp_dir) + if not os.path.exists(check_dir): + os.makedirs(check_dir) cls.spark = ( SparkSession.builder.master("local[*]") @@ -39,6 +42,7 @@ def setUpClass(cls) -> None: ) cls.spark.conf.set("spark.databricks.labs.mosaic.jar.autoattach", "false") cls.spark.conf.set("spark.databricks.labs.mosaic.raster.tmp.prefix", tmp_dir) + cls.spark.conf.set("spark.databricks.labs.mosaic.raster.checkpoint", check_dir) cls.spark.sparkContext.setLogLevel("FATAL") @classmethod From 11d15be2a06d8260957cdd8d7964206c6e33542d Mon Sep 17 00:00:00 2001 From: "milos.colic" Date: Fri, 1 Mar 2024 13:35:00 +0000 Subject: [PATCH 34/47] Fix python build. --- python/test/test_raster_functions.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/test/test_raster_functions.py b/python/test/test_raster_functions.py index be5fd4656..834c02a5c 100644 --- a/python/test/test_raster_functions.py +++ b/python/test/test_raster_functions.py @@ -19,7 +19,7 @@ def test_read_raster(self): result.metadata["LONGNAME"], "MODIS/Terra+Aqua BRDF/Albedo Nadir BRDF-Adjusted Ref Daily L3 Global - 500m", ) - self.assertEqual(result.tile["driver"], "GTiff") + self.assertEqual(result.tile["metadata"]["driver"], "GTiff") def test_raster_scalar_functions(self): result = ( @@ -115,7 +115,7 @@ def test_raster_flatmap_functions(self): ) tessellate_result.write.format("noop").mode("overwrite").save() - self.assertEqual(tessellate_result.count(), 66) + self.assertEqual(tessellate_result.count(), 63) overlap_result = ( self.generate_singleband_raster_df() From 5b4233471692c3c4f29232af6d970e654d0a297a Mon Sep 17 00:00:00 2001 From: "milos.colic" Date: Fri, 1 Mar 2024 13:48:33 +0000 Subject: [PATCH 35/47] Fix python build. --- .../mosaic/core/raster/operator/clip/RasterClipByVector.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/clip/RasterClipByVector.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/clip/RasterClipByVector.scala index 56c29563f..ddae2fed2 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/clip/RasterClipByVector.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/operator/clip/RasterClipByVector.scala @@ -49,7 +49,7 @@ object RasterClipByVector { val result = GDALWarp.executeWarp( resultFileName, Seq(raster), - command = s"gdalwarp -wo CUTLINE_ALL_TOUCHED=TRUE -wo SOURCE_EXTRA=3 -cutline $shapeFileName -crop_to_cutline" + command = s"gdalwarp -wo CUTLINE_ALL_TOUCHED=TRUE -cutline $shapeFileName -crop_to_cutline" ) VectorClipper.cleanUpClipper(shapeFileName) From 86a3901fac68f464f061f4ea375af4618e1b6d3a Mon Sep 17 00:00:00 2001 From: "milos.colic" Date: Fri, 1 Mar 2024 16:46:48 +0000 Subject: [PATCH 36/47] Fix python build. --- .github/actions/scala_build/action.yml | 2 +- python/test/test_raster_functions.py | 3 +- python/test/test_vector_functions.py | 4 +-- .../core/raster/gdal/MosaicRasterGDAL.scala | 33 ++++++++++--------- 4 files changed, 22 insertions(+), 20 deletions(-) diff --git a/.github/actions/scala_build/action.yml b/.github/actions/scala_build/action.yml index e2c19c0a8..b9e366658 100644 --- a/.github/actions/scala_build/action.yml +++ b/.github/actions/scala_build/action.yml @@ -27,7 +27,7 @@ runs: sudo apt-get update -y # - install natives sudo apt-get install -y unixodbc libcurl3-gnutls libsnappy-dev libopenjp2-7 - sudo apt-get install -y gdal-bin libgdal-dev python3-numpy python3-gdal + sudo apt-get install -y gdal-bin libgdal-dev python3-numpy python3-gdal zip unzip # - install pip libs pip install --upgrade pip pip install gdal==${{ matrix.gdal }} diff --git a/python/test/test_raster_functions.py b/python/test/test_raster_functions.py index 834c02a5c..cda55143d 100644 --- a/python/test/test_raster_functions.py +++ b/python/test/test_raster_functions.py @@ -187,11 +187,12 @@ def test_netcdf_load_tessellate_clip_merge(self): df = ( self.spark.read.format("gdal") - .option("raster.read.strategy", "retile_on_read") + .option("raster.read.strategy", "in_memory") .load( "test/data/prAdjust_day_HadGEM2-CC_SMHI-DBSrev930-GFD-1981-2010-postproc_rcp45_r1i1p1_20201201-20201231.nc" ) .select(api.rst_separatebands("tile").alias("tile")) + .repartition(self.spark.sparkContext.defaultParallelism) .withColumn( "timestep", element_at( diff --git a/python/test/test_vector_functions.py b/python/test/test_vector_functions.py index afca19778..c2bb2be74 100644 --- a/python/test/test_vector_functions.py +++ b/python/test/test_vector_functions.py @@ -37,9 +37,7 @@ def test_st_z(self): .select(col("id").cast("double")) .withColumn( "points", - api.st_geomfromwkt( - concat(lit("POINT (9 9 "), "id", lit(")")) - ), + api.st_geomfromwkt(concat(lit("POINT (9 9 "), "id", lit(")"))), ) .withColumn("z", api.st_z("points")) .collect() diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala index 10f04416d..2749d7cfa 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala @@ -76,9 +76,10 @@ case class MosaicRasterGDAL( * @return * The raster's driver short name. */ - def getDriversShortName: String = driverShortName.getOrElse( - Try(raster.GetDriver().getShortName).getOrElse("NONE") - ) + def getDriversShortName: String = + driverShortName.getOrElse( + Try(raster.GetDriver().getShortName).getOrElse("NONE") + ) /** * @return @@ -469,7 +470,8 @@ case class MosaicRasterGDAL( if (Files.isDirectory(Paths.get(tmpPath))) { val parentDir = Paths.get(tmpPath).getParent.toString val fileName = Paths.get(tmpPath).getFileName.toString - SysUtils.runScript(Array("/bin/sh", "-c", s"cd $parentDir && zip -r0 $fileName.zip $fileName")) + val prompt = SysUtils.runScript(Array("/bin/sh", "-c", s"cd $parentDir && zip -r0 $fileName.zip $fileName")) + if (prompt._3.nonEmpty) throw new Exception(s"Error zipping file: ${prompt._3}. Please verify that zip is installed. Run 'apt install zip'.") s"$tmpPath.zip" } else { tmpPath @@ -566,12 +568,11 @@ case class MosaicRasterGDAL( val gdalError = gdal.GetLastErrorMsg() val error = path match { case Some(_) => "" - case None => - s""" - |Subdataset $subsetName not found! - |Available subdatasets: - | ${subdatasets.keys.filterNot(_.startsWith("SUBDATASET_")).mkString(", ")} - | """.stripMargin + case None => s""" + |Subdataset $subsetName not found! + |Available subdatasets: + | ${subdatasets.keys.filterNot(_.startsWith("SUBDATASET_")).mkString(", ")} + | """.stripMargin } val sanitized = PathUtils.getCleanPath(path.getOrElse(PathUtils.NO_PATH_STRING)) val subdatasetPath = PathUtils.getSubdatasetPath(sanitized) @@ -584,11 +585,13 @@ case class MosaicRasterGDAL( "path" -> path.getOrElse(PathUtils.NO_PATH_STRING), "parentPath" -> parentPath, "driver" -> getDriversShortName, - "last_error" -> - s""" - |GDAL Error: $gdalError - |$error - |""".stripMargin + "last_error" -> { + if (gdalError.nonEmpty || error.nonEmpty) s""" + |GDAL Error: $gdalError + |$error + |""".stripMargin + else "" + } ) MosaicRasterGDAL(ds, createInfo, -1) } From 61ba42b30fc2e0787640f7839df1d4a73f25b697 Mon Sep 17 00:00:00 2001 From: "milos.colic" Date: Fri, 1 Mar 2024 16:48:35 +0000 Subject: [PATCH 37/47] Fix python build. --- .github/actions/scala_build/action.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/actions/scala_build/action.yml b/.github/actions/scala_build/action.yml index b9e366658..8a0d20359 100644 --- a/.github/actions/scala_build/action.yml +++ b/.github/actions/scala_build/action.yml @@ -38,7 +38,7 @@ runs: - name: Test and build the scala JAR - skip tests is false if: inputs.skip_tests == 'false' shell: bash - run: sudo mvn -q clean install -DskipTests -Dscoverage.skip + run: sudo mvn -q clean install - name: Build the scala JAR - skip tests is true if: inputs.skip_tests == 'true' shell: bash From 778e3be1a2587b592098f1354ca40d3f6ed04414 Mon Sep 17 00:00:00 2001 From: "milos.colic" Date: Fri, 1 Mar 2024 17:07:35 +0000 Subject: [PATCH 38/47] Fix python build. --- .../databricks/labs/mosaic/functions/MosaicContext.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala index 281dd0eac..5fd5f6e6b 100644 --- a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala +++ b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala @@ -682,15 +682,17 @@ class MosaicContext(indexSystem: IndexSystem, geometryAPI: GeometryAPI) extends def rst_maketiles(input: Column, driver: String, size: Int, withCheckpoint: Boolean): Column = ColumnAdapter(RST_MakeTiles(input.expr, lit(driver).expr, lit(size).expr, lit(withCheckpoint).expr, expressionConfig)) def rst_maketiles(input: Column): Column = - ColumnAdapter(RST_MakeTiles(input.expr, lit(MOSAIC_NO_DRIVER).expr, lit(-1).expr, lit(false).expr, expressionConfig)) + ColumnAdapter(RST_MakeTiles(input.expr, lit("no_driver").expr, lit(-1).expr, lit(false).expr, expressionConfig)) def rst_maketiles(input: Column, size: Int): Column = - ColumnAdapter(RST_MakeTiles(input.expr, lit(MOSAIC_NO_DRIVER).expr, lit(size).expr, lit(false).expr, expressionConfig)) + ColumnAdapter(RST_MakeTiles(input.expr, lit("no_driver").expr, lit(size).expr, lit(false).expr, expressionConfig)) def rst_maketiles(input: Column, driver: String): Column = ColumnAdapter(RST_MakeTiles(input.expr, lit(driver).expr, lit(-1).expr, lit(false).expr, expressionConfig)) + def rst_maketiles(input: Column, driver: String, size: Int): Column = + ColumnAdapter(RST_MakeTiles(input.expr, lit(driver).expr, lit(size).expr, lit(false).expr, expressionConfig)) def rst_maketiles(input: Column, driver: String, withCheckpoint: Boolean): Column = ColumnAdapter(RST_MakeTiles(input.expr, lit(driver).expr, lit(-1).expr, lit(withCheckpoint).expr, expressionConfig)) def rst_maketiles(input: Column, size: Int, withCheckpoint: Boolean): Column = - ColumnAdapter(RST_MakeTiles(input.expr, lit(MOSAIC_NO_DRIVER).expr, lit(size).expr, lit(withCheckpoint).expr, expressionConfig)) + ColumnAdapter(RST_MakeTiles(input.expr, lit("no_driver").expr, lit(size).expr, lit(withCheckpoint).expr, expressionConfig)) def rst_max(raster: Column): Column = ColumnAdapter(RST_Max(raster.expr, expressionConfig)) def rst_min(raster: Column): Column = ColumnAdapter(RST_Min(raster.expr, expressionConfig)) def rst_median(raster: Column): Column = ColumnAdapter(RST_Median(raster.expr, expressionConfig)) From d92b93d8ab9889f8f11903320a6f1f9a9bec5654 Mon Sep 17 00:00:00 2001 From: "milos.colic" Date: Fri, 1 Mar 2024 17:20:57 +0000 Subject: [PATCH 39/47] Fix R build. --- .../labs/mosaic/functions/MosaicContext.scala | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala index 5fd5f6e6b..40be5cb24 100644 --- a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala +++ b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala @@ -681,18 +681,10 @@ class MosaicContext(indexSystem: IndexSystem, geometryAPI: GeometryAPI) extends ColumnAdapter(RST_MakeTiles(input.expr, driver.expr, size.expr, withCheckpoint.expr, expressionConfig)) def rst_maketiles(input: Column, driver: String, size: Int, withCheckpoint: Boolean): Column = ColumnAdapter(RST_MakeTiles(input.expr, lit(driver).expr, lit(size).expr, lit(withCheckpoint).expr, expressionConfig)) - def rst_maketiles(input: Column): Column = - ColumnAdapter(RST_MakeTiles(input.expr, lit("no_driver").expr, lit(-1).expr, lit(false).expr, expressionConfig)) - def rst_maketiles(input: Column, size: Int): Column = - ColumnAdapter(RST_MakeTiles(input.expr, lit("no_driver").expr, lit(size).expr, lit(false).expr, expressionConfig)) - def rst_maketiles(input: Column, driver: String): Column = - ColumnAdapter(RST_MakeTiles(input.expr, lit(driver).expr, lit(-1).expr, lit(false).expr, expressionConfig)) def rst_maketiles(input: Column, driver: String, size: Int): Column = ColumnAdapter(RST_MakeTiles(input.expr, lit(driver).expr, lit(size).expr, lit(false).expr, expressionConfig)) - def rst_maketiles(input: Column, driver: String, withCheckpoint: Boolean): Column = - ColumnAdapter(RST_MakeTiles(input.expr, lit(driver).expr, lit(-1).expr, lit(withCheckpoint).expr, expressionConfig)) - def rst_maketiles(input: Column, size: Int, withCheckpoint: Boolean): Column = - ColumnAdapter(RST_MakeTiles(input.expr, lit("no_driver").expr, lit(size).expr, lit(withCheckpoint).expr, expressionConfig)) + def rst_maketiles(input: Column): Column = + ColumnAdapter(RST_MakeTiles(input.expr, lit("no_driver").expr, lit(-1).expr, lit(false).expr, expressionConfig)) def rst_max(raster: Column): Column = ColumnAdapter(RST_Max(raster.expr, expressionConfig)) def rst_min(raster: Column): Column = ColumnAdapter(RST_Min(raster.expr, expressionConfig)) def rst_median(raster: Column): Column = ColumnAdapter(RST_Median(raster.expr, expressionConfig)) From 7d803bed65316242b5a71e74e04731ea917206d0 Mon Sep 17 00:00:00 2001 From: Stuart Lynn Date: Fri, 1 Mar 2024 17:34:29 +0000 Subject: [PATCH 40/47] fix R tests --- R/sparkR-mosaic/sparkrMosaic/DESCRIPTION | 4 ++-- .../sparkrMosaic/tests/testthat/testRasterFunctions.R | 6 +++--- R/sparklyr-mosaic/sparklyrMosaic/DESCRIPTION | 4 ++-- .../sparklyrMosaic/tests/testthat/testRasterFunctions.R | 6 +++--- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/R/sparkR-mosaic/sparkrMosaic/DESCRIPTION b/R/sparkR-mosaic/sparkrMosaic/DESCRIPTION index f689fe17a..876a46cf7 100644 --- a/R/sparkR-mosaic/sparkrMosaic/DESCRIPTION +++ b/R/sparkR-mosaic/sparkrMosaic/DESCRIPTION @@ -8,7 +8,7 @@ Description: This package extends SparkR to bring the Databricks Mosaic for geos License: Databricks Encoding: UTF-8 Roxygen: list(markdown = TRUE) -RoxygenNote: 7.2.3 +RoxygenNote: 7.3.1 Collate: 'enableGDAL.R' 'enableMosaic.R' @@ -20,4 +20,4 @@ Imports: Suggests: testthat (>= 3.0.0), readr (>= 2.1.5) -Config/testthat/edition: 3 \ No newline at end of file +Config/testthat/edition: 3 diff --git a/R/sparkR-mosaic/sparkrMosaic/tests/testthat/testRasterFunctions.R b/R/sparkR-mosaic/sparkrMosaic/tests/testthat/testRasterFunctions.R index 36296e9d8..6e23454dc 100644 --- a/R/sparkR-mosaic/sparkrMosaic/tests/testthat/testRasterFunctions.R +++ b/R/sparkR-mosaic/sparkrMosaic/tests/testthat/testRasterFunctions.R @@ -15,7 +15,7 @@ test_that("mosaic can read single-band GeoTiff", { expect_equal(row$srid, 0) expect_equal(row$bandCount, 1) expect_equal(row$metadata[[1]]$LONGNAME, "MODIS/Terra+Aqua BRDF/Albedo Nadir BRDF-Adjusted Ref Daily L3 Global - 500m") - expect_equal(row$tile[[1]]$driver, "GTiff") + expect_equal(row$tile[[1]]$metadata$driver, "GTiff") }) @@ -61,7 +61,7 @@ test_that("raster flatmap functions behave as intended", { tessellate_sdf <- withColumn(tessellate_sdf, "rst_tessellate", rst_tessellate(column("tile"), lit(3L))) expect_no_error(write.df(tessellate_sdf, source = "noop", mode = "overwrite")) - expect_equal(nrow(tessellate_sdf), 66) + expect_equal(nrow(tessellate_sdf), 63) overlap_sdf <- generate_singleband_raster_df() overlap_sdf <- withColumn(overlap_sdf, "rst_to_overlapping_tiles", rst_to_overlapping_tiles(column("tile"), lit(200L), lit(200L), lit(10L))) @@ -117,7 +117,7 @@ test_that("the tessellate-join-clip-merge flow works on NetCDF files", { raster_sdf <- read.df( path = "sparkrMosaic/tests/testthat/data/prAdjust_day_HadGEM2-CC_SMHI-DBSrev930-GFD-1981-2010-postproc_rcp45_r1i1p1_20201201-20201231.nc", source = "gdal", - raster.read.strategy = "retile_on_read" + raster.read.strategy = "in_memory" ) raster_sdf <- withColumn(raster_sdf, "tile", rst_separatebands(column("tile"))) diff --git a/R/sparklyr-mosaic/sparklyrMosaic/DESCRIPTION b/R/sparklyr-mosaic/sparklyrMosaic/DESCRIPTION index c9d2048fb..315e6bf3c 100644 --- a/R/sparklyr-mosaic/sparklyrMosaic/DESCRIPTION +++ b/R/sparklyr-mosaic/sparklyrMosaic/DESCRIPTION @@ -8,7 +8,7 @@ Description: This package extends sparklyr to bring the Databricks Mosaic for ge License: Databricks Encoding: UTF-8 Roxygen: list(markdown = TRUE) -RoxygenNote: 7.2.3 +RoxygenNote: 7.3.1 Collate: 'enableGDAL.R' 'enableMosaic.R' @@ -20,4 +20,4 @@ Suggests: testthat (>= 3.0.0), sparklyr.nested (>= 0.0.4), readr (>= 2.1.5) -Config/testthat/edition: 3 \ No newline at end of file +Config/testthat/edition: 3 diff --git a/R/sparklyr-mosaic/sparklyrMosaic/tests/testthat/testRasterFunctions.R b/R/sparklyr-mosaic/sparklyrMosaic/tests/testthat/testRasterFunctions.R index 3bb021c64..3cf016fa7 100644 --- a/R/sparklyr-mosaic/sparklyrMosaic/tests/testthat/testRasterFunctions.R +++ b/R/sparklyr-mosaic/sparklyrMosaic/tests/testthat/testRasterFunctions.R @@ -18,7 +18,7 @@ test_that("mosaic can read single-band GeoTiff", { expect_equal(row$srid, 0) expect_equal(row$bandCount, 1) expect_equal(row$metadata[[1]]$LONGNAME, "MODIS/Terra+Aqua BRDF/Albedo Nadir BRDF-Adjusted Ref Daily L3 Global - 500m") - expect_equal(row$tile[[1]]$driver, "GTiff") + expect_equal(row$tile[[1]]$metadata$driver, "GTiff") }) @@ -90,7 +90,7 @@ test_that("raster flatmap functions behave as intended", { mutate(rst_tessellate = rst_tessellate(tile, 3L)) expect_no_error(spark_write_source(tessellate_sdf, "noop", mode = "overwrite")) - expect_equal(sdf_nrow(tessellate_sdf), 66) + expect_equal(sdf_nrow(tessellate_sdf), 63) overlap_sdf <- generate_singleband_raster_df() %>% mutate(rst_to_overlapping_tiles = rst_to_overlapping_tiles(tile, 200L, 200L, 10L)) @@ -157,7 +157,7 @@ test_that("the tessellate-join-clip-merge flow works on NetCDF files", { name = "raster_raw", source = "gdal", path = "data/prAdjust_day_HadGEM2-CC_SMHI-DBSrev930-GFD-1981-2010-postproc_rcp45_r1i1p1_20201201-20201231.nc", - options = list("raster.read.strategy" = "retile_on_read") + options = list("raster.read.strategy" = "in_memory") ) %>% mutate(tile = rst_separatebands(tile)) %>% sdf_register("raster") From 5e9e84696f1cfb8a5e6c8c3b5643d907d5bde676 Mon Sep 17 00:00:00 2001 From: "milos.colic" Date: Fri, 1 Mar 2024 17:57:38 +0000 Subject: [PATCH 41/47] Fix gribs build. --- .../datasource/GDALFileFormatTest.scala | 66 ++++++++++--------- 1 file changed, 34 insertions(+), 32 deletions(-) diff --git a/src/test/scala/com/databricks/labs/mosaic/datasource/GDALFileFormatTest.scala b/src/test/scala/com/databricks/labs/mosaic/datasource/GDALFileFormatTest.scala index 623993b01..16017d29e 100644 --- a/src/test/scala/com/databricks/labs/mosaic/datasource/GDALFileFormatTest.scala +++ b/src/test/scala/com/databricks/labs/mosaic/datasource/GDALFileFormatTest.scala @@ -34,36 +34,6 @@ class GDALFileFormatTest extends QueryTest with SharedSparkSessionGDAL { .take(1) } - - test("Read grib with GDALFileFormat") { - assume(System.getProperty("os.name") == "Linux") - - val grib = "/binary/grib-cams/" - val filePath = getClass.getResource(grib).getPath - - noException should be thrownBy spark.read - .format("gdal") - .option("extensions", "grb") - .option("raster.read.strategy", "retile_on_read") - .load(filePath) - .take(1) - - noException should be thrownBy spark.read - .format("gdal") - .option("extensions", "grb") - .option("raster.read.strategy", "retile_on_read") - .load(filePath) - .take(1) - - noException should be thrownBy spark.read - .format("gdal") - .option("extensions", "grb") - .option("raster.read.strategy", "retile_on_read") - .load(filePath) - .select("metadata") - .take(1) - - } test("Read tif with GDALFileFormat") { assume(System.getProperty("os.name") == "Linux") @@ -82,14 +52,16 @@ class GDALFileFormatTest extends QueryTest with SharedSparkSessionGDAL { .load(filePath) .take(1) - noException should be thrownBy spark.read + // noException should be thrownBy + + spark.read .format("gdal") .option("driverName", "TIF") .load(filePath) .select("metadata") .take(1) - noException should be thrownBy spark.read + noException should be thrownBy spark.read .format("gdal") .option(MOSAIC_RASTER_READ_STRATEGY, "retile_on_read") .load(filePath) @@ -138,4 +110,34 @@ class GDALFileFormatTest extends QueryTest with SharedSparkSessionGDAL { } + test("Read grib with GDALFileFormat") { + assume(System.getProperty("os.name") == "Linux") + + val grib = "/binary/grib-cams/" + val filePath = getClass.getResource(grib).getPath + + noException should be thrownBy spark.read + .format("gdal") + .option("extensions", "grb") + .option("raster.read.strategy", "retile_on_read") + .load(filePath) + .take(1) + + noException should be thrownBy spark.read + .format("gdal") + .option("extensions", "grb") + .option("raster.read.strategy", "retile_on_read") + .load(filePath) + .take(1) + + noException should be thrownBy spark.read + .format("gdal") + .option("extensions", "grb") + .option("raster.read.strategy", "retile_on_read") + .load(filePath) + .select("metadata") + .take(1) + + } + } From 71db41a9e9126d507ed13c621944d5350a66072b Mon Sep 17 00:00:00 2001 From: "milos.colic" Date: Mon, 4 Mar 2024 12:39:52 +0000 Subject: [PATCH 42/47] Fix gribs build. --- .../mosaic/core/raster/gdal/MosaicRasterGDAL.scala | 3 +++ .../labs/mosaic/datasource/GDALFileFormatTest.scala | 10 ++++------ 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala index 2749d7cfa..bb9d09f25 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala @@ -411,6 +411,9 @@ case class MosaicRasterGDAL( def getMemSize: Long = { if (memSize == -1) { val toRead = if (path.startsWith("/vsizip/")) path.replace("/vsizip/", "") else path + if (Files.notExists(Paths.get(toRead))) { + throw new Exception(s"File not found: ${gdal.GetLastErrorMsg()}") + } Files.size(Paths.get(toRead)) } else { memSize diff --git a/src/test/scala/com/databricks/labs/mosaic/datasource/GDALFileFormatTest.scala b/src/test/scala/com/databricks/labs/mosaic/datasource/GDALFileFormatTest.scala index 16017d29e..c51190ea6 100644 --- a/src/test/scala/com/databricks/labs/mosaic/datasource/GDALFileFormatTest.scala +++ b/src/test/scala/com/databricks/labs/mosaic/datasource/GDALFileFormatTest.scala @@ -51,9 +51,7 @@ class GDALFileFormatTest extends QueryTest with SharedSparkSessionGDAL { .option("driverName", "TIF") .load(filePath) .take(1) - - // noException should be thrownBy - + spark.read .format("gdal") .option("driverName", "TIF") @@ -61,7 +59,7 @@ class GDALFileFormatTest extends QueryTest with SharedSparkSessionGDAL { .select("metadata") .take(1) - noException should be thrownBy spark.read + spark.read .format("gdal") .option(MOSAIC_RASTER_READ_STRATEGY, "retile_on_read") .load(filePath) @@ -116,7 +114,7 @@ class GDALFileFormatTest extends QueryTest with SharedSparkSessionGDAL { val grib = "/binary/grib-cams/" val filePath = getClass.getResource(grib).getPath - noException should be thrownBy spark.read + spark.read .format("gdal") .option("extensions", "grb") .option("raster.read.strategy", "retile_on_read") @@ -130,7 +128,7 @@ class GDALFileFormatTest extends QueryTest with SharedSparkSessionGDAL { .load(filePath) .take(1) - noException should be thrownBy spark.read + spark.read .format("gdal") .option("extensions", "grb") .option("raster.read.strategy", "retile_on_read") From 850cb55a71b9fbb19a5533558826e17e0ee527ed Mon Sep 17 00:00:00 2001 From: "milos.colic" Date: Mon, 4 Mar 2024 17:02:50 +0000 Subject: [PATCH 43/47] Fix gribs build. --- .../mosaic/datasource/gdal/ReTileOnRead.scala | 2 +- .../labs/mosaic/utils/PathUtils.scala | 37 ++++++++++++------- 2 files changed, 25 insertions(+), 14 deletions(-) diff --git a/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReTileOnRead.scala b/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReTileOnRead.scala index 867167c58..b68a580d2 100644 --- a/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReTileOnRead.scala +++ b/src/main/scala/com/databricks/labs/mosaic/datasource/gdal/ReTileOnRead.scala @@ -93,7 +93,7 @@ object ReTileOnRead extends ReadStrategy { val uuid = getUUID(status) val sizeInMB = options.getOrElse("sizeInMB", "16").toInt - val tmpPath = PathUtils.copyToTmp(inPath) + var tmpPath = PathUtils.copyToTmpWithRetry(inPath, 5) val tiles = localSubdivide(tmpPath, inPath, sizeInMB) val rows = tiles.map(tile => { diff --git a/src/main/scala/com/databricks/labs/mosaic/utils/PathUtils.scala b/src/main/scala/com/databricks/labs/mosaic/utils/PathUtils.scala index aacd897f8..2f896c046 100644 --- a/src/main/scala/com/databricks/labs/mosaic/utils/PathUtils.scala +++ b/src/main/scala/com/databricks/labs/mosaic/utils/PathUtils.scala @@ -2,7 +2,8 @@ package com.databricks.labs.mosaic.utils import com.databricks.labs.mosaic.functions.MosaicContext -import java.nio.file.{Files, Paths} +import java.nio.file.{Files, Path, Paths} +import scala.jdk.CollectionConverters._ object PathUtils { @@ -63,7 +64,7 @@ object PathUtils { if (filePath.endsWith("\"")) result = result.dropRight(1) result } - + def getStemRegex(path: String): String = { val cleanPath = replaceDBFSTokens(path) val fileName = Paths.get(cleanPath).getFileName.toString @@ -72,15 +73,25 @@ object PathUtils { val stemRegex = s"$stemEscaped\\..*".r stemRegex.toString } + + def copyToTmpWithRetry(inPath: String, retries: Int = 3): String = { + var tmpPath = copyToTmp(inPath) + var i = 0 + while (Files.notExists(Paths.get(tmpPath)) && i < retries) { + tmpPath = copyToTmp(inPath) + i += 1 + } + tmpPath + } def copyToTmp(inPath: String): String = { val copyFromPath = replaceDBFSTokens(inPath) val inPathDir = Paths.get(copyFromPath).getParent.toString - + val fullFileName = copyFromPath.split("/").last val stemRegex = getStemRegex(inPath) - wildcardCopy(inPathDir, MosaicContext.tmpDir(null), stemRegex.toString) + wildcardCopy(inPathDir, MosaicContext.tmpDir(null), stemRegex) s"${MosaicContext.tmpDir(null)}/$fullFileName" } @@ -93,17 +104,17 @@ object PathUtils { val toCopy = Files .list(Paths.get(copyFromPath)) .filter(_.getFileName.toString.matches(pattern)) - - toCopy.forEach(path => { + .collect(java.util.stream.Collectors.toList[Path]) + .asScala + + for (path <- toCopy) { val destination = Paths.get(copyToPath, path.getFileName.toString) - //noinspection SimplifyBooleanMatch - Files.isDirectory(path) match { - case true => FileUtils.copyDirectory(path.toFile, destination.toFile) - case false => Files.copy(path, destination) - } - }) + // noinspection SimplifyBooleanMatch + if (Files.isDirectory(path)) FileUtils.copyDirectory(path.toFile, destination.toFile) + else FileUtils.copyFile(path.toFile, destination.toFile) + } } - + def parseUnzippedPathFromExtracted(lastExtracted: String, extension: String): String = { val trimmed = lastExtracted.replace("extracting: ", "").replace(" ", "") val indexOfFormat = trimmed.indexOf(s".$extension/") From e84834b198e772435daa33f1a56b8ef75ea77656 Mon Sep 17 00:00:00 2001 From: "milos.colic" Date: Mon, 4 Mar 2024 17:57:10 +0000 Subject: [PATCH 44/47] Fix R build. --- R/sparkR-mosaic/tests.R | 8 ++++++-- R/sparklyr-mosaic/tests.R | 2 ++ 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/R/sparkR-mosaic/tests.R b/R/sparkR-mosaic/tests.R index 556b48b0f..7253cac2d 100644 --- a/R/sparkR-mosaic/tests.R +++ b/R/sparkR-mosaic/tests.R @@ -21,11 +21,15 @@ print("Looking for mosaic jar in") mosaic_jar_path <- paste0(staging_dir, mosaic_jar) print(mosaic_jar_path) +pwd <- getwd() spark <- sparkR.session( master = "local[*]" - ,sparkJars = mosaic_jar_path + ,sparkJars = mosaic_jar_path, + sparkConfig = list( + spark.databricks.labs.mosaic.raster.tmp.prefix = paste0(pwd, "/mosaic_tmp", sep="") + ,spark.databricks.labs.mosaic.raster.checkpoint = paste0(pwd, "/mosaic_checkpoint", sep="") + ) ) - enableMosaic() testthat::test_local(path="./sparkrMosaic") \ No newline at end of file diff --git a/R/sparklyr-mosaic/tests.R b/R/sparklyr-mosaic/tests.R index eef6d7c8b..b806a3d73 100644 --- a/R/sparklyr-mosaic/tests.R +++ b/R/sparklyr-mosaic/tests.R @@ -21,6 +21,8 @@ print(paste("Looking for mosaic jar in", mosaic_jar_path)) config <- sparklyr::spark_config() config$`sparklyr.jars.default` <- c(mosaic_jar_path) +config$`spark.databricks.labs.mosaic.raster.tmp.prefix` <- paste0(getwd(), "/mosaic_tmp", sep="") +config$`spark.databricks.labs.mosaic.raster.checkpoint` <- paste0(getwd(), "/mosaic_checkpoint", sep="") sc <- spark_connect(master="local[*]", config=config) enableMosaic(sc) From c53961521134f288e596dfa56a836470cf84ff2a Mon Sep 17 00:00:00 2001 From: "milos.colic" Date: Tue, 5 Mar 2024 16:54:06 +0000 Subject: [PATCH 45/47] Fix coverage tests. Fix convolve operations. --- python/mosaic/api/raster.py | 91 +++++++++++++++- .../labs/mosaic/core/raster/api/GDAL.scala | 1 - .../mosaic/core/raster/gdal/GDALBlock.scala | 3 +- .../raster/gdal/MosaicRasterBandGDAL.scala | 64 ++++++----- .../core/raster/gdal/MosaicRasterGDAL.scala | 15 ++- .../core/types/model/MosaicRasterTile.scala | 7 +- .../expressions/raster/RST_Convolve.scala | 11 +- .../expressions/raster/RST_FromFile.scala | 3 +- .../expressions/raster/RST_MakeTiles.scala | 22 ++-- .../labs/mosaic/functions/MosaicContext.scala | 4 + .../labs/mosaic/utils/FileUtils.scala | 3 +- .../raster/RST_ConvolveBehaviors.scala | 43 ++++++++ .../expressions/raster/RST_ConvolveTest.scala | 32 ++++++ .../raster/RST_FilterBehaviors.scala | 42 ++++++++ .../raster/RST_MakeTilesBehaviors.scala | 101 ++++++++++++++++++ .../raster/RST_MakeTilesTest.scala | 31 ++++++ 16 files changed, 425 insertions(+), 48 deletions(-) create mode 100644 src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_ConvolveBehaviors.scala create mode 100644 src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_ConvolveTest.scala create mode 100644 src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MakeTilesBehaviors.scala create mode 100644 src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MakeTilesTest.scala diff --git a/python/mosaic/api/raster.py b/python/mosaic/api/raster.py index c61bafcd2..c703b9134 100644 --- a/python/mosaic/api/raster.py +++ b/python/mosaic/api/raster.py @@ -5,7 +5,6 @@ from pyspark.sql.functions import lit from typing import Any - ####################### # Raster functions # ####################### @@ -15,16 +14,19 @@ "rst_boundingbox", "rst_clip", "rst_combineavg", + "rst_convolve", "rst_derivedband", "rst_frombands", "rst_fromcontent", "rst_fromfile", + "rst_filter", "rst_georeference", "rst_getnodata", "rst_getsubdataset", "rst_height", "rst_initnodata", "rst_isempty", + "rst_maketiles", "rst_mapalgebra", "rst_memsize", "rst_merge", @@ -157,6 +159,32 @@ def rst_combineavg(raster_tiles: ColumnOrName) -> Column: ) +def rst_convolve(raster_tile: ColumnOrName, kernel: ColumnOrName) -> Column: + """ + Applies a convolution filter to the raster. + The result is Mosaic raster tile struct column to the filtered raster. + The result is stored in the checkpoint directory. + + Parameters + ---------- + raster_tile : Column (RasterTileType) + Mosaic raster tile struct column. + kernel : Column (ArrayType(ArrayType(DoubleType))) + The kernel to apply to the raster. + + Returns + ------- + Column (RasterTileType) + Mosaic raster tile struct column. + + """ + return config.mosaic_context.invoke_function( + "rst_convolve", + pyspark_to_java_column(raster_tile), + pyspark_to_java_column(kernel), + ) + + def rst_derivedband( raster_tile: ColumnOrName, python_func: ColumnOrName, func_name: ColumnOrName ) -> Column: @@ -317,6 +345,43 @@ def rst_isempty(raster_tile: ColumnOrName) -> Column: ) +def rst_maketiles(input: ColumnOrName, driver: Any = "no_driver", size_in_mb: Any = -1, + with_checkpoint: Any = False) -> Column: + """ + Tiles the raster into tiles of the given size. + :param input: If the raster is stored on disc, the path + to the raster is provided. If the raster is stored in memory, the bytes of + the raster are provided. + :param driver: The driver to use for reading the raster. If not specified, the driver is + inferred from the file extension. If the input is a byte array, the driver + has to be specified. + :param size_in_mb: The size of the tiles in MB. If set to -1, the file is loaded and returned + as a single tile. If set to 0, the file is loaded and subdivided into + tiles of size 64MB. If set to a positive value, the file is loaded and + subdivided into tiles of the specified size. If the file is too big to fit + in memory, it is subdivided into tiles of size 64MB. + :param with_checkpoint: If set to true, the tiles are written to the checkpoint directory. If set + to false, the tiles are returned as a in-memory byte arrays. + :return: A collection of tiles of the raster. + """ + if type(size_in_mb) == int: + size_in_mb = lit(size_in_mb) + + if type(with_checkpoint) == bool: + with_checkpoint = lit(with_checkpoint) + + if type(driver) == str: + driver = lit(driver) + + return config.mosaic_context.invoke_function( + "rst_maketiles", + pyspark_to_java_column(input), + pyspark_to_java_column(driver), + pyspark_to_java_column(size_in_mb), + pyspark_to_java_column(with_checkpoint), + ) + + def rst_mapalgebra(raster_tile: ColumnOrName, json_spec: ColumnOrName) -> Column: """ Parameters @@ -631,7 +696,7 @@ def rst_rastertogridmin(raster_tile: ColumnOrName, resolution: ColumnOrName) -> def rst_rastertoworldcoord( - raster_tile: ColumnOrName, x: ColumnOrName, y: ColumnOrName + raster_tile: ColumnOrName, x: ColumnOrName, y: ColumnOrName ) -> Column: """ Computes the world coordinates of the raster pixel at the given x and y coordinates. @@ -1062,6 +1127,28 @@ def rst_fromfile(raster_path: ColumnOrName, size_in_mb: Any = -1) -> Column: ) +def rst_filter(raster_tile: ColumnOrName, kernel_size: Any, operation: Any) -> Column: + """ + Applies a filter to the raster. + :param raster_tile: Mosaic raster tile struct column. + :param kernel_size: The size of the kernel. Has to be odd. + :param operation: The operation to apply to the kernel. + :return: A new raster tile with the filter applied. + """ + if type(kernel_size) == int: + kernel_size = lit(kernel_size) + + if type(operation) == str: + operation = lit(operation) + + return config.mosaic_context.invoke_function( + "rst_filter", + pyspark_to_java_column(raster_tile), + pyspark_to_java_column(kernel_size), + pyspark_to_java_column(operation), + ) + + def rst_to_overlapping_tiles( raster_tile: ColumnOrName, width: ColumnOrName, diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/api/GDAL.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/api/GDAL.scala index 6e2fee0f6..3517f9368 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/api/GDAL.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/api/GDAL.scala @@ -101,7 +101,6 @@ object GDAL { ): MosaicRasterGDAL = { inputDT match { case StringType => - val path = inputRaster.asInstanceOf[UTF8String].toString MosaicRasterGDAL.readRaster(createInfo) case BinaryType => val bytes = inputRaster.asInstanceOf[Array[Byte]] diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/GDALBlock.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/GDALBlock.scala index 8c5c7a495..5014b0ee3 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/GDALBlock.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/GDALBlock.scala @@ -57,7 +57,6 @@ case class GDALBlock[T: ClassTag]( } } - // TODO: Test and fix, not tested, other filters work. def convolveAt(x: Int, y: Int, kernel: Array[Array[Double]]): Double = { val kernelWidth = kernel.head.length val kernelHeight = kernel.length @@ -70,7 +69,7 @@ case class GDALBlock[T: ClassTag]( val yIndex = y + (i - kernelCenterY) if (xIndex >= 0 && xIndex < width && yIndex >= 0 && yIndex < height) { val maskValue = maskAt(xIndex, yIndex) - val value = rasterElementAt(xIndex, yIndex) + val value = elementAt(xIndex, yIndex) if (maskValue != 0.0 && num.toDouble(value) != noDataValue) { sum += num.toDouble(value) * kernel(i)(j) } diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterBandGDAL.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterBandGDAL.scala index c9181b4f8..683d3791e 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterBandGDAL.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterBandGDAL.scala @@ -265,34 +265,48 @@ case class MosaicRasterBandGDAL(band: Band, id: Int) { stats.getValid_count == 0 } - def convolve(kernel: Array[Array[Double]]): Unit = { - val kernelWidth = kernel.head.length - val kernelHeight = kernel.length - val blockSize = MosaicGDAL.defaultBlockSize - val strideX = kernelWidth / 2 - val strideY = kernelHeight / 2 - - val block = Array.ofDim[Double](blockSize * blockSize) - val maskBlock = Array.ofDim[Double](blockSize * blockSize) - val result = Array.ofDim[Double](blockSize * blockSize) - - for (yOffset <- 0 until ySize by blockSize - strideY) { - for (xOffset <- 0 until xSize by blockSize - strideX) { - val xSize = Math.min(blockSize, this.xSize - xOffset) - val ySize = Math.min(blockSize, this.ySize - yOffset) - - band.ReadRaster(xOffset, yOffset, xSize, ySize, block) - band.GetMaskBand().ReadRaster(xOffset, yOffset, xSize, ySize, maskBlock) - - val currentBlock = GDALBlock[Double](block, maskBlock, noDataValue, xOffset, yOffset, xSize, ySize, Padding.NoPadding) - - for (y <- 0 until ySize) { - for (x <- 0 until xSize) { - result(y * xSize + x) = currentBlock.convolveAt(x, y, kernel) + /** + * Applies a kernel filter to the band. It assumes the kernel is square and + * has an odd number of rows and columns. + * + * @param kernel + * The kernel to apply to the band. + * @return + * The band with the kernel filter applied. + */ + def convolve(kernel: Array[Array[Double]], outputBand: Band): Unit = { + val kernelSize = kernel.length + require(kernelSize % 2 == 1, "Kernel size must be odd") + + val blockSize = MosaicGDAL.blockSize + val stride = kernelSize / 2 + + for (yOffset <- 0 until ySize by blockSize) { + for (xOffset <- 0 until xSize by blockSize) { + + val currentBlock = GDALBlock( + this, + stride, + xOffset, + yOffset, + blockSize + ) + + val result = Array.ofDim[Double](currentBlock.block.length) + + for (y <- 0 until currentBlock.height) { + for (x <- 0 until currentBlock.width) { + result(y * currentBlock.width + x) = currentBlock.convolveAt(x, y, kernel) } } - band.WriteRaster(xOffset, yOffset, xSize, ySize, block) + val trimmedResult = currentBlock.copy(block = result).trimBlock(stride) + + outputBand.WriteRaster(xOffset, yOffset, trimmedResult.width, trimmedResult.height, trimmedResult.block) + outputBand.FlushCache() + outputBand.GetMaskBand().WriteRaster(xOffset, yOffset, trimmedResult.width, trimmedResult.height, trimmedResult.maskBlock) + outputBand.GetMaskBand().FlushCache() + } } } diff --git a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala index bb9d09f25..b399fa0a2 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/raster/gdal/MosaicRasterGDAL.scala @@ -601,13 +601,18 @@ case class MosaicRasterGDAL( def convolve(kernel: Array[Array[Double]]): MosaicRasterGDAL = { val resultRasterPath = PathUtils.createTmpFilePath(getRasterFileExtension) - val outputRaster = this.raster + + this.raster .GetDriver() - .Create(resultRasterPath, this.xSize, this.ySize, this.numBands, this.raster.GetRasterBand(1).getDataType) + .CreateCopy(resultRasterPath, this.raster, 1) + .delete() + val outputRaster = gdal.Open(resultRasterPath, GF_Write) + for (bandIndex <- 1 to this.numBands) { val band = this.getBand(bandIndex) - band.convolve(kernel) + val outputBand = outputRaster.GetRasterBand(bandIndex) + band.convolve(kernel, outputBand) } val createInfo = Map( @@ -616,8 +621,8 @@ case class MosaicRasterGDAL( "driver" -> getDriversShortName ) - MosaicRasterGDAL(outputRaster, createInfo, -1) - + val result = MosaicRasterGDAL(outputRaster, createInfo, this.memSize) + result.flushCache() } def filter(kernelSize: Int, operation: String): MosaicRasterGDAL = { diff --git a/src/main/scala/com/databricks/labs/mosaic/core/types/model/MosaicRasterTile.scala b/src/main/scala/com/databricks/labs/mosaic/core/types/model/MosaicRasterTile.scala index bf36ea8f2..97122a7d7 100644 --- a/src/main/scala/com/databricks/labs/mosaic/core/types/model/MosaicRasterTile.scala +++ b/src/main/scala/com/databricks/labs/mosaic/core/types/model/MosaicRasterTile.scala @@ -100,14 +100,15 @@ case class MosaicRasterTile( rasterDataType: DataType ): InternalRow = { val encodedRaster = encodeRaster(rasterDataType) - val mapData = buildMapString(raster.createInfo) + val path = if (rasterDataType == StringType) encodedRaster.toString else raster.createInfo("path") + val parentPath = if (raster.createInfo("parentPath").isEmpty) raster.createInfo("path") else raster.createInfo("parentPath") + val newCreateInfo = raster.createInfo + ("path" -> path, "parentPath" -> parentPath) + val mapData = buildMapString(newCreateInfo) if (Option(index).isDefined) { if (index.isLeft) InternalRow.fromSeq( Seq(index.left.get, encodedRaster, mapData) ) else { - // Copy from tmp to checkpoint. - // Have to use GDAL Driver to do this since sidecar files are not copied by spark. InternalRow.fromSeq( Seq(UTF8String.fromString(index.right.get), encodedRaster, mapData) ) diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Convolve.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Convolve.scala index db20f8a3a..d831ad849 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Convolve.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_Convolve.scala @@ -9,6 +9,8 @@ import com.databricks.labs.mosaic.functions.MosaicExpressionConfig import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant} +import org.apache.spark.sql.catalyst.util.ArrayData +import org.apache.spark.sql.types._ /** The expression for applying kernel filter on a raster. */ case class RST_Convolve( @@ -39,7 +41,14 @@ case class RST_Convolve( * The clipped raster. */ override def rasterTransform(tile: MosaicRasterTile, arg1: Any): Any = { - val kernel = arg1.asInstanceOf[Array[Array[Double]]] + val kernel = arg1.asInstanceOf[ArrayData].array.map(_.asInstanceOf[ArrayData].array.map( + el => kernelExpr.dataType match { + case ArrayType(ArrayType(DoubleType, false), false) => el.asInstanceOf[Double] + case ArrayType(ArrayType(DecimalType(), false), false) => el.asInstanceOf[java.math.BigDecimal].doubleValue() + case ArrayType(ArrayType(IntegerType, false), false) => el.asInstanceOf[Int].toDouble + case _ => throw new IllegalArgumentException(s"Unsupported kernel type: ${kernelExpr.dataType}") + } + )) tile.copy( raster = tile.getRaster.convolve(kernel) ) diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_FromFile.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_FromFile.scala index cd5808f30..8e1dc213e 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_FromFile.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_FromFile.scala @@ -65,7 +65,8 @@ case class RST_FromFile( val readPath = PathUtils.getCleanPath(path) val driver = MosaicRasterGDAL.identifyDriver(path) val targetSize = sizeInMB.eval(input).asInstanceOf[Int] - if (targetSize <= 0 && Files.size(Paths.get(readPath)) <= Integer.MAX_VALUE) { + val currentSize = Files.size(Paths.get(PathUtils.replaceDBFSTokens(readPath))) + if (targetSize <= 0 && currentSize <= Integer.MAX_VALUE) { val createInfo = Map("path" -> readPath, "parentPath" -> path) var raster = MosaicRasterGDAL.readRaster(createInfo) var tile = MosaicRasterTile(null, raster) diff --git a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MakeTiles.scala b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MakeTiles.scala index 12c70d025..f9cf7099d 100644 --- a/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MakeTiles.scala +++ b/src/main/scala/com/databricks/labs/mosaic/expressions/raster/RST_MakeTiles.scala @@ -97,7 +97,8 @@ case class RST_MakeTiles( private def getInputSize(rawInput: Any): Long = { if (inputExpr.dataType == StringType) { val path = rawInput.asInstanceOf[UTF8String].toString - Files.size(Paths.get(path)) + val cleanPath = PathUtils.replaceDBFSTokens(path) + Files.size(Paths.get(cleanPath)) } else { val bytes = rawInput.asInstanceOf[Array[Byte]] bytes.length @@ -114,7 +115,7 @@ case class RST_MakeTiles( */ override def eval(input: InternalRow): TraversableOnce[InternalRow] = { GDAL.enable(expressionConfig) - + val tileType = dataType.asInstanceOf[StructType].find(_.name == "raster").get.dataType val rawDriver = driverExpr.eval(input).asInstanceOf[UTF8String].toString @@ -122,10 +123,11 @@ case class RST_MakeTiles( val driver = getDriver(rawInput, rawDriver) val targetSize = sizeInMBExpr.eval(input).asInstanceOf[Int] val inputSize = getInputSize(rawInput) + val path = if (inputExpr.dataType == StringType) rawInput.asInstanceOf[UTF8String].toString else PathUtils.NO_PATH_STRING if (targetSize <= 0 && inputSize <= Integer.MAX_VALUE) { // - no split required - val createInfo = Map("parentPath" -> PathUtils.NO_PATH_STRING, "driver" -> driver) + val createInfo = Map("parentPath" -> PathUtils.NO_PATH_STRING, "driver" -> driver, "path" -> path) val raster = GDAL.readRaster(rawInput, createInfo, inputExpr.dataType) val tile = MosaicRasterTile(null, raster) val row = tile.formatCellId(indexSystem).serialize(tileType) @@ -136,9 +138,15 @@ case class RST_MakeTiles( // target size is > 0 and raster size > target size // - write the initial raster to file (unsplit) // - createDirectories in case of context isolation - val rasterPath = PathUtils.createTmpFilePath(GDAL.getExtension(driver)) - Files.createDirectories(Paths.get(rasterPath).getParent) - Files.write(Paths.get(rasterPath), rawInput.asInstanceOf[Array[Byte]]) + val rasterPath = + if (inputExpr.dataType == StringType) { + PathUtils.copyToTmpWithRetry(path, 5) + } else { + val rasterPath = PathUtils.createTmpFilePath(GDAL.getExtension(driver)) + Files.createDirectories(Paths.get(rasterPath).getParent) + Files.write(Paths.get(rasterPath), rawInput.asInstanceOf[Array[Byte]]) + rasterPath + } val size = if (targetSize <= 0) 64 else targetSize var tiles = ReTileOnRead.localSubdivide(rasterPath, PathUtils.NO_PATH_STRING, size) val rows = tiles.map(_.formatCellId(indexSystem).serialize(tileType)) @@ -180,7 +188,7 @@ object RST_MakeTiles extends WithExpressionInfo { def checkChkpnt(chkpnt: Expression) = Try(chkpnt.eval().asInstanceOf[Boolean]).isSuccess def checkDriver(driver: Expression) = Try(driver.eval().asInstanceOf[UTF8String].toString).isSuccess val noSize = new Literal(-1, IntegerType) - val noDriver = new Literal(MOSAIC_NO_DRIVER, StringType) + val noDriver = new Literal(UTF8String.fromString(MOSAIC_NO_DRIVER), StringType) val noCheckpoint = new Literal(false, BooleanType) children match { diff --git a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala index 40be5cb24..7557306ec 100644 --- a/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala +++ b/src/main/scala/com/databricks/labs/mosaic/functions/MosaicContext.scala @@ -263,7 +263,9 @@ class MosaicContext(indexSystem: IndexSystem, geometryAPI: GeometryAPI) extends mosaicRegistry.registerExpression[RST_BoundingBox](expressionConfig) mosaicRegistry.registerExpression[RST_Clip](expressionConfig) mosaicRegistry.registerExpression[RST_CombineAvg](expressionConfig) + mosaicRegistry.registerExpression[RST_Convolve](expressionConfig) mosaicRegistry.registerExpression[RST_DerivedBand](expressionConfig) + mosaicRegistry.registerExpression[RST_Filter](expressionConfig) mosaicRegistry.registerExpression[RST_GeoReference](expressionConfig) mosaicRegistry.registerExpression[RST_GetNoData](expressionConfig) mosaicRegistry.registerExpression[RST_GetSubdataset](expressionConfig) @@ -660,6 +662,8 @@ class MosaicContext(indexSystem: IndexSystem, geometryAPI: GeometryAPI) extends ColumnAdapter(RST_BandMetaData(raster.expr, lit(band).expr, expressionConfig)) def rst_boundingbox(raster: Column): Column = ColumnAdapter(RST_BoundingBox(raster.expr, expressionConfig)) def rst_clip(raster: Column, geometry: Column): Column = ColumnAdapter(RST_Clip(raster.expr, geometry.expr, expressionConfig)) + def rst_convolve(raster: Column, kernel: Column): Column = + ColumnAdapter(RST_Convolve(raster.expr, kernel.expr, expressionConfig)) def rst_pixelcount(raster: Column): Column = ColumnAdapter(RST_PixelCount(raster.expr, expressionConfig)) def rst_combineavg(rasterArray: Column): Column = ColumnAdapter(RST_CombineAvg(rasterArray.expr, expressionConfig)) def rst_derivedband(raster: Column, pythonFunc: Column, funcName: Column): Column = diff --git a/src/main/scala/com/databricks/labs/mosaic/utils/FileUtils.scala b/src/main/scala/com/databricks/labs/mosaic/utils/FileUtils.scala index a36c0bec0..0a881d785 100644 --- a/src/main/scala/com/databricks/labs/mosaic/utils/FileUtils.scala +++ b/src/main/scala/com/databricks/labs/mosaic/utils/FileUtils.scala @@ -7,7 +7,8 @@ object FileUtils { def readBytes(path: String): Array[Byte] = { val bufferSize = 1024 * 1024 // 1MB - val inputStream = new BufferedInputStream(new FileInputStream(path)) + val cleanPath = PathUtils.replaceDBFSTokens(path) + val inputStream = new BufferedInputStream(new FileInputStream(cleanPath)) val buffer = new Array[Byte](bufferSize) var bytesRead = 0 diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_ConvolveBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_ConvolveBehaviors.scala new file mode 100644 index 000000000..fdc09cc28 --- /dev/null +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_ConvolveBehaviors.scala @@ -0,0 +1,43 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.geometry.api.GeometryAPI +import com.databricks.labs.mosaic.core.index.IndexSystem +import com.databricks.labs.mosaic.functions.MosaicContext +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.functions.{array, lit} +import org.scalatest.matchers.should.Matchers._ + +trait RST_ConvolveBehaviors extends QueryTest { + + // noinspection MapGetGet + def behaviors(indexSystem: IndexSystem, geometryAPI: GeometryAPI): Unit = { + spark.sparkContext.setLogLevel("FATAL") + val mc = MosaicContext.build(indexSystem, geometryAPI) + mc.register() + val sc = spark + import mc.functions._ + import sc.implicits._ + + val rastersInMemory = spark.read + .format("gdal") + .option("raster_storage", "in-memory") + .load("src/test/resources/modis") + + val gridTiles = rastersInMemory + .withColumn("result", rst_convolve($"tile", array(array(lit(1.0), lit(2.0), lit(3.0)), array(lit(3.0), lit(2.0), lit(1.0)), array(lit(1.0), lit(3.0), lit(2.0))))) + .select("result") + .collect() + + gridTiles.length should be(7) + + rastersInMemory.createOrReplaceTempView("source") + + spark + .sql(""" + |select rst_convolve(tile, array(array(1, 2, 3), array(2, 3, 1), array(1, 1, 1))) as tile from source + |""".stripMargin) + .collect() + + } + +} diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_ConvolveTest.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_ConvolveTest.scala new file mode 100644 index 000000000..28c049364 --- /dev/null +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_ConvolveTest.scala @@ -0,0 +1,32 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.geometry.api.JTS +import com.databricks.labs.mosaic.core.index.H3IndexSystem +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSessionGDAL + +import scala.util.Try + +class RST_ConvolveTest extends QueryTest with SharedSparkSessionGDAL with RST_ConvolveBehaviors { + + private val noCodegen = + withSQLConf( + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false", + SQLConf.CODEGEN_FACTORY_MODE.key -> CodegenObjectFactoryMode.NO_CODEGEN.toString + ) _ + + // Hotfix for SharedSparkSession afterAll cleanup. + override def afterAll(): Unit = Try(super.afterAll()) + + // These tests are not index system nor geometry API specific. + // Only testing one pairing is sufficient. + test("Testing RST_Convolve with manual GDAL registration (H3, JTS).") { + noCodegen { + assume(System.getProperty("os.name") == "Linux") + behaviors(H3IndexSystem, JTS) + } + } + +} diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_FilterBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_FilterBehaviors.scala index d06923dc1..2d64a633c 100644 --- a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_FilterBehaviors.scala +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_FilterBehaviors.scala @@ -4,6 +4,7 @@ import com.databricks.labs.mosaic.core.geometry.api.GeometryAPI import com.databricks.labs.mosaic.core.index.IndexSystem import com.databricks.labs.mosaic.functions.MosaicContext import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.functions.lit import org.scalatest.matchers.should.Matchers._ trait RST_FilterBehaviors extends QueryTest { @@ -29,7 +30,48 @@ trait RST_FilterBehaviors extends QueryTest { gridTiles.length should be(7) + val gridTiles2 = rastersInMemory + .withColumn("result", rst_filter($"tile", lit(3), lit("mode"))) + .select("result") + .collect() + + gridTiles2.length should be(7) + + val gridTiles3 = rastersInMemory + .withColumn("result", rst_filter($"tile", lit(3), lit("avg"))) + .select("result") + .collect() + + gridTiles3.length should be(7) + + val gridTiles4 = rastersInMemory + .withColumn("result", rst_filter($"tile", lit(3), lit("min"))) + .select("result") + .collect() + + gridTiles4.length should be(7) + + val gridTiles5 = rastersInMemory + .withColumn("result", rst_filter($"tile", lit(3), lit("max"))) + .select("result") + .collect() + + gridTiles5.length should be(7) + + val gridTiles6 = rastersInMemory + .withColumn("result", rst_filter($"tile", lit(3), lit("median"))) + .select("result") + .collect() + + gridTiles6.length should be(7) + rastersInMemory.createOrReplaceTempView("source") + + noException should be thrownBy spark + .sql(""" + |select rst_filter(tile, 3, 'mode') as tile from source + |""".stripMargin) + .collect() } diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MakeTilesBehaviors.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MakeTilesBehaviors.scala new file mode 100644 index 000000000..31caed8b9 --- /dev/null +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MakeTilesBehaviors.scala @@ -0,0 +1,101 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.geometry.api.GeometryAPI +import com.databricks.labs.mosaic.core.index.IndexSystem +import com.databricks.labs.mosaic.functions.MosaicContext +import org.apache.spark.sql.QueryTest +import org.scalatest.matchers.should.Matchers._ + +trait RST_MakeTilesBehaviors extends QueryTest { + + // noinspection MapGetGet + def behaviors(indexSystem: IndexSystem, geometryAPI: GeometryAPI): Unit = { + spark.sparkContext.setLogLevel("ERROR") + val mc = MosaicContext.build(indexSystem, geometryAPI) + mc.register() + val sc = spark + + import mc.functions._ + import sc.implicits._ + + val rastersInMemory = spark.read + .format("binaryFile") + .load("src/test/resources/modis") + + val gridTiles1 = rastersInMemory + .withColumn("tile", rst_maketiles($"content", "GTiff", -1)) + .select(!rst_isempty($"tile")) + .as[Boolean] + .collect() + + gridTiles1.forall(identity) should be(true) + + rastersInMemory.createOrReplaceTempView("source") + + val gridTilesSQL = spark + .sql(""" + |with subquery as ( + | select rst_maketiles(content, 'GTiff', -1) as tile from source + |) + |select not rst_isempty(tile) as result + |from subquery + |""".stripMargin) + .as[Boolean] + .collect() + + gridTilesSQL.forall(identity) should be(true) + + + val gridTilesSQL2 = spark + .sql( + """ + |with subquery as ( + | select rst_maketiles(content, 'GTiff', 4) as tile from source + |) + |select not rst_isempty(tile) as result + |from subquery + |""".stripMargin) + .as[Boolean] + .collect() + + gridTilesSQL2.forall(identity) should be(true) + + val gridTilesSQL3 = spark + .sql( + """ + |with subquery as ( + | select rst_maketiles(path, 'GTiff', 4) as tile from source + |) + |select not rst_isempty(tile) as result + |from subquery + |""".stripMargin) + .as[Boolean] + .collect() + + gridTilesSQL3.forall(identity) should be(true) + + val gridTilesSQL4 = spark + .sql( + """ + |with subquery as ( + | select rst_maketiles(path, 'GTiff', 4, true) as tile from source + |) + |select not rst_isempty(tile) as result + |from subquery + |""".stripMargin) + .as[Boolean] + .collect() + + gridTilesSQL4.forall(identity) should be(true) + + val gridTiles2 = rastersInMemory + .withColumn("tile", rst_maketiles($"path")) + .select(!rst_isempty($"tile")) + .as[Boolean] + .collect() + + gridTiles2.forall(identity) should be(true) + + } + +} diff --git a/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MakeTilesTest.scala b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MakeTilesTest.scala new file mode 100644 index 000000000..7eaae222b --- /dev/null +++ b/src/test/scala/com/databricks/labs/mosaic/expressions/raster/RST_MakeTilesTest.scala @@ -0,0 +1,31 @@ +package com.databricks.labs.mosaic.expressions.raster + +import com.databricks.labs.mosaic.core.geometry.api.JTS +import com.databricks.labs.mosaic.core.index.H3IndexSystem +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSessionGDAL + +import scala.util.Try + +class RST_MakeTilesTest extends QueryTest with SharedSparkSessionGDAL with RST_MakeTilesBehaviors { + + private val noCodegen = + withSQLConf( + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false", + SQLConf.CODEGEN_FACTORY_MODE.key -> CodegenObjectFactoryMode.NO_CODEGEN.toString + ) _ + + // Hotfix for SharedSparkSession afterAll cleanup. + override def afterAll(): Unit = Try(super.afterAll()) + + // These tests are not index system nor geometry API specific. + // Only testing one pairing is sufficient. + test("Testing RST_MakeTiles with manual GDAL registration (H3, JTS).") { + noCodegen { + assume(System.getProperty("os.name") == "Linux") + behaviors(H3IndexSystem, JTS) + } + } +} From 29621c6200a1d77c15f8fa251a9c71609dae8aea Mon Sep 17 00:00:00 2001 From: a0x8o Date: Fri, 28 Apr 2023 18:36:10 +0000 Subject: [PATCH 46/47] Documentation updates --- docs/source/usage/grid-indexes.ipynb | 564 +- docs/source/usage/kepler.ipynb | 591 ++ docs/source/usage/quickstart.ipynb | 10377 ++----------------------- 3 files changed, 1306 insertions(+), 10226 deletions(-) diff --git a/docs/source/usage/grid-indexes.ipynb b/docs/source/usage/grid-indexes.ipynb index 6690c7ae0..6d39eb993 100644 --- a/docs/source/usage/grid-indexes.ipynb +++ b/docs/source/usage/grid-indexes.ipynb @@ -2,495 +2,391 @@ "cells": [ { "cell_type": "markdown", - "source": [ - "# Using grid index systems in Mosaic" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "1deecb01-8ded-4750-a112-5847e410846b" + "nuid": "1deecb01-8ded-4750-a112-5847e410846b", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "# Using grid index systems in Mosaic" + ] }, { "cell_type": "code", - "source": [ - "from pyspark.sql.functions import *\nfrom mosaic import enable_mosaic\nenable_mosaic(spark, dbutils)" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "f6311e22-f0a8-4025-b612-4367b5f1cac2" + "nuid": "f6311e22-f0a8-4025-b612-4367b5f1cac2", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "from pyspark.sql.functions import *\n", + "from mosaic import enable_mosaic\n", + "enable_mosaic(spark, dbutils)" + ] }, { "cell_type": "markdown", - "source": [ - "Set operations over big geospatial datasets become very expensive without some form of spatial indexing.\n\nSpatial indexes not only allow operations like point-in-polygon joins to be partitioned but, if only approximate results are required, can be used to reduce these to deterministic SQL joins directly on the indexes." - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "e5603057-662b-4bf3-9517-84b35ad7a8b5" + "nuid": "e5603057-662b-4bf3-9517-84b35ad7a8b5", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "Set operations over big geospatial datasets become very expensive without some form of spatial indexing.\n", + "\n", + "Spatial indexes not only allow operations like point-in-polygon joins to be partitioned but, if only approximate results are required, can be used to reduce these to deterministic SQL joins directly on the indexes." + ] }, { "cell_type": "markdown", - "source": [ - "![example h3 point-in-poly image](https://databricks.com/wp-content/uploads/2021/01/blog-geospatial-3.jpg)" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "c8199fdd-1372-43a4-8edf-f96129f4195a" + "nuid": "c8199fdd-1372-43a4-8edf-f96129f4195a", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "![example h3 point-in-poly image](https://databricks.com/wp-content/uploads/2021/01/blog-geospatial-3.jpg)" + ] }, { "cell_type": "markdown", - "source": [ - "The workflow for a point-in-poly spatial join might look like the following:" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "39e3cf64-61c9-4930-be5e-91b254cf0984" + "nuid": "39e3cf64-61c9-4930-be5e-91b254cf0984", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "The workflow for a point-in-poly spatial join might look like the following:" + ] }, { "cell_type": "markdown", - "source": [ - "## 1. Read the source point and polygon datasets." - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "947d6e33-06b5-4ea0-bc55-0fdb5b8e8524" + "nuid": "947d6e33-06b5-4ea0-bc55-0fdb5b8e8524", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "## 1. Read the source point and polygon datasets." + ] }, { "cell_type": "code", - "source": [ - "drop_cols = [\n \"rate_code_id\", \"store_and_fwd_flag\", \"dropoff_longitude\",\n \"dropoff_latitude\", \"payment_type\", \"fare_amount\",\n \"extra\", \"mta_tax\", \"tip_amount\", \"tolls_amount\",\n \"total_amount\"\n]\n\ntrips = (\n spark.table(\"delta.`/databricks-datasets/nyctaxi/tables/nyctaxi_yellow`\")\n .drop(*drop_cols)\n .limit(5_000_000)\n .repartition(sc.defaultParallelism * 20)\n)\n\ntrips.show()" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "f1474a53-da53-4955-b944-083fdf3a22a1" + "nuid": "f1474a53-da53-4955-b944-083fdf3a22a1", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+\n|vendor_id| pickup_datetime| dropoff_datetime|passenger_count|trip_distance|pickup_longitude|pickup_latitude|\n+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+\n| CMT|2009-01-01 20:07:33|2009-01-01 20:12:28| 1| 0.8| -74.001041| 40.731|\n| CMT|2009-01-06 15:29:12|2009-01-06 15:51:57| 2| 3.3| -73.996489| 40.725742|\n| CMT|2010-02-14 17:42:16|2010-02-14 17:55:03| 1| 3.4| -74.002949| 40.734254|\n| CMT|2010-02-11 18:19:01|2010-02-11 18:27:54| 1| 1.5| -73.998133| 40.682463|\n| VTS|2009-04-29 12:26:00|2009-04-29 12:35:00| 3| 2.05| -74.001332| 40.72006|\n| VTS|2009-04-24 15:03:00|2009-04-24 15:23:00| 2| 2.89| -73.989952| 40.734625|\n| CMT|2010-02-28 13:55:44|2010-02-28 14:02:37| 1| 1.2| -74.006015| 40.735279|\n| VTS|2009-09-27 08:46:00|2009-09-27 08:59:00| 1| 3.97| -74.000148| 40.717468|\n| CMT|2010-02-18 09:48:52|2010-02-18 10:08:38| 1| 3.0| -73.995177| 40.725297|\n| CMT|2009-04-09 20:33:44|2009-04-09 20:39:33| 2| 0.6| -73.990133| 40.729321|\n| CMT|2010-02-13 22:41:10|2010-02-13 23:07:04| 1| 4.2| -74.009175| 40.706284|\n| CMT|2009-01-25 20:06:51|2009-01-25 20:12:37| 1| 1.3| -74.007384| 40.717929|\n| VTS|2010-02-27 18:19:00|2010-02-27 18:38:00| 1| 4.2| -74.011512| 40.710588|\n| VTS|2010-02-15 10:17:00|2010-02-15 10:24:00| 1| 1.74| -74.016442| 40.711617|\n| CMT|2009-12-26 18:45:49|2009-12-26 18:59:08| 1| 4.8| -74.01014| 40.712263|\n| CMT|2009-12-06 01:00:07|2009-12-06 01:11:41| 2| 4.2| -74.002505| 40.729001|\n| VTS|2009-10-04 14:36:00|2009-10-04 14:42:00| 1| 1.13| -74.006767| 40.718942|\n| CMT|2009-01-18 00:20:50|2009-01-18 00:36:29| 3| 2.1| -73.993258| 40.721401|\n| VTS|2009-05-18 13:24:00|2009-05-18 13:33:00| 1| 1.91| -73.992785| 40.730412|\n| VTS|2009-11-11 21:51:00|2009-11-11 22:13:00| 5| 4.71| -74.010065| 40.733383|\n+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+\nonly showing top 20 rows\n\n
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+\nvendor_id| pickup_datetime| dropoff_datetime|passenger_count|trip_distance|pickup_longitude|pickup_latitude|\n+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+\n CMT|2009-01-01 20:07:33|2009-01-01 20:12:28| 1| 0.8| -74.001041| 40.731|\n CMT|2009-01-06 15:29:12|2009-01-06 15:51:57| 2| 3.3| -73.996489| 40.725742|\n CMT|2010-02-14 17:42:16|2010-02-14 17:55:03| 1| 3.4| -74.002949| 40.734254|\n CMT|2010-02-11 18:19:01|2010-02-11 18:27:54| 1| 1.5| -73.998133| 40.682463|\n VTS|2009-04-29 12:26:00|2009-04-29 12:35:00| 3| 2.05| -74.001332| 40.72006|\n VTS|2009-04-24 15:03:00|2009-04-24 15:23:00| 2| 2.89| -73.989952| 40.734625|\n CMT|2010-02-28 13:55:44|2010-02-28 14:02:37| 1| 1.2| -74.006015| 40.735279|\n VTS|2009-09-27 08:46:00|2009-09-27 08:59:00| 1| 3.97| -74.000148| 40.717468|\n CMT|2010-02-18 09:48:52|2010-02-18 10:08:38| 1| 3.0| -73.995177| 40.725297|\n CMT|2009-04-09 20:33:44|2009-04-09 20:39:33| 2| 0.6| -73.990133| 40.729321|\n CMT|2010-02-13 22:41:10|2010-02-13 23:07:04| 1| 4.2| -74.009175| 40.706284|\n CMT|2009-01-25 20:06:51|2009-01-25 20:12:37| 1| 1.3| -74.007384| 40.717929|\n VTS|2010-02-27 18:19:00|2010-02-27 18:38:00| 1| 4.2| -74.011512| 40.710588|\n VTS|2010-02-15 10:17:00|2010-02-15 10:24:00| 1| 1.74| -74.016442| 40.711617|\n CMT|2009-12-26 18:45:49|2009-12-26 18:59:08| 1| 4.8| -74.01014| 40.712263|\n CMT|2009-12-06 01:00:07|2009-12-06 01:11:41| 2| 4.2| -74.002505| 40.729001|\n VTS|2009-10-04 14:36:00|2009-10-04 14:42:00| 1| 1.13| -74.006767| 40.718942|\n CMT|2009-01-18 00:20:50|2009-01-18 00:36:29| 3| 2.1| -73.993258| 40.721401|\n VTS|2009-05-18 13:24:00|2009-05-18 13:33:00| 1| 1.91| -73.992785| 40.730412|\n VTS|2009-11-11 21:51:00|2009-11-11 22:13:00| 5| 4.71| -74.010065| 40.733383|\n+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+\nonly showing top 20 rows\n\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "drop_cols = [\n", + " \"rate_code_id\", \"store_and_fwd_flag\", \"dropoff_longitude\",\n", + " \"dropoff_latitude\", \"payment_type\", \"fare_amount\",\n", + " \"extra\", \"mta_tax\", \"tip_amount\", \"tolls_amount\",\n", + " \"total_amount\"\n", + "]\n", + "\n", + "trips = (\n", + " spark.table(\"delta.`/databricks-datasets/nyctaxi/tables/nyctaxi_yellow`\")\n", + " .drop(*drop_cols)\n", + " .limit(5_000_000)\n", + " .repartition(sc.defaultParallelism * 20)\n", + ")\n", + "\n", + "trips.show()" + ] }, { "cell_type": "code", - "source": [ - "from mosaic import st_geomfromgeojson\n\nuser = spark.sql(\"select current_user() as user\").collect()[0][\"user\"]\n\nneighbourhoods = (\n spark.read.format(\"json\")\n .load(f\"dbfs:/FileStore/shared_uploads/{user}/NYC_Taxi_Zones.geojson\")\n .repartition(sc.defaultParallelism)\n .withColumn(\"geometry\", st_geomfromgeojson(to_json(col(\"geometry\"))))\n .select(\"properties.*\", \"geometry\")\n .drop(\"shape_area\", \"shape_leng\")\n)\n\nneighbourhoods.show()" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "e3a366bf-4d60-4356-8b7b-eedb0beabc8d" + "nuid": "e3a366bf-4d60-4356-8b7b-eedb0beabc8d", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
+-------------+-----------+--------+-------------------+--------------------+\n| borough|location_id|objectid| zone| geometry|\n+-------------+-----------+--------+-------------------+--------------------+\n| Brooklyn| 123| 123| Homecrest|{6, 4326, [[[-73....|\n| Manhattan| 153| 153| Marble Hill|{6, 4326, [[[-73....|\n| Brooklyn| 112| 112| Greenpoint|{6, 4326, [[[-73....|\n| Manhattan| 233| 233|UN/Turtle Bay South|{6, 4326, [[[-73....|\n| Manhattan| 43| 43| Central Park|{6, 4326, [[[-73....|\n| Queens| 201| 201| Rockaway Park|{6, 4326, [[[-73....|\n| Queens| 131| 131| Jamaica Estates|{6, 4326, [[[-73....|\n| Brooklyn| 111| 111|Green-Wood Cemetery|{6, 4326, [[[-73....|\n| Queens| 226| 226| Sunnyside|{6, 4326, [[[-73....|\n| Queens| 129| 129| Jackson Heights|{6, 4326, [[[-73....|\n| Manhattan| 120| 120| Highbridge Park|{6, 4326, [[[-73....|\n| Brooklyn| 76| 76| East New York|{6, 4326, [[[-73....|\n| Manhattan| 24| 24| Bloomingdale|{6, 4326, [[[-73....|\n| Manhattan| 202| 202| Roosevelt Island|{6, 4326, [[[-73....|\n| Manhattan| 100| 100| Garment District|{6, 4326, [[[-73....|\n|Staten Island| 251| 251| Westerleigh|{6, 4326, [[[-74....|\n| Manhattan| 74| 74| East Harlem North|{6, 4326, [[[-73....|\n| Queens| 98| 98| Fresh Meadows|{6, 4326, [[[-73....|\n| Manhattan| 211| 211| SoHo|{6, 4326, [[[-74....|\n| Bronx| 174| 174| Norwood|{6, 4326, [[[-73....|\n+-------------+-----------+--------+-------------------+--------------------+\nonly showing top 20 rows\n\n
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
+-------------+-----------+--------+-------------------+--------------------+\n borough|location_id|objectid| zone| geometry|\n+-------------+-----------+--------+-------------------+--------------------+\n Brooklyn| 123| 123| Homecrest|{6, 4326, [[[-73....|\n Manhattan| 153| 153| Marble Hill|{6, 4326, [[[-73....|\n Brooklyn| 112| 112| Greenpoint|{6, 4326, [[[-73....|\n Manhattan| 233| 233|UN/Turtle Bay South|{6, 4326, [[[-73....|\n Manhattan| 43| 43| Central Park|{6, 4326, [[[-73....|\n Queens| 201| 201| Rockaway Park|{6, 4326, [[[-73....|\n Queens| 131| 131| Jamaica Estates|{6, 4326, [[[-73....|\n Brooklyn| 111| 111|Green-Wood Cemetery|{6, 4326, [[[-73....|\n Queens| 226| 226| Sunnyside|{6, 4326, [[[-73....|\n Queens| 129| 129| Jackson Heights|{6, 4326, [[[-73....|\n Manhattan| 120| 120| Highbridge Park|{6, 4326, [[[-73....|\n Brooklyn| 76| 76| East New York|{6, 4326, [[[-73....|\n Manhattan| 24| 24| Bloomingdale|{6, 4326, [[[-73....|\n Manhattan| 202| 202| Roosevelt Island|{6, 4326, [[[-73....|\n Manhattan| 100| 100| Garment District|{6, 4326, [[[-73....|\nStaten Island| 251| 251| Westerleigh|{6, 4326, [[[-74....|\n Manhattan| 74| 74| East Harlem North|{6, 4326, [[[-73....|\n Queens| 98| 98| Fresh Meadows|{6, 4326, [[[-73....|\n Manhattan| 211| 211| SoHo|{6, 4326, [[[-74....|\n Bronx| 174| 174| Norwood|{6, 4326, [[[-73....|\n+-------------+-----------+--------+-------------------+--------------------+\nonly showing top 20 rows\n\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "from mosaic import st_geomfromgeojson\n", + "\n", + "user = spark.sql(\"select current_user() as user\").collect()[0][\"user\"]\n", + "\n", + "neighbourhoods = (\n", + " spark.read.format(\"json\")\n", + " .load(f\"dbfs:/FileStore/shared_uploads/{user}/NYC_Taxi_Zones.geojson\")\n", + " .repartition(sc.defaultParallelism)\n", + " .withColumn(\"geometry\", st_geomfromgeojson(to_json(col(\"geometry\"))))\n", + " .select(\"properties.*\", \"geometry\")\n", + " .drop(\"shape_area\", \"shape_leng\")\n", + ")\n", + "\n", + "neighbourhoods.show()" + ] }, { "cell_type": "markdown", - "source": [ - "## 2. Compute the resolution of index required to optimize the join." - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "ae8cc184-574e-458f-9613-d2225b7a73e8" + "nuid": "ae8cc184-574e-458f-9613-d2225b7a73e8", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "## 2. Compute the resolution of index required to optimize the join." + ] }, { "cell_type": "code", - "source": [ - "from mosaic import MosaicFrame\n\nneighbourhoods_mdf = MosaicFrame(neighbourhoods, \"geometry\")\nhelp(neighbourhoods_mdf.get_optimal_resolution)" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "08dc48c1-9da2-4f8a-a233-aae456cac972" + "nuid": "08dc48c1-9da2-4f8a-a233-aae456cac972", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
Help on method get_optimal_resolution in module mosaic.core.mosaic_frame:\n\nget_optimal_resolution(sample_rows: Union[int, NoneType] = None, sample_fraction: Union[float, NoneType] = None) -> int method of mosaic.core.mosaic_frame.MosaicFrame instance\n Analyzes the geometries in the currently selected geometry column and proposes an optimal\n grid-index resolution.\n \n Provide either `sample_rows` or `sample_fraction` parameters to control how much data is passed to the analyzer.\n (Providing too little data to the analyzer may result in a `NotEnoughGeometriesException`)\n \n Parameters\n ----------\n sample_rows: int, optional\n The number of rows to sample.\n sample_fraction: float, optional\n The proportion of rows to sample.\n \n Returns\n -------\n int\n The recommended grid-index resolution to apply to this MosaicFrame.\n\n
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
Help on method get_optimal_resolution in module mosaic.core.mosaic_frame:\n\nget_optimal_resolution(sample_rows: Union[int, NoneType] = None, sample_fraction: Union[float, NoneType] = None) -> int method of mosaic.core.mosaic_frame.MosaicFrame instance\n Analyzes the geometries in the currently selected geometry column and proposes an optimal\n grid-index resolution.\n \n Provide either `sample_rows` or `sample_fraction` parameters to control how much data is passed to the analyzer.\n (Providing too little data to the analyzer may result in a `NotEnoughGeometriesException`)\n \n Parameters\n ----------\n sample_rows: int, optional\n The number of rows to sample.\n sample_fraction: float, optional\n The proportion of rows to sample.\n \n Returns\n -------\n int\n The recommended grid-index resolution to apply to this MosaicFrame.\n\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "from mosaic import MosaicFrame\n", + "\n", + "neighbourhoods_mdf = MosaicFrame(neighbourhoods, \"geometry\")\n", + "help(neighbourhoods_mdf.get_optimal_resolution)" + ] }, { "cell_type": "code", - "source": [ - "(resolution := neighbourhoods_mdf.get_optimal_resolution(sample_fraction=1.))" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "50ed9fb7-3521-40fe-856c-8d5ed9b182d3" + "nuid": "50ed9fb7-3521-40fe-856c-8d5ed9b182d3", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
Out[15]: 9
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
Out[15]: 9
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "(resolution := neighbourhoods_mdf.get_optimal_resolution(sample_fraction=1.))" + ] }, { "cell_type": "markdown", - "source": [ - "## 3. Apply the index to the set of points in your left-hand dataframe.\nThis will generate an index value that corresponds to the grid ‘cell’ that this point occupies." - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "fccff6d5-da76-44c6-b015-749d3dcad3ca" + "nuid": "fccff6d5-da76-44c6-b015-749d3dcad3ca", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "## 3. Apply the index to the set of points in your left-hand dataframe.\n", + "This will generate an index value that corresponds to the grid ‘cell’ that this point occupies." + ] }, { "cell_type": "code", - "source": [ - "from mosaic import grid_longlatascellid\nindexed_trips = trips.withColumn(\"ix\", grid_longlatascellid(lon=\"pickup_longitude\", lat=\"pickup_latitude\", resolution=lit(resolution)))\nindexed_trips.show()" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "aad98fce-a22b-47d0-961d-62007acc23dd" + "nuid": "aad98fce-a22b-47d0-961d-62007acc23dd", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------------+\n|vendor_id| pickup_datetime| dropoff_datetime|passenger_count|trip_distance|pickup_longitude|pickup_latitude| ix|\n+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------------+\n| DDS|2009-01-17 18:49:57|2009-01-17 18:56:29| 3| 1.2| -74.004043| 40.733409|617733151092113407|\n| DDS|2009-12-01 00:47:52|2009-12-01 01:00:16| 1| 3.4| -73.991702| 40.726342|617733151087132671|\n| CMT|2009-02-09 16:50:21|2009-02-09 17:02:47| 1| 2.6| -73.999673| 40.733586|617733123805806591|\n| CMT|2009-12-07 07:15:47|2009-12-07 07:32:07| 1| 3.8| -74.01211| 40.716893|617733151084773375|\n| VTS|2009-10-16 22:02:00|2009-10-16 22:08:00| 1| 1.1| -74.010903| 40.71624|617733151084773375|\n| VTS|2009-12-23 22:13:00|2009-12-23 22:18:00| 1| 0.37| -74.002343| 40.73366|617733151092113407|\n| VTS|2009-12-12 01:24:00|2009-12-12 01:38:00| 2| 3.55| -74.002565| 40.728188|617733151091326975|\n| CMT|2009-12-07 13:10:37|2009-12-07 13:13:45| 1| 0.5| -73.999184| 40.73428|617733123805806591|\n| CMT|2009-11-08 22:20:44|2009-11-08 22:31:23| 1| 1.9| -74.003029| 40.733385|617733151092113407|\n| VTS|2009-12-27 20:01:00|2009-12-27 20:04:00| 1| 1.04| -74.000227| 40.732603|617733151092375551|\n| VTS|2009-02-13 14:33:00|2009-02-13 14:50:00| 3| 1.59| -74.006535| 40.732303|617733151092637695|\n| CMT|2009-11-15 21:13:32|2009-11-15 21:25:56| 3| 3.0| -73.998795| 40.730621|617733151092375551|\n| VTS|2009-01-08 18:13:00|2009-01-08 18:33:00| 2| 4.18| -74.0079| 40.712012|617733151021334527|\n| CMT|2009-11-30 13:30:13|2009-11-30 13:41:55| 1| 1.6| -74.004487| 40.734072|617733151092637695|\n| CMT|2009-01-11 20:02:22|2009-01-11 20:08:15| 1| 1.0| -74.004493| 40.713349|617733151020810239|\n| CMT|2009-12-30 18:46:08|2009-12-30 19:02:23| 1| 2.3| -74.010798| 40.716717|617733151084773375|\n| CMT|2009-11-18 21:50:12|2009-11-18 22:05:19| 1| 5.8| -73.992515| 40.694106|617733151038111743|\n| VTS|2009-11-21 12:51:00|2009-11-21 13:27:00| 1| 14.18| -73.9923| 40.715218|617733151109414911|\n| CMT|2009-01-20 09:34:49|2009-01-20 09:37:15| 1| 0.4| -74.0027| 40.733479|617733151092113407|\n| VTS|2009-01-03 07:07:00|2009-01-03 07:18:00| 1| 7.81| -73.994358| 40.690345|617733151037325311|\n+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------------+\nonly showing top 20 rows\n\n
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------------+\nvendor_id| pickup_datetime| dropoff_datetime|passenger_count|trip_distance|pickup_longitude|pickup_latitude| ix|\n+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------------+\n DDS|2009-01-17 18:49:57|2009-01-17 18:56:29| 3| 1.2| -74.004043| 40.733409|617733151092113407|\n DDS|2009-12-01 00:47:52|2009-12-01 01:00:16| 1| 3.4| -73.991702| 40.726342|617733151087132671|\n CMT|2009-02-09 16:50:21|2009-02-09 17:02:47| 1| 2.6| -73.999673| 40.733586|617733123805806591|\n CMT|2009-12-07 07:15:47|2009-12-07 07:32:07| 1| 3.8| -74.01211| 40.716893|617733151084773375|\n VTS|2009-10-16 22:02:00|2009-10-16 22:08:00| 1| 1.1| -74.010903| 40.71624|617733151084773375|\n VTS|2009-12-23 22:13:00|2009-12-23 22:18:00| 1| 0.37| -74.002343| 40.73366|617733151092113407|\n VTS|2009-12-12 01:24:00|2009-12-12 01:38:00| 2| 3.55| -74.002565| 40.728188|617733151091326975|\n CMT|2009-12-07 13:10:37|2009-12-07 13:13:45| 1| 0.5| -73.999184| 40.73428|617733123805806591|\n CMT|2009-11-08 22:20:44|2009-11-08 22:31:23| 1| 1.9| -74.003029| 40.733385|617733151092113407|\n VTS|2009-12-27 20:01:00|2009-12-27 20:04:00| 1| 1.04| -74.000227| 40.732603|617733151092375551|\n VTS|2009-02-13 14:33:00|2009-02-13 14:50:00| 3| 1.59| -74.006535| 40.732303|617733151092637695|\n CMT|2009-11-15 21:13:32|2009-11-15 21:25:56| 3| 3.0| -73.998795| 40.730621|617733151092375551|\n VTS|2009-01-08 18:13:00|2009-01-08 18:33:00| 2| 4.18| -74.0079| 40.712012|617733151021334527|\n CMT|2009-11-30 13:30:13|2009-11-30 13:41:55| 1| 1.6| -74.004487| 40.734072|617733151092637695|\n CMT|2009-01-11 20:02:22|2009-01-11 20:08:15| 1| 1.0| -74.004493| 40.713349|617733151020810239|\n CMT|2009-12-30 18:46:08|2009-12-30 19:02:23| 1| 2.3| -74.010798| 40.716717|617733151084773375|\n CMT|2009-11-18 21:50:12|2009-11-18 22:05:19| 1| 5.8| -73.992515| 40.694106|617733151038111743|\n VTS|2009-11-21 12:51:00|2009-11-21 13:27:00| 1| 14.18| -73.9923| 40.715218|617733151109414911|\n CMT|2009-01-20 09:34:49|2009-01-20 09:37:15| 1| 0.4| -74.0027| 40.733479|617733151092113407|\n VTS|2009-01-03 07:07:00|2009-01-03 07:18:00| 1| 7.81| -73.994358| 40.690345|617733151037325311|\n+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------------+\nonly showing top 20 rows\n\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "from mosaic import grid_longlatascellid\n", + "indexed_trips = trips.withColumn(\"ix\", grid_longlatascellid(lon=\"pickup_longitude\", lat=\"pickup_latitude\", resolution=lit(resolution)))\n", + "indexed_trips.show()" + ] }, { "cell_type": "markdown", - "source": [ - "## 4. Compute the set of indices that fully covers each polygon in the right-hand dataframe\nThis is commonly referred to as a polyfill operation." - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "e127e5be-30e4-4507-8aa9-59399290b8f7" + "nuid": "e127e5be-30e4-4507-8aa9-59399290b8f7", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "## 4. Compute the set of indices that fully covers each polygon in the right-hand dataframe\n", + "This is commonly referred to as a polyfill operation." + ] }, { "cell_type": "code", - "source": [ - "from mosaic import grid_polyfill\n\nindexed_neighbourhoods = (\n neighbourhoods\n .select(\"*\", grid_polyfill(\"geometry\", lit(resolution)).alias(\"ix_set\"))\n .drop(\"geometry\")\n)\n\nindexed_neighbourhoods.show()" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "06e1f722-8530-47c2-aec5-f0a6cb4f9301" + "nuid": "06e1f722-8530-47c2-aec5-f0a6cb4f9301", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
+-------------+-----------+--------+-------------------+--------------------+\n| borough|location_id|objectid| zone| ix_set|\n+-------------+-----------+--------+-------------------+--------------------+\n| Brooklyn| 123| 123| Homecrest|[6177331514226769...|\n| Manhattan| 153| 153| Marble Hill|[6177331229858201...|\n| Brooklyn| 112| 112| Greenpoint|[6177331237832622...|\n| Manhattan| 233| 233|UN/Turtle Bay South|[6177331238679347...|\n| Manhattan| 43| 43| Central Park|[6177331225792348...|\n| Queens| 201| 201| Rockaway Park|[6177331357831659...|\n| Queens| 131| 131| Jamaica Estates|[6177331242658693...|\n| Brooklyn| 111| 111|Green-Wood Cemetery|[6177331522277212...|\n| Queens| 226| 226| Sunnyside|[6177331238566625...|\n| Queens| 129| 129| Jackson Heights|[6177331243222302...|\n| Manhattan| 120| 120| Highbridge Park|[6177331231976325...|\n| Brooklyn| 76| 76| East New York|[6177331236938711...|\n| Manhattan| 24| 24| Bloomingdale|[6177331226458193...|\n| Manhattan| 202| 202| Roosevelt Island|[6177331237777571...|\n| Manhattan| 100| 100| Garment District|[6177331509717893...|\n|Staten Island| 251| 251| Westerleigh|[6177331466128588...|\n| Manhattan| 74| 74| East Harlem North|[6177331226508001...|\n| Queens| 98| 98| Fresh Meadows|[6177331242448977...|\n| Manhattan| 211| 211| SoHo|[6177331510784819...|\n| Bronx| 174| 174| Norwood|[6177331205497159...|\n+-------------+-----------+--------+-------------------+--------------------+\nonly showing top 20 rows\n\n
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
+-------------+-----------+--------+-------------------+--------------------+\n borough|location_id|objectid| zone| ix_set|\n+-------------+-----------+--------+-------------------+--------------------+\n Brooklyn| 123| 123| Homecrest|[6177331514226769...|\n Manhattan| 153| 153| Marble Hill|[6177331229858201...|\n Brooklyn| 112| 112| Greenpoint|[6177331237832622...|\n Manhattan| 233| 233|UN/Turtle Bay South|[6177331238679347...|\n Manhattan| 43| 43| Central Park|[6177331225792348...|\n Queens| 201| 201| Rockaway Park|[6177331357831659...|\n Queens| 131| 131| Jamaica Estates|[6177331242658693...|\n Brooklyn| 111| 111|Green-Wood Cemetery|[6177331522277212...|\n Queens| 226| 226| Sunnyside|[6177331238566625...|\n Queens| 129| 129| Jackson Heights|[6177331243222302...|\n Manhattan| 120| 120| Highbridge Park|[6177331231976325...|\n Brooklyn| 76| 76| East New York|[6177331236938711...|\n Manhattan| 24| 24| Bloomingdale|[6177331226458193...|\n Manhattan| 202| 202| Roosevelt Island|[6177331237777571...|\n Manhattan| 100| 100| Garment District|[6177331509717893...|\nStaten Island| 251| 251| Westerleigh|[6177331466128588...|\n Manhattan| 74| 74| East Harlem North|[6177331226508001...|\n Queens| 98| 98| Fresh Meadows|[6177331242448977...|\n Manhattan| 211| 211| SoHo|[6177331510784819...|\n Bronx| 174| 174| Norwood|[6177331205497159...|\n+-------------+-----------+--------+-------------------+--------------------+\nonly showing top 20 rows\n\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "from mosaic import grid_polyfill\n", + "\n", + "indexed_neighbourhoods = (\n", + " neighbourhoods\n", + " .select(\"*\", grid_polyfill(\"geometry\", lit(resolution)).alias(\"ix_set\"))\n", + " .drop(\"geometry\")\n", + ")\n", + "\n", + "indexed_neighbourhoods.show()" + ] }, { "cell_type": "markdown", - "source": [ - "## 5. ‘Explode’ the polygon index dataframe, such that each polygon index becomes a row in a new dataframe." - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "be2c88fd-9307-4b9a-ada8-fd714142d2d0" + "nuid": "be2c88fd-9307-4b9a-ada8-fd714142d2d0", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "## 5. ‘Explode’ the polygon index dataframe, such that each polygon index becomes a row in a new dataframe." + ] }, { "cell_type": "code", - "source": [ - "exploded_indexed_neighbourhoods = (\n indexed_neighbourhoods\n .withColumn(\"ix\", explode(\"ix_set\"))\n .drop(\"ix_set\")\n)\n\nexploded_indexed_neighbourhoods.show()" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "132de00e-753f-49d6-ab3f-82517eef296a" + "nuid": "132de00e-753f-49d6-ab3f-82517eef296a", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
+--------+-----------+--------+---------+------------------+\n| borough|location_id|objectid| zone| ix|\n+--------+-----------+--------+---------+------------------+\n|Brooklyn| 123| 123|Homecrest|617733151422676991|\n|Brooklyn| 123| 123|Homecrest|617733151503417343|\n|Brooklyn| 123| 123|Homecrest|617733151502893055|\n|Brooklyn| 123| 123|Homecrest|617733151502368767|\n|Brooklyn| 123| 123|Homecrest|617733151492407295|\n|Brooklyn| 123| 123|Homecrest|617733151488737279|\n|Brooklyn| 123| 123|Homecrest|617733151484542975|\n|Brooklyn| 123| 123|Homecrest|617733151484018687|\n|Brooklyn| 123| 123|Homecrest|617733151483494399|\n|Brooklyn| 123| 123|Homecrest|617733151425560575|\n|Brooklyn| 123| 123|Homecrest|617733151424511999|\n|Brooklyn| 123| 123|Homecrest|617733151423463423|\n|Brooklyn| 123| 123|Homecrest|617733151511019519|\n|Brooklyn| 123| 123|Homecrest|617733151505776639|\n|Brooklyn| 123| 123|Homecrest|617733151505252351|\n|Brooklyn| 123| 123|Homecrest|617733151504203775|\n|Brooklyn| 123| 123|Homecrest|617733151503679487|\n|Brooklyn| 123| 123|Homecrest|617733151503155199|\n|Brooklyn| 123| 123|Homecrest|617733151502630911|\n|Brooklyn| 123| 123|Homecrest|617733151502106623|\n+--------+-----------+--------+---------+------------------+\nonly showing top 20 rows\n\n
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
+--------+-----------+--------+---------+------------------+\n borough|location_id|objectid| zone| ix|\n+--------+-----------+--------+---------+------------------+\nBrooklyn| 123| 123|Homecrest|617733151422676991|\nBrooklyn| 123| 123|Homecrest|617733151503417343|\nBrooklyn| 123| 123|Homecrest|617733151502893055|\nBrooklyn| 123| 123|Homecrest|617733151502368767|\nBrooklyn| 123| 123|Homecrest|617733151492407295|\nBrooklyn| 123| 123|Homecrest|617733151488737279|\nBrooklyn| 123| 123|Homecrest|617733151484542975|\nBrooklyn| 123| 123|Homecrest|617733151484018687|\nBrooklyn| 123| 123|Homecrest|617733151483494399|\nBrooklyn| 123| 123|Homecrest|617733151425560575|\nBrooklyn| 123| 123|Homecrest|617733151424511999|\nBrooklyn| 123| 123|Homecrest|617733151423463423|\nBrooklyn| 123| 123|Homecrest|617733151511019519|\nBrooklyn| 123| 123|Homecrest|617733151505776639|\nBrooklyn| 123| 123|Homecrest|617733151505252351|\nBrooklyn| 123| 123|Homecrest|617733151504203775|\nBrooklyn| 123| 123|Homecrest|617733151503679487|\nBrooklyn| 123| 123|Homecrest|617733151503155199|\nBrooklyn| 123| 123|Homecrest|617733151502630911|\nBrooklyn| 123| 123|Homecrest|617733151502106623|\n+--------+-----------+--------+---------+------------------+\nonly showing top 20 rows\n\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "exploded_indexed_neighbourhoods = (\n", + " indexed_neighbourhoods\n", + " .withColumn(\"ix\", explode(\"ix_set\"))\n", + " .drop(\"ix_set\")\n", + ")\n", + "\n", + "exploded_indexed_neighbourhoods.show()" + ] }, { "cell_type": "markdown", - "source": [ - "## 6. Join the new left- and right-hand dataframes directly on the index." - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "a369d19f-2917-4320-ab4a-e701953db7b5" + "nuid": "a369d19f-2917-4320-ab4a-e701953db7b5", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "## 6. Join the new left- and right-hand dataframes directly on the index." + ] }, { "cell_type": "code", - "source": [ - "joined_df = (\n indexed_trips.alias(\"t\")\n .join(exploded_indexed_neighbourhoods.alias(\"n\"), on=\"ix\", how=\"inner\"))\njoined_df.count()" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "09a92bb5-4c72-4757-8176-01e14dbc3504" + "nuid": "09a92bb5-4c72-4757-8176-01e14dbc3504", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
Out[25]: 4934937
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
Out[25]: 4934937
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "joined_df = (\n", + " indexed_trips.alias(\"t\")\n", + " .join(exploded_indexed_neighbourhoods.alias(\"n\"), on=\"ix\", how=\"inner\"))\n", + "joined_df.count()" + ] }, { "cell_type": "markdown", - "source": [ - "## Final notes\nMosaic provides support for Uber’s H3 spatial indexing library as a core part of the API, but we plan to add support for other index systems, including S2 and British National Grid in due course." - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "9825374a-d5d3-407f-9c0d-94c7514edacf" + "nuid": "9825374a-d5d3-407f-9c0d-94c7514edacf", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "## Final notes\n", + "Mosaic provides support for Uber’s H3 spatial indexing library as a core part of the API, but we plan to add support for other index systems, including S2 and British National Grid in due course." + ] } ], "metadata": { "application/vnd.databricks.v1+notebook": { - "notebookName": "spatial-indexes", "dashboards": [], + "language": "python", "notebookMetadata": { "pythonIndentUnit": 2 }, - "language": "python", - "widgets": {}, - "notebookOrigID": 4066942804256643 + "notebookName": "grid-indexes", + "notebookOrigID": 2666786534675200, + "widgets": {} } }, "nbformat": 4, "nbformat_minor": 0 -} \ No newline at end of file +} diff --git a/docs/source/usage/kepler.ipynb b/docs/source/usage/kepler.ipynb index 3c3ded2c8..87d685ccc 100644 --- a/docs/source/usage/kepler.ipynb +++ b/docs/source/usage/kepler.ipynb @@ -2,8 +2,35 @@ "cells": [ { "cell_type": "markdown", +<<<<<<< HEAD +======= +<<<<<<< HEAD + "source": [ + "# Kepler visualizations" + ], +>>>>>>> 70cc19a1 (Documentation updates) + "metadata": { + "application/vnd.databricks.v1+cell": { + "inputWidgets": {}, + "nuid": "5dbca100-ca8d-4cb3-be39-72d2aeb85077", + "showTitle": false, + "title": "" + } + }, + "source": [ + "# Kepler visualizations" + ] + }, + { + "cell_type": "markdown", +<<<<<<< HEAD "metadata": { "application/vnd.databricks.v1+cell": { +======= +======= + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, "inputWidgets": {}, "nuid": "5dbca100-ca8d-4cb3-be39-72d2aeb85077", "showTitle": false, @@ -18,12 +45,18 @@ "cell_type": "markdown", "metadata": { "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, +>>>>>>> 70cc19a1 (Documentation updates) "inputWidgets": {}, "nuid": "a2712c52-9c84-427e-bff8-814b9fa35d86", "showTitle": false, "title": "" } }, +<<<<<<< HEAD +======= +>>>>>>> efd95270 (Documentation updates) +>>>>>>> 70cc19a1 (Documentation updates) "source": [ "You can use the `%%mosaic_kepler` magic function to visualise data using [Kepler.gl](https://kepler.gl/).\n", "\n", @@ -44,7 +77,20 @@ "```\n", "\n", "This magic function is only available in python. It can be used from notebooks with other default languages by storing the intermediate result in a temporary view, and then adding a python cell that uses the `mosaic_kepler` with the temporary view created from another language." +<<<<<<< HEAD ] +======= +<<<<<<< HEAD + ], + "metadata": { + "application/vnd.databricks.v1+cell": { + "title": "", + "showTitle": false, + "inputWidgets": {}, + "nuid": "a2712c52-9c84-427e-bff8-814b9fa35d86" + } + } +>>>>>>> 70cc19a1 (Documentation updates) }, { "cell_type": "markdown", @@ -114,15 +160,60 @@ }, { "cell_type": "code", +<<<<<<< HEAD "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { +======= +======= + ] + }, + { + "cell_type": "markdown", + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "573fa05d-6dfd-414e-84c4-6f3618540b60", + "showTitle": false, + "title": "" + } + }, + "source": [ + "## Examples" + ] + }, + { + "cell_type": "code", + "execution_count": 0, + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "c73e5956-dec2-4206-be2d-99bca42d2c93", + "showTitle": false, + "title": "" + } + }, + "outputs": [], + "source": [ + "%pip install databricks-mosaic --quiet" + ] + }, + { + "cell_type": "code", + "execution_count": 0, + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, +>>>>>>> 70cc19a1 (Documentation updates) "inputWidgets": {}, "nuid": "a18dcbe7-1a01-45f2-a961-dd0f0284cf9c", "showTitle": false, "title": "" } }, +<<<<<<< HEAD "outputs": [ { "data": { @@ -158,14 +249,23 @@ "output_type": "display_data" } ], +======= + "outputs": [], +>>>>>>> efd95270 (Documentation updates) +>>>>>>> 70cc19a1 (Documentation updates) "source": [ "from pyspark.sql.functions import *\n", "import mosaic as mos\n", "mos.enable_mosaic(spark, dbutils)" +<<<<<<< HEAD ] }, { "cell_type": "markdown", +======= +<<<<<<< HEAD + ], +>>>>>>> 70cc19a1 (Documentation updates) "metadata": { "application/vnd.databricks.v1+cell": { "inputWidgets": {}, @@ -224,17 +324,75 @@ "output_type": "display_data" } ], +<<<<<<< HEAD +======= + "execution_count": 0 + }, + { + "cell_type": "markdown", + "source": [ + "### Download example shapes" + ], + "metadata": { + "application/vnd.databricks.v1+cell": { + "title": "", + "showTitle": false, + "inputWidgets": {}, + "nuid": "2a997c83-5aff-4463-8c64-a1dc7a55d7d3" + } + } + }, + { + "cell_type": "code", +======= + ] + }, + { + "cell_type": "markdown", + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "2a997c83-5aff-4463-8c64-a1dc7a55d7d3", + "showTitle": false, + "title": "" + } + }, + "source": [ + "### Download example shapes" + ] + }, + { + "cell_type": "code", + "execution_count": 0, + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "93072805-dd95-4518-9099-47ca7e85b62d", + "showTitle": false, + "title": "" + } + }, + "outputs": [], +>>>>>>> efd95270 (Documentation updates) +>>>>>>> 70cc19a1 (Documentation updates) "source": [ "import requests\n", "\n", "req = requests.get('https://data.cityofnewyork.us/api/geospatial/d3c5-ddgc?method=export&format=GeoJSON')\n", "with open('/dbfs/tmp/nyc_taxi_zones.geojson', 'wb') as f:\n", " f.write(req.content)" +<<<<<<< HEAD ] }, { "cell_type": "code", "execution_count": null, +======= +<<<<<<< HEAD + ], +>>>>>>> 70cc19a1 (Documentation updates) "metadata": { "application/vnd.databricks.v1+cell": { "inputWidgets": {}, @@ -284,6 +442,30 @@ "output_type": "display_data" } ], +<<<<<<< HEAD +======= + "execution_count": 0 + }, + { + "cell_type": "code", +======= + ] + }, + { + "cell_type": "code", + "execution_count": 0, + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "eb739b60-a368-4dc9-9cf7-2f4346ffb06a", + "showTitle": false, + "title": "" + } + }, + "outputs": [], +>>>>>>> efd95270 (Documentation updates) +>>>>>>> 70cc19a1 (Documentation updates) "source": [ "neighbourhoods = (\n", " spark.read\n", @@ -309,10 +491,15 @@ ")\n", "\n", "neighbourhoods.show()" +<<<<<<< HEAD ] }, { "cell_type": "markdown", +======= +<<<<<<< HEAD + ], +>>>>>>> 70cc19a1 (Documentation updates) "metadata": { "application/vnd.databricks.v1+cell": { "inputWidgets": {}, @@ -997,6 +1184,297 @@ "output_type": "display_data" } ], +<<<<<<< HEAD +======= + "execution_count": 0 + }, + { + "cell_type": "markdown", + "source": [ + "![mosaic kepler map example geometry](../images/kepler-1.png)" + ], + "metadata": { + "application/vnd.databricks.v1+cell": { + "title": "", + "showTitle": false, + "inputWidgets": {}, + "nuid": "9d7dd0c3-64c5-43bb-aecd-287635929970" + } + } + }, + { + "cell_type": "markdown", + "source": [ + "### Plot H3 indexes" + ], + "metadata": { + "application/vnd.databricks.v1+cell": { + "title": "", + "showTitle": false, + "inputWidgets": {}, + "nuid": "d34979d6-45e5-4a6f-a12a-52e46cb1e946" + } + } + }, + { + "cell_type": "code", +======= + ] + }, + { + "cell_type": "markdown", + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "0c74d4f1-dbe6-4117-b752-4761b6a5d692", + "showTitle": false, + "title": "" + } + }, + "source": [ + "### Plot geometries from Spark dataset" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "4109bf9f-479e-47f1-aea5-fb881136c49e", + "showTitle": false, + "title": "" + } + }, + "source": [ + "#### Internal geometry type" + ] + }, + { + "cell_type": "code", + "execution_count": 0, + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "99be3b7a-3678-44bc-8d4a-ac35346cd724", + "showTitle": false, + "title": "" + } + }, + "outputs": [], + "source": [ + "%%mosaic_kepler\n", + "neighbourhoods \"geom_internal\" \"geometry\"" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "34b3328f-2966-4b6f-859f-325eeec04619", + "showTitle": false, + "title": "" + } + }, + "source": [ + "![mosaic kepler map example geometry](../images/kepler-1.png)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "f6d1c289-b007-4e40-b354-4c59fef9c2c4", + "showTitle": false, + "title": "" + } + }, + "source": [ + "#### WKT geometry type" + ] + }, + { + "cell_type": "code", + "execution_count": 0, + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "84ee5934-ed9c-4ee8-affd-dd347f375eec", + "showTitle": false, + "title": "" + } + }, + "outputs": [], + "source": [ + "%%mosaic_kepler\n", + "neighbourhoods \"geom_wkt\" \"geometry\"" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "b81a56b6-3d7c-4c87-91b2-fecc2f77333c", + "showTitle": false, + "title": "" + } + }, + "source": [ + "![mosaic kepler map example geometry](../images/kepler-1.png)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "2acdd70b-decd-4654-8927-9f821e728864", + "showTitle": false, + "title": "" + } + }, + "source": [ + "#### WKB geometry type" + ] + }, + { + "cell_type": "code", + "execution_count": 0, + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "0ddd1cad-dabf-4a9d-9e89-4406e2fda6e4", + "showTitle": false, + "title": "" + } + }, + "outputs": [], + "source": [ + "%%mosaic_kepler\n", + "neighbourhoods \"geom_wkb\" \"geometry\"" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "0f0deddf-3408-470d-8e9a-a6d066d23697", + "showTitle": false, + "title": "" + } + }, + "source": [ + "![mosaic kepler map example geometry](../images/kepler-1.png)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "590cb32e-2b3f-4ad7-b609-ffef9ae17967", + "showTitle": false, + "title": "" + } + }, + "source": [ + "### Plot geometries from table/view" + ] + }, + { + "cell_type": "code", + "execution_count": 0, + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "782fcc2e-d868-4819-bb01-9c4ea4bafb36", + "showTitle": false, + "title": "" + } + }, + "outputs": [], + "source": [ + "neighbourhoods.createOrReplaceTempView(\"temp_view_neighbourhoods\")" + ] + }, + { + "cell_type": "code", + "execution_count": 0, + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "fe84fa59-998d-4b9a-82c5-f280393ae244", + "showTitle": false, + "title": "" + } + }, + "outputs": [], + "source": [ + "%%mosaic_kepler\n", + "\"temp_view_neighbourhoods\" \"geom_wkt\" \"geometry\"" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "9d7dd0c3-64c5-43bb-aecd-287635929970", + "showTitle": false, + "title": "" + } + }, + "source": [ + "![mosaic kepler map example geometry](../images/kepler-1.png)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "d34979d6-45e5-4a6f-a12a-52e46cb1e946", + "showTitle": false, + "title": "" + } + }, + "source": [ + "### Plot H3 indexes" + ] + }, + { + "cell_type": "code", + "execution_count": 0, + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "86cdd4f3-34c5-4365-8108-303ca63d97af", + "showTitle": false, + "title": "" + } + }, + "outputs": [], +>>>>>>> efd95270 (Documentation updates) +>>>>>>> 70cc19a1 (Documentation updates) "source": [ "neighbourhood_chips = (neighbourhoods\n", " .limit(1)\n", @@ -1005,11 +1483,16 @@ " )\n", "\n", "neighbourhood_chips.show()" +<<<<<<< HEAD ] }, { "cell_type": "code", "execution_count": null, +======= +<<<<<<< HEAD + ], +>>>>>>> 70cc19a1 (Documentation updates) "metadata": { "application/vnd.databricks.v1+cell": { "inputWidgets": {}, @@ -1256,25 +1739,133 @@ "showTitle": false, "title": "" } +<<<<<<< HEAD +======= + } +======= + ] + }, + { + "cell_type": "code", + "execution_count": 0, + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "84f6d1d7-b270-498b-b1fe-fe6d17e0e713", + "showTitle": false, + "title": "" + } + }, + "outputs": [], + "source": [ + "%%mosaic_kepler\n", + "neighbourhood_chips \"index_id\" \"h3\"" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "a9d921cb-5994-42a9-9518-626486cd1227", + "showTitle": false, + "title": "" + } + }, + "source": [ + "![mosaic kepler map example H3 indexes](../images/kepler-2.png)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "64cb0e7f-be2e-48f9-bea8-35a147e3f60c", + "showTitle": false, + "title": "" + } + }, + "source": [ + "### Plot H3 chips" + ] + }, + { + "cell_type": "code", + "execution_count": 0, + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "dbe03c2b-a2c2-4b17-a44c-16891e81d3d0", + "showTitle": false, + "title": "" + } + }, + "outputs": [], + "source": [ + "%%mosaic_kepler\n", + "neighbourhood_chips \"wkb\" \"geometry\"" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "application/vnd.databricks.v1+cell": { + "cellMetadata": {}, + "inputWidgets": {}, + "nuid": "6c8244f4-4c03-4d12-9d06-d5174ef94626", + "showTitle": false, + "title": "" + } +>>>>>>> 70cc19a1 (Documentation updates) }, "source": [ "![mosaic kepler map example h3 chips](../images/kepler-3.png)" ] +<<<<<<< HEAD +======= +>>>>>>> efd95270 (Documentation updates) +>>>>>>> 70cc19a1 (Documentation updates) } ], "metadata": { "application/vnd.databricks.v1+notebook": { +<<<<<<< HEAD +======= +<<<<<<< HEAD + "notebookName": "kepler", +>>>>>>> 70cc19a1 (Documentation updates) "dashboards": [], "language": "python", "notebookMetadata": { "pythonIndentUnit": 2 }, +<<<<<<< HEAD "notebookName": "kepler", "notebookOrigID": 2874007245243191, "widgets": {} }, "language_info": { "name": "python" +======= + "language": "python", + "widgets": {}, + "notebookOrigID": 2874007245243191 +======= + "dashboards": [], + "language": "python", + "notebookMetadata": { + "pythonIndentUnit": 2 + }, + "notebookName": "kepler", + "notebookOrigID": 2666786534675682, + "widgets": {} +>>>>>>> efd95270 (Documentation updates) +>>>>>>> 70cc19a1 (Documentation updates) } }, "nbformat": 4, diff --git a/docs/source/usage/quickstart.ipynb b/docs/source/usage/quickstart.ipynb index a643d848b..4eab9ee85 100644 --- a/docs/source/usage/quickstart.ipynb +++ b/docs/source/usage/quickstart.ipynb @@ -2,10235 +2,828 @@ "cells": [ { "cell_type": "markdown", - "source": [ - "# Quickstart notebook\nThe example code here shows how to get up and running with Mosaic using the Python API." - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "c247678b-66ab-4879-b331-bb0532826530" + "nuid": "c247678b-66ab-4879-b331-bb0532826530", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "# Quickstart notebook\n", + "The example code here shows how to get up and running with Mosaic using the Python API." + ] }, { "cell_type": "code", - "source": [ - "from pyspark.sql.functions import *" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "aaf89a78-8b1f-4e27-8cc8-3fa382ff8210" + "nuid": "aaf89a78-8b1f-4e27-8cc8-3fa382ff8210", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "from pyspark.sql.functions import *" + ] }, { "cell_type": "markdown", - "source": [ - "## Enable Mosaic in the notebook\nTo get started, you'll need to attach the python library to your cluster and execute the `enable_mosaic` function." - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "ec0b86d5-e58c-478a-824b-e1165fb267de" + "nuid": "ec0b86d5-e58c-478a-824b-e1165fb267de", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "## Enable Mosaic in the notebook\n", + "To get started, you'll need to attach the python library to your cluster and execute the `enable_mosaic` function." + ] }, { "cell_type": "code", - "source": [ - "from mosaic import enable_mosaic\nenable_mosaic(spark, dbutils)" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "ecb6237c-050b-47c6-98ed-6942a50472e3" + "nuid": "ecb6237c-050b-47c6-98ed-6942a50472e3", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "from mosaic import enable_mosaic\n", + "enable_mosaic(spark, dbutils)" + ] }, { "cell_type": "markdown", - "source": [ - "Mosaic has extra configuration options. Check the docs for more details." - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "d774f893-1152-41de-adde-bdad37e755f7" + "nuid": "d774f893-1152-41de-adde-bdad37e755f7", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "Mosaic has extra configuration options. Check the docs for more details." + ] }, { "cell_type": "code", - "source": [ - "help(enable_mosaic)" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "10776858-768a-450f-9281-6a00e80d04c2" + "nuid": "10776858-768a-450f-9281-6a00e80d04c2", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
Help on function enable_mosaic in module mosaic.api.enable:\n\nenable_mosaic(spark: pyspark.sql.session.SparkSession, dbutils=None) -> None\n Enable Mosaic functions.\n \n Use this function at the start of your workflow to ensure all of the required dependencies are installed and\n Mosaic is configured according to your needs.\n \n Parameters\n ----------\n spark : pyspark.sql.SparkSession\n The active SparkSession.\n dbutils : dbruntime.dbutils.DBUtils\n The dbutils object used for `display` and `displayHTML` functions.\n Optional, only applicable to Databricks users.\n \n Returns\n -------\n \n Notes\n -----\n Users can control various aspects of Mosaic's operation with the following Spark confs:\n \n - `spark.databricks.labs.mosaic.jar.autoattach`: 'true' (default) or 'false'\n Automatically attach the Mosaic JAR to the Databricks cluster? (Optional)\n - `spark.databricks.labs.mosaic.jar.location`\n Explicitly specify the path to the Mosaic JAR.\n (Optional and not required at all in a standard Databricks environment).\n - `spark.databricks.labs.mosaic.geometry.api`: 'OGC' (default) or 'JTS'\n Explicitly specify the underlying geometry library to use for spatial operations. (Optional)\n - `spark.databricks.labs.mosaic.index.system`: 'H3' (default)\n Explicitly specify the index system to use for optimized spatial joins. (Optional)\n\n
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
Help on function enable_mosaic in module mosaic.api.enable:\n\nenable_mosaic(spark: pyspark.sql.session.SparkSession, dbutils=None) -> None\n Enable Mosaic functions.\n \n Use this function at the start of your workflow to ensure all of the required dependencies are installed and\n Mosaic is configured according to your needs.\n \n Parameters\n ----------\n spark : pyspark.sql.SparkSession\n The active SparkSession.\n dbutils : dbruntime.dbutils.DBUtils\n The dbutils object used for `display` and `displayHTML` functions.\n Optional, only applicable to Databricks users.\n \n Returns\n -------\n \n Notes\n -----\n Users can control various aspects of Mosaic's operation with the following Spark confs:\n \n - `spark.databricks.labs.mosaic.jar.autoattach`: 'true' (default) or 'false'\n Automatically attach the Mosaic JAR to the Databricks cluster? (Optional)\n - `spark.databricks.labs.mosaic.jar.location`\n Explicitly specify the path to the Mosaic JAR.\n (Optional and not required at all in a standard Databricks environment).\n - `spark.databricks.labs.mosaic.geometry.api`: 'OGC' (default) or 'JTS'\n Explicitly specify the underlying geometry library to use for spatial operations. (Optional)\n - `spark.databricks.labs.mosaic.index.system`: 'H3' (default)\n Explicitly specify the index system to use for optimized spatial joins. (Optional)\n\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "help(enable_mosaic)" + ] }, { "cell_type": "markdown", - "source": [ - "## Geometry constructors and the Mosaic internal geometry format" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "ac3649d8-5561-4370-b5d3-a36bf5068fd1" + "nuid": "ac3649d8-5561-4370-b5d3-a36bf5068fd1", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "## Geometry constructors and the Mosaic internal geometry format" + ] }, { "cell_type": "markdown", - "source": [ - "Mosaic allows users to create new Point geometries from a pair of Spark DoubleType columns." - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "c9e64d22-ebbc-4ea7-97aa-2dd6e240c9ad" + "nuid": "c9e64d22-ebbc-4ea7-97aa-2dd6e240c9ad", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "Mosaic allows users to create new Point geometries from a pair of Spark DoubleType columns." + ] }, { "cell_type": "code", - "source": [ - "from mosaic import st_point\n\nlons = [-80., -80., -70., -70., -80.]\nlats = [ 35., 45., 45., 35., 35.]\n\nbounds_df = (\n spark\n .createDataFrame({\"lon\": lon, \"lat\": lat} for lon, lat in zip(lons, lats))\n .coalesce(1)\n .withColumn(\"point_geom\", st_point(\"lon\", \"lat\"))\n)\nbounds_df.show()" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "aa85d1b2-8ec1-4879-87b4-cf55eb12820c" + "nuid": "aa85d1b2-8ec1-4879-87b4-cf55eb12820c", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
+----+-----+--------------------+\n| lat| lon| point_geom|\n+----+-----+--------------------+\n|35.0|-80.0|{1, [[[-80.0, 35....|\n|45.0|-80.0|{1, [[[-80.0, 45....|\n|45.0|-70.0|{1, [[[-70.0, 45....|\n|35.0|-70.0|{1, [[[-70.0, 35....|\n|35.0|-80.0|{1, [[[-80.0, 35....|\n+----+-----+--------------------+\n\n
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
+----+-----+--------------------+\n lat| lon| point_geom|\n+----+-----+--------------------+\n35.0|-80.0|{1, [[[-80.0, 35....|\n45.0|-80.0|{1, [[[-80.0, 45....|\n45.0|-70.0|{1, [[[-70.0, 45....|\n35.0|-70.0|{1, [[[-70.0, 35....|\n35.0|-80.0|{1, [[[-80.0, 35....|\n+----+-----+--------------------+\n\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "from mosaic import st_point\n", + "\n", + "lons = [-80., -80., -70., -70., -80.]\n", + "lats = [ 35., 45., 45., 35., 35.]\n", + "\n", + "bounds_df = (\n", + " spark\n", + " .createDataFrame({\"lon\": lon, \"lat\": lat} for lon, lat in zip(lons, lats))\n", + " .coalesce(1)\n", + " .withColumn(\"point_geom\", st_point(\"lon\", \"lat\"))\n", + ")\n", + "bounds_df.show()" + ] }, { "cell_type": "markdown", - "source": [ - "Mosaic Point geometries can be aggregated into LineString and Polygon geometries using the respective constructors." - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "e0229f66-1fe7-4218-905f-6fe265fd2441" + "nuid": "e0229f66-1fe7-4218-905f-6fe265fd2441", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "Mosaic Point geometries can be aggregated into LineString and Polygon geometries using the respective constructors." + ] }, { "cell_type": "code", - "source": [ - "from mosaic import st_makeline\n\nbounds_df = (\n bounds_df\n .groupBy()\n .agg(collect_list(\"point_geom\").alias(\"bounding_coords\"))\n .select(st_makeline(\"bounding_coords\").alias(\"bounding_ring\"))\n)\nbounds_df.show()" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "c143e704-333d-4c15-800d-7dcefc924f71" + "nuid": "c143e704-333d-4c15-800d-7dcefc924f71", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
+--------------------+\n| bounding_ring|\n+--------------------+\n|{3, [[[-80.0, 35....|\n+--------------------+\n\n
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
+--------------------+\n bounding_ring|\n+--------------------+\n{3, [[[-80.0, 35....|\n+--------------------+\n\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "from mosaic import st_makeline\n", + "\n", + "bounds_df = (\n", + " bounds_df\n", + " .groupBy()\n", + " .agg(collect_list(\"point_geom\").alias(\"bounding_coords\"))\n", + " .select(st_makeline(\"bounding_coords\").alias(\"bounding_ring\"))\n", + ")\n", + "bounds_df.show()" + ] }, { "cell_type": "code", - "source": [ - "from mosaic import st_makepolygon\n\nbounds_df = bounds_df.select(st_makepolygon(\"bounding_ring\").alias(\"bounds\"))\nbounds_df.show()" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "0282fcbd-f9b1-4708-b015-dc9e0fac9072" + "nuid": "0282fcbd-f9b1-4708-b015-dc9e0fac9072", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
+--------------------+\n| bounds|\n+--------------------+\n|{5, [[[-80.0, 35....|\n+--------------------+\n\n
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
+--------------------+\n bounds|\n+--------------------+\n{5, [[[-80.0, 35....|\n+--------------------+\n\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "from mosaic import st_makepolygon\n", + "\n", + "bounds_df = bounds_df.select(st_makepolygon(\"bounding_ring\").alias(\"bounds\"))\n", + "bounds_df.show()" + ] }, { "cell_type": "markdown", - "source": [ - "## Geometry clipping without an index" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "459a46e8-e7cb-4bb0-a167-e359889d15a9" + "nuid": "459a46e8-e7cb-4bb0-a167-e359889d15a9", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "## Geometry clipping without an index" + ] }, { "cell_type": "markdown", - "source": [ - "Mosaic implements set intersection functions: contains, intersects, overlaps etc. Here you can see `st_contains` being used to clip points by a polygon geometry." - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "02d90b72-81b4-48a9-b117-b7976bca13aa" + "nuid": "02d90b72-81b4-48a9-b117-b7976bca13aa", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "Mosaic implements set intersection functions: contains, intersects, overlaps etc. Here you can see `st_contains` being used to clip points by a polygon geometry." + ] }, { "cell_type": "code", - "source": [ - "tripsTable = spark.table(\"delta.`/databricks-datasets/nyctaxi/tables/nyctaxi_yellow`\")" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "e5ec8d70-8d6b-44b9-ad2a-ef593e2011bd" + "nuid": "e5ec8d70-8d6b-44b9-ad2a-ef593e2011bd", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "tripsTable = spark.table(\"delta.`/databricks-datasets/nyctaxi/tables/nyctaxi_yellow`\")" + ] }, { "cell_type": "code", - "source": [ - "from mosaic import st_contains\ntrips = (\n tripsTable\n .limit(5_000_000)\n .repartition(sc.defaultParallelism * 20)\n .drop(\"vendorId\", \"rateCodeId\", \"store_and_fwd_flag\", \"payment_type\")\n .withColumn(\"pickup_geom\", st_point(\"pickup_longitude\", \"pickup_latitude\"))\n .withColumn(\"dropoff_geom\", st_point(\"dropoff_longitude\", \"dropoff_latitude\"))\n .crossJoin(bounds_df)\n .where(st_contains(\"bounds\", \"pickup_geom\"))\n .where(st_contains(\"bounds\", \"dropoff_geom\"))\n .cache()\n)" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "ad0e72a9-4e2e-45fe-ba39-d2933859154d" + "nuid": "ad0e72a9-4e2e-45fe-ba39-d2933859154d", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "from mosaic import st_contains\n", + "trips = (\n", + " tripsTable\n", + " .limit(5_000_000)\n", + " .repartition(sc.defaultParallelism * 20)\n", + " .drop(\"vendorId\", \"rateCodeId\", \"store_and_fwd_flag\", \"payment_type\")\n", + " .withColumn(\"pickup_geom\", st_point(\"pickup_longitude\", \"pickup_latitude\"))\n", + " .withColumn(\"dropoff_geom\", st_point(\"dropoff_longitude\", \"dropoff_latitude\"))\n", + " .crossJoin(bounds_df)\n", + " .where(st_contains(\"bounds\", \"pickup_geom\"))\n", + " .where(st_contains(\"bounds\", \"dropoff_geom\"))\n", + " .cache()\n", + ")" + ] }, { "cell_type": "code", - "source": [ - "trips.show()" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "9aa22657-47c6-4de6-8e48-733a150b00a0" + "nuid": "9aa22657-47c6-4de6-8e48-733a150b00a0", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------+-----------------+----------------+-----------+-----+-------+----------+------------+------------+--------------------+--------------------+--------------------+\n|vendor_id| pickup_datetime| dropoff_datetime|passenger_count|trip_distance|pickup_longitude|pickup_latitude|rate_code_id|dropoff_longitude|dropoff_latitude|fare_amount|extra|mta_tax|tip_amount|tolls_amount|total_amount| pickup_geom| dropoff_geom| bounds|\n+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------+-----------------+----------------+-----------+-----+-------+----------+------------+------------+--------------------+--------------------+--------------------+\n| CMT|2009-10-12 19:35:21|2009-10-12 19:44:36| 1| 1.4| -74.000018| 40.720534| null| -73.995008| 40.734112| 6.5| 0.0| null| 0.0| 0.0| 6.5|{1, [[[-74.000018...|{1, [[[-73.995008...|{5, [[[-80.0, 35....|\n| CMT|2010-09-02 07:37:40|2010-09-02 07:53:17| 1| 6.8| -74.016477| 40.70971| 1| -73.97509| 40.759392| 17.3| 0.0| 0.5| 1.0| 0.0| 18.8|{1, [[[-74.016477...|{1, [[[-73.97509,...|{5, [[[-80.0, 35....|\n| CMT|2009-11-28 13:00:00|2009-11-28 13:01:57| 1| 0.5| -73.999666| 40.733384| null| -73.991751| 40.730273| 3.3| 0.0| 0.5| 0.0| 0.0| 3.8|{1, [[[-73.999666...|{1, [[[-73.991751...|{5, [[[-80.0, 35....|\n| CMT|2009-11-30 07:49:13|2009-11-30 08:12:38| 1| 12.9| -74.007033| 40.734931| null| -73.87277| 40.774319| 29.7| 0.0| 0.5| 6.04| 0.0| 36.24|{1, [[[-74.007033...|{1, [[[-73.87277,...|{5, [[[-80.0, 35....|\n| VTS|2009-01-03 01:55:00|2009-01-03 02:04:00| 5| 2.09| -73.993217| 40.72416| null| -73.9944| 40.74624| 7.7| 0.5| null| 0.0| 0.0| 8.2|{1, [[[-73.993217...|{1, [[[-73.9944, ...|{5, [[[-80.0, 35....|\n| VTS|2009-11-09 17:21:00|2009-11-09 17:32:00| 1| 5.19| -74.010003| 40.707022| null| -73.963853| 40.75677| 13.3| 1.0| 0.5| 0.0| 0.0| 14.8|{1, [[[-74.010003...|{1, [[[-73.963853...|{5, [[[-80.0, 35....|\n| VTS|2010-06-09 01:13:00|2010-06-09 01:27:00| 1| 8.09| -74.008697| 40.707335| 1| -73.949605| 40.78761| 19.7| 0.5| 0.5| 0.0| 0.0| 20.7|{1, [[[-74.008697...|{1, [[[-73.949605...|{5, [[[-80.0, 35....|\n| VTS|2009-01-28 18:36:00|2009-01-28 18:50:00| 1| 2.92| -74.008823| 40.72089| null| -73.994055| 40.747588| 10.1| 1.0| null| 3.0| 0.0| 14.1|{1, [[[-74.008823...|{1, [[[-73.994055...|{5, [[[-80.0, 35....|\n| CMT|2010-06-02 14:21:05|2010-06-02 14:28:43| 1| 5.0| -74.011653| 40.702604| 1| -73.971851| 40.749807| 12.5| 0.0| 0.5| 2.6| 0.0| 15.6|{1, [[[-74.011653...|{1, [[[-73.971851...|{5, [[[-80.0, 35....|\n| VTS|2009-11-26 15:38:00|2009-11-26 15:47:00| 5| 2.36| -73.997208| 40.722108| null| -73.99647| 40.748185| 8.1| 0.0| 0.5| 2.0| 0.0| 10.6|{1, [[[-73.997208...|{1, [[[-73.99647,...|{5, [[[-80.0, 35....|\n| CMT|2009-12-07 10:36:06|2009-12-07 10:42:26| 1| 1.3| -73.99067| 40.73065| null| -73.975539| 40.741353| 6.1| 0.0| 0.5| 0.0| 0.0| 6.6|{1, [[[-73.99067,...|{1, [[[-73.975539...|{5, [[[-80.0, 35....|\n| CMT|2009-11-15 09:11:42|2009-11-15 09:26:34| 1| 3.7| -73.990065| 40.687341| null| -73.999566| 40.730219| 12.1| 0.0| 0.5| 0.0| 0.0| 12.6|{1, [[[-73.990065...|{1, [[[-73.999566...|{5, [[[-80.0, 35....|\n| CMT|2010-06-27 17:27:25|2010-06-27 17:47:44| 2| 5.0| -74.008459| 40.712233| 1| -73.9516| 40.718877| 15.7| 0.0| 0.5| 3.0| 0.0| 19.2|{1, [[[-74.008459...|{1, [[[-73.9516, ...|{5, [[[-80.0, 35....|\n| VTS|2009-01-30 21:05:00|2009-01-30 21:34:00| 1| 5.99| -73.994397| 40.694843| null| -73.921257| 40.685107| 18.9| 0.5| null| 0.6| 0.0| 20.0|{1, [[[-73.994397...|{1, [[[-73.921257...|{5, [[[-80.0, 35....|\n| CMT|2009-01-14 07:43:53|2009-01-14 07:55:14| 2| 2.9| -74.007602| 40.725098| null| -73.977846| 40.751608| 9.3| 0.0| null| 0.0| 0.0| 9.3|{1, [[[-74.007602...|{1, [[[-73.977846...|{5, [[[-80.0, 35....|\n| VTS|2010-05-11 16:33:00|2010-05-11 16:41:00| 1| 0.99| -74.011205| 40.715468| 1| -74.011072| 40.708882| 5.7| 1.0| 0.5| 0.0| 0.0| 7.2|{1, [[[-74.011205...|{1, [[[-74.011072...|{5, [[[-80.0, 35....|\n| VTS|2009-01-19 09:43:00|2009-01-19 09:47:00| 1| 0.56| -73.991123| 40.731297| null| -73.98681| 40.736703| 4.1| 0.0| null| 0.0| 0.0| 4.1|{1, [[[-73.991123...|{1, [[[-73.98681,...|{5, [[[-80.0, 35....|\n| CMT|2009-11-14 15:46:36|2009-11-14 15:51:01| 2| 1.2| -74.008448| 40.734271| null| -74.006075| 40.720237| 4.9| 0.0| 0.5| 0.0| 0.0| 5.4|{1, [[[-74.008448...|{1, [[[-74.006075...|{5, [[[-80.0, 35....|\n| CMT|2009-11-17 16:50:35|2009-11-17 17:01:34| 1| 1.6| -74.00679| 40.730546| null| -73.9904| 40.73815| 7.7| 1.0| 0.5| 1.5| 0.0| 10.7|{1, [[[-74.00679,...|{1, [[[-73.9904, ...|{5, [[[-80.0, 35....|\n| CMT|2009-11-30 23:30:43|2009-11-30 23:42:54| 2| 2.9| -74.009605| 40.710155| null| -73.993875| 40.741554| 10.1| 0.5| 0.5| 2.22| 0.0| 13.32|{1, [[[-74.009605...|{1, [[[-73.993875...|{5, [[[-80.0, 35....|\n+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------+-----------------+----------------+-----------+-----+-------+----------+------------+------------+--------------------+--------------------+--------------------+\nonly showing top 20 rows\n\n
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------+-----------------+----------------+-----------+-----+-------+----------+------------+------------+--------------------+--------------------+--------------------+\nvendor_id| pickup_datetime| dropoff_datetime|passenger_count|trip_distance|pickup_longitude|pickup_latitude|rate_code_id|dropoff_longitude|dropoff_latitude|fare_amount|extra|mta_tax|tip_amount|tolls_amount|total_amount| pickup_geom| dropoff_geom| bounds|\n+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------+-----------------+----------------+-----------+-----+-------+----------+------------+------------+--------------------+--------------------+--------------------+\n CMT|2009-10-12 19:35:21|2009-10-12 19:44:36| 1| 1.4| -74.000018| 40.720534| null| -73.995008| 40.734112| 6.5| 0.0| null| 0.0| 0.0| 6.5|{1, [[[-74.000018...|{1, [[[-73.995008...|{5, [[[-80.0, 35....|\n CMT|2010-09-02 07:37:40|2010-09-02 07:53:17| 1| 6.8| -74.016477| 40.70971| 1| -73.97509| 40.759392| 17.3| 0.0| 0.5| 1.0| 0.0| 18.8|{1, [[[-74.016477...|{1, [[[-73.97509,...|{5, [[[-80.0, 35....|\n CMT|2009-11-28 13:00:00|2009-11-28 13:01:57| 1| 0.5| -73.999666| 40.733384| null| -73.991751| 40.730273| 3.3| 0.0| 0.5| 0.0| 0.0| 3.8|{1, [[[-73.999666...|{1, [[[-73.991751...|{5, [[[-80.0, 35....|\n CMT|2009-11-30 07:49:13|2009-11-30 08:12:38| 1| 12.9| -74.007033| 40.734931| null| -73.87277| 40.774319| 29.7| 0.0| 0.5| 6.04| 0.0| 36.24|{1, [[[-74.007033...|{1, [[[-73.87277,...|{5, [[[-80.0, 35....|\n VTS|2009-01-03 01:55:00|2009-01-03 02:04:00| 5| 2.09| -73.993217| 40.72416| null| -73.9944| 40.74624| 7.7| 0.5| null| 0.0| 0.0| 8.2|{1, [[[-73.993217...|{1, [[[-73.9944, ...|{5, [[[-80.0, 35....|\n VTS|2009-11-09 17:21:00|2009-11-09 17:32:00| 1| 5.19| -74.010003| 40.707022| null| -73.963853| 40.75677| 13.3| 1.0| 0.5| 0.0| 0.0| 14.8|{1, [[[-74.010003...|{1, [[[-73.963853...|{5, [[[-80.0, 35....|\n VTS|2010-06-09 01:13:00|2010-06-09 01:27:00| 1| 8.09| -74.008697| 40.707335| 1| -73.949605| 40.78761| 19.7| 0.5| 0.5| 0.0| 0.0| 20.7|{1, [[[-74.008697...|{1, [[[-73.949605...|{5, [[[-80.0, 35....|\n VTS|2009-01-28 18:36:00|2009-01-28 18:50:00| 1| 2.92| -74.008823| 40.72089| null| -73.994055| 40.747588| 10.1| 1.0| null| 3.0| 0.0| 14.1|{1, [[[-74.008823...|{1, [[[-73.994055...|{5, [[[-80.0, 35....|\n CMT|2010-06-02 14:21:05|2010-06-02 14:28:43| 1| 5.0| -74.011653| 40.702604| 1| -73.971851| 40.749807| 12.5| 0.0| 0.5| 2.6| 0.0| 15.6|{1, [[[-74.011653...|{1, [[[-73.971851...|{5, [[[-80.0, 35....|\n VTS|2009-11-26 15:38:00|2009-11-26 15:47:00| 5| 2.36| -73.997208| 40.722108| null| -73.99647| 40.748185| 8.1| 0.0| 0.5| 2.0| 0.0| 10.6|{1, [[[-73.997208...|{1, [[[-73.99647,...|{5, [[[-80.0, 35....|\n CMT|2009-12-07 10:36:06|2009-12-07 10:42:26| 1| 1.3| -73.99067| 40.73065| null| -73.975539| 40.741353| 6.1| 0.0| 0.5| 0.0| 0.0| 6.6|{1, [[[-73.99067,...|{1, [[[-73.975539...|{5, [[[-80.0, 35....|\n CMT|2009-11-15 09:11:42|2009-11-15 09:26:34| 1| 3.7| -73.990065| 40.687341| null| -73.999566| 40.730219| 12.1| 0.0| 0.5| 0.0| 0.0| 12.6|{1, [[[-73.990065...|{1, [[[-73.999566...|{5, [[[-80.0, 35....|\n CMT|2010-06-27 17:27:25|2010-06-27 17:47:44| 2| 5.0| -74.008459| 40.712233| 1| -73.9516| 40.718877| 15.7| 0.0| 0.5| 3.0| 0.0| 19.2|{1, [[[-74.008459...|{1, [[[-73.9516, ...|{5, [[[-80.0, 35....|\n VTS|2009-01-30 21:05:00|2009-01-30 21:34:00| 1| 5.99| -73.994397| 40.694843| null| -73.921257| 40.685107| 18.9| 0.5| null| 0.6| 0.0| 20.0|{1, [[[-73.994397...|{1, [[[-73.921257...|{5, [[[-80.0, 35....|\n CMT|2009-01-14 07:43:53|2009-01-14 07:55:14| 2| 2.9| -74.007602| 40.725098| null| -73.977846| 40.751608| 9.3| 0.0| null| 0.0| 0.0| 9.3|{1, [[[-74.007602...|{1, [[[-73.977846...|{5, [[[-80.0, 35....|\n VTS|2010-05-11 16:33:00|2010-05-11 16:41:00| 1| 0.99| -74.011205| 40.715468| 1| -74.011072| 40.708882| 5.7| 1.0| 0.5| 0.0| 0.0| 7.2|{1, [[[-74.011205...|{1, [[[-74.011072...|{5, [[[-80.0, 35....|\n VTS|2009-01-19 09:43:00|2009-01-19 09:47:00| 1| 0.56| -73.991123| 40.731297| null| -73.98681| 40.736703| 4.1| 0.0| null| 0.0| 0.0| 4.1|{1, [[[-73.991123...|{1, [[[-73.98681,...|{5, [[[-80.0, 35....|\n CMT|2009-11-14 15:46:36|2009-11-14 15:51:01| 2| 1.2| -74.008448| 40.734271| null| -74.006075| 40.720237| 4.9| 0.0| 0.5| 0.0| 0.0| 5.4|{1, [[[-74.008448...|{1, [[[-74.006075...|{5, [[[-80.0, 35....|\n CMT|2009-11-17 16:50:35|2009-11-17 17:01:34| 1| 1.6| -74.00679| 40.730546| null| -73.9904| 40.73815| 7.7| 1.0| 0.5| 1.5| 0.0| 10.7|{1, [[[-74.00679,...|{1, [[[-73.9904, ...|{5, [[[-80.0, 35....|\n CMT|2009-11-30 23:30:43|2009-11-30 23:42:54| 2| 2.9| -74.009605| 40.710155| null| -73.993875| 40.741554| 10.1| 0.5| 0.5| 2.22| 0.0| 13.32|{1, [[[-74.009605...|{1, [[[-73.993875...|{5, [[[-80.0, 35....|\n+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------+-----------------+----------------+-----------+-----+-------+----------+------------+------------+--------------------+--------------------+--------------------+\nonly showing top 20 rows\n\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "trips.show()" + ] }, { "cell_type": "markdown", - "source": [ - "## Read from GeoJson, compute some basic geometry attributes" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "c9353ac7-aa07-4d75-8b16-0031803ea31e" + "nuid": "c9353ac7-aa07-4d75-8b16-0031803ea31e", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "## Read from GeoJson, compute some basic geometry attributes" + ] }, { "cell_type": "markdown", - "source": [ - "You've seen how Mosaic can create geometries from Spark native data types but it also provides functions to translate Well Known Text (WKT), Well Known Binary (WKB) and GeoJSON representations to Mosaic geometries." - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "db5daf84-fc48-45ce-81a7-5af76b7b6010" + "nuid": "db5daf84-fc48-45ce-81a7-5af76b7b6010", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "You've seen how Mosaic can create geometries from Spark native data types but it also provides functions to translate Well Known Text (WKT), Well Known Binary (WKB) and GeoJSON representations to Mosaic geometries." + ] }, { "cell_type": "code", - "source": [ - "from mosaic import st_geomfromgeojson\n\ngeoJsonDF = (\n spark.read.format(\"json\")\n .load(\"dbfs:/FileStore/shared_uploads/stuart.lynn@databricks.com/NYC_Taxi_Zones.geojson\")\n .withColumn(\"geometry\", st_geomfromgeojson(to_json(col(\"geometry\"))))\n .select(\"properties.*\", \"geometry\")\n .drop(\"shape_area\", \"shape_leng\")\n)\n\ngeoJsonDF.show()" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "49505635-4e84-480e-9892-e3e816af2226" + "nuid": "49505635-4e84-480e-9892-e3e816af2226", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
+-------------+-----------+--------+--------------------+--------------------+\n| borough|location_id|objectid| zone| geometry|\n+-------------+-----------+--------+--------------------+--------------------+\n| EWR| 1| 1| Newark Airport|{6, [[[-74.184452...|\n| Queens| 2| 2| Jamaica Bay|{6, [[[-73.823375...|\n| Bronx| 3| 3|Allerton/Pelham G...|{6, [[[-73.847926...|\n| Manhattan| 4| 4| Alphabet City|{6, [[[-73.971774...|\n|Staten Island| 5| 5| Arden Heights|{6, [[[-74.174217...|\n|Staten Island| 6| 6|Arrochar/Fort Wad...|{6, [[[-74.063673...|\n| Queens| 7| 7| Astoria|{6, [[[-73.904136...|\n| Queens| 8| 8| Astoria Park|{6, [[[-73.923340...|\n| Queens| 9| 9| Auburndale|{6, [[[-73.785024...|\n| Manhattan| 24| 24| Bloomingdale|{6, [[[-73.959536...|\n| Queens| 10| 10| Baisley Park|{6, [[[-73.783266...|\n| Brooklyn| 11| 11| Bath Beach|{6, [[[-74.001098...|\n| Manhattan| 12| 12| Battery Park|{6, [[[-74.015657...|\n| Manhattan| 13| 13| Battery Park City|{6, [[[-74.012441...|\n| Bronx| 18| 18| Bedford Park|{6, [[[-73.885139...|\n| Brooklyn| 25| 25| Boerum Hill|{6, [[[-73.981552...|\n| Brooklyn| 14| 14| Bay Ridge|{6, [[[-74.034073...|\n| Queens| 15| 15|Bay Terrace/Fort ...|{6, [[[-73.777403...|\n| Brooklyn| 22| 22| Bensonhurst West|{6, [[[-73.992549...|\n|Staten Island| 23| 23|Bloomfield/Emerso...|{6, [[[-74.195686...|\n+-------------+-----------+--------+--------------------+--------------------+\nonly showing top 20 rows\n\n
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
+-------------+-----------+--------+--------------------+--------------------+\n borough|location_id|objectid| zone| geometry|\n+-------------+-----------+--------+--------------------+--------------------+\n EWR| 1| 1| Newark Airport|{6, [[[-74.184452...|\n Queens| 2| 2| Jamaica Bay|{6, [[[-73.823375...|\n Bronx| 3| 3|Allerton/Pelham G...|{6, [[[-73.847926...|\n Manhattan| 4| 4| Alphabet City|{6, [[[-73.971774...|\nStaten Island| 5| 5| Arden Heights|{6, [[[-74.174217...|\nStaten Island| 6| 6|Arrochar/Fort Wad...|{6, [[[-74.063673...|\n Queens| 7| 7| Astoria|{6, [[[-73.904136...|\n Queens| 8| 8| Astoria Park|{6, [[[-73.923340...|\n Queens| 9| 9| Auburndale|{6, [[[-73.785024...|\n Manhattan| 24| 24| Bloomingdale|{6, [[[-73.959536...|\n Queens| 10| 10| Baisley Park|{6, [[[-73.783266...|\n Brooklyn| 11| 11| Bath Beach|{6, [[[-74.001098...|\n Manhattan| 12| 12| Battery Park|{6, [[[-74.015657...|\n Manhattan| 13| 13| Battery Park City|{6, [[[-74.012441...|\n Bronx| 18| 18| Bedford Park|{6, [[[-73.885139...|\n Brooklyn| 25| 25| Boerum Hill|{6, [[[-73.981552...|\n Brooklyn| 14| 14| Bay Ridge|{6, [[[-74.034073...|\n Queens| 15| 15|Bay Terrace/Fort ...|{6, [[[-73.777403...|\n Brooklyn| 22| 22| Bensonhurst West|{6, [[[-73.992549...|\nStaten Island| 23| 23|Bloomfield/Emerso...|{6, [[[-74.195686...|\n+-------------+-----------+--------+--------------------+--------------------+\nonly showing top 20 rows\n\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "from mosaic import st_geomfromgeojson\n", + "\n", + "geoJsonDF = (\n", + " spark.read.format(\"json\")\n", + " .load(\"dbfs:/FileStore/shared_uploads/stuart.lynn@databricks.com/NYC_Taxi_Zones.geojson\")\n", + " .withColumn(\"geometry\", st_geomfromgeojson(to_json(col(\"geometry\"))))\n", + " .select(\"properties.*\", \"geometry\")\n", + " .drop(\"shape_area\", \"shape_leng\")\n", + ")\n", + "\n", + "geoJsonDF.show()" + ] }, { "cell_type": "markdown", - "source": [ - "Mosaic provides a number of functions for extracting the properties of geometries. Here are some that are relevant to Polygon geometries:" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "880c5fb4-c0b2-456b-9167-cb6327876dbd" + "nuid": "880c5fb4-c0b2-456b-9167-cb6327876dbd", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "Mosaic provides a number of functions for extracting the properties of geometries. Here are some that are relevant to Polygon geometries:" + ] }, { "cell_type": "code", - "source": [ - "from mosaic import st_area, st_length\n(\n geoJsonDF\n .withColumn(\"calculatedArea\", abs(st_area(\"geometry\")))\n .withColumn(\"calculatedLength\", st_length(\"geometry\"))\n .select(\"geometry\", \"calculatedArea\", \"calculatedLength\")\n).show()" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "939d35fd-d868-4e73-8338-3cb8142fffe9" + "nuid": "939d35fd-d868-4e73-8338-3cb8142fffe9", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
+--------------------+--------------------+--------------------+\n| geometry| calculatedArea| calculatedLength|\n+--------------------+--------------------+--------------------+\n|{6, [[[-74.184452...|7.823067885002562E-4| 0.1163574531886787|\n|{6, [[[-73.823375...|0.001422779097814599| 0.8431218810128789|\n|{6, [[[-73.847926...|3.144141568206508E-4| 0.08434110590105784|\n|{6, [[[-73.971774...| 7.94539194214528E-5| 0.03559982116907461|\n|{6, [[[-74.174217...|4.979574893632193E-4| 0.09214648985735088|\n|{6, [[[-74.063673...|4.046076598541714E-4| 0.12905753305870837|\n|{6, [[[-73.904136...|3.897879892739018...| 0.10741717112348065|\n|{6, [[[-73.923340...|2.658771690499072E-5|0.027590691195158792|\n|{6, [[[-73.785024...|3.384438031986125...| 0.0997840924710119|\n|{6, [[[-73.959536...|4.193691000471793E-5|0.034022318280805826|\n|{6, [[[-73.783266...|4.358238180810105...| 0.09983947941552027|\n|{6, [[[-74.001098...|2.029913532381591E-4| 0.07243934539937844|\n|{6, [[[-74.015657...|1.078595394871048...| 0.01759380786230351|\n|{6, [[[-74.012441...|4.874002713725957E-5| 0.04726120195723124|\n|{6, [[[-73.885139...|1.488501639473639E-4| 0.06979954985700013|\n|{6, [[[-73.981552...|1.241682673558662...| 0.04714581993191898|\n|{6, [[[-74.034073...|6.636590898190788E-4| 0.13622530962992194|\n|{6, [[[-73.777403...|4.588438269978550...| 0.1295148763926726|\n|{6, [[[-73.992549...|4.613649657988681...| 0.11844001178496774|\n|{6, [[[-74.195686...|0.002090822778761513| 0.3007595308122533|\n+--------------------+--------------------+--------------------+\nonly showing top 20 rows\n\n
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
+--------------------+--------------------+--------------------+\n geometry| calculatedArea| calculatedLength|\n+--------------------+--------------------+--------------------+\n{6, [[[-74.184452...|7.823067885002562E-4| 0.1163574531886787|\n{6, [[[-73.823375...|0.001422779097814599| 0.8431218810128789|\n{6, [[[-73.847926...|3.144141568206508E-4| 0.08434110590105784|\n{6, [[[-73.971774...| 7.94539194214528E-5| 0.03559982116907461|\n{6, [[[-74.174217...|4.979574893632193E-4| 0.09214648985735088|\n{6, [[[-74.063673...|4.046076598541714E-4| 0.12905753305870837|\n{6, [[[-73.904136...|3.897879892739018...| 0.10741717112348065|\n{6, [[[-73.923340...|2.658771690499072E-5|0.027590691195158792|\n{6, [[[-73.785024...|3.384438031986125...| 0.0997840924710119|\n{6, [[[-73.959536...|4.193691000471793E-5|0.034022318280805826|\n{6, [[[-73.783266...|4.358238180810105...| 0.09983947941552027|\n{6, [[[-74.001098...|2.029913532381591E-4| 0.07243934539937844|\n{6, [[[-74.015657...|1.078595394871048...| 0.01759380786230351|\n{6, [[[-74.012441...|4.874002713725957E-5| 0.04726120195723124|\n{6, [[[-73.885139...|1.488501639473639E-4| 0.06979954985700013|\n{6, [[[-73.981552...|1.241682673558662...| 0.04714581993191898|\n{6, [[[-74.034073...|6.636590898190788E-4| 0.13622530962992194|\n{6, [[[-73.777403...|4.588438269978550...| 0.1295148763926726|\n{6, [[[-73.992549...|4.613649657988681...| 0.11844001178496774|\n{6, [[[-74.195686...|0.002090822778761513| 0.3007595308122533|\n+--------------------+--------------------+--------------------+\nonly showing top 20 rows\n\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "from mosaic import st_area, st_length\n", + "(\n", + " geoJsonDF\n", + " .withColumn(\"calculatedArea\", abs(st_area(\"geometry\")))\n", + " .withColumn(\"calculatedLength\", st_length(\"geometry\"))\n", + " .select(\"geometry\", \"calculatedArea\", \"calculatedLength\")\n", + ").show()" + ] }, { "cell_type": "code", - "source": [ - "geoJsonDF.count()" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "920599a2-6c52-40e4-8547-a99f757406a6" + "nuid": "920599a2-6c52-40e4-8547-a99f757406a6", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
Out[12]: 263
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
Out[12]: 263
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "geoJsonDF.count()" + ] }, { "cell_type": "markdown", - "source": [ - "## Example point-in-poly with indexing" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "e0d34f66-7498-4a86-9dd3-b2664ebc535c" + "nuid": "e0d34f66-7498-4a86-9dd3-b2664ebc535c", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "## Example point-in-poly with indexing" + ] }, { "cell_type": "markdown", - "source": [ - "Mosaic has built-in support for the popular spatial indexing library, H3. The user has access to functions for generating point indices and the sets of indices covering polygons, allowing point-in-polygon joins to be transformed into deterministic SQL joins." - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "cb687786-0494-468d-b2e4-dbe847ea887b" + "nuid": "cb687786-0494-468d-b2e4-dbe847ea887b", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "Mosaic has built-in support for the popular spatial indexing library, H3. The user has access to functions for generating point indices and the sets of indices covering polygons, allowing point-in-polygon joins to be transformed into deterministic SQL joins." + ] }, { "cell_type": "code", - "source": [ - "from mosaic import grid_longlatascellid\n\ntrips_with_geom = (\n trips\n .withColumn(\"pickup_h3\", grid_longlatascellid(lon=\"pickup_longitude\", lat=\"pickup_latitude\", resolution=lit(10)))\n .withColumn(\"dropoff_h3\", grid_longlatascellid(lon=\"dropoff_longitude\", lat=\"dropoff_latitude\", resolution=lit(10)))\n)\n\ntrips_with_geom.show()" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "0ef8a536-fda8-4a9a-a2e6-58ad0c39cc8b" + "nuid": "0ef8a536-fda8-4a9a-a2e6-58ad0c39cc8b", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------+-----------------+----------------+-----------+-----+-------+----------+------------+------------+--------------------+--------------------+--------------------+------------------+------------------+\n|vendor_id| pickup_datetime| dropoff_datetime|passenger_count|trip_distance|pickup_longitude|pickup_latitude|rate_code_id|dropoff_longitude|dropoff_latitude|fare_amount|extra|mta_tax|tip_amount|tolls_amount|total_amount| pickup_geom| dropoff_geom| bounds| pickup_h3| dropoff_h3|\n+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------+-----------------+----------------+-----------+-----+-------+----------+------------+------------+--------------------+--------------------+--------------------+------------------+------------------+\n| CMT|2009-10-12 19:35:21|2009-10-12 19:44:36| 1| 1.4| -74.000018| 40.720534| null| -73.995008| 40.734112| 6.5| 0.0| null| 0.0| 0.0| 6.5|{1, [[[-74.000018...|{1, [[[-73.995008...|{5, [[[-80.0, 35....|622236750705885183|622236723432488959|\n| CMT|2010-09-02 07:37:40|2010-09-02 07:53:17| 1| 6.8| -74.016477| 40.70971| 1| -73.97509| 40.759392| 17.3| 0.0| 0.5| 1.0| 0.0| 18.8|{1, [[[-74.016477...|{1, [[[-73.97509,...|{5, [[[-80.0, 35....|622236750651359231|622236723493961727|\n| CMT|2009-11-28 13:00:00|2009-11-28 13:01:57| 1| 0.5| -73.999666| 40.733384| null| -73.991751| 40.730273| 3.3| 0.0| 0.5| 0.0| 0.0| 3.8|{1, [[[-73.999666...|{1, [[[-73.991751...|{5, [[[-80.0, 35....|622236723433046015|622236750715092991|\n| CMT|2009-11-30 07:49:13|2009-11-30 08:12:38| 1| 12.9| -74.007033| 40.734931| null| -73.87277| 40.774319| 29.7| 0.0| 0.5| 6.04| 0.0| 36.24|{1, [[[-74.007033...|{1, [[[-73.87277,...|{5, [[[-80.0, 35....|622236750719975423|622236724015759359|\n| VTS|2009-01-03 01:55:00|2009-01-03 02:04:00| 5| 2.09| -73.993217| 40.72416| null| -73.9944| 40.74624| 7.7| 0.5| null| 0.0| 0.0| 8.2|{1, [[[-73.993217...|{1, [[[-73.9944, ...|{5, [[[-80.0, 35....|622236750717878271|622236723431276543|\n| VTS|2009-11-09 17:21:00|2009-11-09 17:32:00| 1| 5.19| -74.010003| 40.707022| null| -73.963853| 40.75677| 13.3| 1.0| 0.5| 0.0| 0.0| 14.8|{1, [[[-74.010003...|{1, [[[-73.963853...|{5, [[[-80.0, 35....|622236750647689215|622236723504381951|\n| VTS|2010-06-09 01:13:00|2010-06-09 01:27:00| 1| 8.09| -74.008697| 40.707335| 1| -73.949605| 40.78761| 19.7| 0.5| 0.5| 0.0| 0.0| 20.7|{1, [[[-74.008697...|{1, [[[-73.949605...|{5, [[[-80.0, 35....|622236750647820287|622236722210668543|\n| VTS|2009-01-28 18:36:00|2009-01-28 18:50:00| 1| 2.92| -74.008823| 40.72089| null| -73.994055| 40.747588| 10.1| 1.0| null| 3.0| 0.0| 14.1|{1, [[[-74.008823...|{1, [[[-73.994055...|{5, [[[-80.0, 35....|622236750713159679|622236723439304703|\n| CMT|2010-06-02 14:21:05|2010-06-02 14:28:43| 1| 5.0| -74.011653| 40.702604| 1| -73.971851| 40.749807| 12.5| 0.0| 0.5| 2.6| 0.0| 15.6|{1, [[[-74.011653...|{1, [[[-73.971851...|{5, [[[-80.0, 35....|622236750639857663|622236723496026111|\n| VTS|2009-11-26 15:38:00|2009-11-26 15:47:00| 5| 2.36| -73.997208| 40.722108| null| -73.99647| 40.748185| 8.1| 0.0| 0.5| 2.0| 0.0| 10.6|{1, [[[-73.997208...|{1, [[[-73.99647,...|{5, [[[-80.0, 35....|622236750706966527|622236723431309311|\n| CMT|2009-12-07 10:36:06|2009-12-07 10:42:26| 1| 1.3| -73.99067| 40.73065| null| -73.975539| 40.741353| 6.1| 0.0| 0.5| 0.0| 0.0| 6.6|{1, [[[-73.99067,...|{1, [[[-73.975539...|{5, [[[-80.0, 35....|622236750715092991|622236723437502463|\n| CMT|2009-11-15 09:11:42|2009-11-15 09:26:34| 1| 3.7| -73.990065| 40.687341| null| -73.999566| 40.730219| 12.1| 0.0| 0.5| 0.0| 0.0| 12.6|{1, [[[-73.990065...|{1, [[[-73.999566...|{5, [[[-80.0, 35....|622236750667481087|622236750719549439|\n| CMT|2010-06-27 17:27:25|2010-06-27 17:47:44| 2| 5.0| -74.008459| 40.712233| 1| -73.9516| 40.718877| 15.7| 0.0| 0.5| 3.0| 0.0| 19.2|{1, [[[-74.008459...|{1, [[[-73.9516, ...|{5, [[[-80.0, 35....|622236750648639487|622236723415482367|\n| VTS|2009-01-30 21:05:00|2009-01-30 21:34:00| 1| 5.99| -73.994397| 40.694843| null| -73.921257| 40.685107| 18.9| 0.5| null| 0.6| 0.0| 20.0|{1, [[[-73.994397...|{1, [[[-73.921257...|{5, [[[-80.0, 35....|622236750665449471|622236723618283519|\n| CMT|2009-01-14 07:43:53|2009-01-14 07:55:14| 2| 2.9| -74.007602| 40.725098| null| -73.977846| 40.751608| 9.3| 0.0| null| 0.0| 0.0| 9.3|{1, [[[-74.007602...|{1, [[[-73.977846...|{5, [[[-80.0, 35....|622236750711291903|622236723435503615|\n| VTS|2010-05-11 16:33:00|2010-05-11 16:41:00| 1| 0.99| -74.011205| 40.715468| 1| -74.011072| 40.708882| 5.7| 1.0| 0.5| 0.0| 0.0| 7.2|{1, [[[-74.011205...|{1, [[[-74.011072...|{5, [[[-80.0, 35....|622236750712831999|622236750647459839|\n| VTS|2009-01-19 09:43:00|2009-01-19 09:47:00| 1| 0.56| -73.991123| 40.731297| null| -73.98681| 40.736703| 4.1| 0.0| null| 0.0| 0.0| 4.1|{1, [[[-73.991123...|{1, [[[-73.98681,...|{5, [[[-80.0, 35....|622236750715060223|622236723429539839|\n| CMT|2009-11-14 15:46:36|2009-11-14 15:51:01| 2| 1.2| -74.008448| 40.734271| null| -74.006075| 40.720237| 4.9| 0.0| 0.5| 0.0| 0.0| 5.4|{1, [[[-74.008448...|{1, [[[-74.006075...|{5, [[[-80.0, 35....|622236750719844351|622236750713126911|\n| CMT|2009-11-17 16:50:35|2009-11-17 17:01:34| 1| 1.6| -74.00679| 40.730546| null| -73.9904| 40.73815| 7.7| 1.0| 0.5| 1.5| 0.0| 10.7|{1, [[[-74.00679,...|{1, [[[-73.9904, ...|{5, [[[-80.0, 35....|622236750718894079|622236723426754559|\n| CMT|2009-11-30 23:30:43|2009-11-30 23:42:54| 2| 2.9| -74.009605| 40.710155| null| -73.993875| 40.741554| 10.1| 0.5| 0.5| 2.22| 0.0| 13.32|{1, [[[-74.009605...|{1, [[[-73.993875...|{5, [[[-80.0, 35....|622236750646968319|622236723430490111|\n+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------+-----------------+----------------+-----------+-----+-------+----------+------------+------------+--------------------+--------------------+--------------------+------------------+------------------+\nonly showing top 20 rows\n\n
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------+-----------------+----------------+-----------+-----+-------+----------+------------+------------+--------------------+--------------------+--------------------+------------------+------------------+\nvendor_id| pickup_datetime| dropoff_datetime|passenger_count|trip_distance|pickup_longitude|pickup_latitude|rate_code_id|dropoff_longitude|dropoff_latitude|fare_amount|extra|mta_tax|tip_amount|tolls_amount|total_amount| pickup_geom| dropoff_geom| bounds| pickup_h3| dropoff_h3|\n+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------+-----------------+----------------+-----------+-----+-------+----------+------------+------------+--------------------+--------------------+--------------------+------------------+------------------+\n CMT|2009-10-12 19:35:21|2009-10-12 19:44:36| 1| 1.4| -74.000018| 40.720534| null| -73.995008| 40.734112| 6.5| 0.0| null| 0.0| 0.0| 6.5|{1, [[[-74.000018...|{1, [[[-73.995008...|{5, [[[-80.0, 35....|622236750705885183|622236723432488959|\n CMT|2010-09-02 07:37:40|2010-09-02 07:53:17| 1| 6.8| -74.016477| 40.70971| 1| -73.97509| 40.759392| 17.3| 0.0| 0.5| 1.0| 0.0| 18.8|{1, [[[-74.016477...|{1, [[[-73.97509,...|{5, [[[-80.0, 35....|622236750651359231|622236723493961727|\n CMT|2009-11-28 13:00:00|2009-11-28 13:01:57| 1| 0.5| -73.999666| 40.733384| null| -73.991751| 40.730273| 3.3| 0.0| 0.5| 0.0| 0.0| 3.8|{1, [[[-73.999666...|{1, [[[-73.991751...|{5, [[[-80.0, 35....|622236723433046015|622236750715092991|\n CMT|2009-11-30 07:49:13|2009-11-30 08:12:38| 1| 12.9| -74.007033| 40.734931| null| -73.87277| 40.774319| 29.7| 0.0| 0.5| 6.04| 0.0| 36.24|{1, [[[-74.007033...|{1, [[[-73.87277,...|{5, [[[-80.0, 35....|622236750719975423|622236724015759359|\n VTS|2009-01-03 01:55:00|2009-01-03 02:04:00| 5| 2.09| -73.993217| 40.72416| null| -73.9944| 40.74624| 7.7| 0.5| null| 0.0| 0.0| 8.2|{1, [[[-73.993217...|{1, [[[-73.9944, ...|{5, [[[-80.0, 35....|622236750717878271|622236723431276543|\n VTS|2009-11-09 17:21:00|2009-11-09 17:32:00| 1| 5.19| -74.010003| 40.707022| null| -73.963853| 40.75677| 13.3| 1.0| 0.5| 0.0| 0.0| 14.8|{1, [[[-74.010003...|{1, [[[-73.963853...|{5, [[[-80.0, 35....|622236750647689215|622236723504381951|\n VTS|2010-06-09 01:13:00|2010-06-09 01:27:00| 1| 8.09| -74.008697| 40.707335| 1| -73.949605| 40.78761| 19.7| 0.5| 0.5| 0.0| 0.0| 20.7|{1, [[[-74.008697...|{1, [[[-73.949605...|{5, [[[-80.0, 35....|622236750647820287|622236722210668543|\n VTS|2009-01-28 18:36:00|2009-01-28 18:50:00| 1| 2.92| -74.008823| 40.72089| null| -73.994055| 40.747588| 10.1| 1.0| null| 3.0| 0.0| 14.1|{1, [[[-74.008823...|{1, [[[-73.994055...|{5, [[[-80.0, 35....|622236750713159679|622236723439304703|\n CMT|2010-06-02 14:21:05|2010-06-02 14:28:43| 1| 5.0| -74.011653| 40.702604| 1| -73.971851| 40.749807| 12.5| 0.0| 0.5| 2.6| 0.0| 15.6|{1, [[[-74.011653...|{1, [[[-73.971851...|{5, [[[-80.0, 35....|622236750639857663|622236723496026111|\n VTS|2009-11-26 15:38:00|2009-11-26 15:47:00| 5| 2.36| -73.997208| 40.722108| null| -73.99647| 40.748185| 8.1| 0.0| 0.5| 2.0| 0.0| 10.6|{1, [[[-73.997208...|{1, [[[-73.99647,...|{5, [[[-80.0, 35....|622236750706966527|622236723431309311|\n CMT|2009-12-07 10:36:06|2009-12-07 10:42:26| 1| 1.3| -73.99067| 40.73065| null| -73.975539| 40.741353| 6.1| 0.0| 0.5| 0.0| 0.0| 6.6|{1, [[[-73.99067,...|{1, [[[-73.975539...|{5, [[[-80.0, 35....|622236750715092991|622236723437502463|\n CMT|2009-11-15 09:11:42|2009-11-15 09:26:34| 1| 3.7| -73.990065| 40.687341| null| -73.999566| 40.730219| 12.1| 0.0| 0.5| 0.0| 0.0| 12.6|{1, [[[-73.990065...|{1, [[[-73.999566...|{5, [[[-80.0, 35....|622236750667481087|622236750719549439|\n CMT|2010-06-27 17:27:25|2010-06-27 17:47:44| 2| 5.0| -74.008459| 40.712233| 1| -73.9516| 40.718877| 15.7| 0.0| 0.5| 3.0| 0.0| 19.2|{1, [[[-74.008459...|{1, [[[-73.9516, ...|{5, [[[-80.0, 35....|622236750648639487|622236723415482367|\n VTS|2009-01-30 21:05:00|2009-01-30 21:34:00| 1| 5.99| -73.994397| 40.694843| null| -73.921257| 40.685107| 18.9| 0.5| null| 0.6| 0.0| 20.0|{1, [[[-73.994397...|{1, [[[-73.921257...|{5, [[[-80.0, 35....|622236750665449471|622236723618283519|\n CMT|2009-01-14 07:43:53|2009-01-14 07:55:14| 2| 2.9| -74.007602| 40.725098| null| -73.977846| 40.751608| 9.3| 0.0| null| 0.0| 0.0| 9.3|{1, [[[-74.007602...|{1, [[[-73.977846...|{5, [[[-80.0, 35....|622236750711291903|622236723435503615|\n VTS|2010-05-11 16:33:00|2010-05-11 16:41:00| 1| 0.99| -74.011205| 40.715468| 1| -74.011072| 40.708882| 5.7| 1.0| 0.5| 0.0| 0.0| 7.2|{1, [[[-74.011205...|{1, [[[-74.011072...|{5, [[[-80.0, 35....|622236750712831999|622236750647459839|\n VTS|2009-01-19 09:43:00|2009-01-19 09:47:00| 1| 0.56| -73.991123| 40.731297| null| -73.98681| 40.736703| 4.1| 0.0| null| 0.0| 0.0| 4.1|{1, [[[-73.991123...|{1, [[[-73.98681,...|{5, [[[-80.0, 35....|622236750715060223|622236723429539839|\n CMT|2009-11-14 15:46:36|2009-11-14 15:51:01| 2| 1.2| -74.008448| 40.734271| null| -74.006075| 40.720237| 4.9| 0.0| 0.5| 0.0| 0.0| 5.4|{1, [[[-74.008448...|{1, [[[-74.006075...|{5, [[[-80.0, 35....|622236750719844351|622236750713126911|\n CMT|2009-11-17 16:50:35|2009-11-17 17:01:34| 1| 1.6| -74.00679| 40.730546| null| -73.9904| 40.73815| 7.7| 1.0| 0.5| 1.5| 0.0| 10.7|{1, [[[-74.00679,...|{1, [[[-73.9904, ...|{5, [[[-80.0, 35....|622236750718894079|622236723426754559|\n CMT|2009-11-30 23:30:43|2009-11-30 23:42:54| 2| 2.9| -74.009605| 40.710155| null| -73.993875| 40.741554| 10.1| 0.5| 0.5| 2.22| 0.0| 13.32|{1, [[[-74.009605...|{1, [[[-73.993875...|{5, [[[-80.0, 35....|622236750646968319|622236723430490111|\n+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------+-----------------+----------------+-----------+-----+-------+----------+------------+------------+--------------------+--------------------+--------------------+------------------+------------------+\nonly showing top 20 rows\n\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "from mosaic import grid_longlatascellid\n", + "\n", + "trips_with_geom = (\n", + " trips\n", + " .withColumn(\"pickup_h3\", grid_longlatascellid(lon=\"pickup_longitude\", lat=\"pickup_latitude\", resolution=lit(10)))\n", + " .withColumn(\"dropoff_h3\", grid_longlatascellid(lon=\"dropoff_longitude\", lat=\"dropoff_latitude\", resolution=lit(10)))\n", + ")\n", + "\n", + "trips_with_geom.show()" + ] }, { "cell_type": "code", - "source": [ - "from mosaic import grid_polyfill\n\nneighbourhoods = (\n geoJsonDF\n .repartition(sc.defaultParallelism)\n .select(\"*\", explode(grid_polyfill(\"geometry\", lit(10))).alias(\"h3\"))\n .drop(\"geometry\")\n)\n\nneighbourhoods.show()" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "21627a45-71e0-4286-907b-94de6f04ec43" + "nuid": "21627a45-71e0-4286-907b-94de6f04ec43", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
+-------------+-----------+--------+---------------+------------------+\n| borough|location_id|objectid| zone| h3|\n+-------------+-----------+--------+---------------+------------------+\n|Staten Island| 99| 99|Freshkills Park|622236745731309567|\n|Staten Island| 99| 99|Freshkills Park|622236745690841087|\n|Staten Island| 99| 99|Freshkills Park|622236745723674623|\n|Staten Island| 99| 99|Freshkills Park|622236745683206143|\n|Staten Island| 99| 99|Freshkills Park|622236745691987967|\n|Staten Island| 99| 99|Freshkills Park|622236745684353023|\n|Staten Island| 99| 99|Freshkills Park|622236745733603327|\n|Staten Island| 99| 99|Freshkills Park|622236745725968383|\n|Staten Island| 99| 99|Freshkills Park|622236745701916671|\n|Staten Island| 99| 99|Freshkills Park|622236745734750207|\n|Staten Island| 99| 99|Freshkills Park|622236745694281727|\n|Staten Island| 99| 99|Freshkills Park|622236745703063551|\n|Staten Island| 99| 99|Freshkills Park|622236745679011839|\n|Staten Island| 99| 99|Freshkills Park|622236745761095679|\n|Staten Island| 99| 99|Freshkills Park|622236745680158719|\n|Staten Island| 99| 99|Freshkills Park|622236745696575487|\n|Staten Island| 99| 99|Freshkills Park|622236745762242559|\n|Staten Island| 99| 99|Freshkills Park|622236745681305599|\n|Staten Island| 99| 99|Freshkills Park|622236745730555903|\n|Staten Island| 99| 99|Freshkills Park|622236745690087423|\n+-------------+-----------+--------+---------------+------------------+\nonly showing top 20 rows\n\n
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
+-------------+-----------+--------+---------------+------------------+\n borough|location_id|objectid| zone| h3|\n+-------------+-----------+--------+---------------+------------------+\nStaten Island| 99| 99|Freshkills Park|622236745731309567|\nStaten Island| 99| 99|Freshkills Park|622236745690841087|\nStaten Island| 99| 99|Freshkills Park|622236745723674623|\nStaten Island| 99| 99|Freshkills Park|622236745683206143|\nStaten Island| 99| 99|Freshkills Park|622236745691987967|\nStaten Island| 99| 99|Freshkills Park|622236745684353023|\nStaten Island| 99| 99|Freshkills Park|622236745733603327|\nStaten Island| 99| 99|Freshkills Park|622236745725968383|\nStaten Island| 99| 99|Freshkills Park|622236745701916671|\nStaten Island| 99| 99|Freshkills Park|622236745734750207|\nStaten Island| 99| 99|Freshkills Park|622236745694281727|\nStaten Island| 99| 99|Freshkills Park|622236745703063551|\nStaten Island| 99| 99|Freshkills Park|622236745679011839|\nStaten Island| 99| 99|Freshkills Park|622236745761095679|\nStaten Island| 99| 99|Freshkills Park|622236745680158719|\nStaten Island| 99| 99|Freshkills Park|622236745696575487|\nStaten Island| 99| 99|Freshkills Park|622236745762242559|\nStaten Island| 99| 99|Freshkills Park|622236745681305599|\nStaten Island| 99| 99|Freshkills Park|622236745730555903|\nStaten Island| 99| 99|Freshkills Park|622236745690087423|\n+-------------+-----------+--------+---------------+------------------+\nonly showing top 20 rows\n\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "from mosaic import grid_polyfill\n", + "\n", + "neighbourhoods = (\n", + " geoJsonDF\n", + " .repartition(sc.defaultParallelism)\n", + " .select(\"*\", explode(grid_polyfill(\"geometry\", lit(10))).alias(\"h3\"))\n", + " .drop(\"geometry\")\n", + ")\n", + "\n", + "neighbourhoods.show()" + ] }, { "cell_type": "code", - "source": [ - "joined_df = trips_with_geom.alias(\"t\").join(neighbourhoods.alias(\"n\"), on=expr(\"t.pickup_h3 = n.h3\"), how=\"inner\")\njoined_df.count()" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "c3e68351-52d4-4a8b-8ce6-4898dedd0ecc" + "nuid": "c3e68351-52d4-4a8b-8ce6-4898dedd0ecc", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
Out[15]: 4934151
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
Out[15]: 4934151
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "joined_df = trips_with_geom.alias(\"t\").join(neighbourhoods.alias(\"n\"), on=expr(\"t.pickup_h3 = n.h3\"), how=\"inner\")\n", + "joined_df.count()" + ] }, { "cell_type": "markdown", - "source": [ - "## Mosaic spatial join optimizations" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "c4fb8772-94b2-4fbe-aaf9-6e261609b009" + "nuid": "c4fb8772-94b2-4fbe-aaf9-6e261609b009", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "## Mosaic spatial join optimizations" + ] }, { "cell_type": "markdown", - "source": [ - "Mosaic provides easy access to the optimized spatial join technique described in [this](https://databricks.com/blog/2021/10/11/efficient-point-in-polygon-joins-via-pyspark-and-bng-geospatial-indexing.html) blog post." - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "c7696c95-820a-434e-9535-48807e1fb27c" + "nuid": "c7696c95-820a-434e-9535-48807e1fb27c", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "Mosaic provides easy access to the optimized spatial join technique described in [this](https://databricks.com/blog/2021/10/11/efficient-point-in-polygon-joins-via-pyspark-and-bng-geospatial-indexing.html) blog post." + ] }, { "cell_type": "code", - "source": [ - "from mosaic import grid_tessellateexplode\n\nmosaic_neighbourhoods = (\n geoJsonDF\n .repartition(sc.defaultParallelism)\n .select(\"*\", grid_tessellateexplode(\"geometry\", lit(10)))\n .drop(\"geometry\")\n)\n\nmosaic_neighbourhoods.show()" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "1319c9ca-149c-4208-8512-f035ffff467b" + "nuid": "1319c9ca-149c-4208-8512-f035ffff467b", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
+-------------+-----------+--------+---------------+-------+------------------+----+\n| borough|location_id|objectid| zone|is_core| h3| wkb|\n+-------------+-----------+--------+---------------+-------+------------------+----+\n|Staten Island| 99| 99|Freshkills Park| true|622236745694314495|null|\n|Staten Island| 99| 99|Freshkills Park| true|622236745679077375|null|\n|Staten Island| 99| 99|Freshkills Park| true|622236745734160383|null|\n|Staten Island| 99| 99|Freshkills Park| true|622236745762275327|null|\n|Staten Island| 99| 99|Freshkills Park| true|622236745731801087|null|\n|Staten Island| 99| 99|Freshkills Park| true|622236745725935615|null|\n|Staten Island| 99| 99|Freshkills Park| true|622236745680224255|null|\n|Staten Island| 99| 99|Freshkills Park| true|622236745689595903|null|\n|Staten Island| 99| 99|Freshkills Park| true|622236745683730431|null|\n|Staten Island| 99| 99|Freshkills Park| true|622236745723576319|null|\n|Staten Island| 99| 99|Freshkills Park| true|622236745763422207|null|\n|Staten Island| 99| 99|Freshkills Park| true|622236745693102079|null|\n|Staten Island| 99| 99|Freshkills Park| true|622236745732947967|null|\n|Staten Island| 99| 99|Freshkills Park| true|622236745702473727|null|\n|Staten Island| 99| 99|Freshkills Park| true|622236745696608255|null|\n|Staten Island| 99| 99|Freshkills Park| true|622236745736454143|null|\n|Staten Island| 99| 99|Freshkills Park| true|622236745761062911|null|\n|Staten Island| 99| 99|Freshkills Park| true|622236745690742783|null|\n|Staten Island| 99| 99|Freshkills Park| true|622236745675505663|null|\n|Staten Island| 99| 99|Freshkills Park| true|622236745730588671|null|\n+-------------+-----------+--------+---------------+-------+------------------+----+\nonly showing top 20 rows\n\n
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
+-------------+-----------+--------+---------------+-------+------------------+----+\n borough|location_id|objectid| zone|is_core| h3| wkb|\n+-------------+-----------+--------+---------------+-------+------------------+----+\nStaten Island| 99| 99|Freshkills Park| true|622236745694314495|null|\nStaten Island| 99| 99|Freshkills Park| true|622236745679077375|null|\nStaten Island| 99| 99|Freshkills Park| true|622236745734160383|null|\nStaten Island| 99| 99|Freshkills Park| true|622236745762275327|null|\nStaten Island| 99| 99|Freshkills Park| true|622236745731801087|null|\nStaten Island| 99| 99|Freshkills Park| true|622236745725935615|null|\nStaten Island| 99| 99|Freshkills Park| true|622236745680224255|null|\nStaten Island| 99| 99|Freshkills Park| true|622236745689595903|null|\nStaten Island| 99| 99|Freshkills Park| true|622236745683730431|null|\nStaten Island| 99| 99|Freshkills Park| true|622236745723576319|null|\nStaten Island| 99| 99|Freshkills Park| true|622236745763422207|null|\nStaten Island| 99| 99|Freshkills Park| true|622236745693102079|null|\nStaten Island| 99| 99|Freshkills Park| true|622236745732947967|null|\nStaten Island| 99| 99|Freshkills Park| true|622236745702473727|null|\nStaten Island| 99| 99|Freshkills Park| true|622236745696608255|null|\nStaten Island| 99| 99|Freshkills Park| true|622236745736454143|null|\nStaten Island| 99| 99|Freshkills Park| true|622236745761062911|null|\nStaten Island| 99| 99|Freshkills Park| true|622236745690742783|null|\nStaten Island| 99| 99|Freshkills Park| true|622236745675505663|null|\nStaten Island| 99| 99|Freshkills Park| true|622236745730588671|null|\n+-------------+-----------+--------+---------------+-------+------------------+----+\nonly showing top 20 rows\n\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "from mosaic import grid_tessellateexplode\n", + "\n", + "mosaic_neighbourhoods = (\n", + " geoJsonDF\n", + " .repartition(sc.defaultParallelism)\n", + " .select(\"*\", grid_tessellateexplode(\"geometry\", lit(10)))\n", + " .drop(\"geometry\")\n", + ")\n", + "\n", + "mosaic_neighbourhoods.show()" + ] }, { "cell_type": "markdown", - "source": [ - "Mosaic also includes a convenience function for displaying dataframes with geometry columns." - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "ee5d5a0a-2f2f-4095-9ec8-ce3ba0af6eaa" + "nuid": "ee5d5a0a-2f2f-4095-9ec8-ce3ba0af6eaa", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "Mosaic also includes a convenience function for displaying dataframes with geometry columns." + ] }, { "cell_type": "code", - "source": [ - "from mosaic import displayMosaic\ndisplayMosaic(mosaic_neighbourhoods)" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "162fc850-a289-416f-af6d-2dd73f30e93e" + "nuid": "162fc850-a289-416f-af6d-2dd73f30e93e", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "overflow": true, - "datasetInfos": [], - "data": [ - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694314495, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745679077375, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734160383, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745762275327, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745731801087, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745725935615, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745680224255, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745689595903, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683730431, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745723576319, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745763422207, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745693102079, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732947967, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702473727, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745696608255, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745736454143, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745761062911, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745690742783, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675505663, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745730588671, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694248959, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745679011839, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734094847, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745762209791, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745731735551, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745725870079, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745680158719, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745689530367, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683664895, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745763356671, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745693036543, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732882431, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702408191, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745681305599, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745760997375, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675440127, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745730523135, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745703555071, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745762144255, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745725804543, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745680093183, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683599359, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745763291135, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745692971007, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732816895, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745913303039, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702342655, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745681240063, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745690611711, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675374591, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694117887, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745733963775, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745703489535, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745731604479, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745689399295, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683533823, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745763225599, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745692905471, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732751359, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745913237503, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702277119, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745742123007, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745681174527, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745760866303, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745730392063, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745684680703, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745724526591, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745764372479, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694052351, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745733898239, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745703423999, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745682321407, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745731538943, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745725673471, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745695199231, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745735045119, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745689333759, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683468287, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745763160063, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732685823, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702211583, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745742057471, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745681108991, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745760800767, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745690480639, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675243519, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745684615167, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745764306943, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745693986815, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745733832703, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745672884223, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745682255871, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236746168631295, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745761947647, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745731473407, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745725607935, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745695133695, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734979583, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745689268223, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683402751, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745723248639, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732620287, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702146047, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745741991935, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745760735231, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745690415103, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675177983, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745684549631, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745724395519, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745764241407, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745703292927, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745682190335, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236746168565759, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745676324863, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745725542399, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745695068159, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734914047, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683337215, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745723183103, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732554751, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702080511, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745741926399, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745760669695, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745690349567, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675112447, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745693855743, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745703227391, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745743073279, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745672753151, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236746168500223, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745676259327, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745731342335, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745725476863, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745695002623, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734848511, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745673900031, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683271679, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745723117567, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732489215, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702014975, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745741860863, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745726623743, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745760604159, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745690284031, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745684418559, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745724264447, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745693790207, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745733636095, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745703161855, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745743007743, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745697296383, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236746168434687, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745676193791, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745731276799, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745725411327, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694937087, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745679699967, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734782975, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683206143, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745723052031, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745692577791, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732423679, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745701949439, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745741795327, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745726558207, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745760538623, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745690218495, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745674981375, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745684353023, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745724198911, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745693724671, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745697230847, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745761685503, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745676128255, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745731211263, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745725345791, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694871551, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734717439, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683140607, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745722986495, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732358143, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745701883903, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745741729791, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745726492671, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745690152959, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745674915839, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745684287487, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745724133375, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745703030783, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745697165311, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745761619967, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745676062719, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745725280255, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694806015, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734651903, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683075071, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745722920959, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745692446719, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732292607, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745701818367, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745741664255, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745735798783, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745690087423, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745674850303, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745693593599, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745733439487, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702965247, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745742811135, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745697099775, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675997183, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745725214719, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694740479, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734586367, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745698246655, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683009535, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745722855423, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745762701311, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732227071, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745726361599, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745735733247, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745690021887, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745684156415, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745724002303, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745693528063, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745733373951, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702899711, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745742745599, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745697034239, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745681797119, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745736880127, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745761488895, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675931647, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745725149183, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694674943, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734520831, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745682943999, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745722789887, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732161535, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745726296063, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745735667711, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745689956351, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745674719231, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745684090879, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745723936767, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745693462527, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745733308415, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745696968703, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745736814591, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745761423359, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675866111, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745725083647, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694609407, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734455295, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732095999, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745726230527, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745689890815, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745674653695, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745684025343, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745723871231, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702768639, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745742614527, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745681666047, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745736749055, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745761357823, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675800575, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745725018111, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694543871, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734389759, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732030463, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745689825279, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745674588159, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702703103, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745742548991, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745736683519, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745690972159, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675735039, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745724952575, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694478335, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734324223, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745731964927, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745726099455, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745689759743, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683894271, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745723740159, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702637567, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745742483455, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745696772095, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745761226751, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745690906623, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675669503, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745724887039, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694412799, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745679175679, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734258687, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745731899391, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745726033919, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745689694207, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683828735, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745723674623, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745763520511, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745696706559, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745761161215, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745690841087, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675603967, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694347263, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745679110143, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734193151, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745762308095, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745731833855, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745725968383, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745689628671, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745729474559, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683763199, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745723609087, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745763454975, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702506495, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745742352383, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745696641023, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745761095679, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675538431, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694281727, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745679044607, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734127615, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745762242559, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745731768319, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745680191487, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745689563135, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745693069311, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732915199, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702440959, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745696575487, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675472895, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745730555903, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694216191, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734062079, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745703587839, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745762177023, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745731702783, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745725837311, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745689497599, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683632127, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745723478015, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745763323903, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745693003775, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732849663, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702375423, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745681272831, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745690644479, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675407359, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745730490367, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694150655, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745733996543, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745703522303, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745762111487, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745731637247, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745680060415, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745689432063, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683566591, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745763258367, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745692938239, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732784127, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745681207295, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745760899071, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675341823, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745730424831, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745684713471, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745764405247, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694085119, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745733931007, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745703456767, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745682354175, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745762045951, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745731571711, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745725706239, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745689366527, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683501055, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745763192831, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702244351, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745742090239, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745681141759, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675276287, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745684647935, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745724493823, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745764339711, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694019583, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745678782463, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745733865471, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745703391231, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745682288639, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745761980415, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745731506175, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745695166463, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745735012351, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745689300991, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732653055, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702178815, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745742024703, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745690447871, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675210751, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745684582399, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745764274175, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745693954047, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745733799935, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745703325695, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745743171583, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745682223103, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745761914879, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745731440639, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745725575167, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745695100927, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734946815, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745689235455, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683369983, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745723215871, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732587519, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702113279, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745741959167, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745696247807, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745681010687, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745760702463, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745690382335, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675145215, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745684516863, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745724362751, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745764208639, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745693888511, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745733734399, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745703260159, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745743106047, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745682157567, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745761849343, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745691529215, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745676292095, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745731375103, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745725509631, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745695035391, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734881279, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683304447, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745723150335, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732521983, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745726656511, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745696182271, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745760636927, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745690316799, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675079679, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745684451327, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745724297215, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745693822975, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745733668863, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745703194623, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745743040511, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745761783807, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745691463679, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745676226559, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745731309567, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745725444095, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683238911, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745723084799, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745701982207, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745741828095, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745726590975, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745696116735, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745760571391, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675014143, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745684385791, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745724231679, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745693757439, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745733603327, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745703129087, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745742974975, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745697263615, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745676161023, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745731244031, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694904319, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734750207, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732390911, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745701916671, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745741762559, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745726525439, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745690185727, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745674948607, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745684320255, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745724166143, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745693691903, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745733537791, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745703063551, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745697198079, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745761652735, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745731178495, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694838783, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734684671, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745698344959, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683107839, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745722953727, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745692479487, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732325375, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745701851135, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745741697023, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745726459903, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745735831551, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745690120191, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745674883071, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745684254719, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745724100607, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745693626367, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745733472255, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702998015, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745697132543, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745676029951, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745725247487, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694773247, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734619135, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683042303, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745722888191, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745762734079, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732259839, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745726394367, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745735766015, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745690054655, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745674817535, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745684189183, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745724035071, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745693560831, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745733406719, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702932479, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745761521663, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675964415, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745698213887, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745682976767, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745722822655, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745726328831, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745674751999, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745684123647, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745723969535, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745693495295, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745733341183, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702866943, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745736847359, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745761456127, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675898879, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694642175, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734488063, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745698148351, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732128767, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745726263295, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745735634943, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745689923583, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745674686463, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745684058111, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745723903999, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745693429759, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745733275647, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702801407, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745742647295, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745736781823, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745761390591, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745725050879, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694576639, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734422527, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745732063231, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745726197759, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745689858047, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745674620927, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683992575, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745723838463, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702735871, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745742581759, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745736716287, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745761325055, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745691004927, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675767807, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745724985343, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694511103, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734356991, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745731997695, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745726132223, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745689792511, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745674555391, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683927039, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745723772927, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745763618815, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702670335, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745742516223, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745761259519, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745690939391, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675702271, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745724919807, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745679208447, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745726066687, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683861503, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745723707391, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702604799, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745742450687, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745696739327, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745761193983, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745690873855, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745675636735, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745694380031, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745679142911, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745734225919, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745731866623, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745726001151, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745689661439, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745729507327, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745683795967, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745723641855, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745763487743, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745702539263, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745696673791, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745761128447, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745690808319, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745725739007, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745696935935, - "AQMAAAABAAAABAAAAIzvORt9i1LAUzWXQ0NIRECs3yEigotSwIct0/1KSERAz1Bl94WLUsD10IC1SUhEQIzvORt9i1LAUzWXQ0NIREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236746167451647, - "AQMAAAABAAAABQAAALLLamedjVLA7PTzHUVHREBNNifZjI1SwIx+IDVNR0RAo5WznI2NUsAYmXlhTkdEQOZxFHmcjVLAepg7ZklHRECyy2pnnY1SwOz08x1FR0RA" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745725181951, - "AQMAAAABAAAACAAAAKO97SfLjFLAIFImkFVHRECDqWsmwIxSwIgY7T9ZR0RAaiCr/7mMUsAVstwXZUdEQKmCacK3jFLAJQi4Ym9HREBTwBAWwoxSwGytiUB/R0RAlcOy8tCMUsB7rxhGekdEQE9Ql3vVjFLA5tnfbWVHRECjve0= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745692708863, - "AQMAAAABAAAACgAAABxTRq7gilLAmdfeuPlHREDD33S73YpSwKGHSLX6R0RAyUDDbd2KUsAKqHCXDEhEQHoKDC7dilLAXpp1LhtIRECzsMwt3YpSwIA5Dj0bSERAJM9mHN2KUsARqtk4H0hEQKbuUhbdilLAAg+akiFIREAwyi0= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745763880959, - "AQMAAAABAAAABAAAAMJtrBbEilLAuXWkiSxLREAKJ7ZcwYpSwLrfZjFBS0RAuOwva8WKUsAVGCeVLktEQMJtrBbEilLAuXWkiSxLREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745673801727, - "AQMAAAABAAAAEAAAAEusd47ri1LARkK8hxlLREB67WSv3ItSwK+/oIEeS0RAiTnvJNiLUsC9w3NbM0tEQKosoHnii1LAw55wO0NLRECON47p6otSwA05smhAS0RAQTKHHt2LUsCWtkiYMUtEQGXeF4jfi1LAns6gKjFLREAkqVc= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745762766847, - "AQMAAAABAAAABAAAABHqBw3NilLA5PFQ+cpKREA20j3kzIpSwHeLCgrNSkRAic/LRM2KUsBDJg1Py0pEQBHqBw3NilLA5PFQ+cpKREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745769549823, - "AQMAAAABAAAACAAAAEsfeSjQilLAvI5xoTlMREBSSHP3yYpSwGY4QrM7TERAOPSwLMWKUsBX9gTsV0xEQOaTaHXEilLAzKLeUl9MREDTO0MRx4pSwLxEjlVjTERAUghS8dWKUsAn/qZcXkxEQAYG43zailLAFXeRgklMREBLH3k= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236746168893439, - "AQMAAAABAAAACgAAAEFnNV4qjVLAwD4mP1tHREBSQQgeKo1SwJ1uO2ZcR0RAZL/3cTSNUsCXXLZDbEdEQMYbEV82jVLAcXR5nmtHREBx/B14P41SwJ9IGV9eR0RAd4R8yzqNUsDHj1iuXkdEQDyyRxU2jVLAdJqzbl5HREDzpns= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745673244671, - "AQYAAAACAAAAAQMAAAABAAAABQAAAGfnWjFYjFLAcCT/1uFKREBEWyhGUYxSwCD74ifkSkRAow3VNFKMUsA4U1F95kpEQD0y/b9ZjFLAYpJwO+RKREBn51oxWIxSwHAk/9bhSkRAAQMAAAABAAAABQAAALTRzktOjFLAkEgTJ+U= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745730293759, - "AQMAAAABAAAADwAAAHsrqeBrjVLALNBqRGhIREC72IoDXY1SwL+PlD9tSERAycTpeliNUsAWLoUYgkhEQF/qes9ijVLAaVta9pFIREB0K4b3bo1SwMamceONSERAC8KjXG+NUsBsmGihjEhEQKB950BwjVLAOz81o4lIREDlS8k= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745757982719, - "AQMAAAABAAAACAAAAFvg22aMilLANALWn7VLREB+k+jbiopSwE9M1CO2S0RAGDjhCnqKUsDZZAofz0tEQO+Cb7V5ilLATtUVkNBLREBwrGJPg4pSwGznOlPfS0RAL1ApL5KKUsBiUqda2ktEQNP227qWilLAaLPtgMVLREBb4Ns= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745736519679, - "AQMAAAABAAAABgAAABIxNdUtjVLALY8s+hJKRECy44BiKo1SwHiQ5CEUSkRA9g1i2SWNUsCgwZ/7KEpEQBUkeHMujVLAEIjHMTZKREByI0BpLo1SwE8n89smSkRAEjE11S2NUsAtjyz6EkpEQA==" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745678913535, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745682026495, - "AQMAAAABAAAABwAAABhV/Kp6i1LAh31HSGFMRECUDmrva4tSwAQrrjVmTERABZFSZGeLUsARqwQQe0xEQKfzQ7lxi1LAY4jX8IpMRED1jqXhcotSwCtxt42KTERAYIM7BnqLUsBWLJGmYkxEQBhV/Kp6i1LAh31HSGFMREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745735405567, - "AQMAAAABAAAALAAAACBtiIU+jVLASpeWu6xJREC8PLExMo1SwFki59ywSURAVV6sqC2NUsD4YnG2xUlEQFL7/MY3jVLATD7rQNVJREBHH4DjN41SwDCgssfTSURAIsTaMjiNUsBjWkyr0UlEQI02BZ04jVLAtALEwc9JRED26fE= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745696149503, - "AQMAAAABAAAACQAAAORf+0wejFLAur1XzyRIRECcUfVWHYxSwICcAXIlSERA39+yMxqMUsA/LFb4JkhEQMyzxCgEjFLA7KJzdS9IRECy6bdfA4xSwLMk9RAzSERAMuo1sw2MUsD2iZTvQkhEQLzpcZAcjFLAsm3J9T1IRECArRk= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745742188543, - "AQMAAAABAAAACAAAAIprKi5ti1LAwRpDy59MREAWlwtOXotSwFh5zcSkTERAjxrcwlmLUsBkxz+fuUxEQE1c3xdki1LAZg02gMlMREAFjWoXZ4tSwK+BhX/ITERAVAUJCGiLUsDfm3gqx0xEQOtdaKtui1LA144nFaJMRECKayo= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745699262463, - "AQMAAAABAAAACQAAAJjNyNXrilLAKAnxzP9JREDMTz333IpSwFwbsMUESkRA/2kpzNqKUsC/hKG4DkpEQJ1M8g/ailLAz8UH5RlKREAWV8fm2YpSwFyfwuQbSkRA9IGfv+KKUsAImNB+KUpEQB9AQZ7xilLAEb4HhiRKRECw80s= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745695592447, - "AQMAAAABAAAAFAAAACHyBz3Zi1LAvWeqxEVIREAEgGFh14tSwOf3dj5ISERATDZXYNeLUsCOiL8/SEhEQOccQV/Xi1LAvJIdQUhIRECJnmNZ04tSwD3fZQVNSERAwUAYJM+LUsD72D9wUUhEQC2BQSPPi1LAkpgWcVFIREBFkXY= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745912778751, - "AQMAAAABAAAABAAAAGogq/+5jFLAFbLcF2VHREDDmav/s4xSwCv2LqVwR0RAqYJpwreMUsAlCLhib0dEQGogq/+5jFLAFbLcF2VHREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745769877503, - "AQMAAAABAAAABAAAAAESmsN+ilLAN0D7LPRLRECZpv25fIpSwPYgS9v0S0RAJSoxv3+KUsA+gcyv9UtEQAESmsN+ilLAN0D7LPRLREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745691365375, - "AQMAAAABAAAACwAAAGozUIn1ilLAEKQIQdlHREAw8SYW9IpSwMcGf+jfR0RALiPBaP6KUsDAqrbH70dEQBJ93QEMi1LA/HtxO+tHREA3kig6BotSwLZ1sPHlR0RAxGtyuQCLUsAGDz7t4EdEQAg8IJ7/ilLA9ibo6t9HREDs7PU= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745736290303, - "AQMAAAABAAAABgAAAO0EkPJDjVLAR0N/nJ5JREBBXEPvQo1SwJNXp0SjSURA56I3SkONUsBSb/DdoklEQNmKbSVEjVLAIcjG8KFJRECgZKwxRI1SwIaCYj+gSURA7QSQ8kONUsBHQ3+cnklEQA==" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745921560575, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745913106431, - "AQMAAAABAAAAEAAAAIIWNIjqjFLAZ2Od0SZHREBZDm1g6IxSwLkJSe8mR0RAA+xnOuaMUsDe2MI4J0dEQNYVpRfkjFLAj2zZrSdHREA05cPV4oxSwDR9Fg0oR0RAKXtIjd6MUsChDoq9O0dEQJJi9ODojFLA9Hgrm0tHREDMPG8= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745698476031, - "AQMAAAABAAAACQAAALL+ux3oilLAISejGFNJREBRTn3L4IpSwJM6VYtVSURAr2wazOCKUsAX3FEnVklEQKdUfongilLARGuvEmdJRED8vVvz34pSwAZAevV3SURAzrEi0N+KUsB9ldCGfElEQEwaBObtilLAFHgZ0XdJREBOsPA= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745769648127, - "AQMAAAABAAAABwAAAJ5JnIXCilLA0IWqL3hMRECiHMIJwopSwI4IEFl4TERA/atadMCKUsCS30q4iExEQO271ie+ilLAeH+vT6BMREBgDW9OyIpSwLcrH+ucTERAEwwY2syKUsCQ2O0QiExEQJ5JnIXCilLA0IWqL3hMREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236746168991743, - "AQMAAAABAAAAbgAAAOZxFHmcjVLAepg7ZklHRECjlbOcjY1SwBiZeWFOR0RAeDp0FImNUsAwnOU5Y0dEQNRjiUWTjVLAJcIu4XJHRED/Zstpk41SwACBE3xyR0RAaf7Oo5ONUsAQfUPlcUdEQB0fx+CTjVLAt2EwUXFHRED02aI= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745673342975, - "AQMAAAABAAAACQAAAB+zGpprjFLA/inKAshKRECwB1G7XIxSwNN6Kv3MSkRAZ+daMViMUsBwJP/W4UpEQD0y/b9ZjFLAYpJwO+RKREDd+3RsX4xSwLrmzYjiSkRArimgTm6MUsBukC0X5kpEQBjpkTV0jFLAQxtKzt9KRED9fQQ= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745680125951, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745672785919, - "AQMAAAABAAAABwAAACtBCaInjFLAVe8AGPJKREBS8sO1HIxSwGna4b/1SkRArQe3bRuMUsDdVY+i+0pEQACZCuIdjFLA31deePlKREB6WJ8NIYxSwBYZS+72SkRASHDjSSSMUsDgOSWX9EpEQCtBCaInjFLAVe8AGPJKREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745690578943, - "AQMAAAABAAAACQAAADmf/S3qilLA0p0dH8tIREDeljZq24pSwFzACA/QSERAHQuVdduKUsDBMc0y00hEQP7D8CDcilLA/mL9X+hIREAm+Thi3IpSwFEUV5HtSERABbJIGOGKUsBnDLvP9EhEQA6OJ/bvilLAP+gT1+9IREBuE/g= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745761751039, - "AQMAAAABAAAACAAAAIfQ/ZbEilLAbsZnK/BLRECF0TG3tYpSwH21MST1S0RAZUAtzbGKUsAxnLUYB0xEQIMkNPu8ilLAoHafRg5MREA7iZpExYpSwPaBRZsWTERAImKyX8qKUsAJfFzmFExEQDhHP+vOilLAxBlxDABMRECH0P0= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745679568895, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745743400959, - "AQMAAAABAAAACAAAAG3Nwusyi1LABCIZR+1MREAhLGkLJItSwLFPb0DyTERAyND3fx+LUsBWzvUaB01EQJsA9NQpi1LAtHY0/BZNREC4lME/MotSwGymuysUTURAS5QPEDuLUsB9Dzo1B01EQNZYwUA9i1LAkKk/KP1MREBtzcI= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745742843903, - "AQMAAAABAAAABwAAAAEzn97yilLA2oyXBhZNREClgCsC5IpSwAfxVf4aTURA2LgymeuKUsAqkB94HE1EQCuus3/wilLAjowFRh1NREDVFnRh9YpSwF42raweTURAx31npfmKUsB65hRxIE1EQAEzn97yilLA2oyXBhZNREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745735503871, - "AQMAAAABAAAAJwAAAGmcpnQzjVLA6wiKbupJREBKr4qWJI1SwJq/oWnvSURAfd5tDSCNUsBt20dDBEpEQLLjgGIqjVLAeJDkIRRKREASMTXVLY1SwC2PLPoSSkRAUz3cvy2NUsD9FD0cEEpEQKFRPQkujVLABAdpMg5KREAF9AY= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745763749887, - "AQMAAAABAAAACAAAALtW8YHOilLArlVM4gRLREDBzi5tyIpSwG9it+oGS0RA4J4bi8eKUsBsOitdEktEQMJtrBbEilLAuHWkiSxLREC47C9rxYpSwBUYJ5UuS0RAq7F7StSKUsCSfmGcKUtEQKxz0tXYilLAjfHswhRLREC7VvE= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745742286847, - "AQMAAAABAAAABQAAAPWOpeFyi1LAK3G3jYpMRECn80O5cYtSwGOI1/CKTERAimsqLm2LUsDBGkPLn0xEQOpdaKtui1LA1o4nFaJMRED1jqXhcotSwCtxt42KTERA" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745762635775, - "AQMAAAABAAAABwAAAHYlWVvWilLAhyqUnKFKREBbb3kU0IpSwFIVyLWjSkRAEeoHDc2KUsDk8VD5ykpEQInPy0TNilLAQyYNT8tKRECketcj3IpSwOq4SVbGSkRA5zIUr+CKUsBhDwZ9sUpEQHYlWVvWilLAhyqUnKFKREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745730162687, - "AQMAAAABAAAAGgAAAIWgZ8V9jVLAs+4/8U5IREDJVDRpcI1SwFX6gGtTSERAeyup4GuNUsAs0GpEaEhEQJIooa11jVLAxTnhUXdIREArfNixdY1SwDytwUR3SERAuryaHHaNUsDfR6badUhEQHJmIY12jVLAAyiXwHNIRECYmyg= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745760964607, - "AQMAAAABAAAADwAAAA6xuwjBilLASTCfrUNLREAMPY6+wIpSwIpDl99FS0RADlNtYMCKUsASCxWsSUtEQDbSl+C/ilLA1AOZb01LREC/N2w/v4pSwAJlaydRS0RAwVRcfb6KUsC3/NzQVEtEQOpFM529ilLA4aMZP1hLREAiICs= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745681895423, - "AQMAAAABAAAABAAAACCTZnWFi1LAwuEBV0pMREDpjmBaf4tSwCRuzGFMTERAFEPU6XqLUsBUWqrCYExEQCCTZnWFi1LAwuEBV0pMREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745913204735, - "AQMAAAABAAAACwAAABXjZov2jFLAiJgF+yhHREBY9ydG/IxSwGiLbMgxR0RARIHViAONUsD2Ja9ZL0dEQP5xVegCjVLAOz+bzi5HRECGIiypAY1SwJeud90tR0RAGzyPYQCNUsCfkFYILUdEQEWuBTf+jFLARiql8CtHRECeYPs= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745691791359, - "AQMAAAABAAAADAAAACoGIm3eilLAkiWYz9FHREAB8qRI3opSwD3ytDXaR0RAQqf0OOWKUsCSOQ7h5EdEQDDxJhb0ilLAxwZ/6N9HREBqM1CJ9YpSwBCkCEHZR0RAJkRisPKKUsB/rkns10dEQHVLi13vilLAjx6hh9ZHRECs1T8= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745762963455, - "AQMAAAABAAAABAAAANaM/WrUilLANt5z9mZKRED0jon204pSwBTVNlZtSkRAJZolHtWKUsCeeuUJaEpEQNaM/WrUilLANt5z9mZKREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745912647679, - "AQMAAAABAAAABwAAANhuy9LNjFLAokyR2T1HREBeY4fGzYxSwL/8KPI9R0RAxheJEMSMUsCuK4O2UUdEQISpaybAjFLAhxjtP1lHRECjve0ny4xSwCBSJpBVR0RA40TQsM+MUsBWlAK4QEdEQNhuy9LNjFLAokyR2T1HREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745769746431, - "AQMAAAABAAAACAAAAC9QKS+SilLAYlKnWtpLREBwrGJPg4pSwGznOlPfS0RAARKaw36KUsA3QPss9EtEQCUqMb9/ilLAPoHMr/VLREDbr8BfhYpSwEVfuDv3S0RA56Ztm5iKUsBVd9Ul/EtEQMadPYOcilLALk/KO+pLREAvUCk= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236746169090047, - "AQMAAAABAAAAWQAAAG9CaOCOjVLAOgej74dHREAwmN0DgI1SwM+E2eqMR0RA6FCGe3uNUsAlamHDoUdEQHGZzc+FjVLAnx/BoLFHREDG46MAiY1SwK+q9o6wR0RArlsIMomNUsDMdGTwr0dEQNW/eIWJjVLAbZaq8K5HRECmLPQ= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236746168532991, - "AQMAAAABAAAABwAAANOJOWFxjVLA9XyFoVpHREBd8nW9X41SwGstTkBjR0RAGO8Ff1yNUsAjLT8rckdEQM57ItNmjVLAvnCgCIJHREBMhKqvdY1SwAgXhQ19R0RAFsj/N3qNUsCVSRI1aEdEQNOJOWFxjVLA9XyFoVpHREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745690677247, - "AQMAAAABAAAABwAAAJaTnyPeilLAwauyGAxJREAEAkaS3opSwJtPO5QSSURAs//Tnd+KUsAmHsVpI0lEQJfWoh3gilLAx6cvHi9JREDsCoZV4opSwAM5TWAuSURA4olu4OaKUsDtgduHGUlEQJaTnyPeilLAwauyGAxJREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745679667199, - "AQMAAAABAAAABwAAAMOqaqPMjFLAADEi3UZKRECt/ALFvYxSwIZS2ddLSkRA44l6O7mMUsCOL4yxYEpEQNmtbZDDjFLAVzyWkHBKRECI/Em7y4xSwP8/YdRtSkRAJAV3dNGMUsDK4jpDTkpEQMOqaqPMjFLAADEi3UZKREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745750839295, - "AQMAAAABAAAABgAAAFzYInTAilLAKfO+NQBNREBB/k2MtIpSwMpgZzAETURAyb1QWLOKUsBkYsfNEE1EQCMFdJHJilLAtfyqqxVNREAm6djIyopSwNdDPBcQTURAXNgidMCKUsAp8741AE1EQA==" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745729376255, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745696903167, - "AQMAAAABAAAABwAAAM9QZfeFi1LA9tCAtUlIRECs3yEigotSwIct0/1KSERAaMvol32LUsCB4A7WX0hEQO0eHOuHi1LAnRMdtW9IREAy4IrIlotSwAp217tqSERAF8kzsZqLUsDi+w3JWEhEQM9QZfeFi1LA9tCAtUlIREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745757065215, - "AQMAAAABAAAABQAAAB2rT82hilLAK9g3t5JLREAufZ0+nIpSwPDVNVCcS0RAoABJppuKUsAe8X8ynUtEQKlDKdKfilLAYkKWzZtLREAdq0/NoYpSwCvYN7eSS0RA" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745742385151, - "AQMAAAABAAAABAAAAAWNahdni1LAroGFf8hMREBNXN8XZItSwGYNNoDJTERAhFTWEWOLUsBFdtUyzkxEQAWNahdni1LAroGFf8hMREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745673768959, - "AQMAAAABAAAACAAAAI43junqi1LADDmyaEBLRECqLKB54otSwMOecDtDS0RAuW8o792LUsCLtlgVWEtEQDyb7UPoi1LAvdlg9WdLREBL45x39otSwD0i0TRjS0RATkGm0feLUsDL/C7aX0tEQOKa9+v6i1LAfkZ+mlFLRECON44= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745769517055, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745730260991, - "AQMAAAABAAAAJgAAAHUrhvdujVLAxqZx441IREBf6nrPYo1SwGlbWvaRSERAa9LXRl6NUsCgEGDPpkhEQCQuXl9mjVLAFu6JPrNIRECqx/yQZo1SwKFpRfiySERA89HMMmeNUsDb/RiHskhEQMQxL5dnjVLAgs8Jg7JIREDFrU4= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745743269887, - "AQMAAAABAAAACAAAAEU7ZFdGi1LAF8oxc9NMRECSMx53N4tSwNddmWzYTERAbc3C6zKLUsAEIhlH7UxEQNZYwUA9i1LAkKk/KP1MREB8ENtiQ4tSwFa2PBv7TERA10BbHU+LUsAOw4d76kxEQPwhZaxQi1LA2TBAVONMREBFO2Q= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745700343807, - "AQMAAAABAAAACAAAAF424J3eilLALDWSr4FIREDzmPH+24pSwM2v64+CSERALwjLvNuKUsD5/EBKkEhEQKkTtzrbilLAQoMUz6hIREBjPyg624pSwFetVXipSERAb9flZeSKUsBWt0dnpkhEQP5FsvDoilLAZgAcj5FIREBeNuA= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745678880767, - "AQMAAAABAAAABgAAANJGsZ6rjFLA33bcPp9KRECeJwzAnIxSwMGtejmkSkRA6c9VNpiMUsCaJVATuUpEQKAIEl2YjFLApnjRTrlKREDlerXBrYxSwCmzKoeiSkRA0kaxnquMUsDfdtw+n0pEQA==" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745742712831, - "AQMAAAABAAAACgAAAAz8pEoGi1LAyBTvMvxMREB/bTpq94pSwA6sGCwBTURAATOf3vKKUsDajJcGFk1EQMd9Z6X5ilLAeuYUcSBNREBJ5nIy+opSwDKGhKsgTURA/wLa5v6KUsBkDCo/I01EQCtf+PEAi1LA3S+QpyRNREAaRQM= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745735372799, - "AQMAAAABAAAAJAAAAMRo/LY3jVLAZffo2dZJREBpnKZ0M41SwOsIim7qSURA+d1JyzSNUsD6t8l87ElEQDOCDd40jVLAKUIqbuxJREACiayTNY1SwJK7PRLsSURAMcec9zWNUsC0/EBQ60lEQG9eIGI2jVLAxnHpRupJRECXcwE= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745920643071, - "AQMAAAABAAAABgAAANgYEA6VjFLAkR28UKlHREBPXNQyhIxSwBUZLAvHR0RAmBuE5IeMUsDtzke4zEdEQEHcYMGWjFLAZtQKvsdHREA1JodKm4xSwMaoveWyR0RA2BgQDpWMUsCRHbxQqUdEQA==" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745680879615, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745691332607, - "AQMAAAABAAAACAAAABJ93QEMi1LA/HtxO+tHREAuI8Fo/opSwMCqtsfvR0RA7q8m3vmKUsCTnpWfBEhEQCAT1TAEi1LAk5HYfhRIREDBRCAOE4tSwIZzJIYPSERA1NammBeLUsAwKjeu+kdEQKCuRdAPi1LAtqLotu5HREASfd0= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745733701631, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745750937599, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745697001471, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745760833535, - "AQMAAAABAAAACAAAAKuxe0rUilLAkn5hnClLREC47C9rxYpSwBUYJ5UuS0RACie2XMGKUsC632YxQUtEQA+xuwjBilLASTCfrUNLREBEL7Yzy4pSwLZnZk9TS0RA7DYYE9qKUsB2BpdWTktEQPoEcZ7eilLAFWYNfTlLRECrsXs= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745735700479, - "AQMAAAABAAAAEwAAALQHo0M7jVLA7nzuKYdJRED5LMdlLI1SwMHtAyWMSURAWFPE3CeNUsCnGHn+oElEQLw8sTEyjVLAWSLn3LBJREAgbYiFPo1SwEqXlrusSURAcUhlED+NUsAdsK/OqklEQKg1mTlBjVLAWq13QqVJRECkwDw= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745681764351, - "AQMAAAABAAAACAAAAIwmheW8i1LA9Z0od7BLRECoGgsGrotSwGaO7HC1S0RAbX5Pe6mLUsAx9f1KyktEQPfWIdCzi1LAuMBZK9pLREBb/ekiwItSwL+n7gvWS0RA89LTEsSLUsBCCgbTzktEQJTYWTrHi1LAUUlsV8BLRECMJoU= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745762832383, - "AQMAAAABAAAACAAAAKR61yPcilLA6rhJVsZKRECJz8tEzYpSwEMmDU/LSkRAN9I95MyKUsB0iwoKzUpEQGNgliHLilLAj5WO2+NKRECiDEYN04pSwCjc7AjwSkRArPln7OGKUsATpx8Q60pEQNS9pnfmilLAx+nGNtZKRECketc= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236746168958975, - "AQMAAAABAAAABgAAAM3Ma5ePjVLABgLhpYRHREBvQmjgjo1SwDoHo++HR0RAaOCiCI+NUsBKW2wtiEdEQEa8qCGPjVLAAsfclIdHRECeZNxxj41SwKi7GpuFR0RAzcxrl4+NUsAGAuGlhEdEQA==" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745690546175, - "AQMAAAABAAAACAAAAA6OJ/bvilLAP+gT1+9IREAFskgY4YpSwGcMu8/0SERAjlqPpN2KUsD/cP+mBElEQJaTnyPeilLAwauyGAxJREDiiW7g5opSwO2B24cZSURAQ6RjvvWKUsA/liqPFElEQPk0Nkn6ilLARJ6/tv9IREAOjic= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745761718271, - "AQMAAAABAAAABwAAACJisl/KilLACXxc5hRMREA6iZpExYpSwPeBRZsWTERAM5Feqc6KUsC21skMIExEQFNIc/fJilLAZjhCsztMREBLH3ko0IpSwLyOcaE5TERAtBAItNSKUsABGnHHJExEQCJisl/KilLACXxc5hRMREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745679536127, - "AQMAAAABAAAAJQAAAMpDMQvgjFLAkZqtCC1KREDVLt0s0YxSwLIXdgMySkRAw6pqo8yMUsAAMSLdRkpEQCQFd3TRjFLAy+I6Q05KREDMy6rN1IxSwH/qWMs7SkRAsvtiPdWMUsAl1UwkPEpEQDNcjp7VjFLAviqGWjxKRECH/M4= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745736585215, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745729245183, - "AQMAAAABAAAALAAAADLQxNOGjVLAz4V57BFIRECv9KUCfo1SwPc4++AUSERAULkyenmNUsCkLsm5KUhEQDva5neBjVLAiAm1/zVIREDhkTR5gY1SwLvnIvo1SERAjqF22YGNUsBbvuRsNEhEQKLNoDyCjVLA1TJT4TJIREAoKbI= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745743368191, - "AQMAAAABAAAABQAAALiUwT8yi1LAbKa7KxRNRECbAPTUKYtSwLR2NPwWTURAqtP19CiLUsA8pAQAG01EQP+oH2Qxi1LAZxLDbhVNREC4lME/MotSwGymuysUTURA" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745756934143, - "AQMAAAABAAAABwAAABc4Zze2ilLAMPCOuGpLREDAxvqotIpSwA3YGShuS0RAlKxE5a6KUsAfGV4bektEQNHHlx6pilLAGoA+XYVLRECyHU09s4pSwO3XS/uBS0RABtHRyLeKUsDPwLQhbUtEQBc4Zze2ilLAMPCOuGpLREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745696215039, - "AQMAAAABAAAACwAAALXuThv0i1LAX6sObDdIRECpl7jr64tSwJZhodE7SERAviuZm+OLUsCvTCm8P0hEQCZswprji1LAt++JvD9IRECwmeqZ44tSwLKS6rw/SERA/9ZrM9uLUsDnWTwnQ0hEQCHyBz3Zi1LAvWeqxEVIREDXEmA= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745692545023, - "AQMAAAABAAAABAAAACgicrTcilLA83wbF0dIREArG9yo3IpSwKOPqGtLSERAQxO1YN2KUsDDjPsfSEhEQCgicrTcilLA83wbF0dIREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745724461055, - "AQMAAAABAAAACgAAAJkOOndRjVLA525JvwtJREB5rTCbR41SwEbzBw0PSURAGLZdEkONUsDlZEXmI0lEQHzYJmdNjVLAyKNsxDNJREDHUhhOT41SwGiaQiEzSURAVbkEolKNUsDNYIVVDUlEQBEZy0dSjVLAsnLBeQxJREDOZYg= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745695657983, - "AQMAAAABAAAADgAAABfJM7Gai1LA4vsNyVhIREAy4IrIlotSwAp217tqSERAAcDUG6GLUsAf2NiaekhEQGgqRvmvi1LAXRV4oXVIRECQ4BJKtItSwLrTGtBhSERAV+uEyrGLUsCmx90dYkhEQNwdcPWti1LAPRtxCGJIREAleDs= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745691987967, - "AQMAAAABAAAACAAAAKooeOkvi1LAZfYDlQlIREAcETM+LItSwNxVi2waSERAQI0OkTaLUsBkq7RLKkhEQIwUX25Fi1LAAEXKUiVIRECyP2O7SItSwFMVnSsWSERAGWXJvjmLUsDyC945D0hEQCyKW1sxi1LA8nYzsgpIRECqKHg= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745680977919, - "AQMAAAABAAAABwAAANOw7Nqbi1LA+8pp2QhMREA2MDX7jItSwAXBFNMNTERAT6RLcIiLUsCRtUitIkxEQCoAeomRi1LAx5K2qDBMREAKMIfLlItSwMZjWLspTERA4lHgu6CLUsD3hrhYEExEQNOw7Nqbi1LA+8pp2QhMREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745691430911, - "AQMAAAABAAAABQAAAKGuRdAPi1LAt6Lotu5HREDU1qaYF4tSwDAqN676R0RANvJdkBuLUsBQenpa+UdEQERPK2Eai1LAqpOQYfhHREChrkXQD4tSwLei6LbuR0RA" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745723346943, - "AQMAAAABAAAAHAAAAGvS10ZejVLAoBBgz6ZIREDprY9pT41SwNBJgsqrSERA0KrW4EqNUsAEx46jwEhEQE+zeTVVjVLA1lmHgdBIREAyjfLbX41SwFCY5O/MSERAe70cJGCNUsDujN4AzEhEQFS6eo1gjVLAyM+DecpIREA8O1w= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745921626111, - "AQMAAAABAAAAEQAAALDgyKhgjFLA/P5Tsv9HREAmjv71XIxSwJoITWsDSERALyUTaVuMUsDV0oQvBEhEQHb+39FZjFLAiBZRugRIREB0ovMzWIxSwJm8fgoFSERArsftklaMUsCQYVkfBUhEQO6VdPJUjFLAy8ax+ARIREBH/Co= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745743138815, - "AQMAAAABAAAABwAAAGANb07IilLAtysf65xMREDuu9YnvopSwHh/r0+gTERAv4zxC7uKUsAFUjo3wExEQG4OBTe/ilLAgZ2dn8ZMREB3/VMXzopSwJUttKbBTERAnAj/otKKUsB5yG3MrExEQGANb07IilLAtysf65xMREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745760931839, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745678749695, - "AQMAAAABAAAABgAAANo04wa/jFLANC5eaoVKREAJr1EosIxSwNDBDWWKSkRA0kaxnquMUsDfdtw+n0pEQOV6tcGtjFLAKbMqh6JKREATkUgmw4xSwD9Zlb+LSkRA2jTjBr+MUsA0Ll5qhUpEQA==" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745920511999, - "AQMAAAABAAAABgAAAB+VN+mljFLAHNlvlotHREDYGBAOlYxSwJEdvFCpR0RANSaHSpuMUsDGqL3lskdEQGtSUCeqjFLAWFdv661HREBgsmCwroxSwKzyKBOZR0RAH5U36aWMUsAc2W+Wi0dEQA==" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745913171967, - "AQMAAAABAAAADwAAAESB1YgDjVLA9iWvWS9HREBY9ydG/IxSwGiLbMgxR0RAzDxvvfeMUsBI0JegRkdEQNurMRECjVLA62YsflZHRECtKa/tEI1SwCGbfYNRR0RA4we/chONUsD4CLrtRUdEQOQrAhUOjVLAbjrjPT9HREDV1s8= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745673965567, - "AQMAAAABAAAABgAAAEvjnHf2i1LAPSLRNGNLREA8m+1D6ItSwL3ZYPVnS0RAVdVzueOLUsAaBV7PfEtEQFNTPCboi1LAeowAnINLREAh5ikQ9YtSwJ9Au7BmS0RAS+Ocd/aLUsA9ItE0Y0tEQA==" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745673408511, - "AQMAAAABAAAACAAAALu+rP0+jFLAFZO+8dZKREBwIdIeMIxSwFla8uvbSkRAsHDnACyMUsCsmrvU7kpEQD17KOM1jFLA0yHRc+dKREDaHSmNPoxSwP+vXF34SkRA6nN7k0eMUsBq4qLW7kpEQPSifVJJjFLAABdO0eZKREC7vqw= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745730457599, - "AQMAAAABAAAAFgAAAFC5Mnp5jVLApC7JuSlIRECCNz6dao1SwMdz+rQuSERAAxK1FGaNUsB/Ms+NQ0hEQMlUNGlwjVLAVfqAa1NIRECGoGfFfY1SwLPuP/FOSERAJLS19n2NUsCxrrQ2TUhEQG7V+1p+jVLAQGxP70lIREBH65U= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745733570559, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745672851455, - "AQMAAAABAAAADwAAAKGV1vf+i1LAVCD6s/9KREAkb9cY8ItSwL79760ES0RAS6x3juuLUsBGQryHGUtEQGwbygf0i1LAJW8njSZLRECDK1bD9ItSwG9lkhwmS0RA+S3loviLUsClnVOJJUtEQHh2Qfb6i1LACQXJVCRLREB4iTo= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745679634431, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745729343487, - "AQMAAAABAAAAMAAAALZOdEeBjVLAuBxeecZHRECt1b9qco1SwHIWjXTLR0RAkaFQ4m2NUsA83TBN4EdEQF/MqTZ4jVLASvizKvBHREC1qGG0go1SwD9bjabsR0RA/if6toKNUsDrgP2F7EdEQFP048mCjVLAMdgQxutHREB3w1Q= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745696870399, - "AQMAAAABAAAABgAAAEWNn5Zfi1LAgCxlxitIRECZTqSeXotSwGFA4TgwSERAUeOs8WiLUsC89vAXQEhEQHlj38Rzi1LAr3/MeDxIREBIZEZ1Y4tSwFVgYpswSERARY2fll+LUsCALGXGK0hEQA==" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745742909439, - "AQMAAAABAAAABwAAABpFAxQMi1LAKc7H7iBNREArX/jxAItSwN0vkKckTURAt+MycwOLUsDWUVVhJk1EQFn34MwHi1LAdi8zCSpNREDxR4LqC4tSwO4heSsuTURAVQ/kDRKLUsBEojUeKk1EQBpFAxQMi1LAKc7H7iBNREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745724559359, - "AQMAAAABAAAABwAAALzcUd5IjVLAoCu/nUhJREC81p8AOo1SwFsl3JhNSURAlu60dzWNUsAEdDVyYklEQGz0j8w/jVLASxiAUHJJRECAqMkSRo1SwIYIUzZwSURADEdtl0yNUsBlF5tVTklEQLzcUd5IjVLAoCu/nUhJREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745681633279, - "AQMAAAABAAAACAAAAHFDkU/Qi1LA+6JIo5ZLREBo0CpwwYtSwD31HZ2bS0RAjCaF5byLUsD1nSh3sEtEQJTYWTrHi1LAUUlsV8BLREA6o7cGzYtSwEBezGa+S0RAtPl/1dGLUsCbaA+VtUtEQKNTumjZi1LA7hZenqRLREBxQ5E= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745723445247, - "AQMAAAABAAAAIgAAAP+rvqxQjVLAp+2oWuVIREDrtEzPQY1SwKyfw1XqSERA9sF7Rj2NUsDw+usu/0hEQHmtMJtHjVLARvMHDQ9JRECZDjp3UY1SwOduSb8LSURAmy3qFFGNUsC8VxJ1C0lEQGJCHqRQjVLAXAF+CQtJREBL7aw= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745698312191, - "AQMAAAABAAAABgAAAG08OvjeilLAAk6WhphJREAXiRYn3opSwHp3p6WzSURASdt4DN6KUsCTTEAZt0lEQAdjDkXgilLAksMkW7ZJREC98xLQ5IpSwK4ObYKhSURAbTw6+N6KUsACTpaGmElEQA==" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236746168827903, - "AQMAAAABAAAACwAAAOMHv3ITjVLA+Ai67UVHRECtKa/tEI1SwCGbfYNRR0RASSCIQRuNUsCoXQVhYUdEQFJBCB4qjVLAnW47ZlxHREBBZzVeKo1SwL8+Jj9bR0RAEqjAJyiNUsBQpshPWkdEQGmfbbkjjVLAMafh2ldHRECVqug= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745679962111, - "AQMAAAABAAAABAAAAFysMkV+jFLAjn+zF9VKRED9fQTvdYxSwKC1NOLXSkRAGOmRNXSMUsBEG0rO30pEQFysMkV+jFLAjn+zF9VKREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745761587199, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745743237119, - "AQMAAAABAAAABQAAAHwQ22JDi1LAVrY8G/tMREDWWMFAPYtSwJCpPyj9TERAS5QPEDuLUsB8Dzo1B01EQIhW+cFAi1LAdqb41P5MREB8ENtiQ4tSwFa2PBv7TERA" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745700311039, - "AQMAAAABAAAABwAAAG/X5WXkilLAVrdHZ6ZIREBjPyg624pSwFetVXipSERARaTUKNuKUsDEqJYAvkhEQN6WNmrbilLAW8AID9BIREA5n/0t6opSwNKdHR/LSERANhnMuO6KUsAT0dxGtkhEQG/X5WXkilLAVrdHZ6ZIREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745678847999, - "AQMAAAABAAAABgAAAIj8SbvLjFLA/z9h1G1KREDZrW2Qw4xSwFc8lpBwSkRA2jTjBr+MUsA0Ll5qhUpEQBORSCbDjFLAP1mVv4tKREAyXwIQx4xSwCYt7JSHSkRAiPxJu8uMUsD/P2HUbUpEQA==" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745921167359, - "AQMAAAABAAAAEAAAAKpBmSoxjFLAVBV0KgpIRED1+tsXMIxSwFhnw8cLSERApyPfVi6MUsDCi142D0hEQMWNn+ksjFLAf5uV5RJIREDhGj3pKoxSwHFZtOIWSERACuO+oCiMUsBPdzyYGkhEQP3niRcmjFLAunh29B1IREBiNAs= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745696083967, - "AQMAAAABAAAABwAAAMyzxCgEjFLA7KJzdS9IREBu3gM3/ItSwO2BqIQySERASSVnIfSLUsB7p8hoN0hEQLXuThv0i1LAX6sObDdIRECckX6C9ItSwBocpQo4SERAsum3XwOMUsCzJPUQM0hEQMyzxCgEjFLA7KJzdS9IREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745692413951, - "AQMAAAABAAAACgAAADDKLXbmilLAKU0UcB5IRECm7lIW3YpSwAIPmpIhSERAH2+7+9yKUsCZ6njbK0hEQOuxW87cilLApw88aD1IREAE2tXF3IpSwEW4R5dASERAKCJytNyKUsDzfBsXR0hEQEMTtWDdilLAw4z7H0hIREA0Zyc= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745763586047, - "AQMAAAABAAAABwAAACTefjTeilLAaUjOVz5KREBedQAm14pSwAbCx7NASkRA1oz9atSKUsA23nP2ZkpEQCWaJR7VilLAnnrlCWhKRECJLfH844pSwGA6JBFjSkRAId0TiOiKUsC8dRE4TkpEQCTefjTeilLAaUjOVz5KREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745913270271, - "AQMAAAABAAAADgAAADTlw9XijFLANH0WDShHRECJpqD54YxSwC4EOU4oR0RAlFrR4d+MUsAMQHYZKUdEQEjurdHdjFLAvhj/DipHREALh6TK24xSwD4NLC4rR0RA2usdztmMUsBZZjV2LEdEQJ+QvQ7XjFLAZJ+IVy9HREC48dQ= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745699196927, - "AQMAAAABAAAABQAAALrsGn7bilLAuFfbgQJKRECdmFAc24pSwJ6EgfYJSkRA/2kpzNqKUsC/hKG4DkpEQMxPPffcilLAXBuwxQRKREC67Bp+24pSwLhX24ECSkRA" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745695526911, - "AQMAAAABAAAABQAAAIRo44zDi1LALs7IBFtIRECnGKKbwYtSwLNuDmBcSERA5S3Pf8GLUsACxJtwXEhEQMdPtWDDi1LAoZytz1tIRECEaOOMw4tSwC7OyARbSERA" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745680846847, - "AQMAAAABAAAABwAAABMxZEWvi1LAlzqABe9LRECXScBloItSwJGTPP/zS0RA07Ds2puLUsD7ymnZCExEQOJR4Lugi1LA94a4WBBMREAQZNodo4tSwNinvkcLTERA+pIzCrGLUsBy1VS98UtEQBMxZEWvi1LAlzqABe9LREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745769811967, - "AQMAAAABAAAABwAAAFbvBmOrilLANo0bQ+VLREDGnT2DnIpSwC5PyjvqS0RA56Ztm5iKUsBVd9Ul/EtEQG61ciKmilLAQ0i3mv9LREBlQC3NsYpSwDKctRgHTERAhdExt7WKUsB9tTEk9UtEQFbvBmOrilLANo0bQ+VLREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236746168598527, - "AQYAAAACAAAAAQMAAAABAAAABAAAAMYbEV82jVLAcHR5nmtHREBkv/dxNI1SwJdctkNsR0RA1WQIwjONUsCPAKtsb0dEQMYbEV82jVLAcHR5nmtHREABAwAAAAEAAAAHAAAAJsRYLUiNUsDiJhvEbkdEQPRca30xjVLAU6dZ2nk= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745750904831, - "AQMAAAABAAAABgAAAPBjVKnZilLAqUM/HgtNREAm6djIyopSwNdDPBcQTURAIwV0kcmKUsC1/KqrFU1EQPpPG/vQilLAY0o+SxdNREBfton844pSwGCcPf0aTURA8GNUqdmKUsCpQz8eC01EQA==" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236746168041471, - "AQMAAAABAAAAKQAAALlJtLqljVLATwxtDEFHRECyy2pnnY1SwOz08x1FR0RA5nEUeZyNUsB6mDtmSUdEQEx0E7ihjVLA5SbidFFHREDc8I68oY1SwC+Lt25RR0RAZ42B8qGNUsBV3b8dUUdEQBLr4S6ijVLAvRVN0VBHRED9au0= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745729441791, - "AQMAAAABAAAARQAAAMbjowCJjVLAr6r2jrBHREBxmc3PhY1SwJ8fwaCxR0RAtk50R4GNUsC4HF55xkdEQILDmUyGjVLADwMzL85HREBCxyVMho1SwFOq+gvOR0RAB4btRoaNUsBkrEybzUdEQEuJ9T2GjVLAxewqK81HREBtW0M= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745725771775, - "AQMAAAABAAAACgAAAFJBCB4qjVLAnW47ZlxHREBJIIhBG41SwKhdBWFhR0RAFkXhuBaNUsCFCVQ5dkdEQGpvzgwhjVLALRXnFoZHREDEyGTpL41SwD5XExyBR0RA9FxrfTGNUsBTp1naeUdEQGAQxh8qjVLA7wfWc31HREDVZAg= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745921495039, - "AQMAAAABAAAADgAAAFlfhFdzjFLA98u/xeRHREBmDkG8a4xSwLzO+y/yR0RAwYsfP2iMUsD/Vnzg9kdEQG5Dez5ojFLAohZT4fZHREAf+9Y9aIxSwOI2EuL2R0RAqA5AnWSMUsC/VRtS+0dEQBSyNJxkjFLAIAxmU/tHREAbjiw= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745692741631, - "AQMAAAABAAAABQAAAAHypEjeilLAPfK0NdpHREDE33S73YpSwKCHSLX6R0RAHFNGruCKUsCZ1964+UdEQEKn9DjlilLAkjkO4eRHREAB8qRI3opSwD3ytDXaR0RA" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745681731583, - "AQMAAAABAAAABgAAAFXVc7nji1LAGgVez3xLREAv+yDa1ItSwLy4RMmBS0RAcUORT9CLUsD7okijlktEQKNTumjZi1LA7hZenqRLREBSUzwm6ItSwHmMAJyDS0RAVdVzueOLUsAaBV7PfEtEQA==" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745673834495, - "AQMAAAABAAAABAAAAOKa9+v6i1LAfkZ+mlFLREBOQabR94tSwMz8LtpfS0RA7geSw/yLUsAOyQGVU0tEQOKa9+v6i1LAfkZ+mlFLREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745673277439, - "AQMAAAABAAAACgAAACvMfWZSjFLAqBazHb1KRECTx7aHQ4xSwGY8+BfCSkRAu76s/T6MUsAVk77x1kpEQPSifVJJjFLAABdO0eZKREC00c5LToxSwJBIEyflSkRAPBFL/U6MUsCHseZv3kpEQERbKEZRjFLAIPviJ+RKREBn51o= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745672720383, - "AQMAAAABAAAADAAAAMb1C2ESjFLAwV4t4OVKRECjZyCCA4xSwL2bNNrqSkRAoZXW9/6LUsBUIPqz/0pEQPs5jEwJjFLAu7vGkw9LREBzbpq/DoxSwJzt5sANS0RAx10I9RGMUsDA5h+HBUtEQOl6uNAUjFLAw0fzPAJLREARTko= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745758015487, - "AQMAAAABAAAABAAAACGkevmMilLAdpNQ/7JLREB+k+jbiopSwE9M1CO2S0RAW+DbZoyKUsA0AtaftUtEQCGkevmMilLAdpNQ/7JLREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745736552447, - "AQMAAAABAAAAEwAAABCG53ESjVLA7uNx0EJKREBIKI6TA41SwD21cMtHSkRAz+HDFwONUsAsgW4ESkpEQNhqxDYDjVLAFwaoE0pKRECpDesZBI1SwP/FAJ1KSkRANEitAgWNUsA42e4tS0pEQIxZd6UFjVLAmzLynUtKREB/wbU= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745743335423, - "AQMAAAABAAAABAAAABTMGT1Vi1LAduFtzeFMRED8IWWsUItSwNkwQFTjTERA10BbHU+LUsAOw4d76kxEQBTMGT1Vi1LAduFtzeFMREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745742778367, - "AQMAAAABAAAABwAAAMjQ938fi1LAVs71GgdNREDnlYqfEItSwHeVOhQMTURAGkUDFAyLUsApzsfuIE1EQFUP5A0Si1LARKI1HipNRECq0/X0KItSwD2kBAAbTURAmwD01CmLUsC0djT8Fk1EQMjQ938fi1LAVs71GgdNREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745682059263, - "AQMAAAABAAAACAAAAEoLgQV1i1LAgfkcgTxMREB+WKIlZotSwFEopXpBTERAh+WMmmGLUsDtleZUVkxEQJQOau9ri1LABCuuNWZMREAXVfyqeotSwIh9R0hhTERAFEPU6XqLUsBTWqrCYExEQOmOYFp/i1LAJG7MYUxMREBKC4E= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745692512255, - "AQMAAAABAAAABwAAAALwKmvcilLAqbNYemJIREB2vJ853IpSwNmdr/50SERAWHcvAdyKUsBCN9QYgkhEQPOY8f7bilLAza/rj4JIREBeNuCd3opSwCw1kq+BSERAdJmqKOOKUsAqlHvXbEhEQALwKmvcilLAqbNYemJIREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745724428287, - "AQMAAAABAAAABgAAAMZSGE5PjVLAaJpCITNJREB82CZnTY1SwMijbMQzSURAvNxR3kiNUsCgK7+dSElEQAxHbZdMjVLAZRebVU5JRECrqaUtTY1SwHvM1khLSURAxlIYTk+NUsBomkIhM0lEQA==" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745742221311, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745699295231, - "AQMAAAABAAAACQAAAAGEYg3milLAh7X6E9tJREAZaS/H3IpSwBcHzi3eSURAuf/rg9yKUsBGXMgC6klEQPc6uO3bilLAfz1k//lJREC67Bp+24pSwLhX24ECSkRAzE8999yKUsBcG7DFBEpEQJjNyNXrilLAKAnxzP9JRECWddE= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745680945151, - "AQMAAAABAAAABQAAAFz96SLAi1LAv6fuC9ZLRED31iHQs4tSwLjAWSvaS0RAEzFkRa+LUsCXOoAF70tEQPqSMwqxi1LAc9VUvfFLREBc/ekiwItSwL+n7gvWS0RA" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745691398143, - "AQMAAAABAAAACAAAADbyXZAbi1LAUHp6WvlHREDU1qaYF4tSwDAqN676R0RAwUQgDhOLUsCGcySGD0hEQHg05WAdi1LAAJhaZR9IREAcETM+LItSwNxVi2waSERAqih46S+LUsBl9gOVCUhEQIlyqGAmi1LAkGwjOwJIREA28l0= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745723314175, - "AQMAAAABAAAAKwAAADKN8ttfjVLAUJjk78xIREBPs3k1VY1SwNZZh4HQSERA/6u+rFCNUsCn7aha5UhEQKQUpW5ZjVLAaqP9zfJIREAdsel8WY1SwFHP2pzySERAloK+AlqNUsC/MdhB8UhEQOsjZ41ajVLArDxZTPBIREBNQOs= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745698181119, - "AQMAAAABAAAABwAAAEwaBObtilLAFHgZ0XdJREDPsSLQ34pSwH2V0IZ8SURAbTw6+N6KUsADTpaGmElEQL3zEtDkilLArg5tgqFJREAPXl6u84pSwHArsImcSURAhv9MOfiKUsDgNP+wh0lEQEwaBObtilLAFHgZ0XdJREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745762013183, - "AQMAAAABAAAACAAAAKlDKdKfilLAYkKWzZtLRECgAEmmm4pSwB7xfzKdS0RAIaR6+YyKUsB2k1D/sktEQFvg22aMilLANALWn7VLREDT9tu6lopSwGiz7YDFS0RAUASPmqWKUsCzuEiIwEtEQGu3KyaqilLA9NOVrqtLRECpQyk= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745679831039, - "AQMAAAABAAAABQAAAOnPVTaYjFLAmiVQE7lKREBaF51XiYxSwE3/3A2+SkRAf3rCJ4aMUsDRNUqyzEpEQKAIEl2YjFLApnjRTrlKREDpz1U2mIxSwJolUBO5SkRA" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745729540095, - "AQMAAAABAAAAKQAAALWoYbSCjVLAP1uNpuxHREBfzKk2eI1SwEr4syrwR0RAnZQ4rnONUsCW1mwDBUhEQK/0pQJ+jVLA9zj74BRIREAy0MTTho1SwM+FeewRSERAqDP8z4aNUsCKDkn/D0hEQNO6OsaGjVLA+gFRSQ9IRECKmx8= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745736323071, - "AQMAAAABAAAAFwAAAIGoyRJGjVLAhghTNnBJREBs9I/MP41SwEsYgFBySURAtAejQzuNUsDufO4ph0lEQFwy2K1AjVLAX3L/eo9JREDjJ6yvQI1SwEDIwS2PSURAKCQp/ECNUsCZuHejjElEQDpT429BjVLA+8AvVItJRECuUCM= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745921593343, - "AQMAAAABAAAABgAAAE9c1DKEjFLAFRksC8dHREBZX4RXc4xSwPfLv8XkR0RASZJXfnSMUsBAZMeK5kdEQGDnR1uDjFLAFsGbkOFHRECYG4Tkh4xSwO3OR7jMR0RAT1zUMoSMUsAVGSwLx0dEQA==" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236746167582719, - "AQMAAAABAAAABgAAAE02J9mMjVLAjH4gNU1HREDTiTlhcY1SwPV8haFaR0RAFsj/N3qNUsCVSRI1aEdEQHg6dBSJjVLAMJzlOWNHRECjlbOcjY1SwBiZeWFOR0RATTYn2YyNUsCMfiA1TUdEQA==" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745725313023, - "AQMAAAABAAAACAAAAKmCacK3jFLAJQi4Ym9HREDEmav/s4xSwCv2LqVwR0RAOggkCKeMUsAoi2mciUdEQB+VN+mljFLAHNlvlotHREBgsmCwroxSwKzyKBOZR0RAHUoWjb2MUsDCSskYlEdEQFPAEBbCjFLAbK2JQH9HRECpgmk= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745921036287, - "AQMAAAABAAAADAAAACj9D6ZEjFLASZNCgQBIRECd6uHiQIxSwKd1+6IASERAsP9vKT2MUsBo2DU7AUhEQN7N6oI5jFLA86gvRgJIREALTGHqNoxSwC+sruUDSERA6dS0bzSMUsCkpNQMBkhEQNFvTSQyjFLASmvOsghIRECqQZk= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745681829887, - "AQMAAAABAAAABAAAADqjtwbNi1LAQF7MZr5LRECU2Fk6x4tSwFFJbFfAS0RA89LTEsSLUsBCCgbTzktEQDqjtwbNi1LAQF7MZr5LREA=" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745673932799, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745698508799, - "AQMAAAABAAAACAAAAOwKhlXiilLAAzlNYC5JRECY1qId4IpSwMenLx4vSURAojQ9VuCKUsCemTlNNElEQCGXDLvgilLAyTTvOEVJREBRTn3L4IpSwJM6VYtVSURAsv67HeiKUsAhJ6MYU0lEQDSJpqjsilLAw1ocQD5JREDsCoY= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236746168467455, - "AQMAAAABAAAABQAAAF3ydb1fjVLAay1OQGNHREAmxFgtSI1SwOMmG8RuR0RAuIiAok2NUsAnYz8md0dEQBjvBX9cjVLAIy0/K3JHREBd8nW9X41SwGstTkBjR0RA" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745672818687, - "AQMAAAABAAAACAAAAPvMF8oljFLAZv5VDMxKREA31z/rFoxSwIGabgbRSkRAxvULYRKMUsDBXi3g5UpEQFLyw7UcjFLAadrhv/VKREArQQmiJ4xSwFXvABjySkRAsHDnACyMUsCsmrvU7kpEQHAh0h4wjFLAWVry69tKRED7zBc= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745758113791, - "AQMAAAABAAAADQAAAP+Cb7V5ilLAZdUVkNBLREDS+7j2eIpSwIjMzMfTS0RAcQsE0XiKUsDcdJ9q1EtEQJE1R9x3ilLA4cEU4NlLRED8yekvd4pSwJt2BnPfS0RAbF7jzXaKUsAmLEsW5UtEQF9QvLZ2ilLA0RtOvOpLREBJd6Y= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745679601663, - "AQMAAAABAAAAIQAAACzMfz75jFLACcGj7DdKRECfEilg6oxSwC5oh+c8SkRA6TTabOmMUsC/y8lFQUpEQEvVcqTpjFLA1/1jR0FKRED+DH1K6oxSwE5D04ZBSkRArUzC/OqMUsAfZjgpQkpEQAgR94vrjFLApwFgu0JKRECaQlU= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745696837631, - "AQMAAAABAAAACAAAAHpj38Rzi1LAr3/MeDxIREBR46zxaItSwLz28BdASERAfuxfZ2SLUsBWVR7wVEhEQKGzfLpui1LAClk5z2RIREBoy+iXfYtSwIHgDtZfSERArN8hIoKLUsCHLdP9SkhEQIvvORt9i1LAUjWXQ0NIREB6Y98= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745742876671, - "AQMAAAABAAAACAAAALVpaxXtilLAe1PASvFMREDZhwM13opSwAy7zkP2TERA8GNUqdmKUsCpQz8eC01EQF+2ifzjilLAYJw9/RpNREClgCsC5IpSwAfxVf4aTURAATOf3vKKUsDajJcGFk1EQH9tOmr3ilLADqwYLAFNREC1aWs= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745735536639, - "AQMAAAABAAAACQAAAFVerKgtjVLA+GJxtsVJREAQrqbKHo1SwBhKf7HKSURAPuKLQRqNUsB0UBCL30lEQEqvipYkjVLAmr+hae9JREBpnKZ0M41SwOsIim7qSURAxGj8tjeNUsBl9+jZ1klEQBupNLw3jVLAlaqSz9VJREBS+/w= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745763782655, - "AQMAAAABAAAABQAAAGNgliHLilLAj5WO2+NKREDBzi5tyIpSwG9it+oGS0RAu1bxgc6KUsCuVUziBEtEQKIMRg3TilLAKNzsCPBKREBjYJYhy4pSwI+VjtvjSkRA" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745742319615, - "AQMAAAABAAAACAAAAI8a3MJZi1LAZMc/n7lMREB5rKniSotSwMfAuJi+TERARTtkV0aLUsAXyjFz00xEQPwhZaxQi1LA2TBAVONMREAUzBk9VYtSwHbhbc3hTERAhFTWEWOLUsBEdtUyzkxEQE1c3xdki1LAZg02gMlMRECPGtw= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745699393535, - "AQMAAAABAAAABgAAABZXx+bZilLAXJ/C5BtKREDY2MHI2IpSwGK2GsgpSkRAXnUAJteKUsAGwsezQEpEQCTefjTeilLAaUjOVz5KRED0gZ+/4opSwAiY0H4pSkRAFlfH5tmKUsBcn8LkG0pEQA==" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745695723519, - "AQMAAAABAAAACwAAAMdPtWDDi1LAoZytz1tIREDlLc9/wYtSwALEm3BcSERA8V9t4LyLUsCXZJYwX0hEQLhP40u5i1LA8J8uq2BIREAdbySWtYtSwIHEuqdhSERAkOASSrSLUsC60xrQYUhEQGgqRvmvi1LAXRV4oXVIREBplqY= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745770565631, - "AQMAAAABAAAABgAAAL+M8Qu7ilLABVI6N8BMRED3i+BMuopSwKqQ99/HTERAau9xW7iKUsDrU7c/3ExEQAYORKu6ilLA97vqedtMREBuDgU3v4pSwIGdnZ/GTERAv4zxC7uKUsAFUjo3wExEQA==" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745692053503, - "AQMAAAABAAAACAAAALI/Y7tIi1LAUxWdKxZIRECMFF9uRYtSwABFylIlSERABR1RwU+LUsAry+YxNUhEQJlOpJ5ei1LAYUDhODBIREBFjZ+WX4tSwIAsZcYrSERATCsLel2LUsDt3mQjKUhEQF2Yfk9Ni1LAbw26ShhIRECyP2M= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745691496447, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745762668543, - "AQMAAAABAAAADQAAAMnMvqDSilLAgrRmC4BKREBM92Bw0opSwNJWGbGCSkRApebsSdKKUsAXBA/MhEpEQABiZi3SilLAPll8WYZKRECduvzL0YpSwMTZ76eLSkRA5LxJsNGKUsAQ+JMpjUpEQAzII43RilLAL38yF49KREAmVOM= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745698279423, - "AQMAAAABAAAACAAAAAdjDkXgilLAksMkW7ZJREBJ23gM3opSwJNMQBm3SURAKLotRd2KUsBYv2fx0ElEQHze3N7cilLA+ZyQA9pJREAZaS/H3IpSwBcHzi3eSURAAYRiDeaKUsCHtfoT20lEQGEgaZjqilLA2ustO8ZJREAHYw4= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745769451519, - "AQMAAAABAAAABQAAAOaTaHXEilLAzKLeUl9MRECiHMIJwopSwI4IEFl4TERAnkmchcKKUsDQhaoveExEQNM7QxHHilLAvESOVWNMREDmk2h1xIpSwMyi3lJfTERA" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745736421375, - "AQMAAAABAAAAKgAAAPYNYtkljVLAoMGf+yhKRED1SRz7Fo1SwK7tr/YtSkRAEIbncRKNUsDu43HQQkpEQMJl/UgVjVLAWxMyLUdKREBindMnF41SwDO4waRGSkRAAF87kheNUsBzHJ6iRkpEQKRTDvsXjVLAkmamhkZKREBYXJ4= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745921691647, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745681928191, - "AQMAAAABAAAACQAAAE+kS3CIi1LAkbVIrSJMRECXioCQeYtSwDdI4qYnTERASguBBXWLUsCB+RyBPExEQOmOYFp/i1LAJG7MYUxMREAgk2Z1hYtSwMLhAVdKTERAEo+1M4qLUsBorzhBQExEQBOkhXCRi1LA14/F3TBMREAqAHo= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - true, - 622236745696051199, - null - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745692381183, - "AQMAAAABAAAACAAAADRnJz7silLALShqJ0NIREBDE7Vg3YpSwMOM+x9ISERAKxvcqNyKUsClj6hrS0hEQALwKmvcilLAqLNYemJIREB0maoo44pSwCqUe9dsSERAeyozBvKKUsBHaODeZ0hEQJGg55D2ilLAR4fQBlNIREA0Zyc= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745763553279, - "AQMAAAABAAAACQAAAIkt8fzjilLAYDokEWNKREAlmiUe1YpSwJ565QloSkRA9Y6J9tOKUsAS1TZWbUpEQOeGhijTilLAx5TjnHhKREDJzL6g0opSwIK0ZguASkRAydGT5tqKUsDclGXDjEpEQFSmdcXpilLA34yayodKRECnYZo= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236746169122815, - "AQMAAAABAAAAHwAAAHg6dBSJjVLAMJzlOWNHREAWyP83eo1SwJVJEjVoR0RATISqr3WNUsAIF4UNfUdEQDCY3QOAjVLAz4TZ6oxHREBvQmjgjo1SwDoHo++HR0RAzcxrl4+NUsAGAuGlhEdEQKUCd7+PjVLA2zRwoINHREALQHY= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745680257023, - "AQMAAAABAAAACAAAAMv1sAJ/jFLAcpOKLq5KREBk4/ojcIxSwBpC/CizSkRAH7MammuMUsD+KcoCyEpEQP19BO91jFLAoLU04tdKREBbrDJFfoxSwI5/sxfVSkRAf3rCJ4aMUsDRNUqyzEpEQFoXnVeJjFLATf/cDb5KREDL9bA= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745672916991, - "AQMAAAABAAAACAAAAHNumr8OjFLAnO3mwA1LRED7OYxMCYxSwLu7xpMPS0RAaicA9gaMUsAFuptQGktEQDmF8sQHjFLAJ/EeZRlLRECgGY+oCYxSwPpCVQAXS0RATQaFdguMUsAxYeF1FEtEQNcrki0NjFLAOGqIxxFLREBzbpo= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745690710015, - "AQMAAAABAAAABQAAACb5OGLcilLAURRXke1IREDfA6Iq3YpSwAHiVIL9SERAjlqPpN2KUsD/cP+mBElEQAWySBjhilLAZwy7z/RIREAm+Thi3IpSwFEUV5HtSERA" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745761882111, - "AQMAAAABAAAACgAAALIdTT2zilLA7ddL+4FLREDRx5ceqYpSwBqAPl2FS0RAlwJC5qiKUsDU7QnLhUtEQCcu9q2iilLAtVg/M5FLREAdq0/NoYpSwCvYN7eSS0RAqUMp0p+KUsBiQpbNm0tEQGu3KyaqilLA9NOVrqtLRECiLss= (truncated)" - ], - [ - "Staten Island", - "99", - "99", - "Freshkills Park", - false, - 622236745750872063, - "AQMAAAABAAAABgAAAAYORKu6ilLA97vqedtMREBq73FbuIpSwOtTtz/cTERAQv5NjLSKUsDKYGcwBE1EQFzYInTAilLAKfO+NQBNREBr5eX/xIpSwATDXFvrTERABg5Eq7qKUsD3u+p520xEQA==" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751937470463, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751857385471, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751873212415, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751936913407, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751933079551, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751873376255, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751937077247, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751872098303, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751872819199, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751937241087, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751936684031, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751871705087, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751938125823, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751873146879, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751937568767, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751872589823, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751933014015, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751873310719, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751937011711, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751872032767, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751872753663, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751933177855, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751937175551, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751933341695, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751871639551, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751937339391, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751938060287, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751873081343, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751871803391, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751873245183, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751936946175, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751871967231, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751872688127, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751873409023, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751937110015, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751872851967, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751937273855, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751872294911, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751937994751, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751873015807, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751936716799, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751937437695, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751871901695, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751937601535, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751933046783, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751938322431, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751873343487, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751937044479, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751872786431, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751937208319, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751871672319, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751937372159, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751936815103, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751871836159, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751872557055, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751932981247, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751938256895, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751873277951, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751936978943, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751872720895, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751872884735, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751933308927, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751937306623, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751857221631, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751873048575, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751936749567, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751871770623, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751617425407, - "AQMAAAABAAAABwAAAFuk1FT4fVLAaMHh5nxRRECVVYdY9H1SwD3Fi8d9UURADxBsX/d9UsBOjuzxklFEQAULzkP5fVLAuN//AaBRRECTVndQ/H1SwNAtCQCfUURAb3JF6AB+UsBq1q8nilFEQFuk1FT4fVLAaMHh5nxRREA=" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751877046271, - "AQMAAAABAAAABwAAAEGj2xvoflLAoddU7BdSRECDgRJf4H5SwCrlDPAdUkRAYSTMy9l+UsBwJFt5I1JEQO2jvNjTflLA02SqbChSREDWWrny1X5SwGVRuqsrUkRAHANr2OR+UsBt5Ja+JlJEQEGj2xvoflLAoddU7BdSREA=" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751872458751, - "AQMAAAABAAAABwAAANEgXKx6flLAQcX68e1QREAZnLrUdX5SwEXm0YvvUERAs7lwPXF+UsCcDwRkBFFEQCyjM4t7flLAtKykTxRRREA4YGM/fn5SwLNpy2oTUURAhY2b4n1+UsA3GTSeD1FEQNEgXKx6flLAQcX68e1QREA=" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751871999999, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751933276159, - "AQMAAAABAAAABgAAAJNWd1D8fVLA0C0JAJ9RREAFC85D+X1SwLjf/wGgUURAky60m/59UsDJ767lxFFEQA27WgYCflLAhN+oxMNRREDdCiueBn5SwAyAOuyuUURAk1Z3UPx9UsDQLQkAn1FEQA==" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751876882431, - "AQMAAAABAAAACAAAAAyzeGDpflLAYhwA8BZSREBBo9sb6H5SwKHXVOwXUkRAHANr2OR+UsBt5Ja+JlJEQFBI/ybvflLAlUNhqjZSREC0zLMM/n5SwFigIr0xUkRALIZ1i/5+UsAfvWh9L1JEQLRsbafuflLAqiI0+RtSREAMs3g= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751938158591, - "AQMAAAABAAAABgAAAPOiYf1QflLAByZ+nZFSRECuJFQXQn5SwDEvE4qWUkRA3uhx7D5+UsBjMLvqpFJEQNvH5kpHflLAoYs4I55SREDgVFj3Un5SwHEgOquUUkRA86Jh/VB+UsAHJn6dkVJEQA==" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751875506175, - "AQMAAAABAAAACAAAAH/eC4drflLAF7Q8oN9QREB2wvSuYH5SwIGh+jXjUERAkiKR8Vt+UsDtGiWu51BEQAgOuApYflLAfDblZPlQREBvQ2RYYn5SwKmGklAJUURAs7lwPXF+UsCcDwRkBFFEQBmcutR1flLARebRi+9QREB/3gs= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751936782335, - "AQMAAAABAAAACQAAACzUaK99flLAJ2tl14JSREDP+mvJbn5SwO5DJ8SHUkRAbjHMMWp+UsCSTSqdnFJEQBH63i9uflLAX9E8yKJSREAE7b91dX5SwNFT2jGhUkRAA4jWq4R+UsBX1BnYnVJEQML7W5iFflLAOCL0pJ1SREB7Rdw= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751857319935, - "AQMAAAABAAAABgAAAIIjIicff1LA7++ImFdSREDtoN/4HH9SwI1HWoBhUkRAIoC1Ryd/UsDInhZscVJEQHrUUoUxf1LA1/bvCG5SREBqbveYJn9SwM4+8qFgUkRAgiMiJx9/UsDv74iYV1JEQA==" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751933112319, - "AQMAAAABAAAACAAAANq8cCgTflLAyqZKEzJSRECf3uGPDn5SwDhJFpgzUkRAdwxdVg9+UsBaq8O4OFJEQHrwDZcSflLA1fkX0U5SREBdN7n3E35SwB2yGndYUkRAUgWc3hh+UsBXI2zYVlJEQIwkdXYdflLA4aSp/0FSREDavHA= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751937535999, - "AQMAAAABAAAABwAAAGXD31O5flLAxhigJftRREB2j2sHt35SwFTBM+j7UURAJSUmcLJ+UsC2pQfBEFJEQFD/jL68flLANHLrrCBSREBwl37cvH5SwMO9BKMgUkRATYF/ibl+UsAGpXM0/VFEQGXD31O5flLAxhigJftRREA=" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751877013503, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751938289663, - "AQMAAAABAAAABwAAAH87dgTPflLA3aFLmshSREC9aFoewH5SwG2/aIfNUkRAAUFvPb1+UsATCp2Z2lJEQHJVohfOflLAbw1c2dZSREDBgg/m1X5SwLyEbhXVUkRAiBumsNZ+UsAgQ1l01FJEQH87dgTPflLA3aFLmshSREA=" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751858368511, - "AQMAAAABAAAABgAAAGCGxjMPf1LAAyT6C6BSREDi4M5NAH9SwB2oVfmkUkRAcztnW/x+UsBh+V3mtlJEQMbBKOUOf1LA6EEXhqhSREBc+TC8En9SwBuE4IClUkRAYIbGMw9/UsADJPoLoFJEQA==" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751872425983, - "AQMAAAABAAAACAAAADhgYz9+flLAs2nLahNRREAsozOLe35SwLSspE8UUURAT47n83Z+UsDh3usnKVFEQO+jvkGBflLAneaXEzlRREBzq4fSgX5SwP17ueM4UURAxlGhNoF+UsDFpImBMlFEQN9+Qs5/flLAsfwLwCNRREA4YGM= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751933243391, - "AQMAAAABAAAACQAAAA27WgYCflLAhN+oxMNRRECTLrSb/n1SwMnvruXEUURAHMxUi/99UsCnHwVcy1FEQCnT7qsAflLAFwFvI9NRREAct22yAn5SwGnKB5DjUURACQfUowN+UsC3++Dj6VFEQAoYULwHflLAgARpiehRREDDmyI= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751877308415, - "AQMAAAABAAAABwAAANA9HiyyflLAKsS9MrBRREDto16Zq35SwKGyJV+yUURAhJ0dAqd+UsC+hs83x1FEQMcbXFCxflLAWIGcI9dRRECMc4qvtX5SwBZSjLHVUURADJleB7N+UsDJ0u02uVFEQNA9HiyyflLAKsS9MrBRREA=" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751857287167, - "AQMAAAABAAAACAAAAHrUUoUxf1LA1/bvCG5SREAigLVHJ39SwMieFmxxUkRAEBm7sCJ/UsCFj05FhlJEQM4FhrQqf1LAw2pIppJSREAOYLMgP39SwI/dAZaCUkRAm3Zi0j9/UsDikdVuf1JEQLpltTAzf1LA50pRFXBSREB61FI= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751931244543, - "AQMAAAABAAAABQAAADYY5W4yflLA+hE0C69SREBFdWmZLn5SwM6tkU+wUkRAITioFS5+UsA1gJKlslJEQI2xqw0xflLAhlAUKrBSREA2GOVuMn5SwPoRNAuvUkRA" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751937961983, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751937503231, - "AQMAAAABAAAACgAAAHCXfty8flLAw70EoyBSREBQ/4y+vH5SwDRy66wgUkRAG2NFJ7h+UsCDXtSFNVJEQKdrwHXCflLAmJPDcUVSRECsl4Vb0X5SwEi9sYRAUkRA1lq58tV+UsBlUbqrK1JEQO2jvNjTflLA02SqbChSRED7TwA= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751875309567, - "AQMAAAABAAAABQAAAI/Jx15tflLAtVNtQtdQRED0ZJE1aX5SwF5T4ivbUERAdsL0rmB+UsCBofo141BEQH/eC4drflLAF7Q8oN9QRECPycdebX5SwLVTbULXUERA" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751857876991, - "AQMAAAABAAAABAAAAJt2YtI/f1LA4pHVbn9SREANYLMgP39SwJDdAZaCUkRAUfHMH0F/UsCnG/8DgVJEQJt2YtI/f1LA4pHVbn9SREA=" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751857418239, - "AQMAAAABAAAABgAAABAZu7Aif1LAhY9ORYZSRECM+NbKE39SwFmquzKLUkRAYIbGMw9/UsADJPoLoFJEQFz5MLwSf1LAGoTggKVSREDOBYa0Kn9SwMNqSKaSUkRAEBm7sCJ/UsCFj05FhlJEQA==" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751934128127, - "AQMAAAABAAAABwAAAFIFnN4YflLAVyNs2FZSREBdN7n3E35SwB2yGndYUkRADq4U8BV+UsAv3oBDZlJEQC/lyEIZflLAn0XIX31SRECdR9mUHn5SwKwRrp17UkRAlZq0LCN+UsDHi9bEZlJEQFIFnN4YflLAVyNs2FZSREA=" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751871934463, - "AQMAAAABAAAACwAAADNjLjFHflLA9ZPyS/tQRED5HWuIQn5SwLU1S73/UERAk673Ezx+UsCJ8yHiBVFEQGxUb1k6flLA491zgQdRREBoSLPkN35SwMl0Nc0JUURAYLL4rTN+UsBQSQivDVFEQPPOX70wflLA7G05exBRREBxmx0= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751877275647, - "AQMAAAABAAAACQAAAItziq+1flLAFlKMsdVRREDHG1xQsX5SwFiBnCPXUURAZOMYuax+UsCpXVv861FEQHaPawe3flLAVMEz6PtRREBlw99TuX5SwMYYoCX7UURA45/9brh+UsArw/hc8lFEQPNdute2flLASqcZPuFRREBTxsU= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751938093055, - "AQMAAAABAAAACQAAAC+qEMk3flLA3xGqnYZSREC3CgbjKH5SwDjkI4qLUkRACYQoSyR+UsBmcRBjoFJEQEV1aZkuflLAzq2RT7BSREA2GOVuMn5SwPoRNAuvUkRAVSQkpzZ+UsDuy92dq1JEQN7ocew+flLAYzC76qRSRECuJFQ= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751937634303, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751875440639, - "AQMAAAABAAAABwAAAJIikfFbflLA7BolrudQREDoDGM9W35SwES4C1joUERAz+nU8Eh+UsBaYBih+VBEQDNjLjFHflLA9pPyS/tQREC0da4lSX5SwF56WFH+UERACA64Clh+UsB8NuVk+VBEQJIikfFbflLA7BolrudQREA=" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751857254399, - "AQMAAAABAAAACAAAAOPvGj0Mf1LAIyI1YEBSREBJx15bCH9SwCxI4KhBUkRACrdSxAN/UsBhsvSBVlJEQLjgERMOf1LAu8G9bWZSREDtoN/4HH9SwI1HWoBhUkRAgiMiJx9/UsDv74iYV1JEQBohZ+0Yf1LALNL4CVBSREDj7xo= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751938387967, - "AQMAAAABAAAABgAAAMIN7pqcflLAddQKnLJSREDu5sdWln5SwAzhW660UkRAuy2L6pl+UsAEur0o2lJEQLEbElKiflLAqip2YddSREDiZIzppn5SwKAYQYjCUkRAwg3umpx+UsB11AqcslJEQA==" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751872983039, - "AQMAAAABAAAABgAAAGxjw5QdflLAy2ycskZRREAJHTUFFX5SwNam5YZJUURAJly2lxh+UsDDCRsAb1FEQNj2xUojflLA0GDedmtRRECD/mfiJ35SwCUXkp5WUURAbGPDlB1+UsDLbJyyRlFEQA==" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751872065535, - "AQMAAAABAAAABwAAAPPOX70wflLA7G05exBRRECTTLzTIn5SwPmVk7gdUURALiBBVRp+UsAHwLS8JVFEQJY3YywiflLAkytl2jFRREC70JMRMX5SwLATFe4sUURAcZsdqTV+UsATWuQVGFFEQPPOX70wflLA7G05exBRREA=" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751876947967, - "AQMAAAABAAAACAAAACyGdYv+flLAH71ofS9SREC0zLMM/n5SwFigIr0xUkRAScdeWwh/UsAsSOCoQVJEQOPvGj0Mf1LAIyI1YEBSREASkbjMC39SwL1netU/UkRAb6UCegN/UsBMUi+kNVJEQBeAuNH+flLA6ayz0y9SREAshnU= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751938224127, - "AQMAAAABAAAACQAAAK9apc+1flLApTI/m71SREDiZIzppn5SwKAYQYjCUkRAsRsSUqJ+UsCqKnZh11JEQHGZHYanflLAy/Mza99SREByYXbfqX5SwPFANuzeUkRAGdIcF71+UsDgpCSi2lJEQAFBbz29flLAEwqdmdpSREC9aFo= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751936847871, - "AQMAAAABAAAACAAAAJj82+OWflLA4O2/1o1SREB7Rdz9h35SwJL9nMOSUkRAwvtbmIV+UsA4IvSknVJEQPTpldiTflLAIjoIkJpSREDu5sdWln5SwAzhW660UkRAwg3umpx+UsB11AqcslJEQK4kZjKhflLAuMnqwp1SRECY/Ns= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751858302975, - "AQMAAAABAAAACAAAAEMO/v71flLAFRJpDZVSREC9RAkZ535SwNdeqfqZUkRAA+3OgeJ+UsC/4d/TrlJEQOs4ndDsflLAOpfkv75SREAAuqtD9n5SwMiy65+7UkRAcztnW/x+UsBh+V3mtlJEQOLgzk0Af1LAHahV+aRSREBDDv4= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751934095359, - "AQMAAAABAAAACAAAAJ1H2ZQeflLArBGunXtSREAv5chCGX5SwJ9FyF99UkRAHo2Dqxl+UsAb0CA4gFJEQHuwDmsdflLAyU+QvJlSREB0mRmfHn5SwKi26EKiUkRACYQoSyR+UsBmcRBjoFJEQLcKBuMoflLAOOQjiotSRECdR9k= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751872360447, - "AQMAAAABAAAACAAAAAaMDJquflLAcwfuuIpRRECZJ3PipX5SwItVz5qNUURAM1M0S6F+UsBYIWRzolFEQO2jXpmrflLAobIlX7JRREDQPR4ssn5SwCrEvTKwUURATgz9BLF+UsDnFL0PpFFEQID6xbuvflLAgunWWZdRREAGjAw= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751873114111, - "AQMAAAABAAAABgAAACvdMOoSflLAIaZiHHdRRED7Shf9CX5SwHF8jAN5UURAW6TUVPh9UsBoweHmfFFEQG9yRegAflLAatavJ4pRREAlbLPND35SwKlPeTuFUURAK90w6hJ+UsAhpmIcd1FEQA==" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751872196607, - "AQMAAAABAAAABgAAAIhy58SpflLArzZOHFZRREA/9d3nl35SwGcifcpZUURAs3mJnJZ+UsAMBfARWlFEQHammSugflLAemqZ1mhRRECi9nweq35SwN4W1DdlUURAiHLnxKl+UsCvNk4cVlFEQA==" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751871737855, - "AQMAAAABAAAACgAAAKVccKp8flLAgyH0601RREDlKDfFbX5SwJDUldhSUURALtjSLWl+UsDll/iwZ1FEQC+Su3tzflLAYCjInHdRREALJQthgn5SwCTXHLByUURAtp5b+IZ+UsCpk6vXXVFEQAD4Rr6GflLAmcTufV1RRED2F9A= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751931179007, - "AQMAAAABAAAABwAAAAmEKEskflLAZnEQY6BSREB0mRmfHn5SwKm26EKiUkRAK9x9FyF+UsChkx/Ds1JEQIdU3kIiflLAz6NziLxSREAhOKgVLn5SwDaAkqWyUkRARXVpmS5+UsDOrZFPsFJEQAmEKEskflLAZnEQY6BSREA=" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751877079039, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751938355199, - "AQMAAAABAAAABQAAALEbElKiflLAqip2YddSREC7LYvqmX5SwAS6vSjaUkRAsa5ZrZp+UsCNebch4lJEQHGZHYanflLAy/Mza99SRECxGxJSon5SwKoqdmHXUkRA" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751937437695, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751858434047, - "AQMAAAABAAAACQAAAAPtzoHiflLAv+Hf065SREDbnsab035SwCCXDsGzUkRAfzt2BM9+UsDdoUuayFJEQIgbprDWflLAIENZdNRSRECpvTwo3n5SwGeIZYTOUkRAMLeyrOp+UsCOpsWcxFJEQAZ957LrflLA4zNv0cNSREDrOJ0= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751872950271, - "AQMAAAABAAAACQAAANj2xUojflLA0GDedmtRREAlXLaXGH5SwMQJGwBvUURAeRq/PBl+UsDgx1LDdVFEQCvdMOoSflLAIaZiHHdRREAlbLPND35SwKlPeTuFUURAAappGxp+UsCcg5InlVFEQN+C2gApflLAhshAO5BRRED9vX4= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751616376831, - "AQMAAAABAAAABgAAAC4gQVUaflLAB8C0vCVRREBUdiNUEn5SwAI2hkotUURACR01BRV+UsDWpuWGSVFEQGxjw5QdflLAy2ycskZRRECWN2MsIn5SwJMrZdoxUURALiBBVRp+UsAHwLS8JVFEQA==" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751937732607, - "AQMAAAABAAAABQAAAG4xzDFqflLAkk0qnZxSRECWOp/3Yn5SwL1ktwCfUkRAeOOJkmN+UsCItVYZpVJEQBH63i9uflLAX9E8yKJSREBuMcwxan5SwJJNKp2cUkRA" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751858270207, - "AQMAAAABAAAABAAAAAG6q0P2flLAyLLrn7tSREDrOJ3Q7H5SwDqX5L++UkRABn3nsut+UsDjM2/Rw1JEQAG6q0P2flLAyLLrn7tSREA=" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751857352703, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751872327679, - "AQMAAAABAAAACAAAALN5iZyWflLADAXwEVpRRECwEMFxin5SwD/XnbFcUURAtp5b+IZ+UsCpk6vXXVFEQAslC2GCflLAJNccsHJRREDVkwqvjH5SwHKz35uCUURAHARdlJt+UsDULRmvfVFEQHammSugflLAemqZ1mhRRECzeYk= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751933145087, - "AQMAAAABAAAACAAAAN1tV3INflLAYpxJTg1SREBY0l8hCX5SwMZUbLsOUkRAy7o7ogp+UsAo5urRGFJEQI63eQkMflLA6OQA5SJSRECf3uGPDn5SwDhJFpgzUkRA2rxwKBN+UsDKpkoTMlJEQEioR8AXflLA4C+dOh1SREDdbVc= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751938027519, - "AQMAAAABAAAADQAAAGJAD3tkflLAWfvi13dSREAiRRWVVX5SwFmdicR8UkRA86Jh/VB+UsAHJn6dkVJEQOBUWPdSflLAcSA6q5RSREAwJStPWH5SwDUwwVWQUkRAT4sy9Vx+UsAFeOWCjFJEQM3k3YlgflLADNFTPYlSREAgm+s= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751875375103, - "AQMAAAABAAAABgAAAC7muJd3flLA2IIdps1QRECPycdebX5SwLVTbULXUERAf94Lh2t+UsAXtDyg31BEQBmcutR1flLARebRi+9QREDRIFysen5SwEHF+vHtUERALua4l3d+UsDYgh2mzVBEQA==" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - true, - 622236751936651263, - null - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751872622591, - "AQYAAAACAAAAAQMAAAABAAAABQAAAHOrh9KBflLA/Xu54zhRREDvo75BgX5SwJ3mlxM5UURApVxwqnx+UsCDIfTrTVFEQHJxMRGFflLArfH55lpRREBzq4fSgX5SwP17ueM4UURAAQMAAAABAAAABAAAALAQwXGKflLAP9edsVw= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751872163839, - "AQMAAAABAAAACAAAAKL2fB6rflLA3hbUN2VRREB2ppkroH5SwHpqmdZoUURAHARdlJt+UsDULRmvfVFEQJknc+KlflLAi1XPmo1RREAGjAyarn5SwHMH7riKUURApgB+RK5+UsBx7Tz+hlFEQKy81p6sflLATfgyBXZRRECi9nw= (truncated)" - ], - [ - "Brooklyn", - "133", - "133", - "Kensington", - false, - 622236751933439999, - "AQMAAAABAAAABgAAAAoYULwHflLAgARpiehRREAJB9SjA35SwLf74OPpUURAWNJfIQl+UsDGVGy7DlJEQN1tV3INflLAYpxJTg1SREB1JSwKEn5SwCctsXX4UURAChhQvAd+UsCABGmJ6FFEQA==" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722201296895, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722202345471, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722201591807, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722202116095, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722202640383, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722210209791, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722210504703, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722201133055, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722202181631, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722204803071, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722201952255, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722210045951, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722201493503, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722202017791, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722202542079, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722201264127, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722202312703, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722210406399, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722202083327, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722210177023, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722201329663, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722202378239, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722204999679, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722210242559, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722201395199, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722201919487, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722202443775, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722201165823, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722202214399, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722210308095, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722201460735, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722202509311, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722210078719, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722201231359, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722202279935, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722210373631, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722202050559, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722209619967, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722210144255, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722202476543, - "AQMAAAABAAAACQAAAP+f8/YZfVLA8JO5MJhjREBrKmJ/Fn1SwMBMVVWZY0RASm2uPBZ9UsC69joMmmNEQGVAqBEOfVLA+OxhcbBjRED4yyvzCn1SwOJopQC5Y0RANK6ovhB9UsBSi3bzwWNEQDoh0K8ffVLAqGtmBr1jRECEoOs= (truncated)" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722214010879, - "AQMAAAABAAAABwAAAFlqf9difVLAzzwixkFkREBAHOl2VH1SwNN+woNGZERA36nY2k99UsC1pIplW2REQCWD/h5WfVLAU/pDEmVkRECsbjoEWX1SwNVJZE9dZERA0QvFUGB9UsB/BPqnSGREQFlqf9difVLAzzwixkFkREA=" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722201985023, - "AQMAAAABAAAACQAAANVekW+gfVLAgSja2GtjREDskrJ+kX1SwHgIaMZwY0RATIEB44x9UsB2o9enhWNEQGwvQziXfVLAqenHm5VjREC+95UYo31SwIuG/LCRY0RAiigaj6V9UsBy2+XrimNEQHTdfzWqfVLArD06Vn5jREDqm9U= (truncated)" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722201821183, - "AQMAAAABAAAACQAAACjlj5ySfVLAE3tMfapjREC4+Iarg31SwPA60mqvY0RAmoG9D399UsA7N11MxGNEQBXrEGWJfVLAHf9wQNRjREB2AA0Di31SwMfO57fTY0RAJrmEwo59UsAejzBqyWNEQNrP5mGWfVLAiDRhibRjREDq/a8= (truncated)" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722210340863, - "AQMAAAABAAAACAAAAIcX9JT4fFLAoKhTo/BjREBuepBj9nxSwG8xYFzxY0RAR2JAke98UsBKfc56BGREQOufj07qfFLADa11mBJkREBW+ktc73xSwBUvVGYaZERAUyuxTf58UsDa9V15FWREQChA++kCfVLA+GvWlwBkRECHF/Q= (truncated)" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722208505855, - "AQMAAAABAAAABQAAACdLlxO3fVLAIgROBkRjRECtcvf7s31SwNpZ0QlSY0RAPeXACLh9UsDu+mVKWGNEQFteC3C9fVLAJ7n5YkljREAnS5cTt31SwCIETgZEY0RA" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722215190527, - "AQMAAAABAAAABwAAAFTpiHgufVLAfpVN2rNkREAqu8aGH31SwNOnhce4ZERAgfdTLBt9UsBeyQ5/zGREQFvKzHQrfVLASNILXdpkREBvJaJiM31SwJCKHbjDZERAsniXUzV9UsD28R1wvmREQFTpiHgufVLAfpVN2rNkREA=" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722204835839, - "AQMAAAABAAAABwAAAAJLPg5UfVLAl601h/BiREBJMmN+TH1SwODmSFIFY0RAWgScR0x9UsCgxivoBWNEQFTqeYBTfVLAJQIDDxFjREBw5SVxYn1SwJAdvyEMY0RAnyHuT2V9UsDXNKAg/2JEQAJLPg5UfVLAl601h/BiREA=" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722209849343, - "AQMAAAABAAAABAAAAETav5LYfFLAQhrp2EJkREB7C9kv1nxSwIaybmtJZERACXj8Mdd8UsCZ8VoWSWREQETav5LYfFLAQhrp2EJkREA=" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722209685503, - "AQMAAAABAAAABQAAACp6OozJfFLARtBQBGxkREDWlMXZwXxSwB7BWDWBZERAwPirZcl8UsC8nFuRh2REQGwr9/jNfFLAiQKf2XJkREAqejqMyXxSwEbQUARsZERA" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722202836991, - "AQMAAAABAAAABQAAAE9FueZqfVLA9HMewStkRECbLlQEaH1SwMQGnbQsZERAqR5kKGR9UsCq4lowPmREQEyO6vBnfVLA+PTb4TNkREBPRbnman1SwPRzHsErZERA" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722209521663, - "AQMAAAABAAAABwAAALgbmurcfFLAY7LC7G1kREBsK/f4zXxSwIkCn9lyZERAwPirZcl8UsC8nFuRh2REQAbCFAvafFLA9gdymJVkREArz0mj4nxSwF7ASsOSZERAdbbEP+d8UsC3e4zhfWREQLgbmurcfFLAY7LC7G1kREA=" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722204344319, - "AQMAAAABAAAABgAAAH8U7DY/fVLAgoyPuSljREBSL4srPH1SwI6fmBgyY0RAckTMoDZ9UsDndJVMQWNEQNMa+1c7fVLAWr+Mvj9jREAHzeXzP31SwPG9Xd0qY0RAfxTsNj99UsCCjI+5KWNEQA==" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722211192831, - "AQMAAAABAAAABwAAAEAcWjEVfVLAES/K0qhkREC51Zo/Bn1SwJ3f5r+tZERAxX7u0QN9UsC/5lq/uGREQO9KLAURfVLAg0OL2cNkRECB91MsG31SwF7JDn/MZERAKrvGhh99UsDTp4XHuGREQEAcWjEVfVLAES/K0qhkREA=" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722208866303, - "AQMAAAABAAAACAAAANFQu6SQfVLAQaw8nSNjREBJjscLhn1SwOkuCRwnY0RAD+4acIF9UsCw3E79O2NEQPsHNMWLfVLAwnQo8UtjREBAUfy1mn1SwGc0pANHY0RAHv6UUZ99UsCB+08iMmNEQI7ZtUOcfVLAGU4way1jREDRULs= (truncated)" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722212208639, - "AQMAAAABAAAABgAAAJotUIsefVLASyl2Q4NjREAS8XnSHX1SwBFw1D+FY0RAbCpifxZ9UsDATFVVmWNEQP+f8/YZfVLA8JO5MJhjREBD5QyTHn1SwAjIaE+DY0RAmi1Qix59UsBLKXZDg2NEQA==" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722201690111, - "AQMAAAABAAAABgAAAL73lRijfVLAi4b8sJFjREBsL0M4l31SwKnpx5uVY0RAKOWPnJJ9UsATe0x9qmNEQOr9r16XfVLAfpfk1bFjRECuKyL6nX1SwIiSAMOfY0RAvveVGKN9UsCLhvywkWNEQA==" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722201526271, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722204868607, - null - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722215059455, - "AQMAAAABAAAACAAAAI2CdgZCfVLAQtMZC5pkREDH8scUM31SwM+oY/ieZERAVOmIeC59UsB+lU3as2REQLN4l1M1fVLA9vEdcL5kRECU58fgOn1SwKB4tFWvZERANcvSP0J9UsDxeGQWm2REQC63lWlCfVLARS0kpJpkRECNgnY= (truncated)" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722201198591, - "AQMAAAABAAAACQAAAJqBvQ9/fVLAOzddTMRjRECM96AecH1SwNY20TnJY0RAVVPBgmt9UsCdnWIb3mNEQFUtEth1fVLAgJCOD+5jREAUtiYGg31SwJmV1rbpY0RALneyEId9UsARqYOR3mNEQIcc9zKHfVLATmJOM95jREAV6xA= (truncated)" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722209718271, - "AQMAAAABAAAABgAAACvPSaPifFLAXsBKw5JkREAFwhQL2nxSwPYHcpiVZERAAItY/tt8UsDn9Sw9l2REQNGx03TsfFLAideODKVkREDZtIj47HxSwGrhH7iiZERAK89Jo+J8UsBewErDkmREQA==" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722202869759, - "AQMAAAABAAAACAAAAARQMDxxfVLATe008QJkREBXpOlKYn1SwE3LoN4HZERAaJnxrl19UsAGkk3AHGREQJsuVARofVLAxAadtCxkREBORbnman1SwPRzHsErZERAjTtAWG99UsAipMSNH2REQL9W7q92fVLANpcWXAtkREAEUDA= (truncated)" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722209554431, - "AQMAAAABAAAACQAAAAl4/DHXfFLAmfFaFklkREB7C9kv1nxSwIaybmtJZERAf3rlbdF8UsDeJd6FVmREQN8oM+rJfFLA2lKYAWtkREAqejqMyXxSwEbQUARsZERAbCv3+M18UsCJAp/ZcmREQLgbmurcfFLAY7LC7G1kREAbyBI= (truncated)" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722208899071, - "AQMAAAABAAAACAAAAHf3aat8fVLA1DOu0xJjREALvsK2e31SwEjuOigXY0RASY7HC4Z9UsDpLgkcJ2NEQNFQu6SQfVLAQaw8nSNjRED/1eCZin1SwFgpLYQeY0RA0RvY0Yh9UsBE4JcMHWNEQMp4Vw2HfVLAsz0CkRtjREB392k= (truncated)" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722208735231, - "AQMAAAABAAAABAAAAI7ZtUOcfVLAGU4way1jREAe/pRRn31SwIH7TyIyY0RATr0XJKF9UsAY3G6IMWNEQI7ZtUOcfVLAGU4way1jREA=" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722201559039, - "AQMAAAABAAAABwAAAKPASJorfVLAMBukbl9jREA80aZkJX1SwEtyJW1wY0RAmi1Qix59UsBKKXZDg2NEQEPlDJMefVLACMhoT4NjREAGOAqELX1SwDOFUGJ+Y0RAaFANIDJ9UsASIwaBaWNEQKPASJorfVLAMBukbl9jREA=" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722204901375, - "AQMAAAABAAAABwAAAJ8h7k9lfVLA1zSgIP9iREBw5SVxYn1SwJAdvyEMY0RA3twTxmx9UsDvMJoVHGNEQAu+wrZ7fVLASO46KBdjREB392mrfH1SwNQzrtMSY0RAwIlKS259UsBarOm5BmNEQJ8h7k9lfVLA1zSgIP9iREA=" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - false, - 622236722213257215, - "AQMAAAABAAAABQAAAANsRIn7fFLAE3QCQuNjRECNZ9v59nxSwB+kJbfvY0RAbnqQY/Z8UsBvMWBc8WNEQIcX9JT4fFLAoKhTo/BjREADbESJ+3xSwBN0AkLjY0RA" - ], - [ - "Manhattan", - "236", - "236", - "Upper East Side North", - true, - 622236722209587199, - null - ] - ], - "plotOptions": { - "displayType": "table", - "customPlotOptions": {}, - "pivotColumns": null, - "pivotAggregation": null, - "xColumns": null, - "yColumns": null - }, - "columnCustomDisplayInfos": {}, - "aggType": "", - "isJsonSchema": true, - "removedWidgets": [], - "aggSchema": [], - "schema": [ - { - "name": "borough", - "type": "\"string\"", - "metadata": "{}" - }, - { - "name": "location_id", - "type": "\"string\"", - "metadata": "{}" - }, - { - "name": "objectid", - "type": "\"string\"", - "metadata": "{}" - }, - { - "name": "zone", - "type": "\"string\"", - "metadata": "{}" - }, - { - "name": "is_core", - "type": "\"boolean\"", - "metadata": "{}" - }, - { - "name": "h3", - "type": "\"long\"", - "metadata": "{}" - }, - { - "name": "wkb", - "type": "\"binary\"", - "metadata": "{}" - } - ], - "aggError": "", - "aggData": [], - "addedWidgets": {}, - "metadata": {}, - "dbfsResultPath": null, - "type": "table", - "aggOverflow": false, - "aggSeriesLimitReached": false, - "arguments": {} - } - }, - "data": { - "text/html": [ - "
boroughlocation_idobjectidzoneis_coreh3wkb
Staten Island9999Freshkills Parktrue622236745694314495null
Staten Island9999Freshkills Parktrue622236745679077375null
Staten Island9999Freshkills Parktrue622236745734160383null
Staten Island9999Freshkills Parktrue622236745762275327null
Staten Island9999Freshkills Parktrue622236745731801087null
Staten Island9999Freshkills Parktrue622236745725935615null
Staten Island9999Freshkills Parktrue622236745680224255null
Staten Island9999Freshkills Parktrue622236745689595903null
Staten Island9999Freshkills Parktrue622236745683730431null
Staten Island9999Freshkills Parktrue622236745723576319null
Staten Island9999Freshkills Parktrue622236745763422207null
Staten Island9999Freshkills Parktrue622236745693102079null
Staten Island9999Freshkills Parktrue622236745732947967null
Staten Island9999Freshkills Parktrue622236745702473727null
Staten Island9999Freshkills Parktrue622236745696608255null
Staten Island9999Freshkills Parktrue622236745736454143null
Staten Island9999Freshkills Parktrue622236745761062911null
Staten Island9999Freshkills Parktrue622236745690742783null
Staten Island9999Freshkills Parktrue622236745675505663null
Staten Island9999Freshkills Parktrue622236745730588671null
Staten Island9999Freshkills Parktrue622236745694248959null
Staten Island9999Freshkills Parktrue622236745679011839null
Staten Island9999Freshkills Parktrue622236745734094847null
Staten Island9999Freshkills Parktrue622236745762209791null
Staten Island9999Freshkills Parktrue622236745731735551null
Staten Island9999Freshkills Parktrue622236745725870079null
Staten Island9999Freshkills Parktrue622236745680158719null
Staten Island9999Freshkills Parktrue622236745689530367null
Staten Island9999Freshkills Parktrue622236745683664895null
Staten Island9999Freshkills Parktrue622236745763356671null
Staten Island9999Freshkills Parktrue622236745693036543null
Staten Island9999Freshkills Parktrue622236745732882431null
Staten Island9999Freshkills Parktrue622236745702408191null
Staten Island9999Freshkills Parktrue622236745681305599null
Staten Island9999Freshkills Parktrue622236745760997375null
Staten Island9999Freshkills Parktrue622236745675440127null
Staten Island9999Freshkills Parktrue622236745730523135null
Staten Island9999Freshkills Parktrue622236745703555071null
Staten Island9999Freshkills Parktrue622236745762144255null
Staten Island9999Freshkills Parktrue622236745725804543null
Staten Island9999Freshkills Parktrue622236745680093183null
Staten Island9999Freshkills Parktrue622236745683599359null
Staten Island9999Freshkills Parktrue622236745763291135null
Staten Island9999Freshkills Parktrue622236745692971007null
Staten Island9999Freshkills Parktrue622236745732816895null
Staten Island9999Freshkills Parktrue622236745913303039null
Staten Island9999Freshkills Parktrue622236745702342655null
Staten Island9999Freshkills Parktrue622236745681240063null
Staten Island9999Freshkills Parktrue622236745690611711null
Staten Island9999Freshkills Parktrue622236745675374591null
Staten Island9999Freshkills Parktrue622236745694117887null
Staten Island9999Freshkills Parktrue622236745733963775null
Staten Island9999Freshkills Parktrue622236745703489535null
Staten Island9999Freshkills Parktrue622236745731604479null
Staten Island9999Freshkills Parktrue622236745689399295null
Staten Island9999Freshkills Parktrue622236745683533823null
Staten Island9999Freshkills Parktrue622236745763225599null
Staten Island9999Freshkills Parktrue622236745692905471null
Staten Island9999Freshkills Parktrue622236745732751359null
Staten Island9999Freshkills Parktrue622236745913237503null
Staten Island9999Freshkills Parktrue622236745702277119null
Staten Island9999Freshkills Parktrue622236745742123007null
Staten Island9999Freshkills Parktrue622236745681174527null
Staten Island9999Freshkills Parktrue622236745760866303null
Staten Island9999Freshkills Parktrue622236745730392063null
Staten Island9999Freshkills Parktrue622236745684680703null
Staten Island9999Freshkills Parktrue622236745724526591null
Staten Island9999Freshkills Parktrue622236745764372479null
Staten Island9999Freshkills Parktrue622236745694052351null
Staten Island9999Freshkills Parktrue622236745733898239null
Staten Island9999Freshkills Parktrue622236745703423999null
Staten Island9999Freshkills Parktrue622236745682321407null
Staten Island9999Freshkills Parktrue622236745731538943null
Staten Island9999Freshkills Parktrue622236745725673471null
Staten Island9999Freshkills Parktrue622236745695199231null
Staten Island9999Freshkills Parktrue622236745735045119null
Staten Island9999Freshkills Parktrue622236745689333759null
Staten Island9999Freshkills Parktrue622236745683468287null
Staten Island9999Freshkills Parktrue622236745763160063null
Staten Island9999Freshkills Parktrue622236745732685823null
Staten Island9999Freshkills Parktrue622236745702211583null
Staten Island9999Freshkills Parktrue622236745742057471null
Staten Island9999Freshkills Parktrue622236745681108991null
Staten Island9999Freshkills Parktrue622236745760800767null
Staten Island9999Freshkills Parktrue622236745690480639null
Staten Island9999Freshkills Parktrue622236745675243519null
Staten Island9999Freshkills Parktrue622236745684615167null
Staten Island9999Freshkills Parktrue622236745764306943null
Staten Island9999Freshkills Parktrue622236745693986815null
Staten Island9999Freshkills Parktrue622236745733832703null
Staten Island9999Freshkills Parktrue622236745672884223null
Staten Island9999Freshkills Parktrue622236745682255871null
Staten Island9999Freshkills Parktrue622236746168631295null
Staten Island9999Freshkills Parktrue622236745761947647null
Staten Island9999Freshkills Parktrue622236745731473407null
Staten Island9999Freshkills Parktrue622236745725607935null
Staten Island9999Freshkills Parktrue622236745695133695null
Staten Island9999Freshkills Parktrue622236745734979583null
Staten Island9999Freshkills Parktrue622236745689268223null
Staten Island9999Freshkills Parktrue622236745683402751null
Staten Island9999Freshkills Parktrue622236745723248639null
Staten Island9999Freshkills Parktrue622236745732620287null
Staten Island9999Freshkills Parktrue622236745702146047null
Staten Island9999Freshkills Parktrue622236745741991935null
Staten Island9999Freshkills Parktrue622236745760735231null
Staten Island9999Freshkills Parktrue622236745690415103null
Staten Island9999Freshkills Parktrue622236745675177983null
Staten Island9999Freshkills Parktrue622236745684549631null
Staten Island9999Freshkills Parktrue622236745724395519null
Staten Island9999Freshkills Parktrue622236745764241407null
Staten Island9999Freshkills Parktrue622236745703292927null
Staten Island9999Freshkills Parktrue622236745682190335null
Staten Island9999Freshkills Parktrue622236746168565759null
Staten Island9999Freshkills Parktrue622236745676324863null
Staten Island9999Freshkills Parktrue622236745725542399null
Staten Island9999Freshkills Parktrue622236745695068159null
Staten Island9999Freshkills Parktrue622236745734914047null
Staten Island9999Freshkills Parktrue622236745683337215null
Staten Island9999Freshkills Parktrue622236745723183103null
Staten Island9999Freshkills Parktrue622236745732554751null
Staten Island9999Freshkills Parktrue622236745702080511null
Staten Island9999Freshkills Parktrue622236745741926399null
Staten Island9999Freshkills Parktrue622236745760669695null
Staten Island9999Freshkills Parktrue622236745690349567null
Staten Island9999Freshkills Parktrue622236745675112447null
Staten Island9999Freshkills Parktrue622236745693855743null
Staten Island9999Freshkills Parktrue622236745703227391null
Staten Island9999Freshkills Parktrue622236745743073279null
Staten Island9999Freshkills Parktrue622236745672753151null
Staten Island9999Freshkills Parktrue622236746168500223null
Staten Island9999Freshkills Parktrue622236745676259327null
Staten Island9999Freshkills Parktrue622236745731342335null
Staten Island9999Freshkills Parktrue622236745725476863null
Staten Island9999Freshkills Parktrue622236745695002623null
Staten Island9999Freshkills Parktrue622236745734848511null
Staten Island9999Freshkills Parktrue622236745673900031null
Staten Island9999Freshkills Parktrue622236745683271679null
Staten Island9999Freshkills Parktrue622236745723117567null
Staten Island9999Freshkills Parktrue622236745732489215null
Staten Island9999Freshkills Parktrue622236745702014975null
Staten Island9999Freshkills Parktrue622236745741860863null
Staten Island9999Freshkills Parktrue622236745726623743null
Staten Island9999Freshkills Parktrue622236745760604159null
Staten Island9999Freshkills Parktrue622236745690284031null
Staten Island9999Freshkills Parktrue622236745684418559null
Staten Island9999Freshkills Parktrue622236745724264447null
Staten Island9999Freshkills Parktrue622236745693790207null
Staten Island9999Freshkills Parktrue622236745733636095null
Staten Island9999Freshkills Parktrue622236745703161855null
Staten Island9999Freshkills Parktrue622236745743007743null
Staten Island9999Freshkills Parktrue622236745697296383null
Staten Island9999Freshkills Parktrue622236746168434687null
Staten Island9999Freshkills Parktrue622236745676193791null
Staten Island9999Freshkills Parktrue622236745731276799null
Staten Island9999Freshkills Parktrue622236745725411327null
Staten Island9999Freshkills Parktrue622236745694937087null
Staten Island9999Freshkills Parktrue622236745679699967null
Staten Island9999Freshkills Parktrue622236745734782975null
Staten Island9999Freshkills Parktrue622236745683206143null
Staten Island9999Freshkills Parktrue622236745723052031null
Staten Island9999Freshkills Parktrue622236745692577791null
Staten Island9999Freshkills Parktrue622236745732423679null
Staten Island9999Freshkills Parktrue622236745701949439null
Staten Island9999Freshkills Parktrue622236745741795327null
Staten Island9999Freshkills Parktrue622236745726558207null
Staten Island9999Freshkills Parktrue622236745760538623null
Staten Island9999Freshkills Parktrue622236745690218495null
Staten Island9999Freshkills Parktrue622236745674981375null
Staten Island9999Freshkills Parktrue622236745684353023null
Staten Island9999Freshkills Parktrue622236745724198911null
Staten Island9999Freshkills Parktrue622236745693724671null
Staten Island9999Freshkills Parktrue622236745697230847null
Staten Island9999Freshkills Parktrue622236745761685503null
Staten Island9999Freshkills Parktrue622236745676128255null
Staten Island9999Freshkills Parktrue622236745731211263null
Staten Island9999Freshkills Parktrue622236745725345791null
Staten Island9999Freshkills Parktrue622236745694871551null
Staten Island9999Freshkills Parktrue622236745734717439null
Staten Island9999Freshkills Parktrue622236745683140607null
Staten Island9999Freshkills Parktrue622236745722986495null
Staten Island9999Freshkills Parktrue622236745732358143null
Staten Island9999Freshkills Parktrue622236745701883903null
Staten Island9999Freshkills Parktrue622236745741729791null
Staten Island9999Freshkills Parktrue622236745726492671null
Staten Island9999Freshkills Parktrue622236745690152959null
Staten Island9999Freshkills Parktrue622236745674915839null
Staten Island9999Freshkills Parktrue622236745684287487null
Staten Island9999Freshkills Parktrue622236745724133375null
Staten Island9999Freshkills Parktrue622236745703030783null
Staten Island9999Freshkills Parktrue622236745697165311null
Staten Island9999Freshkills Parktrue622236745761619967null
Staten Island9999Freshkills Parktrue622236745676062719null
Staten Island9999Freshkills Parktrue622236745725280255null
Staten Island9999Freshkills Parktrue622236745694806015null
Staten Island9999Freshkills Parktrue622236745734651903null
Staten Island9999Freshkills Parktrue622236745683075071null
Staten Island9999Freshkills Parktrue622236745722920959null
Staten Island9999Freshkills Parktrue622236745692446719null
Staten Island9999Freshkills Parktrue622236745732292607null
Staten Island9999Freshkills Parktrue622236745701818367null
Staten Island9999Freshkills Parktrue622236745741664255null
Staten Island9999Freshkills Parktrue622236745735798783null
Staten Island9999Freshkills Parktrue622236745690087423null
Staten Island9999Freshkills Parktrue622236745674850303null
Staten Island9999Freshkills Parktrue622236745693593599null
Staten Island9999Freshkills Parktrue622236745733439487null
Staten Island9999Freshkills Parktrue622236745702965247null
Staten Island9999Freshkills Parktrue622236745742811135null
Staten Island9999Freshkills Parktrue622236745697099775null
Staten Island9999Freshkills Parktrue622236745675997183null
Staten Island9999Freshkills Parktrue622236745725214719null
Staten Island9999Freshkills Parktrue622236745694740479null
Staten Island9999Freshkills Parktrue622236745734586367null
Staten Island9999Freshkills Parktrue622236745698246655null
Staten Island9999Freshkills Parktrue622236745683009535null
Staten Island9999Freshkills Parktrue622236745722855423null
Staten Island9999Freshkills Parktrue622236745762701311null
Staten Island9999Freshkills Parktrue622236745732227071null
Staten Island9999Freshkills Parktrue622236745726361599null
Staten Island9999Freshkills Parktrue622236745735733247null
Staten Island9999Freshkills Parktrue622236745690021887null
Staten Island9999Freshkills Parktrue622236745684156415null
Staten Island9999Freshkills Parktrue622236745724002303null
Staten Island9999Freshkills Parktrue622236745693528063null
Staten Island9999Freshkills Parktrue622236745733373951null
Staten Island9999Freshkills Parktrue622236745702899711null
Staten Island9999Freshkills Parktrue622236745742745599null
Staten Island9999Freshkills Parktrue622236745697034239null
Staten Island9999Freshkills Parktrue622236745681797119null
Staten Island9999Freshkills Parktrue622236745736880127null
Staten Island9999Freshkills Parktrue622236745761488895null
Staten Island9999Freshkills Parktrue622236745675931647null
Staten Island9999Freshkills Parktrue622236745725149183null
Staten Island9999Freshkills Parktrue622236745694674943null
Staten Island9999Freshkills Parktrue622236745734520831null
Staten Island9999Freshkills Parktrue622236745682943999null
Staten Island9999Freshkills Parktrue622236745722789887null
Staten Island9999Freshkills Parktrue622236745732161535null
Staten Island9999Freshkills Parktrue622236745726296063null
Staten Island9999Freshkills Parktrue622236745735667711null
Staten Island9999Freshkills Parktrue622236745689956351null
Staten Island9999Freshkills Parktrue622236745674719231null
Staten Island9999Freshkills Parktrue622236745684090879null
Staten Island9999Freshkills Parktrue622236745723936767null
Staten Island9999Freshkills Parktrue622236745693462527null
Staten Island9999Freshkills Parktrue622236745733308415null
Staten Island9999Freshkills Parktrue622236745696968703null
Staten Island9999Freshkills Parktrue622236745736814591null
Staten Island9999Freshkills Parktrue622236745761423359null
Staten Island9999Freshkills Parktrue622236745675866111null
Staten Island9999Freshkills Parktrue622236745725083647null
Staten Island9999Freshkills Parktrue622236745694609407null
Staten Island9999Freshkills Parktrue622236745734455295null
Staten Island9999Freshkills Parktrue622236745732095999null
Staten Island9999Freshkills Parktrue622236745726230527null
Staten Island9999Freshkills Parktrue622236745689890815null
Staten Island9999Freshkills Parktrue622236745674653695null
Staten Island9999Freshkills Parktrue622236745684025343null
Staten Island9999Freshkills Parktrue622236745723871231null
Staten Island9999Freshkills Parktrue622236745702768639null
Staten Island9999Freshkills Parktrue622236745742614527null
Staten Island9999Freshkills Parktrue622236745681666047null
Staten Island9999Freshkills Parktrue622236745736749055null
Staten Island9999Freshkills Parktrue622236745761357823null
Staten Island9999Freshkills Parktrue622236745675800575null
Staten Island9999Freshkills Parktrue622236745725018111null
Staten Island9999Freshkills Parktrue622236745694543871null
Staten Island9999Freshkills Parktrue622236745734389759null
Staten Island9999Freshkills Parktrue622236745732030463null
Staten Island9999Freshkills Parktrue622236745689825279null
Staten Island9999Freshkills Parktrue622236745674588159null
Staten Island9999Freshkills Parktrue622236745702703103null
Staten Island9999Freshkills Parktrue622236745742548991null
Staten Island9999Freshkills Parktrue622236745736683519null
Staten Island9999Freshkills Parktrue622236745690972159null
Staten Island9999Freshkills Parktrue622236745675735039null
Staten Island9999Freshkills Parktrue622236745724952575null
Staten Island9999Freshkills Parktrue622236745694478335null
Staten Island9999Freshkills Parktrue622236745734324223null
Staten Island9999Freshkills Parktrue622236745731964927null
Staten Island9999Freshkills Parktrue622236745726099455null
Staten Island9999Freshkills Parktrue622236745689759743null
Staten Island9999Freshkills Parktrue622236745683894271null
Staten Island9999Freshkills Parktrue622236745723740159null
Staten Island9999Freshkills Parktrue622236745702637567null
Staten Island9999Freshkills Parktrue622236745742483455null
Staten Island9999Freshkills Parktrue622236745696772095null
Staten Island9999Freshkills Parktrue622236745761226751null
Staten Island9999Freshkills Parktrue622236745690906623null
Staten Island9999Freshkills Parktrue622236745675669503null
Staten Island9999Freshkills Parktrue622236745724887039null
Staten Island9999Freshkills Parktrue622236745694412799null
Staten Island9999Freshkills Parktrue622236745679175679null
Staten Island9999Freshkills Parktrue622236745734258687null
Staten Island9999Freshkills Parktrue622236745731899391null
Staten Island9999Freshkills Parktrue622236745726033919null
Staten Island9999Freshkills Parktrue622236745689694207null
Staten Island9999Freshkills Parktrue622236745683828735null
Staten Island9999Freshkills Parktrue622236745723674623null
Staten Island9999Freshkills Parktrue622236745763520511null
Staten Island9999Freshkills Parktrue622236745696706559null
Staten Island9999Freshkills Parktrue622236745761161215null
Staten Island9999Freshkills Parktrue622236745690841087null
Staten Island9999Freshkills Parktrue622236745675603967null
Staten Island9999Freshkills Parktrue622236745694347263null
Staten Island9999Freshkills Parktrue622236745679110143null
Staten Island9999Freshkills Parktrue622236745734193151null
Staten Island9999Freshkills Parktrue622236745762308095null
Staten Island9999Freshkills Parktrue622236745731833855null
Staten Island9999Freshkills Parktrue622236745725968383null
Staten Island9999Freshkills Parktrue622236745689628671null
Staten Island9999Freshkills Parktrue622236745729474559null
Staten Island9999Freshkills Parktrue622236745683763199null
Staten Island9999Freshkills Parktrue622236745723609087null
Staten Island9999Freshkills Parktrue622236745763454975null
Staten Island9999Freshkills Parktrue622236745702506495null
Staten Island9999Freshkills Parktrue622236745742352383null
Staten Island9999Freshkills Parktrue622236745696641023null
Staten Island9999Freshkills Parktrue622236745761095679null
Staten Island9999Freshkills Parktrue622236745675538431null
Staten Island9999Freshkills Parktrue622236745694281727null
Staten Island9999Freshkills Parktrue622236745679044607null
Staten Island9999Freshkills Parktrue622236745734127615null
Staten Island9999Freshkills Parktrue622236745762242559null
Staten Island9999Freshkills Parktrue622236745731768319null
Staten Island9999Freshkills Parktrue622236745680191487null
Staten Island9999Freshkills Parktrue622236745689563135null
Staten Island9999Freshkills Parktrue622236745693069311null
Staten Island9999Freshkills Parktrue622236745732915199null
Staten Island9999Freshkills Parktrue622236745702440959null
Staten Island9999Freshkills Parktrue622236745696575487null
Staten Island9999Freshkills Parktrue622236745675472895null
Staten Island9999Freshkills Parktrue622236745730555903null
Staten Island9999Freshkills Parktrue622236745694216191null
Staten Island9999Freshkills Parktrue622236745734062079null
Staten Island9999Freshkills Parktrue622236745703587839null
Staten Island9999Freshkills Parktrue622236745762177023null
Staten Island9999Freshkills Parktrue622236745731702783null
Staten Island9999Freshkills Parktrue622236745725837311null
Staten Island9999Freshkills Parktrue622236745689497599null
Staten Island9999Freshkills Parktrue622236745683632127null
Staten Island9999Freshkills Parktrue622236745723478015null
Staten Island9999Freshkills Parktrue622236745763323903null
Staten Island9999Freshkills Parktrue622236745693003775null
Staten Island9999Freshkills Parktrue622236745732849663null
Staten Island9999Freshkills Parktrue622236745702375423null
Staten Island9999Freshkills Parktrue622236745681272831null
Staten Island9999Freshkills Parktrue622236745690644479null
Staten Island9999Freshkills Parktrue622236745675407359null
Staten Island9999Freshkills Parktrue622236745730490367null
Staten Island9999Freshkills Parktrue622236745694150655null
Staten Island9999Freshkills Parktrue622236745733996543null
Staten Island9999Freshkills Parktrue622236745703522303null
Staten Island9999Freshkills Parktrue622236745762111487null
Staten Island9999Freshkills Parktrue622236745731637247null
Staten Island9999Freshkills Parktrue622236745680060415null
Staten Island9999Freshkills Parktrue622236745689432063null
Staten Island9999Freshkills Parktrue622236745683566591null
Staten Island9999Freshkills Parktrue622236745763258367null
Staten Island9999Freshkills Parktrue622236745692938239null
Staten Island9999Freshkills Parktrue622236745732784127null
Staten Island9999Freshkills Parktrue622236745681207295null
Staten Island9999Freshkills Parktrue622236745760899071null
Staten Island9999Freshkills Parktrue622236745675341823null
Staten Island9999Freshkills Parktrue622236745730424831null
Staten Island9999Freshkills Parktrue622236745684713471null
Staten Island9999Freshkills Parktrue622236745764405247null
Staten Island9999Freshkills Parktrue622236745694085119null
Staten Island9999Freshkills Parktrue622236745733931007null
Staten Island9999Freshkills Parktrue622236745703456767null
Staten Island9999Freshkills Parktrue622236745682354175null
Staten Island9999Freshkills Parktrue622236745762045951null
Staten Island9999Freshkills Parktrue622236745731571711null
Staten Island9999Freshkills Parktrue622236745725706239null
Staten Island9999Freshkills Parktrue622236745689366527null
Staten Island9999Freshkills Parktrue622236745683501055null
Staten Island9999Freshkills Parktrue622236745763192831null
Staten Island9999Freshkills Parktrue622236745702244351null
Staten Island9999Freshkills Parktrue622236745742090239null
Staten Island9999Freshkills Parktrue622236745681141759null
Staten Island9999Freshkills Parktrue622236745675276287null
Staten Island9999Freshkills Parktrue622236745684647935null
Staten Island9999Freshkills Parktrue622236745724493823null
Staten Island9999Freshkills Parktrue622236745764339711null
Staten Island9999Freshkills Parktrue622236745694019583null
Staten Island9999Freshkills Parktrue622236745678782463null
Staten Island9999Freshkills Parktrue622236745733865471null
Staten Island9999Freshkills Parktrue622236745703391231null
Staten Island9999Freshkills Parktrue622236745682288639null
Staten Island9999Freshkills Parktrue622236745761980415null
Staten Island9999Freshkills Parktrue622236745731506175null
Staten Island9999Freshkills Parktrue622236745695166463null
Staten Island9999Freshkills Parktrue622236745735012351null
Staten Island9999Freshkills Parktrue622236745689300991null
Staten Island9999Freshkills Parktrue622236745732653055null
Staten Island9999Freshkills Parktrue622236745702178815null
Staten Island9999Freshkills Parktrue622236745742024703null
Staten Island9999Freshkills Parktrue622236745690447871null
Staten Island9999Freshkills Parktrue622236745675210751null
Staten Island9999Freshkills Parktrue622236745684582399null
Staten Island9999Freshkills Parktrue622236745764274175null
Staten Island9999Freshkills Parktrue622236745693954047null
Staten Island9999Freshkills Parktrue622236745733799935null
Staten Island9999Freshkills Parktrue622236745703325695null
Staten Island9999Freshkills Parktrue622236745743171583null
Staten Island9999Freshkills Parktrue622236745682223103null
Staten Island9999Freshkills Parktrue622236745761914879null
Staten Island9999Freshkills Parktrue622236745731440639null
Staten Island9999Freshkills Parktrue622236745725575167null
Staten Island9999Freshkills Parktrue622236745695100927null
Staten Island9999Freshkills Parktrue622236745734946815null
Staten Island9999Freshkills Parktrue622236745689235455null
Staten Island9999Freshkills Parktrue622236745683369983null
Staten Island9999Freshkills Parktrue622236745723215871null
Staten Island9999Freshkills Parktrue622236745732587519null
Staten Island9999Freshkills Parktrue622236745702113279null
Staten Island9999Freshkills Parktrue622236745741959167null
Staten Island9999Freshkills Parktrue622236745696247807null
Staten Island9999Freshkills Parktrue622236745681010687null
Staten Island9999Freshkills Parktrue622236745760702463null
Staten Island9999Freshkills Parktrue622236745690382335null
Staten Island9999Freshkills Parktrue622236745675145215null
Staten Island9999Freshkills Parktrue622236745684516863null
Staten Island9999Freshkills Parktrue622236745724362751null
Staten Island9999Freshkills Parktrue622236745764208639null
Staten Island9999Freshkills Parktrue622236745693888511null
Staten Island9999Freshkills Parktrue622236745733734399null
Staten Island9999Freshkills Parktrue622236745703260159null
Staten Island9999Freshkills Parktrue622236745743106047null
Staten Island9999Freshkills Parktrue622236745682157567null
Staten Island9999Freshkills Parktrue622236745761849343null
Staten Island9999Freshkills Parktrue622236745691529215null
Staten Island9999Freshkills Parktrue622236745676292095null
Staten Island9999Freshkills Parktrue622236745731375103null
Staten Island9999Freshkills Parktrue622236745725509631null
Staten Island9999Freshkills Parktrue622236745695035391null
Staten Island9999Freshkills Parktrue622236745734881279null
Staten Island9999Freshkills Parktrue622236745683304447null
Staten Island9999Freshkills Parktrue622236745723150335null
Staten Island9999Freshkills Parktrue622236745732521983null
Staten Island9999Freshkills Parktrue622236745726656511null
Staten Island9999Freshkills Parktrue622236745696182271null
Staten Island9999Freshkills Parktrue622236745760636927null
Staten Island9999Freshkills Parktrue622236745690316799null
Staten Island9999Freshkills Parktrue622236745675079679null
Staten Island9999Freshkills Parktrue622236745684451327null
Staten Island9999Freshkills Parktrue622236745724297215null
Staten Island9999Freshkills Parktrue622236745693822975null
Staten Island9999Freshkills Parktrue622236745733668863null
Staten Island9999Freshkills Parktrue622236745703194623null
Staten Island9999Freshkills Parktrue622236745743040511null
Staten Island9999Freshkills Parktrue622236745761783807null
Staten Island9999Freshkills Parktrue622236745691463679null
Staten Island9999Freshkills Parktrue622236745676226559null
Staten Island9999Freshkills Parktrue622236745731309567null
Staten Island9999Freshkills Parktrue622236745725444095null
Staten Island9999Freshkills Parktrue622236745683238911null
Staten Island9999Freshkills Parktrue622236745723084799null
Staten Island9999Freshkills Parktrue622236745701982207null
Staten Island9999Freshkills Parktrue622236745741828095null
Staten Island9999Freshkills Parktrue622236745726590975null
Staten Island9999Freshkills Parktrue622236745696116735null
Staten Island9999Freshkills Parktrue622236745760571391null
Staten Island9999Freshkills Parktrue622236745675014143null
Staten Island9999Freshkills Parktrue622236745684385791null
Staten Island9999Freshkills Parktrue622236745724231679null
Staten Island9999Freshkills Parktrue622236745693757439null
Staten Island9999Freshkills Parktrue622236745733603327null
Staten Island9999Freshkills Parktrue622236745703129087null
Staten Island9999Freshkills Parktrue622236745742974975null
Staten Island9999Freshkills Parktrue622236745697263615null
Staten Island9999Freshkills Parktrue622236745676161023null
Staten Island9999Freshkills Parktrue622236745731244031null
Staten Island9999Freshkills Parktrue622236745694904319null
Staten Island9999Freshkills Parktrue622236745734750207null
Staten Island9999Freshkills Parktrue622236745732390911null
Staten Island9999Freshkills Parktrue622236745701916671null
Staten Island9999Freshkills Parktrue622236745741762559null
Staten Island9999Freshkills Parktrue622236745726525439null
Staten Island9999Freshkills Parktrue622236745690185727null
Staten Island9999Freshkills Parktrue622236745674948607null
Staten Island9999Freshkills Parktrue622236745684320255null
Staten Island9999Freshkills Parktrue622236745724166143null
Staten Island9999Freshkills Parktrue622236745693691903null
Staten Island9999Freshkills Parktrue622236745733537791null
Staten Island9999Freshkills Parktrue622236745703063551null
Staten Island9999Freshkills Parktrue622236745697198079null
Staten Island9999Freshkills Parktrue622236745761652735null
Staten Island9999Freshkills Parktrue622236745731178495null
Staten Island9999Freshkills Parktrue622236745694838783null
Staten Island9999Freshkills Parktrue622236745734684671null
Staten Island9999Freshkills Parktrue622236745698344959null
Staten Island9999Freshkills Parktrue622236745683107839null
Staten Island9999Freshkills Parktrue622236745722953727null
Staten Island9999Freshkills Parktrue622236745692479487null
Staten Island9999Freshkills Parktrue622236745732325375null
Staten Island9999Freshkills Parktrue622236745701851135null
Staten Island9999Freshkills Parktrue622236745741697023null
Staten Island9999Freshkills Parktrue622236745726459903null
Staten Island9999Freshkills Parktrue622236745735831551null
Staten Island9999Freshkills Parktrue622236745690120191null
Staten Island9999Freshkills Parktrue622236745674883071null
Staten Island9999Freshkills Parktrue622236745684254719null
Staten Island9999Freshkills Parktrue622236745724100607null
Staten Island9999Freshkills Parktrue622236745693626367null
Staten Island9999Freshkills Parktrue622236745733472255null
Staten Island9999Freshkills Parktrue622236745702998015null
Staten Island9999Freshkills Parktrue622236745697132543null
Staten Island9999Freshkills Parktrue622236745676029951null
Staten Island9999Freshkills Parktrue622236745725247487null
Staten Island9999Freshkills Parktrue622236745694773247null
Staten Island9999Freshkills Parktrue622236745734619135null
Staten Island9999Freshkills Parktrue622236745683042303null
Staten Island9999Freshkills Parktrue622236745722888191null
Staten Island9999Freshkills Parktrue622236745762734079null
Staten Island9999Freshkills Parktrue622236745732259839null
Staten Island9999Freshkills Parktrue622236745726394367null
Staten Island9999Freshkills Parktrue622236745735766015null
Staten Island9999Freshkills Parktrue622236745690054655null
Staten Island9999Freshkills Parktrue622236745674817535null
Staten Island9999Freshkills Parktrue622236745684189183null
Staten Island9999Freshkills Parktrue622236745724035071null
Staten Island9999Freshkills Parktrue622236745693560831null
Staten Island9999Freshkills Parktrue622236745733406719null
Staten Island9999Freshkills Parktrue622236745702932479null
Staten Island9999Freshkills Parktrue622236745761521663null
Staten Island9999Freshkills Parktrue622236745675964415null
Staten Island9999Freshkills Parktrue622236745698213887null
Staten Island9999Freshkills Parktrue622236745682976767null
Staten Island9999Freshkills Parktrue622236745722822655null
Staten Island9999Freshkills Parktrue622236745726328831null
Staten Island9999Freshkills Parktrue622236745674751999null
Staten Island9999Freshkills Parktrue622236745684123647null
Staten Island9999Freshkills Parktrue622236745723969535null
Staten Island9999Freshkills Parktrue622236745693495295null
Staten Island9999Freshkills Parktrue622236745733341183null
Staten Island9999Freshkills Parktrue622236745702866943null
Staten Island9999Freshkills Parktrue622236745736847359null
Staten Island9999Freshkills Parktrue622236745761456127null
Staten Island9999Freshkills Parktrue622236745675898879null
Staten Island9999Freshkills Parktrue622236745694642175null
Staten Island9999Freshkills Parktrue622236745734488063null
Staten Island9999Freshkills Parktrue622236745698148351null
Staten Island9999Freshkills Parktrue622236745732128767null
Staten Island9999Freshkills Parktrue622236745726263295null
Staten Island9999Freshkills Parktrue622236745735634943null
Staten Island9999Freshkills Parktrue622236745689923583null
Staten Island9999Freshkills Parktrue622236745674686463null
Staten Island9999Freshkills Parktrue622236745684058111null
Staten Island9999Freshkills Parktrue622236745723903999null
Staten Island9999Freshkills Parktrue622236745693429759null
Staten Island9999Freshkills Parktrue622236745733275647null
Staten Island9999Freshkills Parktrue622236745702801407null
Staten Island9999Freshkills Parktrue622236745742647295null
Staten Island9999Freshkills Parktrue622236745736781823null
Staten Island9999Freshkills Parktrue622236745761390591null
Staten Island9999Freshkills Parktrue622236745725050879null
Staten Island9999Freshkills Parktrue622236745694576639null
Staten Island9999Freshkills Parktrue622236745734422527null
Staten Island9999Freshkills Parktrue622236745732063231null
Staten Island9999Freshkills Parktrue622236745726197759null
Staten Island9999Freshkills Parktrue622236745689858047null
Staten Island9999Freshkills Parktrue622236745674620927null
Staten Island9999Freshkills Parktrue622236745683992575null
Staten Island9999Freshkills Parktrue622236745723838463null
Staten Island9999Freshkills Parktrue622236745702735871null
Staten Island9999Freshkills Parktrue622236745742581759null
Staten Island9999Freshkills Parktrue622236745736716287null
Staten Island9999Freshkills Parktrue622236745761325055null
Staten Island9999Freshkills Parktrue622236745691004927null
Staten Island9999Freshkills Parktrue622236745675767807null
Staten Island9999Freshkills Parktrue622236745724985343null
Staten Island9999Freshkills Parktrue622236745694511103null
Staten Island9999Freshkills Parktrue622236745734356991null
Staten Island9999Freshkills Parktrue622236745731997695null
Staten Island9999Freshkills Parktrue622236745726132223null
Staten Island9999Freshkills Parktrue622236745689792511null
Staten Island9999Freshkills Parktrue622236745674555391null
Staten Island9999Freshkills Parktrue622236745683927039null
Staten Island9999Freshkills Parktrue622236745723772927null
Staten Island9999Freshkills Parktrue622236745763618815null
Staten Island9999Freshkills Parktrue622236745702670335null
Staten Island9999Freshkills Parktrue622236745742516223null
Staten Island9999Freshkills Parktrue622236745761259519null
Staten Island9999Freshkills Parktrue622236745690939391null
Staten Island9999Freshkills Parktrue622236745675702271null
Staten Island9999Freshkills Parktrue622236745724919807null
Staten Island9999Freshkills Parktrue622236745679208447null
Staten Island9999Freshkills Parktrue622236745726066687null
Staten Island9999Freshkills Parktrue622236745683861503null
Staten Island9999Freshkills Parktrue622236745723707391null
Staten Island9999Freshkills Parktrue622236745702604799null
Staten Island9999Freshkills Parktrue622236745742450687null
Staten Island9999Freshkills Parktrue622236745696739327null
Staten Island9999Freshkills Parktrue622236745761193983null
Staten Island9999Freshkills Parktrue622236745690873855null
Staten Island9999Freshkills Parktrue622236745675636735null
Staten Island9999Freshkills Parktrue622236745694380031null
Staten Island9999Freshkills Parktrue622236745679142911null
Staten Island9999Freshkills Parktrue622236745734225919null
Staten Island9999Freshkills Parktrue622236745731866623null
Staten Island9999Freshkills Parktrue622236745726001151null
Staten Island9999Freshkills Parktrue622236745689661439null
Staten Island9999Freshkills Parktrue622236745729507327null
Staten Island9999Freshkills Parktrue622236745683795967null
Staten Island9999Freshkills Parktrue622236745723641855null
Staten Island9999Freshkills Parktrue622236745763487743null
Staten Island9999Freshkills Parktrue622236745702539263null
Staten Island9999Freshkills Parktrue622236745696673791null
Staten Island9999Freshkills Parktrue622236745761128447null
Staten Island9999Freshkills Parktrue622236745690808319null
Staten Island9999Freshkills Parktrue622236745725739007null
Staten Island9999Freshkills Parkfalse622236745696935935AQMAAAABAAAABAAAAIzvORt9i1LAUzWXQ0NIRECs3yEigotSwIct0/1KSERAz1Bl94WLUsD10IC1SUhEQIzvORt9i1LAUzWXQ0NIREA=
Staten Island9999Freshkills Parkfalse622236746167451647AQMAAAABAAAABQAAALLLamedjVLA7PTzHUVHREBNNifZjI1SwIx+IDVNR0RAo5WznI2NUsAYmXlhTkdEQOZxFHmcjVLAepg7ZklHRECyy2pnnY1SwOz08x1FR0RA
Staten Island9999Freshkills Parkfalse622236745725181951AQMAAAABAAAACAAAAKO97SfLjFLAIFImkFVHRECDqWsmwIxSwIgY7T9ZR0RAaiCr/7mMUsAVstwXZUdEQKmCacK3jFLAJQi4Ym9HREBTwBAWwoxSwGytiUB/R0RAlcOy8tCMUsB7rxhGekdEQE9Ql3vVjFLA5tnfbWVHRECjve0= (truncated)
Staten Island9999Freshkills Parkfalse622236745692708863AQMAAAABAAAACgAAABxTRq7gilLAmdfeuPlHREDD33S73YpSwKGHSLX6R0RAyUDDbd2KUsAKqHCXDEhEQHoKDC7dilLAXpp1LhtIRECzsMwt3YpSwIA5Dj0bSERAJM9mHN2KUsARqtk4H0hEQKbuUhbdilLAAg+akiFIREAwyi0= (truncated)
Staten Island9999Freshkills Parkfalse622236745763880959AQMAAAABAAAABAAAAMJtrBbEilLAuXWkiSxLREAKJ7ZcwYpSwLrfZjFBS0RAuOwva8WKUsAVGCeVLktEQMJtrBbEilLAuXWkiSxLREA=
Staten Island9999Freshkills Parkfalse622236745673801727AQMAAAABAAAAEAAAAEusd47ri1LARkK8hxlLREB67WSv3ItSwK+/oIEeS0RAiTnvJNiLUsC9w3NbM0tEQKosoHnii1LAw55wO0NLRECON47p6otSwA05smhAS0RAQTKHHt2LUsCWtkiYMUtEQGXeF4jfi1LAns6gKjFLREAkqVc= (truncated)
Staten Island9999Freshkills Parkfalse622236745762766847AQMAAAABAAAABAAAABHqBw3NilLA5PFQ+cpKREA20j3kzIpSwHeLCgrNSkRAic/LRM2KUsBDJg1Py0pEQBHqBw3NilLA5PFQ+cpKREA=
Staten Island9999Freshkills Parkfalse622236745769549823AQMAAAABAAAACAAAAEsfeSjQilLAvI5xoTlMREBSSHP3yYpSwGY4QrM7TERAOPSwLMWKUsBX9gTsV0xEQOaTaHXEilLAzKLeUl9MREDTO0MRx4pSwLxEjlVjTERAUghS8dWKUsAn/qZcXkxEQAYG43zailLAFXeRgklMREBLH3k= (truncated)
Staten Island9999Freshkills Parkfalse622236746168893439AQMAAAABAAAACgAAAEFnNV4qjVLAwD4mP1tHREBSQQgeKo1SwJ1uO2ZcR0RAZL/3cTSNUsCXXLZDbEdEQMYbEV82jVLAcXR5nmtHREBx/B14P41SwJ9IGV9eR0RAd4R8yzqNUsDHj1iuXkdEQDyyRxU2jVLAdJqzbl5HREDzpns= (truncated)
Staten Island9999Freshkills Parkfalse622236745673244671AQYAAAACAAAAAQMAAAABAAAABQAAAGfnWjFYjFLAcCT/1uFKREBEWyhGUYxSwCD74ifkSkRAow3VNFKMUsA4U1F95kpEQD0y/b9ZjFLAYpJwO+RKREBn51oxWIxSwHAk/9bhSkRAAQMAAAABAAAABQAAALTRzktOjFLAkEgTJ+U= (truncated)
Staten Island9999Freshkills Parkfalse622236745730293759AQMAAAABAAAADwAAAHsrqeBrjVLALNBqRGhIREC72IoDXY1SwL+PlD9tSERAycTpeliNUsAWLoUYgkhEQF/qes9ijVLAaVta9pFIREB0K4b3bo1SwMamceONSERAC8KjXG+NUsBsmGihjEhEQKB950BwjVLAOz81o4lIREDlS8k= (truncated)
Staten Island9999Freshkills Parkfalse622236745757982719AQMAAAABAAAACAAAAFvg22aMilLANALWn7VLREB+k+jbiopSwE9M1CO2S0RAGDjhCnqKUsDZZAofz0tEQO+Cb7V5ilLATtUVkNBLREBwrGJPg4pSwGznOlPfS0RAL1ApL5KKUsBiUqda2ktEQNP227qWilLAaLPtgMVLREBb4Ns= (truncated)
Staten Island9999Freshkills Parkfalse622236745736519679AQMAAAABAAAABgAAABIxNdUtjVLALY8s+hJKRECy44BiKo1SwHiQ5CEUSkRA9g1i2SWNUsCgwZ/7KEpEQBUkeHMujVLAEIjHMTZKREByI0BpLo1SwE8n89smSkRAEjE11S2NUsAtjyz6EkpEQA==
Staten Island9999Freshkills Parktrue622236745678913535null
Staten Island9999Freshkills Parkfalse622236745682026495AQMAAAABAAAABwAAABhV/Kp6i1LAh31HSGFMRECUDmrva4tSwAQrrjVmTERABZFSZGeLUsARqwQQe0xEQKfzQ7lxi1LAY4jX8IpMRED1jqXhcotSwCtxt42KTERAYIM7BnqLUsBWLJGmYkxEQBhV/Kp6i1LAh31HSGFMREA=
Staten Island9999Freshkills Parkfalse622236745735405567AQMAAAABAAAALAAAACBtiIU+jVLASpeWu6xJREC8PLExMo1SwFki59ywSURAVV6sqC2NUsD4YnG2xUlEQFL7/MY3jVLATD7rQNVJREBHH4DjN41SwDCgssfTSURAIsTaMjiNUsBjWkyr0UlEQI02BZ04jVLAtALEwc9JRED26fE= (truncated)
Staten Island9999Freshkills Parkfalse622236745696149503AQMAAAABAAAACQAAAORf+0wejFLAur1XzyRIRECcUfVWHYxSwICcAXIlSERA39+yMxqMUsA/LFb4JkhEQMyzxCgEjFLA7KJzdS9IRECy6bdfA4xSwLMk9RAzSERAMuo1sw2MUsD2iZTvQkhEQLzpcZAcjFLAsm3J9T1IRECArRk= (truncated)
Staten Island9999Freshkills Parkfalse622236745742188543AQMAAAABAAAACAAAAIprKi5ti1LAwRpDy59MREAWlwtOXotSwFh5zcSkTERAjxrcwlmLUsBkxz+fuUxEQE1c3xdki1LAZg02gMlMREAFjWoXZ4tSwK+BhX/ITERAVAUJCGiLUsDfm3gqx0xEQOtdaKtui1LA144nFaJMRECKayo= (truncated)
Staten Island9999Freshkills Parkfalse622236745699262463AQMAAAABAAAACQAAAJjNyNXrilLAKAnxzP9JREDMTz333IpSwFwbsMUESkRA/2kpzNqKUsC/hKG4DkpEQJ1M8g/ailLAz8UH5RlKREAWV8fm2YpSwFyfwuQbSkRA9IGfv+KKUsAImNB+KUpEQB9AQZ7xilLAEb4HhiRKRECw80s= (truncated)
Staten Island9999Freshkills Parkfalse622236745695592447AQMAAAABAAAAFAAAACHyBz3Zi1LAvWeqxEVIREAEgGFh14tSwOf3dj5ISERATDZXYNeLUsCOiL8/SEhEQOccQV/Xi1LAvJIdQUhIRECJnmNZ04tSwD3fZQVNSERAwUAYJM+LUsD72D9wUUhEQC2BQSPPi1LAkpgWcVFIREBFkXY= (truncated)
Staten Island9999Freshkills Parkfalse622236745912778751AQMAAAABAAAABAAAAGogq/+5jFLAFbLcF2VHREDDmav/s4xSwCv2LqVwR0RAqYJpwreMUsAlCLhib0dEQGogq/+5jFLAFbLcF2VHREA=
Staten Island9999Freshkills Parkfalse622236745769877503AQMAAAABAAAABAAAAAESmsN+ilLAN0D7LPRLRECZpv25fIpSwPYgS9v0S0RAJSoxv3+KUsA+gcyv9UtEQAESmsN+ilLAN0D7LPRLREA=
Staten Island9999Freshkills Parkfalse622236745691365375AQMAAAABAAAACwAAAGozUIn1ilLAEKQIQdlHREAw8SYW9IpSwMcGf+jfR0RALiPBaP6KUsDAqrbH70dEQBJ93QEMi1LA/HtxO+tHREA3kig6BotSwLZ1sPHlR0RAxGtyuQCLUsAGDz7t4EdEQAg8IJ7/ilLA9ibo6t9HREDs7PU= (truncated)
Staten Island9999Freshkills Parkfalse622236745736290303AQMAAAABAAAABgAAAO0EkPJDjVLAR0N/nJ5JREBBXEPvQo1SwJNXp0SjSURA56I3SkONUsBSb/DdoklEQNmKbSVEjVLAIcjG8KFJRECgZKwxRI1SwIaCYj+gSURA7QSQ8kONUsBHQ3+cnklEQA==
Staten Island9999Freshkills Parktrue622236745921560575null
Staten Island9999Freshkills Parkfalse622236745913106431AQMAAAABAAAAEAAAAIIWNIjqjFLAZ2Od0SZHREBZDm1g6IxSwLkJSe8mR0RAA+xnOuaMUsDe2MI4J0dEQNYVpRfkjFLAj2zZrSdHREA05cPV4oxSwDR9Fg0oR0RAKXtIjd6MUsChDoq9O0dEQJJi9ODojFLA9Hgrm0tHREDMPG8= (truncated)
Staten Island9999Freshkills Parkfalse622236745698476031AQMAAAABAAAACQAAALL+ux3oilLAISejGFNJREBRTn3L4IpSwJM6VYtVSURAr2wazOCKUsAX3FEnVklEQKdUfongilLARGuvEmdJRED8vVvz34pSwAZAevV3SURAzrEi0N+KUsB9ldCGfElEQEwaBObtilLAFHgZ0XdJREBOsPA= (truncated)
Staten Island9999Freshkills Parkfalse622236745769648127AQMAAAABAAAABwAAAJ5JnIXCilLA0IWqL3hMRECiHMIJwopSwI4IEFl4TERA/atadMCKUsCS30q4iExEQO271ie+ilLAeH+vT6BMREBgDW9OyIpSwLcrH+ucTERAEwwY2syKUsCQ2O0QiExEQJ5JnIXCilLA0IWqL3hMREA=
Staten Island9999Freshkills Parkfalse622236746168991743AQMAAAABAAAAbgAAAOZxFHmcjVLAepg7ZklHRECjlbOcjY1SwBiZeWFOR0RAeDp0FImNUsAwnOU5Y0dEQNRjiUWTjVLAJcIu4XJHRED/Zstpk41SwACBE3xyR0RAaf7Oo5ONUsAQfUPlcUdEQB0fx+CTjVLAt2EwUXFHRED02aI= (truncated)
Staten Island9999Freshkills Parkfalse622236745673342975AQMAAAABAAAACQAAAB+zGpprjFLA/inKAshKRECwB1G7XIxSwNN6Kv3MSkRAZ+daMViMUsBwJP/W4UpEQD0y/b9ZjFLAYpJwO+RKREDd+3RsX4xSwLrmzYjiSkRArimgTm6MUsBukC0X5kpEQBjpkTV0jFLAQxtKzt9KRED9fQQ= (truncated)
Staten Island9999Freshkills Parktrue622236745680125951null
Staten Island9999Freshkills Parkfalse622236745672785919AQMAAAABAAAABwAAACtBCaInjFLAVe8AGPJKREBS8sO1HIxSwGna4b/1SkRArQe3bRuMUsDdVY+i+0pEQACZCuIdjFLA31deePlKREB6WJ8NIYxSwBYZS+72SkRASHDjSSSMUsDgOSWX9EpEQCtBCaInjFLAVe8AGPJKREA=
Staten Island9999Freshkills Parkfalse622236745690578943AQMAAAABAAAACQAAADmf/S3qilLA0p0dH8tIREDeljZq24pSwFzACA/QSERAHQuVdduKUsDBMc0y00hEQP7D8CDcilLA/mL9X+hIREAm+Thi3IpSwFEUV5HtSERABbJIGOGKUsBnDLvP9EhEQA6OJ/bvilLAP+gT1+9IREBuE/g= (truncated)
Staten Island9999Freshkills Parkfalse622236745761751039AQMAAAABAAAACAAAAIfQ/ZbEilLAbsZnK/BLRECF0TG3tYpSwH21MST1S0RAZUAtzbGKUsAxnLUYB0xEQIMkNPu8ilLAoHafRg5MREA7iZpExYpSwPaBRZsWTERAImKyX8qKUsAJfFzmFExEQDhHP+vOilLAxBlxDABMRECH0P0= (truncated)
Staten Island9999Freshkills Parktrue622236745679568895null
Staten Island9999Freshkills Parkfalse622236745743400959AQMAAAABAAAACAAAAG3Nwusyi1LABCIZR+1MREAhLGkLJItSwLFPb0DyTERAyND3fx+LUsBWzvUaB01EQJsA9NQpi1LAtHY0/BZNREC4lME/MotSwGymuysUTURAS5QPEDuLUsB9Dzo1B01EQNZYwUA9i1LAkKk/KP1MREBtzcI= (truncated)
Staten Island9999Freshkills Parkfalse622236745742843903AQMAAAABAAAABwAAAAEzn97yilLA2oyXBhZNREClgCsC5IpSwAfxVf4aTURA2LgymeuKUsAqkB94HE1EQCuus3/wilLAjowFRh1NREDVFnRh9YpSwF42raweTURAx31npfmKUsB65hRxIE1EQAEzn97yilLA2oyXBhZNREA=
Staten Island9999Freshkills Parkfalse622236745735503871AQMAAAABAAAAJwAAAGmcpnQzjVLA6wiKbupJREBKr4qWJI1SwJq/oWnvSURAfd5tDSCNUsBt20dDBEpEQLLjgGIqjVLAeJDkIRRKREASMTXVLY1SwC2PLPoSSkRAUz3cvy2NUsD9FD0cEEpEQKFRPQkujVLABAdpMg5KREAF9AY= (truncated)
Staten Island9999Freshkills Parkfalse622236745763749887AQMAAAABAAAACAAAALtW8YHOilLArlVM4gRLREDBzi5tyIpSwG9it+oGS0RA4J4bi8eKUsBsOitdEktEQMJtrBbEilLAuHWkiSxLREC47C9rxYpSwBUYJ5UuS0RAq7F7StSKUsCSfmGcKUtEQKxz0tXYilLAjfHswhRLREC7VvE= (truncated)
Staten Island9999Freshkills Parkfalse622236745742286847AQMAAAABAAAABQAAAPWOpeFyi1LAK3G3jYpMRECn80O5cYtSwGOI1/CKTERAimsqLm2LUsDBGkPLn0xEQOpdaKtui1LA1o4nFaJMRED1jqXhcotSwCtxt42KTERA
Staten Island9999Freshkills Parkfalse622236745762635775AQMAAAABAAAABwAAAHYlWVvWilLAhyqUnKFKREBbb3kU0IpSwFIVyLWjSkRAEeoHDc2KUsDk8VD5ykpEQInPy0TNilLAQyYNT8tKRECketcj3IpSwOq4SVbGSkRA5zIUr+CKUsBhDwZ9sUpEQHYlWVvWilLAhyqUnKFKREA=
Staten Island9999Freshkills Parkfalse622236745730162687AQMAAAABAAAAGgAAAIWgZ8V9jVLAs+4/8U5IREDJVDRpcI1SwFX6gGtTSERAeyup4GuNUsAs0GpEaEhEQJIooa11jVLAxTnhUXdIREArfNixdY1SwDytwUR3SERAuryaHHaNUsDfR6badUhEQHJmIY12jVLAAyiXwHNIRECYmyg= (truncated)
Staten Island9999Freshkills Parkfalse622236745760964607AQMAAAABAAAADwAAAA6xuwjBilLASTCfrUNLREAMPY6+wIpSwIpDl99FS0RADlNtYMCKUsASCxWsSUtEQDbSl+C/ilLA1AOZb01LREC/N2w/v4pSwAJlaydRS0RAwVRcfb6KUsC3/NzQVEtEQOpFM529ilLA4aMZP1hLREAiICs= (truncated)
Staten Island9999Freshkills Parkfalse622236745681895423AQMAAAABAAAABAAAACCTZnWFi1LAwuEBV0pMREDpjmBaf4tSwCRuzGFMTERAFEPU6XqLUsBUWqrCYExEQCCTZnWFi1LAwuEBV0pMREA=
Staten Island9999Freshkills Parkfalse622236745913204735AQMAAAABAAAACwAAABXjZov2jFLAiJgF+yhHREBY9ydG/IxSwGiLbMgxR0RARIHViAONUsD2Ja9ZL0dEQP5xVegCjVLAOz+bzi5HRECGIiypAY1SwJeud90tR0RAGzyPYQCNUsCfkFYILUdEQEWuBTf+jFLARiql8CtHRECeYPs= (truncated)
Staten Island9999Freshkills Parkfalse622236745691791359AQMAAAABAAAADAAAACoGIm3eilLAkiWYz9FHREAB8qRI3opSwD3ytDXaR0RAQqf0OOWKUsCSOQ7h5EdEQDDxJhb0ilLAxwZ/6N9HREBqM1CJ9YpSwBCkCEHZR0RAJkRisPKKUsB/rkns10dEQHVLi13vilLAjx6hh9ZHRECs1T8= (truncated)
Staten Island9999Freshkills Parkfalse622236745762963455AQMAAAABAAAABAAAANaM/WrUilLANt5z9mZKRED0jon204pSwBTVNlZtSkRAJZolHtWKUsCeeuUJaEpEQNaM/WrUilLANt5z9mZKREA=
Staten Island9999Freshkills Parkfalse622236745912647679AQMAAAABAAAABwAAANhuy9LNjFLAokyR2T1HREBeY4fGzYxSwL/8KPI9R0RAxheJEMSMUsCuK4O2UUdEQISpaybAjFLAhxjtP1lHRECjve0ny4xSwCBSJpBVR0RA40TQsM+MUsBWlAK4QEdEQNhuy9LNjFLAokyR2T1HREA=
Staten Island9999Freshkills Parkfalse622236745769746431AQMAAAABAAAACAAAAC9QKS+SilLAYlKnWtpLREBwrGJPg4pSwGznOlPfS0RAARKaw36KUsA3QPss9EtEQCUqMb9/ilLAPoHMr/VLREDbr8BfhYpSwEVfuDv3S0RA56Ztm5iKUsBVd9Ul/EtEQMadPYOcilLALk/KO+pLREAvUCk= (truncated)
Staten Island9999Freshkills Parkfalse622236746169090047AQMAAAABAAAAWQAAAG9CaOCOjVLAOgej74dHREAwmN0DgI1SwM+E2eqMR0RA6FCGe3uNUsAlamHDoUdEQHGZzc+FjVLAnx/BoLFHREDG46MAiY1SwK+q9o6wR0RArlsIMomNUsDMdGTwr0dEQNW/eIWJjVLAbZaq8K5HRECmLPQ= (truncated)
Staten Island9999Freshkills Parkfalse622236746168532991AQMAAAABAAAABwAAANOJOWFxjVLA9XyFoVpHREBd8nW9X41SwGstTkBjR0RAGO8Ff1yNUsAjLT8rckdEQM57ItNmjVLAvnCgCIJHREBMhKqvdY1SwAgXhQ19R0RAFsj/N3qNUsCVSRI1aEdEQNOJOWFxjVLA9XyFoVpHREA=
Staten Island9999Freshkills Parkfalse622236745690677247AQMAAAABAAAABwAAAJaTnyPeilLAwauyGAxJREAEAkaS3opSwJtPO5QSSURAs//Tnd+KUsAmHsVpI0lEQJfWoh3gilLAx6cvHi9JREDsCoZV4opSwAM5TWAuSURA4olu4OaKUsDtgduHGUlEQJaTnyPeilLAwauyGAxJREA=
Staten Island9999Freshkills Parkfalse622236745679667199AQMAAAABAAAABwAAAMOqaqPMjFLAADEi3UZKRECt/ALFvYxSwIZS2ddLSkRA44l6O7mMUsCOL4yxYEpEQNmtbZDDjFLAVzyWkHBKRECI/Em7y4xSwP8/YdRtSkRAJAV3dNGMUsDK4jpDTkpEQMOqaqPMjFLAADEi3UZKREA=
Staten Island9999Freshkills Parkfalse622236745750839295AQMAAAABAAAABgAAAFzYInTAilLAKfO+NQBNREBB/k2MtIpSwMpgZzAETURAyb1QWLOKUsBkYsfNEE1EQCMFdJHJilLAtfyqqxVNREAm6djIyopSwNdDPBcQTURAXNgidMCKUsAp8741AE1EQA==
Staten Island9999Freshkills Parktrue622236745729376255null
Staten Island9999Freshkills Parkfalse622236745696903167AQMAAAABAAAABwAAAM9QZfeFi1LA9tCAtUlIRECs3yEigotSwIct0/1KSERAaMvol32LUsCB4A7WX0hEQO0eHOuHi1LAnRMdtW9IREAy4IrIlotSwAp217tqSERAF8kzsZqLUsDi+w3JWEhEQM9QZfeFi1LA9tCAtUlIREA=
Staten Island9999Freshkills Parkfalse622236745757065215AQMAAAABAAAABQAAAB2rT82hilLAK9g3t5JLREAufZ0+nIpSwPDVNVCcS0RAoABJppuKUsAe8X8ynUtEQKlDKdKfilLAYkKWzZtLREAdq0/NoYpSwCvYN7eSS0RA
Staten Island9999Freshkills Parkfalse622236745742385151AQMAAAABAAAABAAAAAWNahdni1LAroGFf8hMREBNXN8XZItSwGYNNoDJTERAhFTWEWOLUsBFdtUyzkxEQAWNahdni1LAroGFf8hMREA=
Staten Island9999Freshkills Parkfalse622236745673768959AQMAAAABAAAACAAAAI43junqi1LADDmyaEBLRECqLKB54otSwMOecDtDS0RAuW8o792LUsCLtlgVWEtEQDyb7UPoi1LAvdlg9WdLREBL45x39otSwD0i0TRjS0RATkGm0feLUsDL/C7aX0tEQOKa9+v6i1LAfkZ+mlFLRECON44= (truncated)
Staten Island9999Freshkills Parktrue622236745769517055null
Staten Island9999Freshkills Parkfalse622236745730260991AQMAAAABAAAAJgAAAHUrhvdujVLAxqZx441IREBf6nrPYo1SwGlbWvaRSERAa9LXRl6NUsCgEGDPpkhEQCQuXl9mjVLAFu6JPrNIRECqx/yQZo1SwKFpRfiySERA89HMMmeNUsDb/RiHskhEQMQxL5dnjVLAgs8Jg7JIREDFrU4= (truncated)
Staten Island9999Freshkills Parkfalse622236745743269887AQMAAAABAAAACAAAAEU7ZFdGi1LAF8oxc9NMRECSMx53N4tSwNddmWzYTERAbc3C6zKLUsAEIhlH7UxEQNZYwUA9i1LAkKk/KP1MREB8ENtiQ4tSwFa2PBv7TERA10BbHU+LUsAOw4d76kxEQPwhZaxQi1LA2TBAVONMREBFO2Q= (truncated)
Staten Island9999Freshkills Parkfalse622236745700343807AQMAAAABAAAACAAAAF424J3eilLALDWSr4FIREDzmPH+24pSwM2v64+CSERALwjLvNuKUsD5/EBKkEhEQKkTtzrbilLAQoMUz6hIREBjPyg624pSwFetVXipSERAb9flZeSKUsBWt0dnpkhEQP5FsvDoilLAZgAcj5FIREBeNuA= (truncated)
Staten Island9999Freshkills Parkfalse622236745678880767AQMAAAABAAAABgAAANJGsZ6rjFLA33bcPp9KRECeJwzAnIxSwMGtejmkSkRA6c9VNpiMUsCaJVATuUpEQKAIEl2YjFLApnjRTrlKREDlerXBrYxSwCmzKoeiSkRA0kaxnquMUsDfdtw+n0pEQA==
Staten Island9999Freshkills Parkfalse622236745742712831AQMAAAABAAAACgAAAAz8pEoGi1LAyBTvMvxMREB/bTpq94pSwA6sGCwBTURAATOf3vKKUsDajJcGFk1EQMd9Z6X5ilLAeuYUcSBNREBJ5nIy+opSwDKGhKsgTURA/wLa5v6KUsBkDCo/I01EQCtf+PEAi1LA3S+QpyRNREAaRQM= (truncated)
Staten Island9999Freshkills Parkfalse622236745735372799AQMAAAABAAAAJAAAAMRo/LY3jVLAZffo2dZJREBpnKZ0M41SwOsIim7qSURA+d1JyzSNUsD6t8l87ElEQDOCDd40jVLAKUIqbuxJREACiayTNY1SwJK7PRLsSURAMcec9zWNUsC0/EBQ60lEQG9eIGI2jVLAxnHpRupJRECXcwE= (truncated)
Staten Island9999Freshkills Parkfalse622236745920643071AQMAAAABAAAABgAAANgYEA6VjFLAkR28UKlHREBPXNQyhIxSwBUZLAvHR0RAmBuE5IeMUsDtzke4zEdEQEHcYMGWjFLAZtQKvsdHREA1JodKm4xSwMaoveWyR0RA2BgQDpWMUsCRHbxQqUdEQA==
Staten Island9999Freshkills Parktrue622236745680879615null
Staten Island9999Freshkills Parkfalse622236745691332607AQMAAAABAAAACAAAABJ93QEMi1LA/HtxO+tHREAuI8Fo/opSwMCqtsfvR0RA7q8m3vmKUsCTnpWfBEhEQCAT1TAEi1LAk5HYfhRIREDBRCAOE4tSwIZzJIYPSERA1NammBeLUsAwKjeu+kdEQKCuRdAPi1LAtqLotu5HREASfd0= (truncated)
Staten Island9999Freshkills Parktrue622236745733701631null
Staten Island9999Freshkills Parktrue622236745750937599null
Staten Island9999Freshkills Parktrue622236745697001471null
Staten Island9999Freshkills Parkfalse622236745760833535AQMAAAABAAAACAAAAKuxe0rUilLAkn5hnClLREC47C9rxYpSwBUYJ5UuS0RACie2XMGKUsC632YxQUtEQA+xuwjBilLASTCfrUNLREBEL7Yzy4pSwLZnZk9TS0RA7DYYE9qKUsB2BpdWTktEQPoEcZ7eilLAFWYNfTlLRECrsXs= (truncated)
Staten Island9999Freshkills Parkfalse622236745735700479AQMAAAABAAAAEwAAALQHo0M7jVLA7nzuKYdJRED5LMdlLI1SwMHtAyWMSURAWFPE3CeNUsCnGHn+oElEQLw8sTEyjVLAWSLn3LBJREAgbYiFPo1SwEqXlrusSURAcUhlED+NUsAdsK/OqklEQKg1mTlBjVLAWq13QqVJRECkwDw= (truncated)
Staten Island9999Freshkills Parkfalse622236745681764351AQMAAAABAAAACAAAAIwmheW8i1LA9Z0od7BLRECoGgsGrotSwGaO7HC1S0RAbX5Pe6mLUsAx9f1KyktEQPfWIdCzi1LAuMBZK9pLREBb/ekiwItSwL+n7gvWS0RA89LTEsSLUsBCCgbTzktEQJTYWTrHi1LAUUlsV8BLRECMJoU= (truncated)
Staten Island9999Freshkills Parkfalse622236745762832383AQMAAAABAAAACAAAAKR61yPcilLA6rhJVsZKRECJz8tEzYpSwEMmDU/LSkRAN9I95MyKUsB0iwoKzUpEQGNgliHLilLAj5WO2+NKRECiDEYN04pSwCjc7AjwSkRArPln7OGKUsATpx8Q60pEQNS9pnfmilLAx+nGNtZKRECketc= (truncated)
Staten Island9999Freshkills Parkfalse622236746168958975AQMAAAABAAAABgAAAM3Ma5ePjVLABgLhpYRHREBvQmjgjo1SwDoHo++HR0RAaOCiCI+NUsBKW2wtiEdEQEa8qCGPjVLAAsfclIdHRECeZNxxj41SwKi7GpuFR0RAzcxrl4+NUsAGAuGlhEdEQA==
Staten Island9999Freshkills Parkfalse622236745690546175AQMAAAABAAAACAAAAA6OJ/bvilLAP+gT1+9IREAFskgY4YpSwGcMu8/0SERAjlqPpN2KUsD/cP+mBElEQJaTnyPeilLAwauyGAxJREDiiW7g5opSwO2B24cZSURAQ6RjvvWKUsA/liqPFElEQPk0Nkn6ilLARJ6/tv9IREAOjic= (truncated)
Staten Island9999Freshkills Parkfalse622236745761718271AQMAAAABAAAABwAAACJisl/KilLACXxc5hRMREA6iZpExYpSwPeBRZsWTERAM5Feqc6KUsC21skMIExEQFNIc/fJilLAZjhCsztMREBLH3ko0IpSwLyOcaE5TERAtBAItNSKUsABGnHHJExEQCJisl/KilLACXxc5hRMREA=
Staten Island9999Freshkills Parkfalse622236745679536127AQMAAAABAAAAJQAAAMpDMQvgjFLAkZqtCC1KREDVLt0s0YxSwLIXdgMySkRAw6pqo8yMUsAAMSLdRkpEQCQFd3TRjFLAy+I6Q05KREDMy6rN1IxSwH/qWMs7SkRAsvtiPdWMUsAl1UwkPEpEQDNcjp7VjFLAviqGWjxKRECH/M4= (truncated)
Staten Island9999Freshkills Parktrue622236745736585215null
Staten Island9999Freshkills Parkfalse622236745729245183AQMAAAABAAAALAAAADLQxNOGjVLAz4V57BFIRECv9KUCfo1SwPc4++AUSERAULkyenmNUsCkLsm5KUhEQDva5neBjVLAiAm1/zVIREDhkTR5gY1SwLvnIvo1SERAjqF22YGNUsBbvuRsNEhEQKLNoDyCjVLA1TJT4TJIREAoKbI= (truncated)
Staten Island9999Freshkills Parkfalse622236745743368191AQMAAAABAAAABQAAALiUwT8yi1LAbKa7KxRNRECbAPTUKYtSwLR2NPwWTURAqtP19CiLUsA8pAQAG01EQP+oH2Qxi1LAZxLDbhVNREC4lME/MotSwGymuysUTURA
Staten Island9999Freshkills Parkfalse622236745756934143AQMAAAABAAAABwAAABc4Zze2ilLAMPCOuGpLREDAxvqotIpSwA3YGShuS0RAlKxE5a6KUsAfGV4bektEQNHHlx6pilLAGoA+XYVLRECyHU09s4pSwO3XS/uBS0RABtHRyLeKUsDPwLQhbUtEQBc4Zze2ilLAMPCOuGpLREA=
Staten Island9999Freshkills Parkfalse622236745696215039AQMAAAABAAAACwAAALXuThv0i1LAX6sObDdIRECpl7jr64tSwJZhodE7SERAviuZm+OLUsCvTCm8P0hEQCZswprji1LAt++JvD9IRECwmeqZ44tSwLKS6rw/SERA/9ZrM9uLUsDnWTwnQ0hEQCHyBz3Zi1LAvWeqxEVIREDXEmA= (truncated)
Staten Island9999Freshkills Parkfalse622236745692545023AQMAAAABAAAABAAAACgicrTcilLA83wbF0dIREArG9yo3IpSwKOPqGtLSERAQxO1YN2KUsDDjPsfSEhEQCgicrTcilLA83wbF0dIREA=
Staten Island9999Freshkills Parkfalse622236745724461055AQMAAAABAAAACgAAAJkOOndRjVLA525JvwtJREB5rTCbR41SwEbzBw0PSURAGLZdEkONUsDlZEXmI0lEQHzYJmdNjVLAyKNsxDNJREDHUhhOT41SwGiaQiEzSURAVbkEolKNUsDNYIVVDUlEQBEZy0dSjVLAsnLBeQxJREDOZYg= (truncated)
Staten Island9999Freshkills Parkfalse622236745695657983AQMAAAABAAAADgAAABfJM7Gai1LA4vsNyVhIREAy4IrIlotSwAp217tqSERAAcDUG6GLUsAf2NiaekhEQGgqRvmvi1LAXRV4oXVIRECQ4BJKtItSwLrTGtBhSERAV+uEyrGLUsCmx90dYkhEQNwdcPWti1LAPRtxCGJIREAleDs= (truncated)
Staten Island9999Freshkills Parkfalse622236745691987967AQMAAAABAAAACAAAAKooeOkvi1LAZfYDlQlIREAcETM+LItSwNxVi2waSERAQI0OkTaLUsBkq7RLKkhEQIwUX25Fi1LAAEXKUiVIRECyP2O7SItSwFMVnSsWSERAGWXJvjmLUsDyC945D0hEQCyKW1sxi1LA8nYzsgpIRECqKHg= (truncated)
Staten Island9999Freshkills Parkfalse622236745680977919AQMAAAABAAAABwAAANOw7Nqbi1LA+8pp2QhMREA2MDX7jItSwAXBFNMNTERAT6RLcIiLUsCRtUitIkxEQCoAeomRi1LAx5K2qDBMREAKMIfLlItSwMZjWLspTERA4lHgu6CLUsD3hrhYEExEQNOw7Nqbi1LA+8pp2QhMREA=
Staten Island9999Freshkills Parkfalse622236745691430911AQMAAAABAAAABQAAAKGuRdAPi1LAt6Lotu5HREDU1qaYF4tSwDAqN676R0RANvJdkBuLUsBQenpa+UdEQERPK2Eai1LAqpOQYfhHREChrkXQD4tSwLei6LbuR0RA
Staten Island9999Freshkills Parkfalse622236745723346943AQMAAAABAAAAHAAAAGvS10ZejVLAoBBgz6ZIREDprY9pT41SwNBJgsqrSERA0KrW4EqNUsAEx46jwEhEQE+zeTVVjVLA1lmHgdBIREAyjfLbX41SwFCY5O/MSERAe70cJGCNUsDujN4AzEhEQFS6eo1gjVLAyM+DecpIREA8O1w= (truncated)
Staten Island9999Freshkills Parkfalse622236745921626111AQMAAAABAAAAEQAAALDgyKhgjFLA/P5Tsv9HREAmjv71XIxSwJoITWsDSERALyUTaVuMUsDV0oQvBEhEQHb+39FZjFLAiBZRugRIREB0ovMzWIxSwJm8fgoFSERArsftklaMUsCQYVkfBUhEQO6VdPJUjFLAy8ax+ARIREBH/Co= (truncated)
Staten Island9999Freshkills Parkfalse622236745743138815AQMAAAABAAAABwAAAGANb07IilLAtysf65xMREDuu9YnvopSwHh/r0+gTERAv4zxC7uKUsAFUjo3wExEQG4OBTe/ilLAgZ2dn8ZMREB3/VMXzopSwJUttKbBTERAnAj/otKKUsB5yG3MrExEQGANb07IilLAtysf65xMREA=
Staten Island9999Freshkills Parktrue622236745760931839null
Staten Island9999Freshkills Parkfalse622236745678749695AQMAAAABAAAABgAAANo04wa/jFLANC5eaoVKREAJr1EosIxSwNDBDWWKSkRA0kaxnquMUsDfdtw+n0pEQOV6tcGtjFLAKbMqh6JKREATkUgmw4xSwD9Zlb+LSkRA2jTjBr+MUsA0Ll5qhUpEQA==
Staten Island9999Freshkills Parkfalse622236745920511999AQMAAAABAAAABgAAAB+VN+mljFLAHNlvlotHREDYGBAOlYxSwJEdvFCpR0RANSaHSpuMUsDGqL3lskdEQGtSUCeqjFLAWFdv661HREBgsmCwroxSwKzyKBOZR0RAH5U36aWMUsAc2W+Wi0dEQA==
Staten Island9999Freshkills Parkfalse622236745913171967AQMAAAABAAAADwAAAESB1YgDjVLA9iWvWS9HREBY9ydG/IxSwGiLbMgxR0RAzDxvvfeMUsBI0JegRkdEQNurMRECjVLA62YsflZHRECtKa/tEI1SwCGbfYNRR0RA4we/chONUsD4CLrtRUdEQOQrAhUOjVLAbjrjPT9HREDV1s8= (truncated)
Staten Island9999Freshkills Parkfalse622236745673965567AQMAAAABAAAABgAAAEvjnHf2i1LAPSLRNGNLREA8m+1D6ItSwL3ZYPVnS0RAVdVzueOLUsAaBV7PfEtEQFNTPCboi1LAeowAnINLREAh5ikQ9YtSwJ9Au7BmS0RAS+Ocd/aLUsA9ItE0Y0tEQA==
Staten Island9999Freshkills Parkfalse622236745673408511AQMAAAABAAAACAAAALu+rP0+jFLAFZO+8dZKREBwIdIeMIxSwFla8uvbSkRAsHDnACyMUsCsmrvU7kpEQD17KOM1jFLA0yHRc+dKREDaHSmNPoxSwP+vXF34SkRA6nN7k0eMUsBq4qLW7kpEQPSifVJJjFLAABdO0eZKREC7vqw= (truncated)
Staten Island9999Freshkills Parkfalse622236745730457599AQMAAAABAAAAFgAAAFC5Mnp5jVLApC7JuSlIRECCNz6dao1SwMdz+rQuSERAAxK1FGaNUsB/Ms+NQ0hEQMlUNGlwjVLAVfqAa1NIRECGoGfFfY1SwLPuP/FOSERAJLS19n2NUsCxrrQ2TUhEQG7V+1p+jVLAQGxP70lIREBH65U= (truncated)
Staten Island9999Freshkills Parktrue622236745733570559null
Staten Island9999Freshkills Parkfalse622236745672851455AQMAAAABAAAADwAAAKGV1vf+i1LAVCD6s/9KREAkb9cY8ItSwL79760ES0RAS6x3juuLUsBGQryHGUtEQGwbygf0i1LAJW8njSZLRECDK1bD9ItSwG9lkhwmS0RA+S3loviLUsClnVOJJUtEQHh2Qfb6i1LACQXJVCRLREB4iTo= (truncated)
Staten Island9999Freshkills Parktrue622236745679634431null
Staten Island9999Freshkills Parkfalse622236745729343487AQMAAAABAAAAMAAAALZOdEeBjVLAuBxeecZHRECt1b9qco1SwHIWjXTLR0RAkaFQ4m2NUsA83TBN4EdEQF/MqTZ4jVLASvizKvBHREC1qGG0go1SwD9bjabsR0RA/if6toKNUsDrgP2F7EdEQFP048mCjVLAMdgQxutHREB3w1Q= (truncated)
Staten Island9999Freshkills Parkfalse622236745696870399AQMAAAABAAAABgAAAEWNn5Zfi1LAgCxlxitIRECZTqSeXotSwGFA4TgwSERAUeOs8WiLUsC89vAXQEhEQHlj38Rzi1LAr3/MeDxIREBIZEZ1Y4tSwFVgYpswSERARY2fll+LUsCALGXGK0hEQA==
Staten Island9999Freshkills Parkfalse622236745742909439AQMAAAABAAAABwAAABpFAxQMi1LAKc7H7iBNREArX/jxAItSwN0vkKckTURAt+MycwOLUsDWUVVhJk1EQFn34MwHi1LAdi8zCSpNREDxR4LqC4tSwO4heSsuTURAVQ/kDRKLUsBEojUeKk1EQBpFAxQMi1LAKc7H7iBNREA=
Staten Island9999Freshkills Parkfalse622236745724559359AQMAAAABAAAABwAAALzcUd5IjVLAoCu/nUhJREC81p8AOo1SwFsl3JhNSURAlu60dzWNUsAEdDVyYklEQGz0j8w/jVLASxiAUHJJRECAqMkSRo1SwIYIUzZwSURADEdtl0yNUsBlF5tVTklEQLzcUd5IjVLAoCu/nUhJREA=
Staten Island9999Freshkills Parkfalse622236745681633279AQMAAAABAAAACAAAAHFDkU/Qi1LA+6JIo5ZLREBo0CpwwYtSwD31HZ2bS0RAjCaF5byLUsD1nSh3sEtEQJTYWTrHi1LAUUlsV8BLREA6o7cGzYtSwEBezGa+S0RAtPl/1dGLUsCbaA+VtUtEQKNTumjZi1LA7hZenqRLREBxQ5E= (truncated)
Staten Island9999Freshkills Parkfalse622236745723445247AQMAAAABAAAAIgAAAP+rvqxQjVLAp+2oWuVIREDrtEzPQY1SwKyfw1XqSERA9sF7Rj2NUsDw+usu/0hEQHmtMJtHjVLARvMHDQ9JRECZDjp3UY1SwOduSb8LSURAmy3qFFGNUsC8VxJ1C0lEQGJCHqRQjVLAXAF+CQtJREBL7aw= (truncated)
Staten Island9999Freshkills Parkfalse622236745698312191AQMAAAABAAAABgAAAG08OvjeilLAAk6WhphJREAXiRYn3opSwHp3p6WzSURASdt4DN6KUsCTTEAZt0lEQAdjDkXgilLAksMkW7ZJREC98xLQ5IpSwK4ObYKhSURAbTw6+N6KUsACTpaGmElEQA==
Staten Island9999Freshkills Parkfalse622236746168827903AQMAAAABAAAACwAAAOMHv3ITjVLA+Ai67UVHRECtKa/tEI1SwCGbfYNRR0RASSCIQRuNUsCoXQVhYUdEQFJBCB4qjVLAnW47ZlxHREBBZzVeKo1SwL8+Jj9bR0RAEqjAJyiNUsBQpshPWkdEQGmfbbkjjVLAMafh2ldHRECVqug= (truncated)
Staten Island9999Freshkills Parkfalse622236745679962111AQMAAAABAAAABAAAAFysMkV+jFLAjn+zF9VKRED9fQTvdYxSwKC1NOLXSkRAGOmRNXSMUsBEG0rO30pEQFysMkV+jFLAjn+zF9VKREA=
Staten Island9999Freshkills Parktrue622236745761587199null
Staten Island9999Freshkills Parkfalse622236745743237119AQMAAAABAAAABQAAAHwQ22JDi1LAVrY8G/tMREDWWMFAPYtSwJCpPyj9TERAS5QPEDuLUsB8Dzo1B01EQIhW+cFAi1LAdqb41P5MREB8ENtiQ4tSwFa2PBv7TERA
Staten Island9999Freshkills Parkfalse622236745700311039AQMAAAABAAAABwAAAG/X5WXkilLAVrdHZ6ZIREBjPyg624pSwFetVXipSERARaTUKNuKUsDEqJYAvkhEQN6WNmrbilLAW8AID9BIREA5n/0t6opSwNKdHR/LSERANhnMuO6KUsAT0dxGtkhEQG/X5WXkilLAVrdHZ6ZIREA=
Staten Island9999Freshkills Parkfalse622236745678847999AQMAAAABAAAABgAAAIj8SbvLjFLA/z9h1G1KREDZrW2Qw4xSwFc8lpBwSkRA2jTjBr+MUsA0Ll5qhUpEQBORSCbDjFLAP1mVv4tKREAyXwIQx4xSwCYt7JSHSkRAiPxJu8uMUsD/P2HUbUpEQA==
Staten Island9999Freshkills Parkfalse622236745921167359AQMAAAABAAAAEAAAAKpBmSoxjFLAVBV0KgpIRED1+tsXMIxSwFhnw8cLSERApyPfVi6MUsDCi142D0hEQMWNn+ksjFLAf5uV5RJIREDhGj3pKoxSwHFZtOIWSERACuO+oCiMUsBPdzyYGkhEQP3niRcmjFLAunh29B1IREBiNAs= (truncated)
Staten Island9999Freshkills Parkfalse622236745696083967AQMAAAABAAAABwAAAMyzxCgEjFLA7KJzdS9IREBu3gM3/ItSwO2BqIQySERASSVnIfSLUsB7p8hoN0hEQLXuThv0i1LAX6sObDdIRECckX6C9ItSwBocpQo4SERAsum3XwOMUsCzJPUQM0hEQMyzxCgEjFLA7KJzdS9IREA=
Staten Island9999Freshkills Parkfalse622236745692413951AQMAAAABAAAACgAAADDKLXbmilLAKU0UcB5IRECm7lIW3YpSwAIPmpIhSERAH2+7+9yKUsCZ6njbK0hEQOuxW87cilLApw88aD1IREAE2tXF3IpSwEW4R5dASERAKCJytNyKUsDzfBsXR0hEQEMTtWDdilLAw4z7H0hIREA0Zyc= (truncated)
Staten Island9999Freshkills Parkfalse622236745763586047AQMAAAABAAAABwAAACTefjTeilLAaUjOVz5KREBedQAm14pSwAbCx7NASkRA1oz9atSKUsA23nP2ZkpEQCWaJR7VilLAnnrlCWhKRECJLfH844pSwGA6JBFjSkRAId0TiOiKUsC8dRE4TkpEQCTefjTeilLAaUjOVz5KREA=
Staten Island9999Freshkills Parkfalse622236745913270271AQMAAAABAAAADgAAADTlw9XijFLANH0WDShHRECJpqD54YxSwC4EOU4oR0RAlFrR4d+MUsAMQHYZKUdEQEjurdHdjFLAvhj/DipHREALh6TK24xSwD4NLC4rR0RA2usdztmMUsBZZjV2LEdEQJ+QvQ7XjFLAZJ+IVy9HREC48dQ= (truncated)
Staten Island9999Freshkills Parkfalse622236745699196927AQMAAAABAAAABQAAALrsGn7bilLAuFfbgQJKRECdmFAc24pSwJ6EgfYJSkRA/2kpzNqKUsC/hKG4DkpEQMxPPffcilLAXBuwxQRKREC67Bp+24pSwLhX24ECSkRA
Staten Island9999Freshkills Parkfalse622236745695526911AQMAAAABAAAABQAAAIRo44zDi1LALs7IBFtIRECnGKKbwYtSwLNuDmBcSERA5S3Pf8GLUsACxJtwXEhEQMdPtWDDi1LAoZytz1tIRECEaOOMw4tSwC7OyARbSERA
Staten Island9999Freshkills Parkfalse622236745680846847AQMAAAABAAAABwAAABMxZEWvi1LAlzqABe9LRECXScBloItSwJGTPP/zS0RA07Ds2puLUsD7ymnZCExEQOJR4Lugi1LA94a4WBBMREAQZNodo4tSwNinvkcLTERA+pIzCrGLUsBy1VS98UtEQBMxZEWvi1LAlzqABe9LREA=
Staten Island9999Freshkills Parkfalse622236745769811967AQMAAAABAAAABwAAAFbvBmOrilLANo0bQ+VLREDGnT2DnIpSwC5PyjvqS0RA56Ztm5iKUsBVd9Ul/EtEQG61ciKmilLAQ0i3mv9LREBlQC3NsYpSwDKctRgHTERAhdExt7WKUsB9tTEk9UtEQFbvBmOrilLANo0bQ+VLREA=
Staten Island9999Freshkills Parkfalse622236746168598527AQYAAAACAAAAAQMAAAABAAAABAAAAMYbEV82jVLAcHR5nmtHREBkv/dxNI1SwJdctkNsR0RA1WQIwjONUsCPAKtsb0dEQMYbEV82jVLAcHR5nmtHREABAwAAAAEAAAAHAAAAJsRYLUiNUsDiJhvEbkdEQPRca30xjVLAU6dZ2nk= (truncated)
Staten Island9999Freshkills Parkfalse622236745750904831AQMAAAABAAAABgAAAPBjVKnZilLAqUM/HgtNREAm6djIyopSwNdDPBcQTURAIwV0kcmKUsC1/KqrFU1EQPpPG/vQilLAY0o+SxdNREBfton844pSwGCcPf0aTURA8GNUqdmKUsCpQz8eC01EQA==
Staten Island9999Freshkills Parkfalse622236746168041471AQMAAAABAAAAKQAAALlJtLqljVLATwxtDEFHRECyy2pnnY1SwOz08x1FR0RA5nEUeZyNUsB6mDtmSUdEQEx0E7ihjVLA5SbidFFHREDc8I68oY1SwC+Lt25RR0RAZ42B8qGNUsBV3b8dUUdEQBLr4S6ijVLAvRVN0VBHRED9au0= (truncated)
Staten Island9999Freshkills Parkfalse622236745729441791AQMAAAABAAAARQAAAMbjowCJjVLAr6r2jrBHREBxmc3PhY1SwJ8fwaCxR0RAtk50R4GNUsC4HF55xkdEQILDmUyGjVLADwMzL85HREBCxyVMho1SwFOq+gvOR0RAB4btRoaNUsBkrEybzUdEQEuJ9T2GjVLAxewqK81HREBtW0M= (truncated)
Staten Island9999Freshkills Parkfalse622236745725771775AQMAAAABAAAACgAAAFJBCB4qjVLAnW47ZlxHREBJIIhBG41SwKhdBWFhR0RAFkXhuBaNUsCFCVQ5dkdEQGpvzgwhjVLALRXnFoZHREDEyGTpL41SwD5XExyBR0RA9FxrfTGNUsBTp1naeUdEQGAQxh8qjVLA7wfWc31HREDVZAg= (truncated)
Staten Island9999Freshkills Parkfalse622236745921495039AQMAAAABAAAADgAAAFlfhFdzjFLA98u/xeRHREBmDkG8a4xSwLzO+y/yR0RAwYsfP2iMUsD/Vnzg9kdEQG5Dez5ojFLAohZT4fZHREAf+9Y9aIxSwOI2EuL2R0RAqA5AnWSMUsC/VRtS+0dEQBSyNJxkjFLAIAxmU/tHREAbjiw= (truncated)
Staten Island9999Freshkills Parkfalse622236745692741631AQMAAAABAAAABQAAAAHypEjeilLAPfK0NdpHREDE33S73YpSwKCHSLX6R0RAHFNGruCKUsCZ1964+UdEQEKn9DjlilLAkjkO4eRHREAB8qRI3opSwD3ytDXaR0RA
Staten Island9999Freshkills Parkfalse622236745681731583AQMAAAABAAAABgAAAFXVc7nji1LAGgVez3xLREAv+yDa1ItSwLy4RMmBS0RAcUORT9CLUsD7okijlktEQKNTumjZi1LA7hZenqRLREBSUzwm6ItSwHmMAJyDS0RAVdVzueOLUsAaBV7PfEtEQA==
Staten Island9999Freshkills Parkfalse622236745673834495AQMAAAABAAAABAAAAOKa9+v6i1LAfkZ+mlFLREBOQabR94tSwMz8LtpfS0RA7geSw/yLUsAOyQGVU0tEQOKa9+v6i1LAfkZ+mlFLREA=
Staten Island9999Freshkills Parkfalse622236745673277439AQMAAAABAAAACgAAACvMfWZSjFLAqBazHb1KRECTx7aHQ4xSwGY8+BfCSkRAu76s/T6MUsAVk77x1kpEQPSifVJJjFLAABdO0eZKREC00c5LToxSwJBIEyflSkRAPBFL/U6MUsCHseZv3kpEQERbKEZRjFLAIPviJ+RKREBn51o= (truncated)
Staten Island9999Freshkills Parkfalse622236745672720383AQMAAAABAAAADAAAAMb1C2ESjFLAwV4t4OVKRECjZyCCA4xSwL2bNNrqSkRAoZXW9/6LUsBUIPqz/0pEQPs5jEwJjFLAu7vGkw9LREBzbpq/DoxSwJzt5sANS0RAx10I9RGMUsDA5h+HBUtEQOl6uNAUjFLAw0fzPAJLREARTko= (truncated)
Staten Island9999Freshkills Parkfalse622236745758015487AQMAAAABAAAABAAAACGkevmMilLAdpNQ/7JLREB+k+jbiopSwE9M1CO2S0RAW+DbZoyKUsA0AtaftUtEQCGkevmMilLAdpNQ/7JLREA=
Staten Island9999Freshkills Parkfalse622236745736552447AQMAAAABAAAAEwAAABCG53ESjVLA7uNx0EJKREBIKI6TA41SwD21cMtHSkRAz+HDFwONUsAsgW4ESkpEQNhqxDYDjVLAFwaoE0pKRECpDesZBI1SwP/FAJ1KSkRANEitAgWNUsA42e4tS0pEQIxZd6UFjVLAmzLynUtKREB/wbU= (truncated)
Staten Island9999Freshkills Parkfalse622236745743335423AQMAAAABAAAABAAAABTMGT1Vi1LAduFtzeFMRED8IWWsUItSwNkwQFTjTERA10BbHU+LUsAOw4d76kxEQBTMGT1Vi1LAduFtzeFMREA=
Staten Island9999Freshkills Parkfalse622236745742778367AQMAAAABAAAABwAAAMjQ938fi1LAVs71GgdNREDnlYqfEItSwHeVOhQMTURAGkUDFAyLUsApzsfuIE1EQFUP5A0Si1LARKI1HipNRECq0/X0KItSwD2kBAAbTURAmwD01CmLUsC0djT8Fk1EQMjQ938fi1LAVs71GgdNREA=
Staten Island9999Freshkills Parkfalse622236745682059263AQMAAAABAAAACAAAAEoLgQV1i1LAgfkcgTxMREB+WKIlZotSwFEopXpBTERAh+WMmmGLUsDtleZUVkxEQJQOau9ri1LABCuuNWZMREAXVfyqeotSwIh9R0hhTERAFEPU6XqLUsBTWqrCYExEQOmOYFp/i1LAJG7MYUxMREBKC4E= (truncated)
Staten Island9999Freshkills Parkfalse622236745692512255AQMAAAABAAAABwAAAALwKmvcilLAqbNYemJIREB2vJ853IpSwNmdr/50SERAWHcvAdyKUsBCN9QYgkhEQPOY8f7bilLAza/rj4JIREBeNuCd3opSwCw1kq+BSERAdJmqKOOKUsAqlHvXbEhEQALwKmvcilLAqbNYemJIREA=
Staten Island9999Freshkills Parkfalse622236745724428287AQMAAAABAAAABgAAAMZSGE5PjVLAaJpCITNJREB82CZnTY1SwMijbMQzSURAvNxR3kiNUsCgK7+dSElEQAxHbZdMjVLAZRebVU5JRECrqaUtTY1SwHvM1khLSURAxlIYTk+NUsBomkIhM0lEQA==
Staten Island9999Freshkills Parktrue622236745742221311null
Staten Island9999Freshkills Parkfalse622236745699295231AQMAAAABAAAACQAAAAGEYg3milLAh7X6E9tJREAZaS/H3IpSwBcHzi3eSURAuf/rg9yKUsBGXMgC6klEQPc6uO3bilLAfz1k//lJREC67Bp+24pSwLhX24ECSkRAzE8999yKUsBcG7DFBEpEQJjNyNXrilLAKAnxzP9JRECWddE= (truncated)
Staten Island9999Freshkills Parkfalse622236745680945151AQMAAAABAAAABQAAAFz96SLAi1LAv6fuC9ZLRED31iHQs4tSwLjAWSvaS0RAEzFkRa+LUsCXOoAF70tEQPqSMwqxi1LAc9VUvfFLREBc/ekiwItSwL+n7gvWS0RA
Staten Island9999Freshkills Parkfalse622236745691398143AQMAAAABAAAACAAAADbyXZAbi1LAUHp6WvlHREDU1qaYF4tSwDAqN676R0RAwUQgDhOLUsCGcySGD0hEQHg05WAdi1LAAJhaZR9IREAcETM+LItSwNxVi2waSERAqih46S+LUsBl9gOVCUhEQIlyqGAmi1LAkGwjOwJIREA28l0= (truncated)
Staten Island9999Freshkills Parkfalse622236745723314175AQMAAAABAAAAKwAAADKN8ttfjVLAUJjk78xIREBPs3k1VY1SwNZZh4HQSERA/6u+rFCNUsCn7aha5UhEQKQUpW5ZjVLAaqP9zfJIREAdsel8WY1SwFHP2pzySERAloK+AlqNUsC/MdhB8UhEQOsjZ41ajVLArDxZTPBIREBNQOs= (truncated)
Staten Island9999Freshkills Parkfalse622236745698181119AQMAAAABAAAABwAAAEwaBObtilLAFHgZ0XdJREDPsSLQ34pSwH2V0IZ8SURAbTw6+N6KUsADTpaGmElEQL3zEtDkilLArg5tgqFJREAPXl6u84pSwHArsImcSURAhv9MOfiKUsDgNP+wh0lEQEwaBObtilLAFHgZ0XdJREA=
Staten Island9999Freshkills Parkfalse622236745762013183AQMAAAABAAAACAAAAKlDKdKfilLAYkKWzZtLRECgAEmmm4pSwB7xfzKdS0RAIaR6+YyKUsB2k1D/sktEQFvg22aMilLANALWn7VLREDT9tu6lopSwGiz7YDFS0RAUASPmqWKUsCzuEiIwEtEQGu3KyaqilLA9NOVrqtLRECpQyk= (truncated)
Staten Island9999Freshkills Parkfalse622236745679831039AQMAAAABAAAABQAAAOnPVTaYjFLAmiVQE7lKREBaF51XiYxSwE3/3A2+SkRAf3rCJ4aMUsDRNUqyzEpEQKAIEl2YjFLApnjRTrlKREDpz1U2mIxSwJolUBO5SkRA
Staten Island9999Freshkills Parkfalse622236745729540095AQMAAAABAAAAKQAAALWoYbSCjVLAP1uNpuxHREBfzKk2eI1SwEr4syrwR0RAnZQ4rnONUsCW1mwDBUhEQK/0pQJ+jVLA9zj74BRIREAy0MTTho1SwM+FeewRSERAqDP8z4aNUsCKDkn/D0hEQNO6OsaGjVLA+gFRSQ9IRECKmx8= (truncated)
Staten Island9999Freshkills Parkfalse622236745736323071AQMAAAABAAAAFwAAAIGoyRJGjVLAhghTNnBJREBs9I/MP41SwEsYgFBySURAtAejQzuNUsDufO4ph0lEQFwy2K1AjVLAX3L/eo9JREDjJ6yvQI1SwEDIwS2PSURAKCQp/ECNUsCZuHejjElEQDpT429BjVLA+8AvVItJRECuUCM= (truncated)
Staten Island9999Freshkills Parkfalse622236745921593343AQMAAAABAAAABgAAAE9c1DKEjFLAFRksC8dHREBZX4RXc4xSwPfLv8XkR0RASZJXfnSMUsBAZMeK5kdEQGDnR1uDjFLAFsGbkOFHRECYG4Tkh4xSwO3OR7jMR0RAT1zUMoSMUsAVGSwLx0dEQA==
Staten Island9999Freshkills Parkfalse622236746167582719AQMAAAABAAAABgAAAE02J9mMjVLAjH4gNU1HREDTiTlhcY1SwPV8haFaR0RAFsj/N3qNUsCVSRI1aEdEQHg6dBSJjVLAMJzlOWNHRECjlbOcjY1SwBiZeWFOR0RATTYn2YyNUsCMfiA1TUdEQA==
Staten Island9999Freshkills Parkfalse622236745725313023AQMAAAABAAAACAAAAKmCacK3jFLAJQi4Ym9HREDEmav/s4xSwCv2LqVwR0RAOggkCKeMUsAoi2mciUdEQB+VN+mljFLAHNlvlotHREBgsmCwroxSwKzyKBOZR0RAHUoWjb2MUsDCSskYlEdEQFPAEBbCjFLAbK2JQH9HRECpgmk= (truncated)
Staten Island9999Freshkills Parkfalse622236745921036287AQMAAAABAAAADAAAACj9D6ZEjFLASZNCgQBIRECd6uHiQIxSwKd1+6IASERAsP9vKT2MUsBo2DU7AUhEQN7N6oI5jFLA86gvRgJIREALTGHqNoxSwC+sruUDSERA6dS0bzSMUsCkpNQMBkhEQNFvTSQyjFLASmvOsghIRECqQZk= (truncated)
Staten Island9999Freshkills Parkfalse622236745681829887AQMAAAABAAAABAAAADqjtwbNi1LAQF7MZr5LRECU2Fk6x4tSwFFJbFfAS0RA89LTEsSLUsBCCgbTzktEQDqjtwbNi1LAQF7MZr5LREA=
Staten Island9999Freshkills Parktrue622236745673932799null
Staten Island9999Freshkills Parkfalse622236745698508799AQMAAAABAAAACAAAAOwKhlXiilLAAzlNYC5JRECY1qId4IpSwMenLx4vSURAojQ9VuCKUsCemTlNNElEQCGXDLvgilLAyTTvOEVJREBRTn3L4IpSwJM6VYtVSURAsv67HeiKUsAhJ6MYU0lEQDSJpqjsilLAw1ocQD5JREDsCoY= (truncated)
Staten Island9999Freshkills Parkfalse622236746168467455AQMAAAABAAAABQAAAF3ydb1fjVLAay1OQGNHREAmxFgtSI1SwOMmG8RuR0RAuIiAok2NUsAnYz8md0dEQBjvBX9cjVLAIy0/K3JHREBd8nW9X41SwGstTkBjR0RA
Staten Island9999Freshkills Parkfalse622236745672818687AQMAAAABAAAACAAAAPvMF8oljFLAZv5VDMxKREA31z/rFoxSwIGabgbRSkRAxvULYRKMUsDBXi3g5UpEQFLyw7UcjFLAadrhv/VKREArQQmiJ4xSwFXvABjySkRAsHDnACyMUsCsmrvU7kpEQHAh0h4wjFLAWVry69tKRED7zBc= (truncated)
Staten Island9999Freshkills Parkfalse622236745758113791AQMAAAABAAAADQAAAP+Cb7V5ilLAZdUVkNBLREDS+7j2eIpSwIjMzMfTS0RAcQsE0XiKUsDcdJ9q1EtEQJE1R9x3ilLA4cEU4NlLRED8yekvd4pSwJt2BnPfS0RAbF7jzXaKUsAmLEsW5UtEQF9QvLZ2ilLA0RtOvOpLREBJd6Y= (truncated)
Staten Island9999Freshkills Parkfalse622236745679601663AQMAAAABAAAAIQAAACzMfz75jFLACcGj7DdKRECfEilg6oxSwC5oh+c8SkRA6TTabOmMUsC/y8lFQUpEQEvVcqTpjFLA1/1jR0FKRED+DH1K6oxSwE5D04ZBSkRArUzC/OqMUsAfZjgpQkpEQAgR94vrjFLApwFgu0JKRECaQlU= (truncated)
Staten Island9999Freshkills Parkfalse622236745696837631AQMAAAABAAAACAAAAHpj38Rzi1LAr3/MeDxIREBR46zxaItSwLz28BdASERAfuxfZ2SLUsBWVR7wVEhEQKGzfLpui1LAClk5z2RIREBoy+iXfYtSwIHgDtZfSERArN8hIoKLUsCHLdP9SkhEQIvvORt9i1LAUjWXQ0NIREB6Y98= (truncated)
Staten Island9999Freshkills Parkfalse622236745742876671AQMAAAABAAAACAAAALVpaxXtilLAe1PASvFMREDZhwM13opSwAy7zkP2TERA8GNUqdmKUsCpQz8eC01EQF+2ifzjilLAYJw9/RpNREClgCsC5IpSwAfxVf4aTURAATOf3vKKUsDajJcGFk1EQH9tOmr3ilLADqwYLAFNREC1aWs= (truncated)
Staten Island9999Freshkills Parkfalse622236745735536639AQMAAAABAAAACQAAAFVerKgtjVLA+GJxtsVJREAQrqbKHo1SwBhKf7HKSURAPuKLQRqNUsB0UBCL30lEQEqvipYkjVLAmr+hae9JREBpnKZ0M41SwOsIim7qSURAxGj8tjeNUsBl9+jZ1klEQBupNLw3jVLAlaqSz9VJREBS+/w= (truncated)
Staten Island9999Freshkills Parkfalse622236745763782655AQMAAAABAAAABQAAAGNgliHLilLAj5WO2+NKREDBzi5tyIpSwG9it+oGS0RAu1bxgc6KUsCuVUziBEtEQKIMRg3TilLAKNzsCPBKREBjYJYhy4pSwI+VjtvjSkRA
Staten Island9999Freshkills Parkfalse622236745742319615AQMAAAABAAAACAAAAI8a3MJZi1LAZMc/n7lMREB5rKniSotSwMfAuJi+TERARTtkV0aLUsAXyjFz00xEQPwhZaxQi1LA2TBAVONMREAUzBk9VYtSwHbhbc3hTERAhFTWEWOLUsBEdtUyzkxEQE1c3xdki1LAZg02gMlMRECPGtw= (truncated)
Staten Island9999Freshkills Parkfalse622236745699393535AQMAAAABAAAABgAAABZXx+bZilLAXJ/C5BtKREDY2MHI2IpSwGK2GsgpSkRAXnUAJteKUsAGwsezQEpEQCTefjTeilLAaUjOVz5KRED0gZ+/4opSwAiY0H4pSkRAFlfH5tmKUsBcn8LkG0pEQA==
Staten Island9999Freshkills Parkfalse622236745695723519AQMAAAABAAAACwAAAMdPtWDDi1LAoZytz1tIREDlLc9/wYtSwALEm3BcSERA8V9t4LyLUsCXZJYwX0hEQLhP40u5i1LA8J8uq2BIREAdbySWtYtSwIHEuqdhSERAkOASSrSLUsC60xrQYUhEQGgqRvmvi1LAXRV4oXVIREBplqY= (truncated)
Staten Island9999Freshkills Parkfalse622236745770565631AQMAAAABAAAABgAAAL+M8Qu7ilLABVI6N8BMRED3i+BMuopSwKqQ99/HTERAau9xW7iKUsDrU7c/3ExEQAYORKu6ilLA97vqedtMREBuDgU3v4pSwIGdnZ/GTERAv4zxC7uKUsAFUjo3wExEQA==
Staten Island9999Freshkills Parkfalse622236745692053503AQMAAAABAAAACAAAALI/Y7tIi1LAUxWdKxZIRECMFF9uRYtSwABFylIlSERABR1RwU+LUsAry+YxNUhEQJlOpJ5ei1LAYUDhODBIREBFjZ+WX4tSwIAsZcYrSERATCsLel2LUsDt3mQjKUhEQF2Yfk9Ni1LAbw26ShhIRECyP2M= (truncated)
Staten Island9999Freshkills Parktrue622236745691496447null
Staten Island9999Freshkills Parkfalse622236745762668543AQMAAAABAAAADQAAAMnMvqDSilLAgrRmC4BKREBM92Bw0opSwNJWGbGCSkRApebsSdKKUsAXBA/MhEpEQABiZi3SilLAPll8WYZKRECduvzL0YpSwMTZ76eLSkRA5LxJsNGKUsAQ+JMpjUpEQAzII43RilLAL38yF49KREAmVOM= (truncated)
Staten Island9999Freshkills Parkfalse622236745698279423AQMAAAABAAAACAAAAAdjDkXgilLAksMkW7ZJREBJ23gM3opSwJNMQBm3SURAKLotRd2KUsBYv2fx0ElEQHze3N7cilLA+ZyQA9pJREAZaS/H3IpSwBcHzi3eSURAAYRiDeaKUsCHtfoT20lEQGEgaZjqilLA2ustO8ZJREAHYw4= (truncated)
Staten Island9999Freshkills Parkfalse622236745769451519AQMAAAABAAAABQAAAOaTaHXEilLAzKLeUl9MRECiHMIJwopSwI4IEFl4TERAnkmchcKKUsDQhaoveExEQNM7QxHHilLAvESOVWNMREDmk2h1xIpSwMyi3lJfTERA
Staten Island9999Freshkills Parkfalse622236745736421375AQMAAAABAAAAKgAAAPYNYtkljVLAoMGf+yhKRED1SRz7Fo1SwK7tr/YtSkRAEIbncRKNUsDu43HQQkpEQMJl/UgVjVLAWxMyLUdKREBindMnF41SwDO4waRGSkRAAF87kheNUsBzHJ6iRkpEQKRTDvsXjVLAkmamhkZKREBYXJ4= (truncated)
Staten Island9999Freshkills Parktrue622236745921691647null
Staten Island9999Freshkills Parkfalse622236745681928191AQMAAAABAAAACQAAAE+kS3CIi1LAkbVIrSJMRECXioCQeYtSwDdI4qYnTERASguBBXWLUsCB+RyBPExEQOmOYFp/i1LAJG7MYUxMREAgk2Z1hYtSwMLhAVdKTERAEo+1M4qLUsBorzhBQExEQBOkhXCRi1LA14/F3TBMREAqAHo= (truncated)
Staten Island9999Freshkills Parktrue622236745696051199null
Staten Island9999Freshkills Parkfalse622236745692381183AQMAAAABAAAACAAAADRnJz7silLALShqJ0NIREBDE7Vg3YpSwMOM+x9ISERAKxvcqNyKUsClj6hrS0hEQALwKmvcilLAqLNYemJIREB0maoo44pSwCqUe9dsSERAeyozBvKKUsBHaODeZ0hEQJGg55D2ilLAR4fQBlNIREA0Zyc= (truncated)
Staten Island9999Freshkills Parkfalse622236745763553279AQMAAAABAAAACQAAAIkt8fzjilLAYDokEWNKREAlmiUe1YpSwJ565QloSkRA9Y6J9tOKUsAS1TZWbUpEQOeGhijTilLAx5TjnHhKREDJzL6g0opSwIK0ZguASkRAydGT5tqKUsDclGXDjEpEQFSmdcXpilLA34yayodKRECnYZo= (truncated)
Staten Island9999Freshkills Parkfalse622236746169122815AQMAAAABAAAAHwAAAHg6dBSJjVLAMJzlOWNHREAWyP83eo1SwJVJEjVoR0RATISqr3WNUsAIF4UNfUdEQDCY3QOAjVLAz4TZ6oxHREBvQmjgjo1SwDoHo++HR0RAzcxrl4+NUsAGAuGlhEdEQKUCd7+PjVLA2zRwoINHREALQHY= (truncated)
Staten Island9999Freshkills Parkfalse622236745680257023AQMAAAABAAAACAAAAMv1sAJ/jFLAcpOKLq5KREBk4/ojcIxSwBpC/CizSkRAH7MammuMUsD+KcoCyEpEQP19BO91jFLAoLU04tdKREBbrDJFfoxSwI5/sxfVSkRAf3rCJ4aMUsDRNUqyzEpEQFoXnVeJjFLATf/cDb5KREDL9bA= (truncated)
Staten Island9999Freshkills Parkfalse622236745672916991AQMAAAABAAAACAAAAHNumr8OjFLAnO3mwA1LRED7OYxMCYxSwLu7xpMPS0RAaicA9gaMUsAFuptQGktEQDmF8sQHjFLAJ/EeZRlLRECgGY+oCYxSwPpCVQAXS0RATQaFdguMUsAxYeF1FEtEQNcrki0NjFLAOGqIxxFLREBzbpo= (truncated)
Staten Island9999Freshkills Parkfalse622236745690710015AQMAAAABAAAABQAAACb5OGLcilLAURRXke1IREDfA6Iq3YpSwAHiVIL9SERAjlqPpN2KUsD/cP+mBElEQAWySBjhilLAZwy7z/RIREAm+Thi3IpSwFEUV5HtSERA
Staten Island9999Freshkills Parkfalse622236745761882111AQMAAAABAAAACgAAALIdTT2zilLA7ddL+4FLREDRx5ceqYpSwBqAPl2FS0RAlwJC5qiKUsDU7QnLhUtEQCcu9q2iilLAtVg/M5FLREAdq0/NoYpSwCvYN7eSS0RAqUMp0p+KUsBiQpbNm0tEQGu3KyaqilLA9NOVrqtLRECiLss= (truncated)
Staten Island9999Freshkills Parkfalse622236745750872063AQMAAAABAAAABgAAAAYORKu6ilLA97vqedtMREBq73FbuIpSwOtTtz/cTERAQv5NjLSKUsDKYGcwBE1EQFzYInTAilLAKfO+NQBNREBr5eX/xIpSwATDXFvrTERABg5Eq7qKUsD3u+p520xEQA==
Brooklyn133133Kensingtontrue622236751937470463null
Brooklyn133133Kensingtontrue622236751857385471null
Brooklyn133133Kensingtontrue622236751873212415null
Brooklyn133133Kensingtontrue622236751936913407null
Brooklyn133133Kensingtontrue622236751933079551null
Brooklyn133133Kensingtontrue622236751873376255null
Brooklyn133133Kensingtontrue622236751937077247null
Brooklyn133133Kensingtontrue622236751872098303null
Brooklyn133133Kensingtontrue622236751872819199null
Brooklyn133133Kensingtontrue622236751937241087null
Brooklyn133133Kensingtontrue622236751936684031null
Brooklyn133133Kensingtontrue622236751871705087null
Brooklyn133133Kensingtontrue622236751938125823null
Brooklyn133133Kensingtontrue622236751873146879null
Brooklyn133133Kensingtontrue622236751937568767null
Brooklyn133133Kensingtontrue622236751872589823null
Brooklyn133133Kensingtontrue622236751933014015null
Brooklyn133133Kensingtontrue622236751873310719null
Brooklyn133133Kensingtontrue622236751937011711null
Brooklyn133133Kensingtontrue622236751872032767null
Brooklyn133133Kensingtontrue622236751872753663null
Brooklyn133133Kensingtontrue622236751933177855null
Brooklyn133133Kensingtontrue622236751937175551null
Brooklyn133133Kensingtontrue622236751933341695null
Brooklyn133133Kensingtontrue622236751871639551null
Brooklyn133133Kensingtontrue622236751937339391null
Brooklyn133133Kensingtontrue622236751938060287null
Brooklyn133133Kensingtontrue622236751873081343null
Brooklyn133133Kensingtontrue622236751871803391null
Brooklyn133133Kensingtontrue622236751873245183null
Brooklyn133133Kensingtontrue622236751936946175null
Brooklyn133133Kensingtontrue622236751871967231null
Brooklyn133133Kensingtontrue622236751872688127null
Brooklyn133133Kensingtontrue622236751873409023null
Brooklyn133133Kensingtontrue622236751937110015null
Brooklyn133133Kensingtontrue622236751872851967null
Brooklyn133133Kensingtontrue622236751937273855null
Brooklyn133133Kensingtontrue622236751872294911null
Brooklyn133133Kensingtontrue622236751937994751null
Brooklyn133133Kensingtontrue622236751873015807null
Brooklyn133133Kensingtontrue622236751936716799null
Brooklyn133133Kensingtontrue622236751937437695null
Brooklyn133133Kensingtontrue622236751871901695null
Brooklyn133133Kensingtontrue622236751937601535null
Brooklyn133133Kensingtontrue622236751933046783null
Brooklyn133133Kensingtontrue622236751938322431null
Brooklyn133133Kensingtontrue622236751873343487null
Brooklyn133133Kensingtontrue622236751937044479null
Brooklyn133133Kensingtontrue622236751872786431null
Brooklyn133133Kensingtontrue622236751937208319null
Brooklyn133133Kensingtontrue622236751871672319null
Brooklyn133133Kensingtontrue622236751937372159null
Brooklyn133133Kensingtontrue622236751936815103null
Brooklyn133133Kensingtontrue622236751871836159null
Brooklyn133133Kensingtontrue622236751872557055null
Brooklyn133133Kensingtontrue622236751932981247null
Brooklyn133133Kensingtontrue622236751938256895null
Brooklyn133133Kensingtontrue622236751873277951null
Brooklyn133133Kensingtontrue622236751936978943null
Brooklyn133133Kensingtontrue622236751872720895null
Brooklyn133133Kensingtontrue622236751872884735null
Brooklyn133133Kensingtontrue622236751933308927null
Brooklyn133133Kensingtontrue622236751937306623null
Brooklyn133133Kensingtontrue622236751857221631null
Brooklyn133133Kensingtontrue622236751873048575null
Brooklyn133133Kensingtontrue622236751936749567null
Brooklyn133133Kensingtontrue622236751871770623null
Brooklyn133133Kensingtonfalse622236751617425407AQMAAAABAAAABwAAAFuk1FT4fVLAaMHh5nxRRECVVYdY9H1SwD3Fi8d9UURADxBsX/d9UsBOjuzxklFEQAULzkP5fVLAuN//AaBRRECTVndQ/H1SwNAtCQCfUURAb3JF6AB+UsBq1q8nilFEQFuk1FT4fVLAaMHh5nxRREA=
Brooklyn133133Kensingtonfalse622236751877046271AQMAAAABAAAABwAAAEGj2xvoflLAoddU7BdSRECDgRJf4H5SwCrlDPAdUkRAYSTMy9l+UsBwJFt5I1JEQO2jvNjTflLA02SqbChSREDWWrny1X5SwGVRuqsrUkRAHANr2OR+UsBt5Ja+JlJEQEGj2xvoflLAoddU7BdSREA=
Brooklyn133133Kensingtonfalse622236751872458751AQMAAAABAAAABwAAANEgXKx6flLAQcX68e1QREAZnLrUdX5SwEXm0YvvUERAs7lwPXF+UsCcDwRkBFFEQCyjM4t7flLAtKykTxRRREA4YGM/fn5SwLNpy2oTUURAhY2b4n1+UsA3GTSeD1FEQNEgXKx6flLAQcX68e1QREA=
Brooklyn133133Kensingtontrue622236751871999999null
Brooklyn133133Kensingtonfalse622236751933276159AQMAAAABAAAABgAAAJNWd1D8fVLA0C0JAJ9RREAFC85D+X1SwLjf/wGgUURAky60m/59UsDJ767lxFFEQA27WgYCflLAhN+oxMNRREDdCiueBn5SwAyAOuyuUURAk1Z3UPx9UsDQLQkAn1FEQA==
Brooklyn133133Kensingtonfalse622236751876882431AQMAAAABAAAACAAAAAyzeGDpflLAYhwA8BZSREBBo9sb6H5SwKHXVOwXUkRAHANr2OR+UsBt5Ja+JlJEQFBI/ybvflLAlUNhqjZSREC0zLMM/n5SwFigIr0xUkRALIZ1i/5+UsAfvWh9L1JEQLRsbafuflLAqiI0+RtSREAMs3g= (truncated)
Brooklyn133133Kensingtonfalse622236751938158591AQMAAAABAAAABgAAAPOiYf1QflLAByZ+nZFSRECuJFQXQn5SwDEvE4qWUkRA3uhx7D5+UsBjMLvqpFJEQNvH5kpHflLAoYs4I55SREDgVFj3Un5SwHEgOquUUkRA86Jh/VB+UsAHJn6dkVJEQA==
Brooklyn133133Kensingtonfalse622236751875506175AQMAAAABAAAACAAAAH/eC4drflLAF7Q8oN9QREB2wvSuYH5SwIGh+jXjUERAkiKR8Vt+UsDtGiWu51BEQAgOuApYflLAfDblZPlQREBvQ2RYYn5SwKmGklAJUURAs7lwPXF+UsCcDwRkBFFEQBmcutR1flLARebRi+9QREB/3gs= (truncated)
Brooklyn133133Kensingtonfalse622236751936782335AQMAAAABAAAACQAAACzUaK99flLAJ2tl14JSREDP+mvJbn5SwO5DJ8SHUkRAbjHMMWp+UsCSTSqdnFJEQBH63i9uflLAX9E8yKJSREAE7b91dX5SwNFT2jGhUkRAA4jWq4R+UsBX1BnYnVJEQML7W5iFflLAOCL0pJ1SREB7Rdw= (truncated)
Brooklyn133133Kensingtonfalse622236751857319935AQMAAAABAAAABgAAAIIjIicff1LA7++ImFdSREDtoN/4HH9SwI1HWoBhUkRAIoC1Ryd/UsDInhZscVJEQHrUUoUxf1LA1/bvCG5SREBqbveYJn9SwM4+8qFgUkRAgiMiJx9/UsDv74iYV1JEQA==
Brooklyn133133Kensingtonfalse622236751933112319AQMAAAABAAAACAAAANq8cCgTflLAyqZKEzJSRECf3uGPDn5SwDhJFpgzUkRAdwxdVg9+UsBaq8O4OFJEQHrwDZcSflLA1fkX0U5SREBdN7n3E35SwB2yGndYUkRAUgWc3hh+UsBXI2zYVlJEQIwkdXYdflLA4aSp/0FSREDavHA= (truncated)
Brooklyn133133Kensingtonfalse622236751937535999AQMAAAABAAAABwAAAGXD31O5flLAxhigJftRREB2j2sHt35SwFTBM+j7UURAJSUmcLJ+UsC2pQfBEFJEQFD/jL68flLANHLrrCBSREBwl37cvH5SwMO9BKMgUkRATYF/ibl+UsAGpXM0/VFEQGXD31O5flLAxhigJftRREA=
Brooklyn133133Kensingtontrue622236751877013503null
Brooklyn133133Kensingtonfalse622236751938289663AQMAAAABAAAABwAAAH87dgTPflLA3aFLmshSREC9aFoewH5SwG2/aIfNUkRAAUFvPb1+UsATCp2Z2lJEQHJVohfOflLAbw1c2dZSREDBgg/m1X5SwLyEbhXVUkRAiBumsNZ+UsAgQ1l01FJEQH87dgTPflLA3aFLmshSREA=
Brooklyn133133Kensingtonfalse622236751858368511AQMAAAABAAAABgAAAGCGxjMPf1LAAyT6C6BSREDi4M5NAH9SwB2oVfmkUkRAcztnW/x+UsBh+V3mtlJEQMbBKOUOf1LA6EEXhqhSREBc+TC8En9SwBuE4IClUkRAYIbGMw9/UsADJPoLoFJEQA==
Brooklyn133133Kensingtonfalse622236751872425983AQMAAAABAAAACAAAADhgYz9+flLAs2nLahNRREAsozOLe35SwLSspE8UUURAT47n83Z+UsDh3usnKVFEQO+jvkGBflLAneaXEzlRREBzq4fSgX5SwP17ueM4UURAxlGhNoF+UsDFpImBMlFEQN9+Qs5/flLAsfwLwCNRREA4YGM= (truncated)
Brooklyn133133Kensingtonfalse622236751933243391AQMAAAABAAAACQAAAA27WgYCflLAhN+oxMNRRECTLrSb/n1SwMnvruXEUURAHMxUi/99UsCnHwVcy1FEQCnT7qsAflLAFwFvI9NRREAct22yAn5SwGnKB5DjUURACQfUowN+UsC3++Dj6VFEQAoYULwHflLAgARpiehRREDDmyI= (truncated)
Brooklyn133133Kensingtonfalse622236751877308415AQMAAAABAAAABwAAANA9HiyyflLAKsS9MrBRREDto16Zq35SwKGyJV+yUURAhJ0dAqd+UsC+hs83x1FEQMcbXFCxflLAWIGcI9dRRECMc4qvtX5SwBZSjLHVUURADJleB7N+UsDJ0u02uVFEQNA9HiyyflLAKsS9MrBRREA=
Brooklyn133133Kensingtonfalse622236751857287167AQMAAAABAAAACAAAAHrUUoUxf1LA1/bvCG5SREAigLVHJ39SwMieFmxxUkRAEBm7sCJ/UsCFj05FhlJEQM4FhrQqf1LAw2pIppJSREAOYLMgP39SwI/dAZaCUkRAm3Zi0j9/UsDikdVuf1JEQLpltTAzf1LA50pRFXBSREB61FI= (truncated)
Brooklyn133133Kensingtonfalse622236751931244543AQMAAAABAAAABQAAADYY5W4yflLA+hE0C69SREBFdWmZLn5SwM6tkU+wUkRAITioFS5+UsA1gJKlslJEQI2xqw0xflLAhlAUKrBSREA2GOVuMn5SwPoRNAuvUkRA
Brooklyn133133Kensingtontrue622236751937961983null
Brooklyn133133Kensingtonfalse622236751937503231AQMAAAABAAAACgAAAHCXfty8flLAw70EoyBSREBQ/4y+vH5SwDRy66wgUkRAG2NFJ7h+UsCDXtSFNVJEQKdrwHXCflLAmJPDcUVSRECsl4Vb0X5SwEi9sYRAUkRA1lq58tV+UsBlUbqrK1JEQO2jvNjTflLA02SqbChSRED7TwA= (truncated)
Brooklyn133133Kensingtonfalse622236751875309567AQMAAAABAAAABQAAAI/Jx15tflLAtVNtQtdQRED0ZJE1aX5SwF5T4ivbUERAdsL0rmB+UsCBofo141BEQH/eC4drflLAF7Q8oN9QRECPycdebX5SwLVTbULXUERA
Brooklyn133133Kensingtonfalse622236751857876991AQMAAAABAAAABAAAAJt2YtI/f1LA4pHVbn9SREANYLMgP39SwJDdAZaCUkRAUfHMH0F/UsCnG/8DgVJEQJt2YtI/f1LA4pHVbn9SREA=
Brooklyn133133Kensingtonfalse622236751857418239AQMAAAABAAAABgAAABAZu7Aif1LAhY9ORYZSRECM+NbKE39SwFmquzKLUkRAYIbGMw9/UsADJPoLoFJEQFz5MLwSf1LAGoTggKVSREDOBYa0Kn9SwMNqSKaSUkRAEBm7sCJ/UsCFj05FhlJEQA==
Brooklyn133133Kensingtonfalse622236751934128127AQMAAAABAAAABwAAAFIFnN4YflLAVyNs2FZSREBdN7n3E35SwB2yGndYUkRADq4U8BV+UsAv3oBDZlJEQC/lyEIZflLAn0XIX31SRECdR9mUHn5SwKwRrp17UkRAlZq0LCN+UsDHi9bEZlJEQFIFnN4YflLAVyNs2FZSREA=
Brooklyn133133Kensingtonfalse622236751871934463AQMAAAABAAAACwAAADNjLjFHflLA9ZPyS/tQRED5HWuIQn5SwLU1S73/UERAk673Ezx+UsCJ8yHiBVFEQGxUb1k6flLA491zgQdRREBoSLPkN35SwMl0Nc0JUURAYLL4rTN+UsBQSQivDVFEQPPOX70wflLA7G05exBRREBxmx0= (truncated)
Brooklyn133133Kensingtonfalse622236751877275647AQMAAAABAAAACQAAAItziq+1flLAFlKMsdVRREDHG1xQsX5SwFiBnCPXUURAZOMYuax+UsCpXVv861FEQHaPawe3flLAVMEz6PtRREBlw99TuX5SwMYYoCX7UURA45/9brh+UsArw/hc8lFEQPNdute2flLASqcZPuFRREBTxsU= (truncated)
Brooklyn133133Kensingtonfalse622236751938093055AQMAAAABAAAACQAAAC+qEMk3flLA3xGqnYZSREC3CgbjKH5SwDjkI4qLUkRACYQoSyR+UsBmcRBjoFJEQEV1aZkuflLAzq2RT7BSREA2GOVuMn5SwPoRNAuvUkRAVSQkpzZ+UsDuy92dq1JEQN7ocew+flLAYzC76qRSRECuJFQ= (truncated)
Brooklyn133133Kensingtontrue622236751937634303null
Brooklyn133133Kensingtonfalse622236751875440639AQMAAAABAAAABwAAAJIikfFbflLA7BolrudQREDoDGM9W35SwES4C1joUERAz+nU8Eh+UsBaYBih+VBEQDNjLjFHflLA9pPyS/tQREC0da4lSX5SwF56WFH+UERACA64Clh+UsB8NuVk+VBEQJIikfFbflLA7BolrudQREA=
Brooklyn133133Kensingtonfalse622236751857254399AQMAAAABAAAACAAAAOPvGj0Mf1LAIyI1YEBSREBJx15bCH9SwCxI4KhBUkRACrdSxAN/UsBhsvSBVlJEQLjgERMOf1LAu8G9bWZSREDtoN/4HH9SwI1HWoBhUkRAgiMiJx9/UsDv74iYV1JEQBohZ+0Yf1LALNL4CVBSREDj7xo= (truncated)
Brooklyn133133Kensingtonfalse622236751938387967AQMAAAABAAAABgAAAMIN7pqcflLAddQKnLJSREDu5sdWln5SwAzhW660UkRAuy2L6pl+UsAEur0o2lJEQLEbElKiflLAqip2YddSREDiZIzppn5SwKAYQYjCUkRAwg3umpx+UsB11AqcslJEQA==
Brooklyn133133Kensingtonfalse622236751872983039AQMAAAABAAAABgAAAGxjw5QdflLAy2ycskZRREAJHTUFFX5SwNam5YZJUURAJly2lxh+UsDDCRsAb1FEQNj2xUojflLA0GDedmtRRECD/mfiJ35SwCUXkp5WUURAbGPDlB1+UsDLbJyyRlFEQA==
Brooklyn133133Kensingtonfalse622236751872065535AQMAAAABAAAABwAAAPPOX70wflLA7G05exBRRECTTLzTIn5SwPmVk7gdUURALiBBVRp+UsAHwLS8JVFEQJY3YywiflLAkytl2jFRREC70JMRMX5SwLATFe4sUURAcZsdqTV+UsATWuQVGFFEQPPOX70wflLA7G05exBRREA=
Brooklyn133133Kensingtonfalse622236751876947967AQMAAAABAAAACAAAACyGdYv+flLAH71ofS9SREC0zLMM/n5SwFigIr0xUkRAScdeWwh/UsAsSOCoQVJEQOPvGj0Mf1LAIyI1YEBSREASkbjMC39SwL1netU/UkRAb6UCegN/UsBMUi+kNVJEQBeAuNH+flLA6ayz0y9SREAshnU= (truncated)
Brooklyn133133Kensingtonfalse622236751938224127AQMAAAABAAAACQAAAK9apc+1flLApTI/m71SREDiZIzppn5SwKAYQYjCUkRAsRsSUqJ+UsCqKnZh11JEQHGZHYanflLAy/Mza99SREByYXbfqX5SwPFANuzeUkRAGdIcF71+UsDgpCSi2lJEQAFBbz29flLAEwqdmdpSREC9aFo= (truncated)
Brooklyn133133Kensingtonfalse622236751936847871AQMAAAABAAAACAAAAJj82+OWflLA4O2/1o1SREB7Rdz9h35SwJL9nMOSUkRAwvtbmIV+UsA4IvSknVJEQPTpldiTflLAIjoIkJpSREDu5sdWln5SwAzhW660UkRAwg3umpx+UsB11AqcslJEQK4kZjKhflLAuMnqwp1SRECY/Ns= (truncated)
Brooklyn133133Kensingtonfalse622236751858302975AQMAAAABAAAACAAAAEMO/v71flLAFRJpDZVSREC9RAkZ535SwNdeqfqZUkRAA+3OgeJ+UsC/4d/TrlJEQOs4ndDsflLAOpfkv75SREAAuqtD9n5SwMiy65+7UkRAcztnW/x+UsBh+V3mtlJEQOLgzk0Af1LAHahV+aRSREBDDv4= (truncated)
Brooklyn133133Kensingtonfalse622236751934095359AQMAAAABAAAACAAAAJ1H2ZQeflLArBGunXtSREAv5chCGX5SwJ9FyF99UkRAHo2Dqxl+UsAb0CA4gFJEQHuwDmsdflLAyU+QvJlSREB0mRmfHn5SwKi26EKiUkRACYQoSyR+UsBmcRBjoFJEQLcKBuMoflLAOOQjiotSRECdR9k= (truncated)
Brooklyn133133Kensingtonfalse622236751872360447AQMAAAABAAAACAAAAAaMDJquflLAcwfuuIpRRECZJ3PipX5SwItVz5qNUURAM1M0S6F+UsBYIWRzolFEQO2jXpmrflLAobIlX7JRREDQPR4ssn5SwCrEvTKwUURATgz9BLF+UsDnFL0PpFFEQID6xbuvflLAgunWWZdRREAGjAw= (truncated)
Brooklyn133133Kensingtonfalse622236751873114111AQMAAAABAAAABgAAACvdMOoSflLAIaZiHHdRRED7Shf9CX5SwHF8jAN5UURAW6TUVPh9UsBoweHmfFFEQG9yRegAflLAatavJ4pRREAlbLPND35SwKlPeTuFUURAK90w6hJ+UsAhpmIcd1FEQA==
Brooklyn133133Kensingtonfalse622236751872196607AQMAAAABAAAABgAAAIhy58SpflLArzZOHFZRREA/9d3nl35SwGcifcpZUURAs3mJnJZ+UsAMBfARWlFEQHammSugflLAemqZ1mhRRECi9nweq35SwN4W1DdlUURAiHLnxKl+UsCvNk4cVlFEQA==
Brooklyn133133Kensingtonfalse622236751871737855AQMAAAABAAAACgAAAKVccKp8flLAgyH0601RREDlKDfFbX5SwJDUldhSUURALtjSLWl+UsDll/iwZ1FEQC+Su3tzflLAYCjInHdRREALJQthgn5SwCTXHLByUURAtp5b+IZ+UsCpk6vXXVFEQAD4Rr6GflLAmcTufV1RRED2F9A= (truncated)
Brooklyn133133Kensingtonfalse622236751931179007AQMAAAABAAAABwAAAAmEKEskflLAZnEQY6BSREB0mRmfHn5SwKm26EKiUkRAK9x9FyF+UsChkx/Ds1JEQIdU3kIiflLAz6NziLxSREAhOKgVLn5SwDaAkqWyUkRARXVpmS5+UsDOrZFPsFJEQAmEKEskflLAZnEQY6BSREA=
Brooklyn133133Kensingtontrue622236751877079039null
Brooklyn133133Kensingtonfalse622236751938355199AQMAAAABAAAABQAAALEbElKiflLAqip2YddSREC7LYvqmX5SwAS6vSjaUkRAsa5ZrZp+UsCNebch4lJEQHGZHYanflLAy/Mza99SRECxGxJSon5SwKoqdmHXUkRA
Brooklyn133133Kensingtontrue622236751937437695null
Brooklyn133133Kensingtonfalse622236751858434047AQMAAAABAAAACQAAAAPtzoHiflLAv+Hf065SREDbnsab035SwCCXDsGzUkRAfzt2BM9+UsDdoUuayFJEQIgbprDWflLAIENZdNRSRECpvTwo3n5SwGeIZYTOUkRAMLeyrOp+UsCOpsWcxFJEQAZ957LrflLA4zNv0cNSREDrOJ0= (truncated)
Brooklyn133133Kensingtonfalse622236751872950271AQMAAAABAAAACQAAANj2xUojflLA0GDedmtRREAlXLaXGH5SwMQJGwBvUURAeRq/PBl+UsDgx1LDdVFEQCvdMOoSflLAIaZiHHdRREAlbLPND35SwKlPeTuFUURAAappGxp+UsCcg5InlVFEQN+C2gApflLAhshAO5BRRED9vX4= (truncated)
Brooklyn133133Kensingtonfalse622236751616376831AQMAAAABAAAABgAAAC4gQVUaflLAB8C0vCVRREBUdiNUEn5SwAI2hkotUURACR01BRV+UsDWpuWGSVFEQGxjw5QdflLAy2ycskZRRECWN2MsIn5SwJMrZdoxUURALiBBVRp+UsAHwLS8JVFEQA==
Brooklyn133133Kensingtonfalse622236751937732607AQMAAAABAAAABQAAAG4xzDFqflLAkk0qnZxSRECWOp/3Yn5SwL1ktwCfUkRAeOOJkmN+UsCItVYZpVJEQBH63i9uflLAX9E8yKJSREBuMcwxan5SwJJNKp2cUkRA
Brooklyn133133Kensingtonfalse622236751858270207AQMAAAABAAAABAAAAAG6q0P2flLAyLLrn7tSREDrOJ3Q7H5SwDqX5L++UkRABn3nsut+UsDjM2/Rw1JEQAG6q0P2flLAyLLrn7tSREA=
Brooklyn133133Kensingtontrue622236751857352703null
Brooklyn133133Kensingtonfalse622236751872327679AQMAAAABAAAACAAAALN5iZyWflLADAXwEVpRRECwEMFxin5SwD/XnbFcUURAtp5b+IZ+UsCpk6vXXVFEQAslC2GCflLAJNccsHJRREDVkwqvjH5SwHKz35uCUURAHARdlJt+UsDULRmvfVFEQHammSugflLAemqZ1mhRRECzeYk= (truncated)
Brooklyn133133Kensingtonfalse622236751933145087AQMAAAABAAAACAAAAN1tV3INflLAYpxJTg1SREBY0l8hCX5SwMZUbLsOUkRAy7o7ogp+UsAo5urRGFJEQI63eQkMflLA6OQA5SJSRECf3uGPDn5SwDhJFpgzUkRA2rxwKBN+UsDKpkoTMlJEQEioR8AXflLA4C+dOh1SREDdbVc= (truncated)
Brooklyn133133Kensingtonfalse622236751938027519AQMAAAABAAAADQAAAGJAD3tkflLAWfvi13dSREAiRRWVVX5SwFmdicR8UkRA86Jh/VB+UsAHJn6dkVJEQOBUWPdSflLAcSA6q5RSREAwJStPWH5SwDUwwVWQUkRAT4sy9Vx+UsAFeOWCjFJEQM3k3YlgflLADNFTPYlSREAgm+s= (truncated)
Brooklyn133133Kensingtonfalse622236751875375103AQMAAAABAAAABgAAAC7muJd3flLA2IIdps1QRECPycdebX5SwLVTbULXUERAf94Lh2t+UsAXtDyg31BEQBmcutR1flLARebRi+9QREDRIFysen5SwEHF+vHtUERALua4l3d+UsDYgh2mzVBEQA==
Brooklyn133133Kensingtontrue622236751936651263null
Brooklyn133133Kensingtonfalse622236751872622591AQYAAAACAAAAAQMAAAABAAAABQAAAHOrh9KBflLA/Xu54zhRREDvo75BgX5SwJ3mlxM5UURApVxwqnx+UsCDIfTrTVFEQHJxMRGFflLArfH55lpRREBzq4fSgX5SwP17ueM4UURAAQMAAAABAAAABAAAALAQwXGKflLAP9edsVw= (truncated)
Brooklyn133133Kensingtonfalse622236751872163839AQMAAAABAAAACAAAAKL2fB6rflLA3hbUN2VRREB2ppkroH5SwHpqmdZoUURAHARdlJt+UsDULRmvfVFEQJknc+KlflLAi1XPmo1RREAGjAyarn5SwHMH7riKUURApgB+RK5+UsBx7Tz+hlFEQKy81p6sflLATfgyBXZRRECi9nw= (truncated)
Brooklyn133133Kensingtonfalse622236751933439999AQMAAAABAAAABgAAAAoYULwHflLAgARpiehRREAJB9SjA35SwLf74OPpUURAWNJfIQl+UsDGVGy7DlJEQN1tV3INflLAYpxJTg1SREB1JSwKEn5SwCctsXX4UURAChhQvAd+UsCABGmJ6FFEQA==
Manhattan236236Upper East Side Northtrue622236722201296895null
Manhattan236236Upper East Side Northtrue622236722202345471null
Manhattan236236Upper East Side Northtrue622236722201591807null
Manhattan236236Upper East Side Northtrue622236722202116095null
Manhattan236236Upper East Side Northtrue622236722202640383null
Manhattan236236Upper East Side Northtrue622236722210209791null
Manhattan236236Upper East Side Northtrue622236722210504703null
Manhattan236236Upper East Side Northtrue622236722201133055null
Manhattan236236Upper East Side Northtrue622236722202181631null
Manhattan236236Upper East Side Northtrue622236722204803071null
Manhattan236236Upper East Side Northtrue622236722201952255null
Manhattan236236Upper East Side Northtrue622236722210045951null
Manhattan236236Upper East Side Northtrue622236722201493503null
Manhattan236236Upper East Side Northtrue622236722202017791null
Manhattan236236Upper East Side Northtrue622236722202542079null
Manhattan236236Upper East Side Northtrue622236722201264127null
Manhattan236236Upper East Side Northtrue622236722202312703null
Manhattan236236Upper East Side Northtrue622236722210406399null
Manhattan236236Upper East Side Northtrue622236722202083327null
Manhattan236236Upper East Side Northtrue622236722210177023null
Manhattan236236Upper East Side Northtrue622236722201329663null
Manhattan236236Upper East Side Northtrue622236722202378239null
Manhattan236236Upper East Side Northtrue622236722204999679null
Manhattan236236Upper East Side Northtrue622236722210242559null
Manhattan236236Upper East Side Northtrue622236722201395199null
Manhattan236236Upper East Side Northtrue622236722201919487null
Manhattan236236Upper East Side Northtrue622236722202443775null
Manhattan236236Upper East Side Northtrue622236722201165823null
Manhattan236236Upper East Side Northtrue622236722202214399null
Manhattan236236Upper East Side Northtrue622236722210308095null
Manhattan236236Upper East Side Northtrue622236722201460735null
Manhattan236236Upper East Side Northtrue622236722202509311null
Manhattan236236Upper East Side Northtrue622236722210078719null
Manhattan236236Upper East Side Northtrue622236722201231359null
Manhattan236236Upper East Side Northtrue622236722202279935null
Manhattan236236Upper East Side Northtrue622236722210373631null
Manhattan236236Upper East Side Northtrue622236722202050559null
Manhattan236236Upper East Side Northtrue622236722209619967null
Manhattan236236Upper East Side Northtrue622236722210144255null
Manhattan236236Upper East Side Northfalse622236722202476543AQMAAAABAAAACQAAAP+f8/YZfVLA8JO5MJhjREBrKmJ/Fn1SwMBMVVWZY0RASm2uPBZ9UsC69joMmmNEQGVAqBEOfVLA+OxhcbBjRED4yyvzCn1SwOJopQC5Y0RANK6ovhB9UsBSi3bzwWNEQDoh0K8ffVLAqGtmBr1jRECEoOs= (truncated)
Manhattan236236Upper East Side Northfalse622236722214010879AQMAAAABAAAABwAAAFlqf9difVLAzzwixkFkREBAHOl2VH1SwNN+woNGZERA36nY2k99UsC1pIplW2REQCWD/h5WfVLAU/pDEmVkRECsbjoEWX1SwNVJZE9dZERA0QvFUGB9UsB/BPqnSGREQFlqf9difVLAzzwixkFkREA=
Manhattan236236Upper East Side Northfalse622236722201985023AQMAAAABAAAACQAAANVekW+gfVLAgSja2GtjREDskrJ+kX1SwHgIaMZwY0RATIEB44x9UsB2o9enhWNEQGwvQziXfVLAqenHm5VjREC+95UYo31SwIuG/LCRY0RAiigaj6V9UsBy2+XrimNEQHTdfzWqfVLArD06Vn5jREDqm9U= (truncated)
Manhattan236236Upper East Side Northfalse622236722201821183AQMAAAABAAAACQAAACjlj5ySfVLAE3tMfapjREC4+Iarg31SwPA60mqvY0RAmoG9D399UsA7N11MxGNEQBXrEGWJfVLAHf9wQNRjREB2AA0Di31SwMfO57fTY0RAJrmEwo59UsAejzBqyWNEQNrP5mGWfVLAiDRhibRjREDq/a8= (truncated)
Manhattan236236Upper East Side Northfalse622236722210340863AQMAAAABAAAACAAAAIcX9JT4fFLAoKhTo/BjREBuepBj9nxSwG8xYFzxY0RAR2JAke98UsBKfc56BGREQOufj07qfFLADa11mBJkREBW+ktc73xSwBUvVGYaZERAUyuxTf58UsDa9V15FWREQChA++kCfVLA+GvWlwBkRECHF/Q= (truncated)
Manhattan236236Upper East Side Northfalse622236722208505855AQMAAAABAAAABQAAACdLlxO3fVLAIgROBkRjRECtcvf7s31SwNpZ0QlSY0RAPeXACLh9UsDu+mVKWGNEQFteC3C9fVLAJ7n5YkljREAnS5cTt31SwCIETgZEY0RA
Manhattan236236Upper East Side Northfalse622236722215190527AQMAAAABAAAABwAAAFTpiHgufVLAfpVN2rNkREAqu8aGH31SwNOnhce4ZERAgfdTLBt9UsBeyQ5/zGREQFvKzHQrfVLASNILXdpkREBvJaJiM31SwJCKHbjDZERAsniXUzV9UsD28R1wvmREQFTpiHgufVLAfpVN2rNkREA=
Manhattan236236Upper East Side Northfalse622236722204835839AQMAAAABAAAABwAAAAJLPg5UfVLAl601h/BiREBJMmN+TH1SwODmSFIFY0RAWgScR0x9UsCgxivoBWNEQFTqeYBTfVLAJQIDDxFjREBw5SVxYn1SwJAdvyEMY0RAnyHuT2V9UsDXNKAg/2JEQAJLPg5UfVLAl601h/BiREA=
Manhattan236236Upper East Side Northfalse622236722209849343AQMAAAABAAAABAAAAETav5LYfFLAQhrp2EJkREB7C9kv1nxSwIaybmtJZERACXj8Mdd8UsCZ8VoWSWREQETav5LYfFLAQhrp2EJkREA=
Manhattan236236Upper East Side Northfalse622236722209685503AQMAAAABAAAABQAAACp6OozJfFLARtBQBGxkREDWlMXZwXxSwB7BWDWBZERAwPirZcl8UsC8nFuRh2REQGwr9/jNfFLAiQKf2XJkREAqejqMyXxSwEbQUARsZERA
Manhattan236236Upper East Side Northfalse622236722202836991AQMAAAABAAAABQAAAE9FueZqfVLA9HMewStkRECbLlQEaH1SwMQGnbQsZERAqR5kKGR9UsCq4lowPmREQEyO6vBnfVLA+PTb4TNkREBPRbnman1SwPRzHsErZERA
Manhattan236236Upper East Side Northfalse622236722209521663AQMAAAABAAAABwAAALgbmurcfFLAY7LC7G1kREBsK/f4zXxSwIkCn9lyZERAwPirZcl8UsC8nFuRh2REQAbCFAvafFLA9gdymJVkREArz0mj4nxSwF7ASsOSZERAdbbEP+d8UsC3e4zhfWREQLgbmurcfFLAY7LC7G1kREA=
Manhattan236236Upper East Side Northfalse622236722204344319AQMAAAABAAAABgAAAH8U7DY/fVLAgoyPuSljREBSL4srPH1SwI6fmBgyY0RAckTMoDZ9UsDndJVMQWNEQNMa+1c7fVLAWr+Mvj9jREAHzeXzP31SwPG9Xd0qY0RAfxTsNj99UsCCjI+5KWNEQA==
Manhattan236236Upper East Side Northfalse622236722211192831AQMAAAABAAAABwAAAEAcWjEVfVLAES/K0qhkREC51Zo/Bn1SwJ3f5r+tZERAxX7u0QN9UsC/5lq/uGREQO9KLAURfVLAg0OL2cNkRECB91MsG31SwF7JDn/MZERAKrvGhh99UsDTp4XHuGREQEAcWjEVfVLAES/K0qhkREA=
Manhattan236236Upper East Side Northfalse622236722208866303AQMAAAABAAAACAAAANFQu6SQfVLAQaw8nSNjREBJjscLhn1SwOkuCRwnY0RAD+4acIF9UsCw3E79O2NEQPsHNMWLfVLAwnQo8UtjREBAUfy1mn1SwGc0pANHY0RAHv6UUZ99UsCB+08iMmNEQI7ZtUOcfVLAGU4way1jREDRULs= (truncated)
Manhattan236236Upper East Side Northfalse622236722212208639AQMAAAABAAAABgAAAJotUIsefVLASyl2Q4NjREAS8XnSHX1SwBFw1D+FY0RAbCpifxZ9UsDATFVVmWNEQP+f8/YZfVLA8JO5MJhjREBD5QyTHn1SwAjIaE+DY0RAmi1Qix59UsBLKXZDg2NEQA==
Manhattan236236Upper East Side Northfalse622236722201690111AQMAAAABAAAABgAAAL73lRijfVLAi4b8sJFjREBsL0M4l31SwKnpx5uVY0RAKOWPnJJ9UsATe0x9qmNEQOr9r16XfVLAfpfk1bFjRECuKyL6nX1SwIiSAMOfY0RAvveVGKN9UsCLhvywkWNEQA==
Manhattan236236Upper East Side Northtrue622236722201526271null
Manhattan236236Upper East Side Northtrue622236722204868607null
Manhattan236236Upper East Side Northfalse622236722215059455AQMAAAABAAAACAAAAI2CdgZCfVLAQtMZC5pkREDH8scUM31SwM+oY/ieZERAVOmIeC59UsB+lU3as2REQLN4l1M1fVLA9vEdcL5kRECU58fgOn1SwKB4tFWvZERANcvSP0J9UsDxeGQWm2REQC63lWlCfVLARS0kpJpkRECNgnY= (truncated)
Manhattan236236Upper East Side Northfalse622236722201198591AQMAAAABAAAACQAAAJqBvQ9/fVLAOzddTMRjRECM96AecH1SwNY20TnJY0RAVVPBgmt9UsCdnWIb3mNEQFUtEth1fVLAgJCOD+5jREAUtiYGg31SwJmV1rbpY0RALneyEId9UsARqYOR3mNEQIcc9zKHfVLATmJOM95jREAV6xA= (truncated)
Manhattan236236Upper East Side Northfalse622236722209718271AQMAAAABAAAABgAAACvPSaPifFLAXsBKw5JkREAFwhQL2nxSwPYHcpiVZERAAItY/tt8UsDn9Sw9l2REQNGx03TsfFLAideODKVkREDZtIj47HxSwGrhH7iiZERAK89Jo+J8UsBewErDkmREQA==
Manhattan236236Upper East Side Northfalse622236722202869759AQMAAAABAAAACAAAAARQMDxxfVLATe008QJkREBXpOlKYn1SwE3LoN4HZERAaJnxrl19UsAGkk3AHGREQJsuVARofVLAxAadtCxkREBORbnman1SwPRzHsErZERAjTtAWG99UsAipMSNH2REQL9W7q92fVLANpcWXAtkREAEUDA= (truncated)
Manhattan236236Upper East Side Northfalse622236722209554431AQMAAAABAAAACQAAAAl4/DHXfFLAmfFaFklkREB7C9kv1nxSwIaybmtJZERAf3rlbdF8UsDeJd6FVmREQN8oM+rJfFLA2lKYAWtkREAqejqMyXxSwEbQUARsZERAbCv3+M18UsCJAp/ZcmREQLgbmurcfFLAY7LC7G1kREAbyBI= (truncated)
Manhattan236236Upper East Side Northfalse622236722208899071AQMAAAABAAAACAAAAHf3aat8fVLA1DOu0xJjREALvsK2e31SwEjuOigXY0RASY7HC4Z9UsDpLgkcJ2NEQNFQu6SQfVLAQaw8nSNjRED/1eCZin1SwFgpLYQeY0RA0RvY0Yh9UsBE4JcMHWNEQMp4Vw2HfVLAsz0CkRtjREB392k= (truncated)
Manhattan236236Upper East Side Northfalse622236722208735231AQMAAAABAAAABAAAAI7ZtUOcfVLAGU4way1jREAe/pRRn31SwIH7TyIyY0RATr0XJKF9UsAY3G6IMWNEQI7ZtUOcfVLAGU4way1jREA=
Manhattan236236Upper East Side Northfalse622236722201559039AQMAAAABAAAABwAAAKPASJorfVLAMBukbl9jREA80aZkJX1SwEtyJW1wY0RAmi1Qix59UsBKKXZDg2NEQEPlDJMefVLACMhoT4NjREAGOAqELX1SwDOFUGJ+Y0RAaFANIDJ9UsASIwaBaWNEQKPASJorfVLAMBukbl9jREA=
Manhattan236236Upper East Side Northfalse622236722204901375AQMAAAABAAAABwAAAJ8h7k9lfVLA1zSgIP9iREBw5SVxYn1SwJAdvyEMY0RA3twTxmx9UsDvMJoVHGNEQAu+wrZ7fVLASO46KBdjREB392mrfH1SwNQzrtMSY0RAwIlKS259UsBarOm5BmNEQJ8h7k9lfVLA1zSgIP9iREA=
Manhattan236236Upper East Side Northfalse622236722213257215AQMAAAABAAAABQAAAANsRIn7fFLAE3QCQuNjRECNZ9v59nxSwB+kJbfvY0RAbnqQY/Z8UsBvMWBc8WNEQIcX9JT4fFLAoKhTo/BjREADbESJ+3xSwBN0AkLjY0RA
Manhattan236236Upper East Side Northtrue622236722209587199null
Showing the first 1000 rows.
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "from mosaic import displayMosaic\n", + "displayMosaic(mosaic_neighbourhoods)" + ] }, { "cell_type": "markdown", - "source": [ - "This also extends to plotting maps inside the notebook using the kepler.gl visualisation library using a notebook magic `%%mosaic_kepler`." - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "a304589f-36a7-4bef-8cd2-932ddd3acba4" + "nuid": "a304589f-36a7-4bef-8cd2-932ddd3acba4", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "This also extends to plotting maps inside the notebook using the kepler.gl visualisation library using a notebook magic `%%mosaic_kepler`." + ] }, { "cell_type": "code", - "source": [ - "from mosaic import st_aswkt\n(\n mosaic_neighbourhoods\n .select(st_aswkt(col(\"index.wkb\")).alias(\"wkt\"), col(\"index.index_id\").alias(\"h3\"))\n).createOrReplaceTempView(\"kepler_df\")" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "3be96bd5-e14d-40bd-9fc7-00dea16cb096" + "nuid": "3be96bd5-e14d-40bd-9fc7-00dea16cb096", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "from mosaic import st_aswkt\n", + "(\n", + " mosaic_neighbourhoods\n", + " .select(st_aswkt(col(\"index.wkb\")).alias(\"wkt\"), col(\"index.index_id\").alias(\"h3\"))\n", + ").createOrReplaceTempView(\"kepler_df\")" + ] }, { "cell_type": "code", - "source": [ - "%%mosaic_kepler\n\"kepler_df\" \"h3\" \"h3\"" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "ffe429bb-de6f-4118-8d7b-6484b6b2d678" + "nuid": "ffe429bb-de6f-4118-8d7b-6484b6b2d678", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
User Guide: https://docs.kepler.gl/docs/keplergl-jupyter\n
" - ] - } - }, - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "", - "textData": null, - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "htmlSandbox", - "arguments": {} - } - }, - "data": { - "text/html": [ - "" - ] - } - }, - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "Kepler.gl
", - "textData": null, - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "htmlSandbox", - "arguments": {} - } - }, - "data": { - "text/html": [ - "Kepler.gl
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "%%mosaic_kepler\n", + "\"kepler_df\" \"h3\" \"h3\"" + ] }, { "cell_type": "markdown", - "source": [ - "![mosaic kepler map example](../images/kepler-example.png)" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "4323fa2c-8495-43c6-afb2-9e239e5fb6fc" + "nuid": "4323fa2c-8495-43c6-afb2-9e239e5fb6fc", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "![mosaic kepler map example](../images/kepler-example.png)" + ] }, { "cell_type": "markdown", - "source": [ - "Now the two datasets can be joined first on H3 index, with any false positives removed through a contains filter on a much simpler geometry." - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "11933dab-085c-4503-8ba1-06365caa335c" + "nuid": "11933dab-085c-4503-8ba1-06365caa335c", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "Now the two datasets can be joined first on H3 index, with any false positives removed through a contains filter on a much simpler geometry." + ] }, { "cell_type": "code", - "source": [ - "mosaic_joined_df = (\n trips_with_geom.alias(\"t\")\n .join(mosaic_neighbourhoods.alias(\"n\"), on=expr(\"t.pickup_h3 = n.index.index_id\"), how=\"inner\")\n .where(\n ~col(\"index.is_core\") | \n st_contains(\"index.wkb\", \"pickup_geom\")\n )\n)\n\nmosaic_joined_df.show()" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "98514998-1fe7-4a0c-b4f9-eebf38fb9d81" + "nuid": "98514998-1fe7-4a0c-b4f9-eebf38fb9d81", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------+-----------------+----------------+-----------+-----+-------+----------+------------+------------+--------------------+--------------------+--------------------+------------------+------------------+---------+-----------+--------+--------------------+-------+------------------+--------------------+\n|vendor_id| pickup_datetime| dropoff_datetime|passenger_count|trip_distance|pickup_longitude|pickup_latitude|rate_code_id|dropoff_longitude|dropoff_latitude|fare_amount|extra|mta_tax|tip_amount|tolls_amount|total_amount| pickup_geom| dropoff_geom| bounds| pickup_h3| dropoff_h3| borough|location_id|objectid| zone|is_core| h3| wkb|\n+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------+-----------------+----------------+-----------+-----+-------+----------+------------+------------+--------------------+--------------------+--------------------+------------------+------------------+---------+-----------+--------+--------------------+-------+------------------+--------------------+\n| CMT|2009-11-28 13:00:00|2009-11-28 13:01:57| 1| 0.5| -73.999666| 40.733384| null| -73.991751| 40.730273| 3.3| 0.0| 0.5| 0.0| 0.0| 3.8|{1, [[[-73.999666...|{1, [[[-73.991751...|{5, [[[-80.0, 35....|622236723433046015|622236750715092991|Manhattan| 114| 114|Greenwich Village...| false|622236723433046015|[01 03 00 00 00 0...|\n| CMT|2009-11-28 13:00:00|2009-11-28 13:01:57| 1| 0.5| -73.999666| 40.733384| null| -73.991751| 40.730273| 3.3| 0.0| 0.5| 0.0| 0.0| 3.8|{1, [[[-73.999666...|{1, [[[-73.991751...|{5, [[[-80.0, 35....|622236723433046015|622236750715092991|Manhattan| 249| 249| West Village| false|622236723433046015|[01 03 00 00 00 0...|\n| CMT|2009-11-28 13:00:00|2009-11-28 13:01:57| 1| 0.5| -73.999666| 40.733384| null| -73.991751| 40.730273| 3.3| 0.0| 0.5| 0.0| 0.0| 3.8|{1, [[[-73.999666...|{1, [[[-73.991751...|{5, [[[-80.0, 35....|622236723433046015|622236750715092991|Manhattan| 113| 113|Greenwich Village...| false|622236723433046015|[01 03 00 00 00 0...|\n| CMT|2009-11-30 07:49:13|2009-11-30 08:12:38| 1| 12.9| -74.007033| 40.734931| null| -73.87277| 40.774319| 29.7| 0.0| 0.5| 6.04| 0.0| 36.24|{1, [[[-74.007033...|{1, [[[-73.87277,...|{5, [[[-80.0, 35....|622236750719975423|622236724015759359|Manhattan| 249| 249| West Village| false|622236750719975423|[01 03 00 00 00 0...|\n| CMT|2009-11-30 07:49:13|2009-11-30 08:12:38| 1| 12.9| -74.007033| 40.734931| null| -73.87277| 40.774319| 29.7| 0.0| 0.5| 6.04| 0.0| 36.24|{1, [[[-74.007033...|{1, [[[-73.87277,...|{5, [[[-80.0, 35....|622236750719975423|622236724015759359|Manhattan| 158| 158|Meatpacking/West ...| false|622236750719975423|[01 03 00 00 00 0...|\n| VTS|2009-01-03 01:55:00|2009-01-03 02:04:00| 5| 2.09| -73.993217| 40.72416| null| -73.9944| 40.74624| 7.7| 0.5| null| 0.0| 0.0| 8.2|{1, [[[-73.993217...|{1, [[[-73.9944, ...|{5, [[[-80.0, 35....|622236750717878271|622236723431276543|Manhattan| 114| 114|Greenwich Village...| false|622236750717878271|[01 03 00 00 00 0...|\n| VTS|2009-01-03 01:55:00|2009-01-03 02:04:00| 5| 2.09| -73.993217| 40.72416| null| -73.9944| 40.74624| 7.7| 0.5| null| 0.0| 0.0| 8.2|{1, [[[-73.993217...|{1, [[[-73.9944, ...|{5, [[[-80.0, 35....|622236750717878271|622236723431276543|Manhattan| 144| 144| Little Italy/NoLiTa| false|622236750717878271|[01 03 00 00 00 0...|\n| CMT|2009-12-07 10:36:06|2009-12-07 10:42:26| 1| 1.3| -73.99067| 40.73065| null| -73.975539| 40.741353| 6.1| 0.0| 0.5| 0.0| 0.0| 6.6|{1, [[[-73.99067,...|{1, [[[-73.975539...|{5, [[[-80.0, 35....|622236750715092991|622236723437502463|Manhattan| 113| 113|Greenwich Village...| false|622236750715092991|[01 03 00 00 00 0...|\n| CMT|2009-12-07 10:36:06|2009-12-07 10:42:26| 1| 1.3| -73.99067| 40.73065| null| -73.975539| 40.741353| 6.1| 0.0| 0.5| 0.0| 0.0| 6.6|{1, [[[-73.99067,...|{1, [[[-73.975539...|{5, [[[-80.0, 35....|622236750715092991|622236723437502463|Manhattan| 79| 79| East Village| false|622236750715092991|[01 03 00 00 00 0...|\n| CMT|2010-06-27 17:27:25|2010-06-27 17:47:44| 2| 5.0| -74.008459| 40.712233| 1| -73.9516| 40.718877| 15.7| 0.0| 0.5| 3.0| 0.0| 19.2|{1, [[[-74.008459...|{1, [[[-73.9516, ...|{5, [[[-80.0, 35....|622236750648639487|622236723415482367|Manhattan| 231| 231|TriBeCa/Civic Center| false|622236750648639487|[01 03 00 00 00 0...|\n| CMT|2010-06-27 17:27:25|2010-06-27 17:47:44| 2| 5.0| -74.008459| 40.712233| 1| -73.9516| 40.718877| 15.7| 0.0| 0.5| 3.0| 0.0| 19.2|{1, [[[-74.008459...|{1, [[[-73.9516, ...|{5, [[[-80.0, 35....|622236750648639487|622236723415482367|Manhattan| 261| 261| World Trade Center| false|622236750648639487|[01 03 00 00 00 0...|\n| CMT|2010-06-27 17:27:25|2010-06-27 17:47:44| 2| 5.0| -74.008459| 40.712233| 1| -73.9516| 40.718877| 15.7| 0.0| 0.5| 3.0| 0.0| 19.2|{1, [[[-74.008459...|{1, [[[-73.9516, ...|{5, [[[-80.0, 35....|622236750648639487|622236723415482367|Manhattan| 209| 209| Seaport| false|622236750648639487|[01 03 00 00 00 0...|\n| CMT|2009-11-17 16:50:35|2009-11-17 17:01:34| 1| 1.6| -74.00679| 40.730546| null| -73.9904| 40.73815| 7.7| 1.0| 0.5| 1.5| 0.0| 10.7|{1, [[[-74.00679,...|{1, [[[-73.9904, ...|{5, [[[-80.0, 35....|622236750718894079|622236723426754559|Manhattan| 249| 249| West Village| false|622236750718894079|[01 03 00 00 00 0...|\n| CMT|2009-11-17 16:50:35|2009-11-17 17:01:34| 1| 1.6| -74.00679| 40.730546| null| -73.9904| 40.73815| 7.7| 1.0| 0.5| 1.5| 0.0| 10.7|{1, [[[-74.00679,...|{1, [[[-73.9904, ...|{5, [[[-80.0, 35....|622236750718894079|622236723426754559|Manhattan| 158| 158|Meatpacking/West ...| false|622236750718894079|[01 03 00 00 00 0...|\n| CMT|2009-11-30 23:30:43|2009-11-30 23:42:54| 2| 2.9| -74.009605| 40.710155| null| -73.993875| 40.741554| 10.1| 0.5| 0.5| 2.22| 0.0| 13.32|{1, [[[-74.009605...|{1, [[[-73.993875...|{5, [[[-80.0, 35....|622236750646968319|622236723430490111|Manhattan| 87| 87|Financial Distric...| false|622236750646968319|[01 03 00 00 00 0...|\n| CMT|2009-11-30 23:30:43|2009-11-30 23:42:54| 2| 2.9| -74.009605| 40.710155| null| -73.993875| 40.741554| 10.1| 0.5| 0.5| 2.22| 0.0| 13.32|{1, [[[-74.009605...|{1, [[[-73.993875...|{5, [[[-80.0, 35....|622236750646968319|622236723430490111|Manhattan| 261| 261| World Trade Center| false|622236750646968319|[01 03 00 00 00 0...|\n| CMT|2010-09-21 07:44:41|2010-09-21 08:07:03| 1| 4.2| -74.008626| 40.7042| 1| -73.981636| 40.758581| 14.5| 0.0| 0.5| 1.2| 0.0| 16.2|{1, [[[-74.008626...|{1, [[[-73.981636...|{5, [[[-80.0, 35....|622236750650671103|622236723500384255|Manhattan| 87| 87|Financial Distric...| false|622236750650671103|[01 03 00 00 00 0...|\n| CMT|2010-09-21 07:44:41|2010-09-21 08:07:03| 1| 4.2| -74.008626| 40.7042| 1| -73.981636| 40.758581| 14.5| 0.0| 0.5| 1.2| 0.0| 16.2|{1, [[[-74.008626...|{1, [[[-73.981636...|{5, [[[-80.0, 35....|622236750650671103|622236723500384255|Manhattan| 88| 88|Financial Distric...| false|622236750650671103|[01 03 00 00 00 0...|\n| VTS|2010-10-10 08:56:00|2010-10-10 09:01:00| 1| 1.96| -74.012115| 40.707623| 1| -73.999272| 40.734073| 6.5| 0.0| 0.5| 0.0| 0.0| 7.0|{1, [[[-74.012115...|{1, [[[-73.999272...|{5, [[[-80.0, 35....|622236750639693823|622236723433013247|Manhattan| 87| 87|Financial Distric...| false|622236750639693823|[01 03 00 00 00 0...|\n| VTS|2010-10-10 08:56:00|2010-10-10 09:01:00| 1| 1.96| -74.012115| 40.707623| 1| -73.999272| 40.734073| 6.5| 0.0| 0.5| 0.0| 0.0| 7.0|{1, [[[-74.012115...|{1, [[[-73.999272...|{5, [[[-80.0, 35....|622236750639693823|622236723433013247|Manhattan| 261| 261| World Trade Center| false|622236750639693823|[01 03 00 00 00 0...|\n+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------+-----------------+----------------+-----------+-----+-------+----------+------------+------------+--------------------+--------------------+--------------------+------------------+------------------+---------+-----------+--------+--------------------+-------+------------------+--------------------+\nonly showing top 20 rows\n\n
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------+-----------------+----------------+-----------+-----+-------+----------+------------+------------+--------------------+--------------------+--------------------+------------------+------------------+---------+-----------+--------+--------------------+-------+------------------+--------------------+\nvendor_id| pickup_datetime| dropoff_datetime|passenger_count|trip_distance|pickup_longitude|pickup_latitude|rate_code_id|dropoff_longitude|dropoff_latitude|fare_amount|extra|mta_tax|tip_amount|tolls_amount|total_amount| pickup_geom| dropoff_geom| bounds| pickup_h3| dropoff_h3| borough|location_id|objectid| zone|is_core| h3| wkb|\n+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------+-----------------+----------------+-----------+-----+-------+----------+------------+------------+--------------------+--------------------+--------------------+------------------+------------------+---------+-----------+--------+--------------------+-------+------------------+--------------------+\n CMT|2009-11-28 13:00:00|2009-11-28 13:01:57| 1| 0.5| -73.999666| 40.733384| null| -73.991751| 40.730273| 3.3| 0.0| 0.5| 0.0| 0.0| 3.8|{1, [[[-73.999666...|{1, [[[-73.991751...|{5, [[[-80.0, 35....|622236723433046015|622236750715092991|Manhattan| 114| 114|Greenwich Village...| false|622236723433046015|[01 03 00 00 00 0...|\n CMT|2009-11-28 13:00:00|2009-11-28 13:01:57| 1| 0.5| -73.999666| 40.733384| null| -73.991751| 40.730273| 3.3| 0.0| 0.5| 0.0| 0.0| 3.8|{1, [[[-73.999666...|{1, [[[-73.991751...|{5, [[[-80.0, 35....|622236723433046015|622236750715092991|Manhattan| 249| 249| West Village| false|622236723433046015|[01 03 00 00 00 0...|\n CMT|2009-11-28 13:00:00|2009-11-28 13:01:57| 1| 0.5| -73.999666| 40.733384| null| -73.991751| 40.730273| 3.3| 0.0| 0.5| 0.0| 0.0| 3.8|{1, [[[-73.999666...|{1, [[[-73.991751...|{5, [[[-80.0, 35....|622236723433046015|622236750715092991|Manhattan| 113| 113|Greenwich Village...| false|622236723433046015|[01 03 00 00 00 0...|\n CMT|2009-11-30 07:49:13|2009-11-30 08:12:38| 1| 12.9| -74.007033| 40.734931| null| -73.87277| 40.774319| 29.7| 0.0| 0.5| 6.04| 0.0| 36.24|{1, [[[-74.007033...|{1, [[[-73.87277,...|{5, [[[-80.0, 35....|622236750719975423|622236724015759359|Manhattan| 249| 249| West Village| false|622236750719975423|[01 03 00 00 00 0...|\n CMT|2009-11-30 07:49:13|2009-11-30 08:12:38| 1| 12.9| -74.007033| 40.734931| null| -73.87277| 40.774319| 29.7| 0.0| 0.5| 6.04| 0.0| 36.24|{1, [[[-74.007033...|{1, [[[-73.87277,...|{5, [[[-80.0, 35....|622236750719975423|622236724015759359|Manhattan| 158| 158|Meatpacking/West ...| false|622236750719975423|[01 03 00 00 00 0...|\n VTS|2009-01-03 01:55:00|2009-01-03 02:04:00| 5| 2.09| -73.993217| 40.72416| null| -73.9944| 40.74624| 7.7| 0.5| null| 0.0| 0.0| 8.2|{1, [[[-73.993217...|{1, [[[-73.9944, ...|{5, [[[-80.0, 35....|622236750717878271|622236723431276543|Manhattan| 114| 114|Greenwich Village...| false|622236750717878271|[01 03 00 00 00 0...|\n VTS|2009-01-03 01:55:00|2009-01-03 02:04:00| 5| 2.09| -73.993217| 40.72416| null| -73.9944| 40.74624| 7.7| 0.5| null| 0.0| 0.0| 8.2|{1, [[[-73.993217...|{1, [[[-73.9944, ...|{5, [[[-80.0, 35....|622236750717878271|622236723431276543|Manhattan| 144| 144| Little Italy/NoLiTa| false|622236750717878271|[01 03 00 00 00 0...|\n CMT|2009-12-07 10:36:06|2009-12-07 10:42:26| 1| 1.3| -73.99067| 40.73065| null| -73.975539| 40.741353| 6.1| 0.0| 0.5| 0.0| 0.0| 6.6|{1, [[[-73.99067,...|{1, [[[-73.975539...|{5, [[[-80.0, 35....|622236750715092991|622236723437502463|Manhattan| 113| 113|Greenwich Village...| false|622236750715092991|[01 03 00 00 00 0...|\n CMT|2009-12-07 10:36:06|2009-12-07 10:42:26| 1| 1.3| -73.99067| 40.73065| null| -73.975539| 40.741353| 6.1| 0.0| 0.5| 0.0| 0.0| 6.6|{1, [[[-73.99067,...|{1, [[[-73.975539...|{5, [[[-80.0, 35....|622236750715092991|622236723437502463|Manhattan| 79| 79| East Village| false|622236750715092991|[01 03 00 00 00 0...|\n CMT|2010-06-27 17:27:25|2010-06-27 17:47:44| 2| 5.0| -74.008459| 40.712233| 1| -73.9516| 40.718877| 15.7| 0.0| 0.5| 3.0| 0.0| 19.2|{1, [[[-74.008459...|{1, [[[-73.9516, ...|{5, [[[-80.0, 35....|622236750648639487|622236723415482367|Manhattan| 231| 231|TriBeCa/Civic Center| false|622236750648639487|[01 03 00 00 00 0...|\n CMT|2010-06-27 17:27:25|2010-06-27 17:47:44| 2| 5.0| -74.008459| 40.712233| 1| -73.9516| 40.718877| 15.7| 0.0| 0.5| 3.0| 0.0| 19.2|{1, [[[-74.008459...|{1, [[[-73.9516, ...|{5, [[[-80.0, 35....|622236750648639487|622236723415482367|Manhattan| 261| 261| World Trade Center| false|622236750648639487|[01 03 00 00 00 0...|\n CMT|2010-06-27 17:27:25|2010-06-27 17:47:44| 2| 5.0| -74.008459| 40.712233| 1| -73.9516| 40.718877| 15.7| 0.0| 0.5| 3.0| 0.0| 19.2|{1, [[[-74.008459...|{1, [[[-73.9516, ...|{5, [[[-80.0, 35....|622236750648639487|622236723415482367|Manhattan| 209| 209| Seaport| false|622236750648639487|[01 03 00 00 00 0...|\n CMT|2009-11-17 16:50:35|2009-11-17 17:01:34| 1| 1.6| -74.00679| 40.730546| null| -73.9904| 40.73815| 7.7| 1.0| 0.5| 1.5| 0.0| 10.7|{1, [[[-74.00679,...|{1, [[[-73.9904, ...|{5, [[[-80.0, 35....|622236750718894079|622236723426754559|Manhattan| 249| 249| West Village| false|622236750718894079|[01 03 00 00 00 0...|\n CMT|2009-11-17 16:50:35|2009-11-17 17:01:34| 1| 1.6| -74.00679| 40.730546| null| -73.9904| 40.73815| 7.7| 1.0| 0.5| 1.5| 0.0| 10.7|{1, [[[-74.00679,...|{1, [[[-73.9904, ...|{5, [[[-80.0, 35....|622236750718894079|622236723426754559|Manhattan| 158| 158|Meatpacking/West ...| false|622236750718894079|[01 03 00 00 00 0...|\n CMT|2009-11-30 23:30:43|2009-11-30 23:42:54| 2| 2.9| -74.009605| 40.710155| null| -73.993875| 40.741554| 10.1| 0.5| 0.5| 2.22| 0.0| 13.32|{1, [[[-74.009605...|{1, [[[-73.993875...|{5, [[[-80.0, 35....|622236750646968319|622236723430490111|Manhattan| 87| 87|Financial Distric...| false|622236750646968319|[01 03 00 00 00 0...|\n CMT|2009-11-30 23:30:43|2009-11-30 23:42:54| 2| 2.9| -74.009605| 40.710155| null| -73.993875| 40.741554| 10.1| 0.5| 0.5| 2.22| 0.0| 13.32|{1, [[[-74.009605...|{1, [[[-73.993875...|{5, [[[-80.0, 35....|622236750646968319|622236723430490111|Manhattan| 261| 261| World Trade Center| false|622236750646968319|[01 03 00 00 00 0...|\n CMT|2010-09-21 07:44:41|2010-09-21 08:07:03| 1| 4.2| -74.008626| 40.7042| 1| -73.981636| 40.758581| 14.5| 0.0| 0.5| 1.2| 0.0| 16.2|{1, [[[-74.008626...|{1, [[[-73.981636...|{5, [[[-80.0, 35....|622236750650671103|622236723500384255|Manhattan| 87| 87|Financial Distric...| false|622236750650671103|[01 03 00 00 00 0...|\n CMT|2010-09-21 07:44:41|2010-09-21 08:07:03| 1| 4.2| -74.008626| 40.7042| 1| -73.981636| 40.758581| 14.5| 0.0| 0.5| 1.2| 0.0| 16.2|{1, [[[-74.008626...|{1, [[[-73.981636...|{5, [[[-80.0, 35....|622236750650671103|622236723500384255|Manhattan| 88| 88|Financial Distric...| false|622236750650671103|[01 03 00 00 00 0...|\n VTS|2010-10-10 08:56:00|2010-10-10 09:01:00| 1| 1.96| -74.012115| 40.707623| 1| -73.999272| 40.734073| 6.5| 0.0| 0.5| 0.0| 0.0| 7.0|{1, [[[-74.012115...|{1, [[[-73.999272...|{5, [[[-80.0, 35....|622236750639693823|622236723433013247|Manhattan| 87| 87|Financial Distric...| false|622236750639693823|[01 03 00 00 00 0...|\n VTS|2010-10-10 08:56:00|2010-10-10 09:01:00| 1| 1.96| -74.012115| 40.707623| 1| -73.999272| 40.734073| 6.5| 0.0| 0.5| 0.0| 0.0| 7.0|{1, [[[-74.012115...|{1, [[[-73.999272...|{5, [[[-80.0, 35....|622236750639693823|622236723433013247|Manhattan| 261| 261| World Trade Center| false|622236750639693823|[01 03 00 00 00 0...|\n+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------+-----------------+----------------+-----------+-----+-------+----------+------------+------------+--------------------+--------------------+--------------------+------------------+------------------+---------+-----------+--------+--------------------+-------+------------------+--------------------+\nonly showing top 20 rows\n\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "mosaic_joined_df = (\n", + " trips_with_geom.alias(\"t\")\n", + " .join(mosaic_neighbourhoods.alias(\"n\"), on=expr(\"t.pickup_h3 = n.index.index_id\"), how=\"inner\")\n", + " .where(\n", + " ~col(\"index.is_core\") | \n", + " st_contains(\"index.wkb\", \"pickup_geom\")\n", + " )\n", + ")\n", + "\n", + "mosaic_joined_df.show()" + ] }, { "cell_type": "markdown", - "source": [ - "## MosaicFrame abstraction for simple indexing and joins" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "6ad416fa-3a36-4946-98f9-b8d306d31dc5" + "nuid": "6ad416fa-3a36-4946-98f9-b8d306d31dc5", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "## MosaicFrame abstraction for simple indexing and joins" + ] }, { "cell_type": "markdown", - "source": [ - "By wrapping our Spark DataFrames with `MosaicFrame`, we can simplify the join process. For example:" - ], "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "aba08cb2-fbbb-4de7-acdc-5eac0c12b91b" + "nuid": "aba08cb2-fbbb-4de7-acdc-5eac0c12b91b", + "showTitle": false, + "title": "" } - } + }, + "source": [ + "By wrapping our Spark DataFrames with `MosaicFrame`, we can simplify the join process. For example:" + ] }, { "cell_type": "code", - "source": [ - "from mosaic import MosaicFrame" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "54194240-dc88-42dc-a7e0-8b4a97a496f8" + "nuid": "54194240-dc88-42dc-a7e0-8b4a97a496f8", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "from mosaic import MosaicFrame" + ] }, { "cell_type": "code", - "source": [ - "trips_mdf = MosaicFrame(trips, \"pickup_geom\")\nneighbourhoods_mdf = MosaicFrame(geoJsonDF, \"geometry\")" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "94730cc1-afc1-4227-9c91-709a18bb6eec" + "nuid": "94730cc1-afc1-4227-9c91-709a18bb6eec", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "trips_mdf = MosaicFrame(trips, \"pickup_geom\")\n", + "neighbourhoods_mdf = MosaicFrame(geoJsonDF, \"geometry\")" + ] }, { "cell_type": "code", - "source": [ - "(\n trips_mdf\n .set_index_resolution(10)\n .apply_index()\n .join(\n neighbourhoods_mdf\n .set_index_resolution(10)\n .apply_index()\n )\n).show()" - ], + "execution_count": 0, "metadata": { "application/vnd.databricks.v1+cell": { - "title": "", - "showTitle": false, + "cellMetadata": {}, "inputWidgets": {}, - "nuid": "a33782bb-3d4f-44e8-a33b-faeb11d0de46" + "nuid": "a33782bb-3d4f-44e8-a33b-faeb11d0de46", + "showTitle": false, + "title": "" } }, - "outputs": [ - { - "output_type": "display_data", - "metadata": { - "application/vnd.databricks.v1+output": { - "datasetInfos": [], - "data": "
+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------+-----------------+----------------+-----------+-----+-------+----------+------------+------------+--------------------+--------------------+--------------------+\n|vendor_id| pickup_datetime| dropoff_datetime|passenger_count|trip_distance|pickup_longitude|pickup_latitude|rate_code_id|dropoff_longitude|dropoff_latitude|fare_amount|extra|mta_tax|tip_amount|tolls_amount|total_amount| pickup_geom| dropoff_geom| bounds|\n+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------+-----------------+----------------+-----------+-----+-------+----------+------------+------------+--------------------+--------------------+--------------------+\n| CMT|2009-10-12 19:35:21|2009-10-12 19:44:36| 1| 1.4| -74.000018| 40.720534| null| -73.995008| 40.734112| 6.5| 0.0| null| 0.0| 0.0| 6.5|{1, [[[-74.000018...|{1, [[[-73.995008...|{5, [[[-80.0, 35....|\n| CMT|2010-09-02 07:37:40|2010-09-02 07:53:17| 1| 6.8| -74.016477| 40.70971| 1| -73.97509| 40.759392| 17.3| 0.0| 0.5| 1.0| 0.0| 18.8|{1, [[[-74.016477...|{1, [[[-73.97509,...|{5, [[[-80.0, 35....|\n| CMT|2009-11-28 13:00:00|2009-11-28 13:01:57| 1| 0.5| -73.999666| 40.733384| null| -73.991751| 40.730273| 3.3| 0.0| 0.5| 0.0| 0.0| 3.8|{1, [[[-73.999666...|{1, [[[-73.991751...|{5, [[[-80.0, 35....|\n| CMT|2009-11-30 07:49:13|2009-11-30 08:12:38| 1| 12.9| -74.007033| 40.734931| null| -73.87277| 40.774319| 29.7| 0.0| 0.5| 6.04| 0.0| 36.24|{1, [[[-74.007033...|{1, [[[-73.87277,...|{5, [[[-80.0, 35....|\n| VTS|2009-01-03 01:55:00|2009-01-03 02:04:00| 5| 2.09| -73.993217| 40.72416| null| -73.9944| 40.74624| 7.7| 0.5| null| 0.0| 0.0| 8.2|{1, [[[-73.993217...|{1, [[[-73.9944, ...|{5, [[[-80.0, 35....|\n| VTS|2009-11-09 17:21:00|2009-11-09 17:32:00| 1| 5.19| -74.010003| 40.707022| null| -73.963853| 40.75677| 13.3| 1.0| 0.5| 0.0| 0.0| 14.8|{1, [[[-74.010003...|{1, [[[-73.963853...|{5, [[[-80.0, 35....|\n| VTS|2010-06-09 01:13:00|2010-06-09 01:27:00| 1| 8.09| -74.008697| 40.707335| 1| -73.949605| 40.78761| 19.7| 0.5| 0.5| 0.0| 0.0| 20.7|{1, [[[-74.008697...|{1, [[[-73.949605...|{5, [[[-80.0, 35....|\n| VTS|2009-01-28 18:36:00|2009-01-28 18:50:00| 1| 2.92| -74.008823| 40.72089| null| -73.994055| 40.747588| 10.1| 1.0| null| 3.0| 0.0| 14.1|{1, [[[-74.008823...|{1, [[[-73.994055...|{5, [[[-80.0, 35....|\n| CMT|2010-06-02 14:21:05|2010-06-02 14:28:43| 1| 5.0| -74.011653| 40.702604| 1| -73.971851| 40.749807| 12.5| 0.0| 0.5| 2.6| 0.0| 15.6|{1, [[[-74.011653...|{1, [[[-73.971851...|{5, [[[-80.0, 35....|\n| VTS|2009-11-26 15:38:00|2009-11-26 15:47:00| 5| 2.36| -73.997208| 40.722108| null| -73.99647| 40.748185| 8.1| 0.0| 0.5| 2.0| 0.0| 10.6|{1, [[[-73.997208...|{1, [[[-73.99647,...|{5, [[[-80.0, 35....|\n| CMT|2009-12-07 10:36:06|2009-12-07 10:42:26| 1| 1.3| -73.99067| 40.73065| null| -73.975539| 40.741353| 6.1| 0.0| 0.5| 0.0| 0.0| 6.6|{1, [[[-73.99067,...|{1, [[[-73.975539...|{5, [[[-80.0, 35....|\n| CMT|2009-11-15 09:11:42|2009-11-15 09:26:34| 1| 3.7| -73.990065| 40.687341| null| -73.999566| 40.730219| 12.1| 0.0| 0.5| 0.0| 0.0| 12.6|{1, [[[-73.990065...|{1, [[[-73.999566...|{5, [[[-80.0, 35....|\n| CMT|2010-06-27 17:27:25|2010-06-27 17:47:44| 2| 5.0| -74.008459| 40.712233| 1| -73.9516| 40.718877| 15.7| 0.0| 0.5| 3.0| 0.0| 19.2|{1, [[[-74.008459...|{1, [[[-73.9516, ...|{5, [[[-80.0, 35....|\n| VTS|2009-01-30 21:05:00|2009-01-30 21:34:00| 1| 5.99| -73.994397| 40.694843| null| -73.921257| 40.685107| 18.9| 0.5| null| 0.6| 0.0| 20.0|{1, [[[-73.994397...|{1, [[[-73.921257...|{5, [[[-80.0, 35....|\n| CMT|2009-01-14 07:43:53|2009-01-14 07:55:14| 2| 2.9| -74.007602| 40.725098| null| -73.977846| 40.751608| 9.3| 0.0| null| 0.0| 0.0| 9.3|{1, [[[-74.007602...|{1, [[[-73.977846...|{5, [[[-80.0, 35....|\n| VTS|2010-05-11 16:33:00|2010-05-11 16:41:00| 1| 0.99| -74.011205| 40.715468| 1| -74.011072| 40.708882| 5.7| 1.0| 0.5| 0.0| 0.0| 7.2|{1, [[[-74.011205...|{1, [[[-74.011072...|{5, [[[-80.0, 35....|\n| VTS|2009-01-19 09:43:00|2009-01-19 09:47:00| 1| 0.56| -73.991123| 40.731297| null| -73.98681| 40.736703| 4.1| 0.0| null| 0.0| 0.0| 4.1|{1, [[[-73.991123...|{1, [[[-73.98681,...|{5, [[[-80.0, 35....|\n| CMT|2009-11-14 15:46:36|2009-11-14 15:51:01| 2| 1.2| -74.008448| 40.734271| null| -74.006075| 40.720237| 4.9| 0.0| 0.5| 0.0| 0.0| 5.4|{1, [[[-74.008448...|{1, [[[-74.006075...|{5, [[[-80.0, 35....|\n| CMT|2009-11-17 16:50:35|2009-11-17 17:01:34| 1| 1.6| -74.00679| 40.730546| null| -73.9904| 40.73815| 7.7| 1.0| 0.5| 1.5| 0.0| 10.7|{1, [[[-74.00679,...|{1, [[[-73.9904, ...|{5, [[[-80.0, 35....|\n| CMT|2009-11-30 23:30:43|2009-11-30 23:42:54| 2| 2.9| -74.009605| 40.710155| null| -73.993875| 40.741554| 10.1| 0.5| 0.5| 2.22| 0.0| 13.32|{1, [[[-74.009605...|{1, [[[-73.993875...|{5, [[[-80.0, 35....|\n+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------+-----------------+----------------+-----------+-----+-------+----------+------------+------------+--------------------+--------------------+--------------------+\nonly showing top 20 rows\n\n
", - "removedWidgets": [], - "addedWidgets": {}, - "metadata": {}, - "type": "html", - "arguments": {} - } - }, - "data": { - "text/html": [ - "\n
+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------+-----------------+----------------+-----------+-----+-------+----------+------------+------------+--------------------+--------------------+--------------------+\nvendor_id| pickup_datetime| dropoff_datetime|passenger_count|trip_distance|pickup_longitude|pickup_latitude|rate_code_id|dropoff_longitude|dropoff_latitude|fare_amount|extra|mta_tax|tip_amount|tolls_amount|total_amount| pickup_geom| dropoff_geom| bounds|\n+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------+-----------------+----------------+-----------+-----+-------+----------+------------+------------+--------------------+--------------------+--------------------+\n CMT|2009-10-12 19:35:21|2009-10-12 19:44:36| 1| 1.4| -74.000018| 40.720534| null| -73.995008| 40.734112| 6.5| 0.0| null| 0.0| 0.0| 6.5|{1, [[[-74.000018...|{1, [[[-73.995008...|{5, [[[-80.0, 35....|\n CMT|2010-09-02 07:37:40|2010-09-02 07:53:17| 1| 6.8| -74.016477| 40.70971| 1| -73.97509| 40.759392| 17.3| 0.0| 0.5| 1.0| 0.0| 18.8|{1, [[[-74.016477...|{1, [[[-73.97509,...|{5, [[[-80.0, 35....|\n CMT|2009-11-28 13:00:00|2009-11-28 13:01:57| 1| 0.5| -73.999666| 40.733384| null| -73.991751| 40.730273| 3.3| 0.0| 0.5| 0.0| 0.0| 3.8|{1, [[[-73.999666...|{1, [[[-73.991751...|{5, [[[-80.0, 35....|\n CMT|2009-11-30 07:49:13|2009-11-30 08:12:38| 1| 12.9| -74.007033| 40.734931| null| -73.87277| 40.774319| 29.7| 0.0| 0.5| 6.04| 0.0| 36.24|{1, [[[-74.007033...|{1, [[[-73.87277,...|{5, [[[-80.0, 35....|\n VTS|2009-01-03 01:55:00|2009-01-03 02:04:00| 5| 2.09| -73.993217| 40.72416| null| -73.9944| 40.74624| 7.7| 0.5| null| 0.0| 0.0| 8.2|{1, [[[-73.993217...|{1, [[[-73.9944, ...|{5, [[[-80.0, 35....|\n VTS|2009-11-09 17:21:00|2009-11-09 17:32:00| 1| 5.19| -74.010003| 40.707022| null| -73.963853| 40.75677| 13.3| 1.0| 0.5| 0.0| 0.0| 14.8|{1, [[[-74.010003...|{1, [[[-73.963853...|{5, [[[-80.0, 35....|\n VTS|2010-06-09 01:13:00|2010-06-09 01:27:00| 1| 8.09| -74.008697| 40.707335| 1| -73.949605| 40.78761| 19.7| 0.5| 0.5| 0.0| 0.0| 20.7|{1, [[[-74.008697...|{1, [[[-73.949605...|{5, [[[-80.0, 35....|\n VTS|2009-01-28 18:36:00|2009-01-28 18:50:00| 1| 2.92| -74.008823| 40.72089| null| -73.994055| 40.747588| 10.1| 1.0| null| 3.0| 0.0| 14.1|{1, [[[-74.008823...|{1, [[[-73.994055...|{5, [[[-80.0, 35....|\n CMT|2010-06-02 14:21:05|2010-06-02 14:28:43| 1| 5.0| -74.011653| 40.702604| 1| -73.971851| 40.749807| 12.5| 0.0| 0.5| 2.6| 0.0| 15.6|{1, [[[-74.011653...|{1, [[[-73.971851...|{5, [[[-80.0, 35....|\n VTS|2009-11-26 15:38:00|2009-11-26 15:47:00| 5| 2.36| -73.997208| 40.722108| null| -73.99647| 40.748185| 8.1| 0.0| 0.5| 2.0| 0.0| 10.6|{1, [[[-73.997208...|{1, [[[-73.99647,...|{5, [[[-80.0, 35....|\n CMT|2009-12-07 10:36:06|2009-12-07 10:42:26| 1| 1.3| -73.99067| 40.73065| null| -73.975539| 40.741353| 6.1| 0.0| 0.5| 0.0| 0.0| 6.6|{1, [[[-73.99067,...|{1, [[[-73.975539...|{5, [[[-80.0, 35....|\n CMT|2009-11-15 09:11:42|2009-11-15 09:26:34| 1| 3.7| -73.990065| 40.687341| null| -73.999566| 40.730219| 12.1| 0.0| 0.5| 0.0| 0.0| 12.6|{1, [[[-73.990065...|{1, [[[-73.999566...|{5, [[[-80.0, 35....|\n CMT|2010-06-27 17:27:25|2010-06-27 17:47:44| 2| 5.0| -74.008459| 40.712233| 1| -73.9516| 40.718877| 15.7| 0.0| 0.5| 3.0| 0.0| 19.2|{1, [[[-74.008459...|{1, [[[-73.9516, ...|{5, [[[-80.0, 35....|\n VTS|2009-01-30 21:05:00|2009-01-30 21:34:00| 1| 5.99| -73.994397| 40.694843| null| -73.921257| 40.685107| 18.9| 0.5| null| 0.6| 0.0| 20.0|{1, [[[-73.994397...|{1, [[[-73.921257...|{5, [[[-80.0, 35....|\n CMT|2009-01-14 07:43:53|2009-01-14 07:55:14| 2| 2.9| -74.007602| 40.725098| null| -73.977846| 40.751608| 9.3| 0.0| null| 0.0| 0.0| 9.3|{1, [[[-74.007602...|{1, [[[-73.977846...|{5, [[[-80.0, 35....|\n VTS|2010-05-11 16:33:00|2010-05-11 16:41:00| 1| 0.99| -74.011205| 40.715468| 1| -74.011072| 40.708882| 5.7| 1.0| 0.5| 0.0| 0.0| 7.2|{1, [[[-74.011205...|{1, [[[-74.011072...|{5, [[[-80.0, 35....|\n VTS|2009-01-19 09:43:00|2009-01-19 09:47:00| 1| 0.56| -73.991123| 40.731297| null| -73.98681| 40.736703| 4.1| 0.0| null| 0.0| 0.0| 4.1|{1, [[[-73.991123...|{1, [[[-73.98681,...|{5, [[[-80.0, 35....|\n CMT|2009-11-14 15:46:36|2009-11-14 15:51:01| 2| 1.2| -74.008448| 40.734271| null| -74.006075| 40.720237| 4.9| 0.0| 0.5| 0.0| 0.0| 5.4|{1, [[[-74.008448...|{1, [[[-74.006075...|{5, [[[-80.0, 35....|\n CMT|2009-11-17 16:50:35|2009-11-17 17:01:34| 1| 1.6| -74.00679| 40.730546| null| -73.9904| 40.73815| 7.7| 1.0| 0.5| 1.5| 0.0| 10.7|{1, [[[-74.00679,...|{1, [[[-73.9904, ...|{5, [[[-80.0, 35....|\n CMT|2009-11-30 23:30:43|2009-11-30 23:42:54| 2| 2.9| -74.009605| 40.710155| null| -73.993875| 40.741554| 10.1| 0.5| 0.5| 2.22| 0.0| 13.32|{1, [[[-74.009605...|{1, [[[-73.993875...|{5, [[[-80.0, 35....|\n+---------+-------------------+-------------------+---------------+-------------+----------------+---------------+------------+-----------------+----------------+-----------+-----+-------+----------+------------+------------+--------------------+--------------------+--------------------+\nonly showing top 20 rows\n\n
" - ] - } - } - ], - "execution_count": 0 + "outputs": [], + "source": [ + "(\n", + " trips_mdf\n", + " .set_index_resolution(10)\n", + " .apply_index()\n", + " .join(\n", + " neighbourhoods_mdf\n", + " .set_index_resolution(10)\n", + " .apply_index()\n", + " )\n", + ").show()" + ] } ], "metadata": { "application/vnd.databricks.v1+notebook": { - "notebookName": "quickstart", "dashboards": [], + "language": "python", "notebookMetadata": { "pythonIndentUnit": 2 }, - "language": "python", - "widgets": {}, - "notebookOrigID": 1935524762305611 + "notebookName": "quickstart", + "notebookOrigID": 2666786534675592, + "widgets": {} } }, "nbformat": 4, "nbformat_minor": 0 -} \ No newline at end of file +} From 7be3fdd0119237ff7c5fbd80117bf8c965f57611 Mon Sep 17 00:00:00 2001 From: a0x8o Date: Mon, 1 May 2023 16:35:03 +0000 Subject: [PATCH 47/47] Sedona Raster example --- .../Mosaic + Sedona Raster 1.2.x Example.py | 733 ++++++++++++++++++ 1 file changed, 733 insertions(+) create mode 100644 notebooks/examples/python/Mosaic + Sedona Raster 1.2.x Example.py diff --git a/notebooks/examples/python/Mosaic + Sedona Raster 1.2.x Example.py b/notebooks/examples/python/Mosaic + Sedona Raster 1.2.x Example.py new file mode 100644 index 000000000..11dbafbaa --- /dev/null +++ b/notebooks/examples/python/Mosaic + Sedona Raster 1.2.x Example.py @@ -0,0 +1,733 @@ +# Databricks notebook source +# MAGIC %md # Mosaic + Apache Sedona Raster 1.2.x Example +# MAGIC +# MAGIC * Assumes DBR 10.4 which is Spark 3.2.1 + +# COMMAND ---------- + +# MAGIC %md _Example Cluster Config from demo shard (doesn't need to be single node)_ +# MAGIC ``` +# MAGIC { +# MAGIC "num_workers": 0, +# MAGIC "cluster_name": "mjohns-mosaic-sedona-sn", +# MAGIC "spark_version": "10.4.x-scala2.12", +# MAGIC "spark_conf": { +# MAGIC "spark.databricks.cluster.profile": "singleNode", +# MAGIC "spark.sql.extensions": "org.apache.sedona.viz.sql.SedonaVizExtensions,org.apache.sedona.sql.SedonaSqlExtensions", +# MAGIC "spark.master": "local[*, 4]", +# MAGIC "spark.kryo.registrator": "org.apache.sedona.core.serde.SedonaKryoRegistrator", +# MAGIC "spark.serializer": "org.apache.spark.serializer.KryoSerializer" +# MAGIC }, +# MAGIC "aws_attributes": { +# MAGIC "first_on_demand": 1, +# MAGIC "availability": "SPOT_WITH_FALLBACK", +# MAGIC "zone_id": "us-west-2c", +# MAGIC "spot_bid_price_percent": 100, +# MAGIC "ebs_volume_count": 0 +# MAGIC }, +# MAGIC "node_type_id": "i3.xlarge", +# MAGIC "driver_node_type_id": "i3.xlarge", +# MAGIC "ssh_public_keys": [], +# MAGIC "custom_tags": { +# MAGIC "ResourceClass": "SingleNode" +# MAGIC }, +# MAGIC "spark_env_vars": { +# MAGIC "PYSPARK_PYTHON": "/databricks/python3/bin/python3" +# MAGIC }, +# MAGIC "autotermination_minutes": 60, +# MAGIC "enable_elastic_disk": true, +# MAGIC "cluster_source": "UI", +# MAGIC "init_scripts": [ +# MAGIC { +# MAGIC "dbfs": { +# MAGIC "destination": "dbfs:/FileStore/sedona/sedona-1.2.1-init-master.sh" +# MAGIC } +# MAGIC } +# MAGIC ], +# MAGIC "runtime_engine": "STANDARD", +# MAGIC "cluster_id": "0704-014006-t9f0pvs3" +# MAGIC } +# MAGIC ``` + +# COMMAND ---------- + +# MAGIC %md _For DBR, install python bindings_ +# MAGIC +# MAGIC > Instead of `%pip install apache-sedona==1.2.0` using WHL built from Sedona master (1.2.1) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC +# MAGIC %pip install /dbfs/FileStore/jars/sedona/1.2.1-incubating/apache_sedona-1.2.1-py3-none-any.whl + +# COMMAND ---------- + +import pyspark.sql.functions as f +import mosaic as mos +from sedona.register.geo_registrator import SedonaRegistrator + +mos.enable_mosaic(spark, dbutils) # Enable Mosaic +SedonaRegistrator.registerAll(spark) # Register Sedona SQL functions + +# COMMAND ---------- + +from pyspark.sql.types import * +from pyspark.sql import functions as F +from pyspark.sql.functions import col +from sedona.register import SedonaRegistrator + +import os + +# COMMAND ---------- + +# MAGIC %md +# MAGIC ## Register Sedona functions and types + +# COMMAND ---------- + +SedonaRegistrator.registerAll(spark) + +# COMMAND ---------- + +# MAGIC %scala +# MAGIC // Test for various functions in the catalog +# MAGIC // - rs_add, rs_subtract, rs_divide, and rs_multiply are missing in the Maven 1.2.0 JAR but are in the 1.2.1 from Master. +# MAGIC // - e.g. of one available: "st_polygonfromtext" +# MAGIC println("rs_add",spark.catalog.functionExists("rs_add")) +# MAGIC println("rs_subtract",spark.catalog.functionExists("rs_subtract")) +# MAGIC println("rs_divide",spark.catalog.functionExists("rs_divide")) +# MAGIC println("rs_multiply",spark.catalog.functionExists("rs_multiply")) + +# COMMAND ---------- + +# MAGIC %md ## Setup Data (1x) + +# COMMAND ---------- + +# Path to directory of geotiff images +DATA_DIR = "/FileStore/geospatial/sedona-raster" +DATA_DIR_FUSE = "/dbfs" + DATA_DIR + +# COMMAND ---------- + +# MAGIC %sh +# MAGIC wget https://github.com/apache/incubator-sedona/blob/master/binder/data/raster/T21HUB_4704_4736_8224_8256.tif?raw=true -O /dbfs/FileStore/geospatial/sedona-raster/T21HUB_4704_4736_8224_8256.tif +# MAGIC wget https://github.com/apache/incubator-sedona/blob/master/binder/data/raster/vya_T21HUB_992_1024_4352_4384.tif?raw=true -O /dbfs/FileStore/geospatial/sedona-raster/vya_T21HUB_992_1024_4352_4384.tif +# MAGIC ls $PWD + +# COMMAND ---------- + +# MAGIC %sh ls /dbfs/FileStore/geospatial/sedona-raster/ + +# COMMAND ---------- + +# dbutils.fs.cp("file:/databricks/driver/T21HUB_4704_4736_8224_8256.tif",f"dbfs:{DATA_DIR}/T21HUB_4704_4736_8224_8256.tif") +# dbutils.fs.cp("file:/databricks/driver/vya_T21HUB_992_1024_4352_4384.tif",f"dbfs:{DATA_DIR}/vya_T21HUB_992_1024_4352_4384.tif") +display(dbutils.fs.ls(DATA_DIR)) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC # Geotiff Loader +# MAGIC +# MAGIC > __Note: if using 1.2.0 JAR (Maven) the spec uses `wkt` instead of `geometry` as name of column (1.2.1 JAR from Master uses `geometry`)__ +# MAGIC +# MAGIC 1. Loader takes as input a path to directory which contains geotiff files or a parth to particular geotiff file +# MAGIC 2. Loader will read geotiff image in a struct named image which contains multiple fields as shown in the schema below which can be extracted using spark SQL + +# COMMAND ---------- + +df_base = spark.read.format("geotiff").option("dropInvalid",True).option("readToCRS", "EPSG:4326").option("disableErrorInCRS", False).load(DATA_DIR) +df_base.printSchema() + +# COMMAND ---------- + +display(df_base.select("image.*").drop("data")) + +# COMMAND ---------- + +df_flat = df_base.selectExpr( + "image.origin as origin", + "ST_GeomFromWkt(image.geometry) as Geom", + "image.height as height", + "image.width as width", + "image.data as data", + "image.nBands as bands" +) +display(df_flat) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC # Extract a particular band from geotiff dataframe using RS_GetBand() + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > `RS_GetBand()` will fetch a particular band from given data array which is the concatination of all the bands + +# COMMAND ---------- + +df_band = ( + df_flat.selectExpr( + "Geom", + "RS_GetBand(data, 1,bands) as Band1", + "RS_GetBand(data, 2,bands) as Band2", + "RS_GetBand(data, 3,bands) as Band3", + "RS_GetBand(data, 4,bands) as Band4" + ) +) +df_band.createOrReplaceTempView("allbands") +display(df_band) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC # Map Algebra operations on band values + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > `RS_NormalizedDifference()` can be used to calculate NDVI for a particular geotiff image since it uses same computational formula as ndvi + +# COMMAND ---------- + +df_ND = df_band.selectExpr("RS_NormalizedDifference(Band1, Band2) as normDiff") +display(df_ND) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > `RS_Mean()` can be used to calculate mean of pixel values in a particular spatial band + +# COMMAND ---------- + +df_mean = df_band.selectExpr("RS_Mean(Band1) as mean") +display(df_mean) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > `RS_Mode()` is used to calculate mode in an array of pixels and returns a array of double with size 1 in case of unique mode + +# COMMAND ---------- + +df_mode = df_band.selectExpr("RS_Mode(Band1) as mode") +display(df_mode) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > `RS_GreaterThan()` is used to mask all the values with 1 which are greater than a particular threshold + +# COMMAND ---------- + +df_gt = spark.sql("Select RS_GreaterThan(Band1,1000.0) as greaterthan from allbands") +display(df_gt.limit(1)) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > `RS_GreaterThanEqual()` is used to mask all the values with 1 which are greater than a particular threshold + +# COMMAND ---------- + +df_gte = spark.sql("Select RS_GreaterThanEqual(Band1,360.0) as greaterthanEqual from allbands") +display(df_gte.limit(1)) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > `RS_LessThan()` is used to mask all the values with 1 which are less than a particular threshold + +# COMMAND ---------- + +df_lt = spark.sql("Select RS_LessThan(Band1,1000.0) as lessthan from allbands") +display(df_lt.limit(1)) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > `RS_LessThanEqual()` is used to mask all the values with 1 which are less than equal to a particular threshold + +# COMMAND ---------- + +df_lte = spark.sql("Select RS_LessThanEqual(Band1,2890.0) as lessthanequal from allbands") +display(df_lte.limit(1)) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > `RS_Add()` can add two spatial bands together -- __Not available in 1.2.0 from maven; IS available in 1.2.1 from master.__ + +# COMMAND ---------- + +df_sum = df_band.selectExpr("RS_Add(Band1, Band2) as sumOfBand") +display(df_sum.limit(1)) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > `RS_Subtract()` can subtract two spatial bands together -- __Not available in 1.2.0 from maven; IS available in 1.2.1 from master.__ + +# COMMAND ---------- + +df_subtract = df_band.selectExpr("RS_Subtract(Band1, Band2) as diffOfBand") +display(df_subtract.limit(1)) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > `RS_Multiply()` can multiple two bands together -- __Not available in 1.2.0 from maven; IS available in 1.2.1 from master.__ + +# COMMAND ---------- + +df_multiply = df_band.selectExpr("RS_Multiply(Band1, Band2) as productOfBand") +display(df_multiply.limit(1)) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > `RS_Divide()` can divide two bands together -- __Not available in 1.2.0 from maven; IS available in 1.2.1 from master.__ + +# COMMAND ---------- + +df_divide = df_band.selectExpr("RS_Divide(Band1, Band2) as divisionOfBand") +display(df_divide.limit(1)) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > `RS_MultiplyFactor()` will multiply a factor to a spatial band + +# COMMAND ---------- + +df_mulfac = df_band.selectExpr("RS_MultiplyFactor(Band2, 2) as target") +display(df_mulfac.limit(1)) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > `RS_BitwiseAND()` will return AND between two values of Bands + +# COMMAND ---------- + +df_bitwiseAND = df_band.selectExpr("RS_BitwiseAND(Band1, Band2) as AND") +display(df_bitwiseAND.limit(1)) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > `RS_BitwiseOR()` will return OR between two values of Bands + +# COMMAND ---------- + +df_bitwiseOR = df_band.selectExpr("RS_BitwiseOR(Band1, Band2) as OR") +display(df_bitwiseOR.limit(1)) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > `RS_Count()` will calculate the total number of occurence of a target value + +# COMMAND ---------- + +df_count = df_band.selectExpr("RS_Count(RS_GreaterThan(Band1,1000.0), 1.0) as count") +display(df_count) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > `RS_Modulo()` will calculate the modulus of band value with respect to a given number + +# COMMAND ---------- + +df_modulo = df_band.selectExpr("RS_Modulo(Band1, 21.0) as modulo ") +display(df_modulo.limit(1)) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > `RS_SquareRoot()` will calculate calculate square root of all the band values upto two decimal places + +# COMMAND ---------- + +df_root = df_band.selectExpr("RS_SquareRoot(Band1) as root") +display(df_root.limit(1)) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > `RS_LogicalDifference()` will return value from band1 if value at that particular location is not equal tp band1 else it will return 0 + +# COMMAND ---------- + +df_logDiff = df_band.selectExpr("RS_LogicalDifference(Band1, Band2) as loggDifference") +display(df_logDiff.limit(1)) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > `RS_LogicalOver()` will iterate over two bands and return value of first band if it is not equal to 0 else it will return value from later band + +# COMMAND ---------- + +df_logOver = df_band.selectExpr("RS_LogicalOver(Band3, Band2) as logicalOver") +display(df_logOver.limit(1)) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC ## Visualising Geotiff Images +# MAGIC +# MAGIC 1. Normalize the bands in range [0-255] if values are greater than 255 +# MAGIC 2. Process image using RS_Base64() which converts in into a base64 string +# MAGIC 3. Embedd results of RS_Base64() in RS_HTML() to embedd into IPython notebook +# MAGIC 4. Process results of RS_HTML() as below: + +# COMMAND ---------- + +# MAGIC %md ### Plotting images as a dataframe using geotiff Dataframe + +# COMMAND ---------- + +df_gtif = ( + spark + .read + .format("geotiff") + .option("dropInvalid",True) + .option("readToCRS", "EPSG:4326") + .load(DATA_DIR) + .selectExpr( + "image.origin as origin", + "ST_GeomFromWkt(image.geometry) as Geom", + "image.height as height", + "image.width as width", + "image.data as data", + "image.nBands as bands" + ) + .selectExpr( + "RS_GetBand(data,1,bands) as targetband", + "height", "width", "bands", "Geom" + ) +) + +# COMMAND ---------- + +df_base64 = df_gtif.selectExpr( + "Geom", + "RS_Base64(height,width,RS_Normalize(targetBand), RS_Array(height*width,0.0), RS_Array(height*width, 0.0)) as red", + "RS_Base64(height,width,RS_Array(height*width, 0.0), RS_Normalize(targetBand), RS_Array(height*width, 0.0)) as green", + "RS_Base64(height,width,RS_Array(height*width, 0.0), RS_Array(height*width, 0.0), RS_Normalize(targetBand)) as blue", + "RS_Base64(height,width,RS_Normalize(targetBand), RS_Normalize(targetBand),RS_Normalize(targetBand)) as RGB" +) + +display( + df_base64 + .selectExpr( + "Geom","RS_HTML(red) as RedBand", + "RS_HTML(blue) as BlueBand", + "RS_HTML(green) as GreenBand", + "RS_HTML(RGB) as CombinedBand" + ) +) + +# COMMAND ---------- + +# MAGIC %md > Using `displayHTML` to render in Databricks + +# COMMAND ---------- + +displayHTML( + df_base64.selectExpr( + "Geom","RS_HTML(red) as RedBand", + "RS_HTML(blue) as BlueBand", + "RS_HTML(green) as GreenBand", + "RS_HTML(RGB) as CombinedBand") + .limit(2) + .toPandas() + .to_html(escape=False) +) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC ## Writing GeoTiff Images + +# COMMAND ---------- + +user_str = dbutils.notebook.entry_point.getDbutils().notebook().getContext().tags().get("user").get() +print(f"user_str? '{user_str}'") +#print(f"""files --> {dbutils.fs.ls(f"/home/{user_str}")}""") + +# COMMAND ---------- + +SAVE_DIR = f"/home/{user_str}/geospatial/sedona/raster-out" +SAVE_DIR_FUSE = "/dbfs" + SAVE_DIR + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > Writing GeoTiff DataFrame as GeoTiff Images + +# COMMAND ---------- + +df_save = ( + spark.read.format("geotiff").option("dropInvalid",True).option("readToCRS", "EPSG:4326").load(DATA_DIR) + .selectExpr( + "image.origin as origin", + "ST_GeomFromWkt(image.geometry) as Geom", + "image.height as height", + "image.width as width", + "image.data as data", + "image.nBands as bands" + ) +) + +# COMMAND ---------- + +df_save.write.mode("overwrite").format("geotiff").option("writeToCRS", "EPSG:4326").option("fieldGeometry", "Geom").option("fieldNBands", "bands").save(SAVE_DIR) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > Writing GeoTiff Images in a Single Partition + +# COMMAND ---------- + +( + df_save.coalesce(1) + .write + .mode("overwrite") + .format("geotiff") + .option("writeToCRS", "EPSG:4326") + .option("fieldGeometry", "Geom") + .option("fieldNBands", "bands") + .save(SAVE_DIR) +) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > Find the Partition of the Written GeoTiff Images. If you did not write with coalesce(1), change the below code to adjust the writtenPath. + +# COMMAND ---------- + +import os +writtenPath = SAVE_DIR_FUSE +dirList = os.listdir(writtenPath) +for item in dirList: + if os.path.isdir(writtenPath + "/" + item): + writtenPath += "/" + item + break + +writtenPath = writtenPath[5:] # MLJ: Modify to remove the FUSE portion (aka '/dbfs') +display(dbutils.fs.ls(writtenPath)) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > Load and Visualize Written GeoTiff Image. + +# COMMAND ---------- + +df_load = ( + spark.read.format("geotiff").option("dropInvalid",True).option("readToCRS", "EPSG:4326").load(writtenPath) + .selectExpr( + "image.origin as origin", + "ST_GeomFromWkt(image.geometry) as Geom", + "image.height as height", "image.width as width", "image.data as data", "image.nBands as bands") + .selectExpr( + "RS_GetBand(data,1,bands) as targetband", + "height", "width", "bands", "Geom") +) + +# COMMAND ---------- + +df_base64 = ( + df_load.selectExpr( + "Geom", + "RS_Base64(height,width,RS_Normalize(targetBand), RS_Array(height*width,0.0), RS_Array(height*width, 0.0)) as red", + "RS_Base64(height,width,RS_Array(height*width, 0.0), RS_Normalize(targetBand), RS_Array(height*width, 0.0)) as green", + "RS_Base64(height,width,RS_Array(height*width, 0.0), RS_Array(height*width, 0.0), RS_Normalize(targetBand)) as blue", + "RS_Base64(height,width,RS_Normalize(targetBand), RS_Normalize(targetBand),RS_Normalize(targetBand)) as RGB" + ) + .selectExpr( + "Geom","RS_HTML(red) as RedBand","RS_HTML(blue) as BlueBand","RS_HTML(green) as GreenBand", "RS_HTML(RGB) as CombinedBand" + ) +) +displayHTML(df_base64.limit(2).toPandas().to_html(escape=False)) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC ## Transformation of GeoTiff Images + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > First load GeoTiff Images + +# COMMAND ---------- + +df_t = ( + spark + .read + .format("geotiff") + .option("dropInvalid",True) + .option("readToCRS", "EPSG:4326") + .option("disableErrorInCRS", False) + .load(DATA_DIR) + .selectExpr( + "image.origin as origin", + "ST_GeomFromWkt(image.geometry) as geom", + "image.height as height", + "image.width as width", + "image.data as data", + "image.nBands as bands" + ) +) +display(df_t) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC > First extract the bands for which normalized difference index needs to be calculated + +# COMMAND ---------- + +df_t1 = ( + df_t.selectExpr( + "origin", + "geom", + "width", + "height", + "data", + "bands", + "RS_GetBand(data, 1, bands) as band1", + "RS_GetBand(data, 2, bands) as band2" + ) +) + +# COMMAND ---------- + +# MAGIC %md +# MAGIC +# MAGIC > Get the normalized difference index between the extracted bands + +# COMMAND ---------- + +df_nd = ( + df_t1.selectExpr( + "origin", + "geom", + "width", + "height", + "data", + "bands", + "RS_NormalizedDifference(band2, band1) as normalizedDifference" + ) +) +display(df_nd) + +# COMMAND ---------- + +# MAGIC %md > `RS_Append()` takes the data array containing bands, a new band to be appended, and number of total bands in the data array. It appends the new band to the end of the data array and returns the appended data-- __Not available in 1.2.0 from maven; IS available in 1.2.1 from master.__ + +# COMMAND ---------- + +df_append = ( + df_nd.selectExpr( + "origin", "geom", + "RS_Append(data, normalizedDifference, bands) as data_edited", + "height", "width", "bands" + ) + .drop("data") + .withColumn("nBand_edited", col("bands") + 1) + .drop("bands") +) +display(df_append) + +# COMMAND ---------- + +# MAGIC %md _Could write GeoTiff DataFrame as GeoTiff image similar as before_ +# MAGIC +# MAGIC Example +# MAGIC +# MAGIC ``` +# MAGIC ( +# MAGIC df_append.coalesce(1) +# MAGIC .write.mode("overwrite").format("geotiff") +# MAGIC .option("writeToCRS", "EPSG:4326") +# MAGIC .option("fieldGeometry", "geom") +# MAGIC .option("fieldNBands", "nBand_edited") +# MAGIC .option("fieldData", "data_edited") +# MAGIC .save(SAVE_DIR) +# MAGIC ) +# MAGIC ``` + +# COMMAND ---------- + +# MAGIC %md +# MAGIC ## User can also create some UDF manually to manipulate Geotiff dataframes + +# COMMAND ---------- + +# MAGIC %md +# MAGIC +# MAGIC > Sample UDF calculates sum of all the values in a band which are greater than 1000.0 + +# COMMAND ---------- + +def SumOfValues(band): + total = 0.0 + for num in band: + if num>1000.0: + total+=1 + return total + +df_u1 = ( + spark.read.format("geotiff").option("dropInvalid",True).option("readToCRS", "EPSG:4326").load(DATA_DIR) + .selectExpr("image.origin as origin","ST_GeomFromWkt(image.geometry) as Geom", + "image.height as height", "image.width as width", "image.data as data", "image.nBands as bands") + .selectExpr("RS_GetBand(data,1,bands) as targetband", + "height", "width", "bands", "Geom") +) + +calculateSum = udf(SumOfValues, DoubleType()) +spark.udf.register("RS_Sum", calculateSum) + +df_sum = df_u1.selectExpr("RS_Sum(targetband) as sum") +display(df_sum) + +# COMMAND ---------- + +# MAGIC %md > Sample UDF to visualize a particular region of a GeoTiff image + +# COMMAND ---------- + +def generatemask(band, width,height): + for (i,val) in enumerate(band): + if (i%width>=12 and i%width<26) and (i%height>=12 and i%height<26): + band[i] = 255.0 + else: + band[i] = 0.0 + return band + +maskValues = udf(generatemask, F.ArrayType(DoubleType())) +spark.udf.register("RS_MaskValues", maskValues) + +df_u1b64 = ( + df_u1.selectExpr( + "Geom", + "RS_Base64(height,width,RS_Normalize(targetband), RS_Array(height*width,0.0), RS_Array(height*width, 0.0), RS_MaskValues(targetband,width,height)) as region" + ) + .selectExpr("Geom","RS_HTML(region) as selectedregion") +) + +displayHTML(df_u1b64.limit(2).toPandas().to_html(escape=False)) + +# COMMAND ---------- + +