Skip to content

Commit

Permalink
quick fix load_stac
Browse files Browse the repository at this point in the history
  • Loading branch information
bossie authored May 23, 2024
1 parent e087ce5 commit 2a1cd0d
Show file tree
Hide file tree
Showing 3 changed files with 109 additions and 7 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -988,7 +988,7 @@ class FileLayerProvider private(openSearch: OpenSearchClient, openSearchCollecti
var requiredSpacetimeKeys: RDD[(SpaceTimeKey, vector.Feature[Geometry, (RasterSource, Feature)])] = filteredSources.map(t => (SpaceTimeKey(t._1, TemporalKey(t._2.data._2.nominalDate.toLocalDate.atStartOfDay(ZoneId.of("UTC")))), t._2))

requiredSpacetimeKeys = applySpaceTimeMask(datacubeParams, requiredSpacetimeKeys,metadata)
if (isUTM) {
if (isUTM && !openSearch.isInstanceOf[FixedFeaturesOpenSearchClient]) {
//only for utm is just a safeguard to limit to Sentinel-1/2 for now
//try to resolve overlap before actually reading the data
requiredSpacetimeKeys = requiredSpacetimeKeys.groupByKey().flatMap(t => {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,107 @@
package org.openeo.geotrellis.file

import geotrellis.proj4.LatLng
import geotrellis.proj4.util.UTM
import geotrellis.raster.{CellSize, isData}
import geotrellis.spark._
import geotrellis.spark.util.SparkUtils
import geotrellis.vector.{Extent, MultiPolygon, ProjectedExtent}
import org.apache.spark.SparkContext
import org.junit.jupiter.api.Assertions.assertTrue
import org.junit.jupiter.api.{AfterAll, BeforeAll, Test}
import org.openeo.geotrellis.ProjectedPolygons
import org.openeo.geotrelliscommon.DataCubeParameters
import org.openeo.opensearch.OpenSearchResponses.{Feature, Link}

import java.net.URI
import java.time.ZonedDateTime
import java.util.Collections

import scala.collection.JavaConverters._

object LoadStacPyramidFactoryTest {
private var sc: SparkContext = _

@BeforeAll
def setupSpark(): Unit =
sc = SparkUtils.createLocalSparkContext("local[*]", appName = classOf[LoadStacPyramidFactoryTest].getName)

@AfterAll
def tearDownSpark(): Unit = sc.stop()
}

class LoadStacPyramidFactoryTest {

@Test
def testMissingDataInAdjacentTiles(): Unit = {
val boundingBox = ProjectedExtent(
Extent(11.1427023295687, 47.22033843316067, 11.821519349155245, 47.628952581107114), LatLng)

val bandNames = Seq("TCD")

val topFeature = Feature(
id = "TCD_2018_010m_E44N27_03035_v020",
bbox = Extent(11.064548187608006, 47.38783029804821, 12.36948893966052, 48.3083796083107),
nominalDate = ZonedDateTime.parse("2018-01-01T00:00:00+00:00"),
links = Array(Link(
URI.create("file:/data/projects/OpenEO/automated_test_files/load_stac_TCD_2018_010m_E44N27_03035_v020.tif"),
title = None,
bandNames = Some(bandNames),
)),
resolution = None,
)

val bottomFeature = Feature(
id = "TCD_2018_010m_E44N26_03035_v020",
bbox = Extent(11.046005504476401, 46.48802651961088, 12.329336972791394, 47.40858427433351),
nominalDate = ZonedDateTime.parse("2018-01-01T00:00:00+00:00"),
links = Array(Link(
URI.create("file:/data/projects/OpenEO/automated_test_files/load_stac_TCD_2018_010m_E44N26_03035_v020.tif"),
title = None,
bandNames = Some(bandNames),
)),
resolution = None,
)

val openSearchClient = new FixedFeaturesOpenSearchClient
openSearchClient.addFeature(topFeature)
openSearchClient.addFeature(bottomFeature)

val pyramidFactory = new PyramidFactory(
openSearchClient,
openSearchCollectionId = "doesnotmatter",
openSearchLinkTitles = bandNames.asJava,
rootPath = "/doesnotmatter",
maxSpatialResolution = CellSize(10, 10),
experimental = false,
)

val utmCrs = UTM.getZoneCrs(lon = boundingBox.extent.center.getX, lat = boundingBox.extent.center.getY)
val utmBoundingBox = ProjectedExtent(boundingBox.reproject(utmCrs), utmCrs)

val projectedPolygons = ProjectedPolygons(Array(MultiPolygon(utmBoundingBox.extent.toPolygon())), utmCrs)

val Seq((_, baseLayer)) = pyramidFactory.datacube_seq(
projectedPolygons,
from_date = "1970-01-01T00:00:00Z",
to_date = "2070-01-01T00:00:00Z",
metadata_properties = Collections.emptyMap(),
correlationId = "doesnotmatter",
new DataCubeParameters,
)

val spatialLayer = baseLayer
.withContext(_.mapValues(_.band(0)))
.toSpatial()
.crop(utmBoundingBox.extent)
.cache()

geotrellis.raster.io.geotiff.GeoTiff(spatialLayer.stitch(), spatialLayer.metadata.crs)
.write("/tmp/testMissingDataInAdjacentTiles.tif")

for {
tile <- spatialLayer.values
value <- tile
} assertTrue(isData(value))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -7,13 +7,12 @@ import geotrellis.raster.io.geotiff.GeoTiff
import geotrellis.raster.summary.polygonal.Summary
import geotrellis.raster.summary.polygonal.visitors.MeanVisitor
import geotrellis.raster.testkit.RasterMatchers
import geotrellis.raster.{CellSize, CellType, FloatConstantNoDataCellType, Raster, RasterSource, ShortConstantNoDataCellType, Tile, isNoData}
import geotrellis.raster.{CellSize, CellType, FloatConstantNoDataCellType, RasterSource, ShortConstantNoDataCellType, isNoData}
import geotrellis.spark._
import geotrellis.spark.partition.SpacePartitioner
import geotrellis.spark.summary.polygonal._
import geotrellis.spark.util.SparkUtils
import geotrellis.vector._
import org.apache.commons.io.FileUtils
import org.apache.spark.SparkContext
import org.apache.spark.rdd.RDD
import org.junit.jupiter.api.Assertions.{assertEquals, assertNotSame, assertSame, assertTrue}
Expand All @@ -23,7 +22,6 @@ import org.junit.jupiter.params.provider.ValueSource
import org.openeo.geotrellis.TestImplicits._
import org.openeo.geotrellis.layers.FileLayerProvider.rasterSourceRDD
import org.openeo.geotrellis.geotiff._
import org.openeo.geotrellis.netcdf.{NetCDFOptions, NetCDFRDDWriter}
import org.openeo.geotrellis.{LayerFixtures, ProjectedPolygons}
import org.openeo.geotrelliscommon.DatacubeSupport._
import org.openeo.geotrelliscommon.{ConfigurableSpaceTimePartitioner, DataCubeParameters, DatacubeSupport, NoCloudFilterStrategy, SpaceTimeByMonthPartitioner, SparseSpaceTimePartitioner}
Expand All @@ -32,13 +30,10 @@ import org.openeo.opensearch.backends.CreodiasClient
import org.openeo.opensearch.{OpenSearchClient, OpenSearchResponses}
import org.openeo.sparklisteners.GetInfoSparkListener

import java.io.File
import java.net.{URI, URL}
import java.nio.file.{Files, Paths}
import java.time.ZoneOffset.UTC
import java.time.format.DateTimeFormatter
import java.time.{LocalDate, ZoneId, ZonedDateTime}
import java.util.Collections
import java.util.concurrent.TimeUnit
import scala.collection.immutable
import scala.io.Source
Expand Down

0 comments on commit 2a1cd0d

Please sign in to comment.