, U, T>) pair -> new Tuple2<>(pair.getKey(), pair.getValue()));
}
+ /**
+ * Performs a KNN join where the query side is broadcasted.
+ *
+ * This function performs a K-Nearest Neighbors (KNN) join operation where the query geometries
+ * are broadcasted to all partitions of the object geometries.
+ *
+ *
The function first maps partitions of the indexed raw RDD to perform the KNN join, then
+ * groups the results by the query geometry and keeps the top K pair for each query geometry based
+ * on the distance.
+ *
+ * @param objectRDD The set of geometries (neighbors) to be queried.
+ * @param joinParams The parameters for the join, including index type, number of neighbors (k),
+ * and distance metric.
+ * @param judgement The judgement function used to perform the KNN join.
+ * @param The type of the geometries in the queryRDD set.
+ * @param The type of the geometries in the objectRDD set.
+ * @return A JavaRDD of pairs where each pair contains a geometry from the queryRDD and a matching
+ * geometry from the objectRDD.
+ */
+ private static
+ JavaRDD> querySideBroadcastKNNJoin(
+ SpatialRDD objectRDD,
+ JoinParams joinParams,
+ KnnJoinIndexJudgement judgement,
+ boolean includeTies) {
+ final JavaRDD> joinResult;
+ JavaRDD> joinResultMapped =
+ objectRDD.indexedRawRDD.mapPartitions(
+ iterator -> {
+ List> results = new ArrayList<>();
+ if (iterator.hasNext()) {
+ SpatialIndex spatialIndex = iterator.next();
+ // the broadcast join won't need inputs from the query's shape stream
+ Iterator> callResult =
+ judgement.call(null, Collections.singletonList(spatialIndex).iterator());
+ callResult.forEachRemaining(results::add);
+ }
+ return results.iterator();
+ });
+ // this is to avoid serializable issues with the broadcast variable
+ int k = joinParams.k;
+ DistanceMetric distanceMetric = joinParams.distanceMetric;
+
+ // Transform joinResultMapped to keep the top k pairs for each geometry
+ // (based on a grouping key and distance)
+ joinResult =
+ joinResultMapped
+ .groupBy(pair -> pair.getKey()) // Group by the first geometry
+ .flatMap(
+ (FlatMapFunction>>, Pair>)
+ pair -> {
+ Iterable> values = pair._2;
+
+ // Extract and sort values by distance
+ List> sortedPairs = new ArrayList<>();
+ for (Pair p : values) {
+ Pair newPair =
+ Pair.of(
+ (U) ((UniqueGeometry>) p.getKey()).getOriginalGeometry(),
+ p.getValue());
+ sortedPairs.add(newPair);
+ }
+
+ // Sort pairs based on the distance function between the two geometries
+ sortedPairs.sort(
+ (p1, p2) -> {
+ double distance1 =
+ KnnJoinIndexJudgement.distance(
+ p1.getKey(), p1.getValue(), distanceMetric);
+ double distance2 =
+ KnnJoinIndexJudgement.distance(
+ p2.getKey(), p2.getValue(), distanceMetric);
+ return Double.compare(
+ distance1, distance2); // Sort ascending by distance
+ });
+
+ if (includeTies) {
+ // Keep the top k pairs, including ties
+ List> topPairs = new ArrayList<>();
+ double kthDistance = -1;
+ for (int i = 0; i < sortedPairs.size(); i++) {
+ if (i < k) {
+ topPairs.add(sortedPairs.get(i));
+ if (i == k - 1) {
+ kthDistance =
+ KnnJoinIndexJudgement.distance(
+ sortedPairs.get(i).getKey(),
+ sortedPairs.get(i).getValue(),
+ distanceMetric);
+ }
+ } else {
+ double currentDistance =
+ KnnJoinIndexJudgement.distance(
+ sortedPairs.get(i).getKey(),
+ sortedPairs.get(i).getValue(),
+ distanceMetric);
+ if (currentDistance == kthDistance) {
+ topPairs.add(sortedPairs.get(i));
+ } else {
+ break;
+ }
+ }
+ }
+ return topPairs.iterator();
+ } else {
+ // Keep the top k pairs without ties
+ List> topPairs = new ArrayList<>();
+ for (int i = 0; i < Math.min(k, sortedPairs.size()); i++) {
+ topPairs.add(sortedPairs.get(i));
+ }
+ return topPairs.iterator();
+ }
+ });
+
+ return joinResult;
+ }
+
public static final class JoinParams {
public final boolean useIndex;
public final SpatialPredicate spatialPredicate;
diff --git a/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/EqualPartitioning.java b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/EqualPartitioning.java
index 94793fa68a..2d1b95eb75 100644
--- a/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/EqualPartitioning.java
+++ b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/EqualPartitioning.java
@@ -37,8 +37,19 @@ public class EqualPartitioning implements Serializable {
/** The grids. */
List grids = new ArrayList();
- public EqualPartitioning(List grids) {
+ /**
+ * Whether to discard geometries that do not intersect any grid. If true, geometries that are not
+ * contained in a grid are placed into the overflow container.
+ */
+ Boolean preserveUncontainedGeometries;
+
+ public EqualPartitioning(List grids, boolean preserveUncontainedGeometries) {
this.grids = grids;
+ this.preserveUncontainedGeometries = preserveUncontainedGeometries;
+ }
+
+ public EqualPartitioning(List grids) {
+ this(grids, true);
}
/**
* Instantiates a new equal partitioning.
@@ -100,12 +111,12 @@ public Iterator> placeObject(Geometry geometry) {
if (grid.covers(envelope)) {
result.add(new Tuple2(i, geometry));
containFlag = true;
- } else if (grid.intersects(envelope) || envelope.covers(grid)) {
+ } else if (grid.intersects(envelope)) {
result.add(new Tuple2<>(i, geometry));
}
}
- if (!containFlag) {
+ if (!containFlag && preserveUncontainedGeometries) {
result.add(new Tuple2<>(overflowContainerID, geometry));
}
@@ -133,7 +144,7 @@ public Set getKeys(Geometry geometry) {
}
}
- if (!containFlag) {
+ if (!containFlag && preserveUncontainedGeometries) {
result.add(overflowContainerID);
}
return result;
diff --git a/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/FlatGridPartitioner.java b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/FlatGridPartitioner.java
index e962a965ee..a50ce43f09 100644
--- a/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/FlatGridPartitioner.java
+++ b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/FlatGridPartitioner.java
@@ -27,19 +27,39 @@
import org.locationtech.jts.geom.Geometry;
import scala.Tuple2;
+/**
+ * The FlatGridPartitioner is used when there is already a set of grids which the data should be
+ * partitioned into. It iterates through all the grids to find the grids to place a geometry into.
+ * Unless you have very few objects to place, it may make more sense to use the
+ * IndexedGridPartitioner. If you do not have a strict requirement to use a specific set of grids,
+ * it may make more sense to use another partitioner that generates its own grids from a
+ * space-partitioning tree, e.g. the KDBTreePartitioner or the QuadTreePartitioner.
+ */
public class FlatGridPartitioner extends SpatialPartitioner {
- public FlatGridPartitioner(GridType gridType, List grids) {
+ protected final Boolean preserveUncontainedGeometries;
+
+ public FlatGridPartitioner(
+ GridType gridType, List grids, Boolean preserveUncontainedGeometries) {
super(gridType, grids);
+ this.preserveUncontainedGeometries = preserveUncontainedGeometries;
+ }
+
+ public FlatGridPartitioner(GridType gridType, List grids) {
+ this(gridType, grids, true);
+ }
+
+ public FlatGridPartitioner(List grids, Boolean preserveUncontainedGeometries) {
+ this(null, grids, preserveUncontainedGeometries);
}
// For backwards compatibility (see SpatialRDD.spatialPartitioning(otherGrids))
public FlatGridPartitioner(List grids) {
- super(null, grids);
+ this(null, grids);
}
@Override
public Iterator> placeObject(Geometry spatialObject) throws Exception {
- EqualPartitioning partitioning = new EqualPartitioning(grids);
+ EqualPartitioning partitioning = new EqualPartitioning(grids, preserveUncontainedGeometries);
return partitioning.placeObject(spatialObject);
}
@@ -61,7 +81,7 @@ public DedupParams getDedupParams() {
@Override
public int numPartitions() {
- return grids.size() + 1 /* overflow partition */;
+ return grids.size() + (preserveUncontainedGeometries ? 1 : 0);
}
@Override
diff --git a/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/GenericUniquePartitioner.java b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/GenericUniquePartitioner.java
new file mode 100644
index 0000000000..214446d6dd
--- /dev/null
+++ b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/GenericUniquePartitioner.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.sedona.core.spatialPartitioning;
+
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.sedona.core.enums.GridType;
+import org.apache.sedona.core.joinJudgement.DedupParams;
+import org.locationtech.jts.geom.Envelope;
+import org.locationtech.jts.geom.Geometry;
+import scala.Tuple2;
+
+public class GenericUniquePartitioner extends SpatialPartitioner {
+ private SpatialPartitioner parent;
+
+ public GenericUniquePartitioner(SpatialPartitioner parent) {
+ this.parent = parent;
+ }
+
+ public GridType getGridType() {
+ return parent.gridType;
+ }
+
+ public List getGrids() {
+ return parent.grids;
+ }
+
+ @Override
+ public Iterator> placeObject(Geometry spatialObject) throws Exception {
+ // Rather than take the first result from the parent, consume the entire iterator
+ // and return the partition with the minimum ID. This ensures that given the same
+ // (parent) partitioner, the output partitions from this method will be consistent.
+ Iterator> it = parent.placeObject(spatialObject);
+ int minParitionId = Integer.MAX_VALUE;
+ Geometry minGeometry = null;
+ while (it.hasNext()) {
+ Tuple2 value = it.next();
+ if (value._1() < minParitionId) {
+ minParitionId = value._1();
+ minGeometry = value._2();
+ }
+ }
+
+ HashSet> out = new HashSet>();
+ if (minGeometry != null) {
+ out.add(new Tuple2(minParitionId, minGeometry));
+ }
+
+ return out.iterator();
+ }
+
+ @Override
+ @Nullable
+ public DedupParams getDedupParams() {
+ throw new UnsupportedOperationException("Unique partitioner cannot deduplicate join results");
+ }
+
+ @Override
+ public int numPartitions() {
+ return parent.numPartitions();
+ }
+}
diff --git a/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/IndexedGridPartitioner.java b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/IndexedGridPartitioner.java
new file mode 100644
index 0000000000..ab8d5cde11
--- /dev/null
+++ b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/IndexedGridPartitioner.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.sedona.core.spatialPartitioning;
+
+import java.util.*;
+import org.apache.sedona.core.enums.GridType;
+import org.locationtech.jts.geom.Envelope;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.index.strtree.STRtree;
+import scala.Tuple2;
+
+/**
+ * The IndexedGridPartitioner is used when there is already a set of grids which the data should be
+ * partitioned into. It leverages an STRTree to quickly find the grids to place a geometry into. If
+ * you have very few objects to place, it may make more sense to use the FlatGridPartitioner. If you
+ * do not have a strict requirement to use a specific set of grids, it may make more sense to use
+ * another partitioner that generates its own grids from space-partitioning tree, e.g. the
+ * KDBTreePartitioner or the QuadTreePartitioner.
+ */
+public class IndexedGridPartitioner extends FlatGridPartitioner {
+ private final STRtree index;
+
+ public IndexedGridPartitioner(
+ GridType gridType, List grids, Boolean preserveUncontainedGeometries) {
+ super(gridType, grids, preserveUncontainedGeometries);
+ this.index = new STRtree();
+ for (int i = 0; i < grids.size(); i++) {
+ final Envelope grid = grids.get(i);
+ index.insert(grid, i);
+ }
+ index.build();
+ }
+
+ public IndexedGridPartitioner(GridType gridType, List grids) {
+ this(gridType, grids, true);
+ }
+
+ public IndexedGridPartitioner(List grids, Boolean preserveUncontainedGeometries) {
+ this(null, grids, preserveUncontainedGeometries);
+ }
+
+ public IndexedGridPartitioner(List grids) {
+ this(null, grids);
+ }
+
+ public STRtree getIndex() {
+ return index;
+ }
+
+ @Override
+ public Iterator> placeObject(Geometry spatialObject) throws Exception {
+ List results = index.query(spatialObject.getEnvelopeInternal());
+ if (preserveUncontainedGeometries) {
+ // borrowed from EqualPartitioning.placeObject
+ final int overflowContainerID = grids.size();
+ final Envelope envelope = spatialObject.getEnvelopeInternal();
+
+ Set> result = new HashSet();
+ boolean containFlag = false;
+ for (Object queryResult : results) {
+ Integer i = (Integer) queryResult;
+ final Envelope grid = grids.get(i);
+ if (grid.covers(envelope)) {
+ result.add(new Tuple2(i, spatialObject));
+ containFlag = true;
+ } else if (grid.intersects(envelope)) {
+ result.add(new Tuple2<>(i, spatialObject));
+ }
+ }
+
+ if (!containFlag) {
+ result.add(new Tuple2<>(overflowContainerID, spatialObject));
+ }
+
+ return result.iterator();
+ } else {
+ return results.stream().map(i -> new Tuple2(i, spatialObject)).iterator();
+ }
+ }
+}
diff --git a/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/KDB.java b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/KDB.java
index be3ba72cc3..4bd615383a 100644
--- a/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/KDB.java
+++ b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/KDB.java
@@ -33,7 +33,10 @@
import org.locationtech.jts.geom.Point;
import scala.Tuple2;
-/** see https://en.wikipedia.org/wiki/K-D-B-tree */
+/**
+ * see https://en.wikipedia.org/wiki/K-D-B-tree
+ */
public class KDB extends PartitioningUtils implements Serializable {
private final int maxItemsPerNode;
diff --git a/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/SpatialPartitioner.java b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/SpatialPartitioner.java
index c7deb3b704..96594c5c10 100644
--- a/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/SpatialPartitioner.java
+++ b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/SpatialPartitioner.java
@@ -35,6 +35,11 @@ public abstract class SpatialPartitioner extends Partitioner implements Serializ
protected final GridType gridType;
protected final List grids;
+ protected SpatialPartitioner() {
+ gridType = null;
+ grids = null;
+ }
+
protected SpatialPartitioner(GridType gridType, List grids) {
this.gridType = gridType;
this.grids = Objects.requireNonNull(grids, "grids");
diff --git a/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/quadtree/ExtendedQuadTree.java b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/quadtree/ExtendedQuadTree.java
index 73169363ee..9925e93f8a 100644
--- a/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/quadtree/ExtendedQuadTree.java
+++ b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/quadtree/ExtendedQuadTree.java
@@ -146,8 +146,11 @@ public Iterator> placeObject(Geometry geometry) {
final Set> result = new HashSet<>();
for (QuadRectangle rectangle : matchedPartitions) {
+ // Ignore null or empty point
+ if (point == null || point.isEmpty()) break;
+
// For points, make sure to return only one partition
- if (point != null && !(new HalfOpenRectangle(rectangle.getEnvelope())).contains(point)) {
+ if (!(new HalfOpenRectangle(rectangle.getEnvelope())).contains(point)) {
continue;
}
diff --git a/spark/common/src/main/java/org/apache/sedona/core/spatialRDD/SpatialRDD.java b/spark/common/src/main/java/org/apache/sedona/core/spatialRDD/SpatialRDD.java
index d81b916183..b8b46ae35e 100644
--- a/spark/common/src/main/java/org/apache/sedona/core/spatialRDD/SpatialRDD.java
+++ b/spark/common/src/main/java/org/apache/sedona/core/spatialRDD/SpatialRDD.java
@@ -42,6 +42,7 @@
import org.apache.spark.api.java.function.Function;
import org.apache.spark.api.java.function.Function2;
import org.apache.spark.api.java.function.PairFlatMapFunction;
+import org.apache.spark.sql.types.StructType;
import org.apache.spark.storage.StorageLevel;
import org.apache.spark.util.random.SamplingUtils;
import org.locationtech.jts.geom.Coordinate;
@@ -85,6 +86,8 @@ public class SpatialRDD implements Serializable {
public JavaRDD rawSpatialRDD;
public List fieldNames;
+
+ public StructType schema;
/** The CR stransformation. */
protected boolean CRStransformation = false;
/** The source epsg code. */
@@ -159,11 +162,71 @@ public boolean spatialPartitioning(GridType gridType) throws Exception {
return true;
}
+ public boolean spatialParitioningWithoutDuplicates(GridType gridType) throws Exception {
+ int numPartitions = this.rawSpatialRDD.rdd().partitions().length;
+ spatialPartitioningWithoutDuplicates(gridType, numPartitions);
+ return true;
+ }
+
+ /**
+ * Calculate non-duplicate inducing partitioning
+ *
+ * Note that non-duplicating partitioners are intended for use by distributed partitioned
+ * writers and not able to be used for spatial joins.
+ *
+ * @param gridType The target GridType
+ * @param numPartitions The target number of partitions
+ * @throws Exception
+ */
+ public void spatialPartitioningWithoutDuplicates(GridType gridType, int numPartitions)
+ throws Exception {
+ calc_partitioner(gridType, numPartitions);
+ partitioner = new GenericUniquePartitioner(partitioner);
+ this.spatialPartitionedRDD = partition(partitioner);
+ }
+
+ /**
+ * Calculate non-duplicate inducing partitioning from an existing SpatialPartitioner
+ *
+ *
Note that non-duplicating partitioners are intended for use by distributed partitioned
+ * writers and not able to be used for spatial joins.
+ *
+ * @param partitioner An existing partitioner obtained from the partitioning of another
+ * SpatialRDD.
+ * @throws Exception
+ */
+ public void spatialPartitioningWithoutDuplicates(SpatialPartitioner partitioner) {
+ partitioner = new GenericUniquePartitioner(partitioner);
+ this.spatialPartitionedRDD = partition(partitioner);
+ }
+
+ /**
+ * Calculate non-duplicate inducing partitioning based on a list of existing envelopes
+ *
+ *
This is shorthand for spatialPartitioningWithoutDuplicates(new IndexedGridPartitioner()).
+ * Using spatialPartitioningWithoutDuplicates(gridType, numPartitions) is typically more
+ * appropriate because it is able to adapt to the content of the partition and is able to produce
+ * more consistently balanced partitions.
+ *
+ *
Note that non-duplicating partitioners are intended for use by distributed partitioned
+ * writers and not able to be used for spatial joins.
+ *
+ * @param otherGrids A list of existing envelopes
+ * @return true on success
+ * @throws Exception
+ */
+ public boolean spatialPartitioningWithoutDuplicates(final List otherGrids)
+ throws Exception {
+ this.partitioner = new GenericUniquePartitioner(new IndexedGridPartitioner(otherGrids));
+ this.spatialPartitionedRDD = partition(partitioner);
+ return true;
+ }
+
/**
* Spatial partitioning.
*
* @param gridType the grid type
- * @return true, if successful
+ * @param numPartitions the target number of partitions
* @throws Exception the exception
*/
public void calc_partitioner(GridType gridType, int numPartitions) throws Exception {
@@ -278,7 +341,7 @@ public void spatialPartitioning(SpatialPartitioner partitioner) {
/** @deprecated Use spatialPartitioning(SpatialPartitioner partitioner) */
public boolean spatialPartitioning(final List otherGrids) throws Exception {
- this.partitioner = new FlatGridPartitioner(otherGrids);
+ this.partitioner = new IndexedGridPartitioner(otherGrids);
this.spatialPartitionedRDD = partition(partitioner);
return true;
}
diff --git a/spark/common/src/main/java/org/apache/sedona/core/utils/SedonaConf.java b/spark/common/src/main/java/org/apache/sedona/core/utils/SedonaConf.java
index 28685c6a03..d02e96df93 100644
--- a/spark/common/src/main/java/org/apache/sedona/core/utils/SedonaConf.java
+++ b/spark/common/src/main/java/org/apache/sedona/core/utils/SedonaConf.java
@@ -59,6 +59,9 @@ public class SedonaConf implements Serializable {
// Parameters for knn joins
private boolean includeTieBreakersInKNNJoins = false;
+ // Parameters for geostats
+ private Boolean DBSCANIncludeOutliers = true;
+
public static SedonaConf fromActiveSession() {
return new SedonaConf(SparkSession.active().conf());
}
@@ -98,6 +101,10 @@ public SedonaConf(RuntimeConfig runtimeConfig) {
// Parameters for knn joins
this.includeTieBreakersInKNNJoins =
Boolean.parseBoolean(getConfigValue(runtimeConfig, "join.knn.includeTieBreakers", "false"));
+
+ // Parameters for geostats
+ this.DBSCANIncludeOutliers =
+ Boolean.parseBoolean(runtimeConfig.get("spark.sedona.dbscan.includeOutliers", "true"));
}
// Helper method to prioritize `sedona.*` over `spark.sedona.*`
@@ -182,4 +189,8 @@ static long bytesFromString(String str) {
public SpatialJoinOptimizationMode getSpatialJoinOptimizationMode() {
return spatialJoinOptimizationMode;
}
+
+ public Boolean getDBSCANIncludeOutliers() {
+ return DBSCANIncludeOutliers;
+ }
}
diff --git a/spark/common/src/main/java/org/apache/sedona/core/wrapper/UniqueGeometry.java b/spark/common/src/main/java/org/apache/sedona/core/wrapper/UniqueGeometry.java
new file mode 100644
index 0000000000..01f20f2fa6
--- /dev/null
+++ b/spark/common/src/main/java/org/apache/sedona/core/wrapper/UniqueGeometry.java
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.sedona.core.wrapper;
+
+import java.util.UUID;
+import org.apache.commons.lang3.NotImplementedException;
+import org.locationtech.jts.geom.*;
+
+public class UniqueGeometry extends Geometry {
+ private final T originalGeometry;
+ private final String uniqueId;
+
+ public UniqueGeometry(T originalGeometry) {
+ super(new GeometryFactory());
+ this.originalGeometry = originalGeometry;
+ this.uniqueId = UUID.randomUUID().toString();
+ }
+
+ public T getOriginalGeometry() {
+ return originalGeometry;
+ }
+
+ public String getUniqueId() {
+ return uniqueId;
+ }
+
+ @Override
+ public int hashCode() {
+ return uniqueId.hashCode(); // Uniqueness ensured by uniqueId
+ }
+
+ @Override
+ public String getGeometryType() {
+ throw new NotImplementedException("getGeometryType is not implemented.");
+ }
+
+ @Override
+ public Coordinate getCoordinate() {
+ throw new NotImplementedException("getCoordinate is not implemented.");
+ }
+
+ @Override
+ public Coordinate[] getCoordinates() {
+ throw new NotImplementedException("getCoordinates is not implemented.");
+ }
+
+ @Override
+ public int getNumPoints() {
+ throw new NotImplementedException("getNumPoints is not implemented.");
+ }
+
+ @Override
+ public boolean isEmpty() {
+ throw new NotImplementedException("isEmpty is not implemented.");
+ }
+
+ @Override
+ public int getDimension() {
+ throw new NotImplementedException("getDimension is not implemented.");
+ }
+
+ @Override
+ public Geometry getBoundary() {
+ throw new NotImplementedException("getBoundary is not implemented.");
+ }
+
+ @Override
+ public int getBoundaryDimension() {
+ throw new NotImplementedException("getBoundaryDimension is not implemented.");
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj) return true;
+ if (obj == null || getClass() != obj.getClass()) return false;
+ UniqueGeometry> that = (UniqueGeometry>) obj;
+ return uniqueId.equals(that.uniqueId);
+ }
+
+ @Override
+ public String toString() {
+ return "UniqueGeometry{"
+ + "originalGeometry="
+ + originalGeometry
+ + ", uniqueId='"
+ + uniqueId
+ + '\''
+ + '}';
+ }
+
+ @Override
+ protected Geometry reverseInternal() {
+ throw new NotImplementedException("reverseInternal is not implemented.");
+ }
+
+ @Override
+ public boolean equalsExact(Geometry geometry, double v) {
+ throw new NotImplementedException("equalsExact is not implemented.");
+ }
+
+ @Override
+ public void apply(CoordinateFilter coordinateFilter) {
+ throw new NotImplementedException("apply(CoordinateFilter) is not implemented.");
+ }
+
+ @Override
+ public void apply(CoordinateSequenceFilter coordinateSequenceFilter) {
+ throw new NotImplementedException("apply(CoordinateSequenceFilter) is not implemented.");
+ }
+
+ @Override
+ public void apply(GeometryFilter geometryFilter) {
+ throw new NotImplementedException("apply(GeometryFilter) is not implemented.");
+ }
+
+ @Override
+ public void apply(GeometryComponentFilter geometryComponentFilter) {
+ throw new NotImplementedException("apply(GeometryComponentFilter) is not implemented.");
+ }
+
+ @Override
+ protected Geometry copyInternal() {
+ throw new NotImplementedException("copyInternal is not implemented.");
+ }
+
+ @Override
+ public void normalize() {
+ throw new NotImplementedException("normalize is not implemented.");
+ }
+
+ @Override
+ protected Envelope computeEnvelopeInternal() {
+ throw new NotImplementedException("computeEnvelopeInternal is not implemented.");
+ }
+
+ @Override
+ protected int compareToSameClass(Object o) {
+ throw new NotImplementedException("compareToSameClass(Object) is not implemented.");
+ }
+
+ @Override
+ protected int compareToSameClass(
+ Object o, CoordinateSequenceComparator coordinateSequenceComparator) {
+ throw new NotImplementedException(
+ "compareToSameClass(Object, CoordinateSequenceComparator) is not implemented.");
+ }
+
+ @Override
+ protected int getTypeCode() {
+ throw new NotImplementedException("getTypeCode is not implemented.");
+ }
+}
diff --git a/spark/common/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/spark/common/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
index b664d1db50..162e44369f 100644
--- a/spark/common/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
+++ b/spark/common/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
@@ -1,3 +1,4 @@
org.apache.spark.sql.sedona_sql.io.raster.RasterFileFormat
org.apache.spark.sql.sedona_sql.io.geojson.GeoJSONFileFormat
org.apache.sedona.sql.datasources.spider.SpiderDataSource
+org.apache.spark.sql.sedona_sql.io.stac.StacDataSource
diff --git a/spark/common/src/main/scala/org/apache/sedona/spark/SedonaContext.scala b/spark/common/src/main/scala/org/apache/sedona/spark/SedonaContext.scala
index 4c8fcab692..fe2926fc51 100644
--- a/spark/common/src/main/scala/org/apache/sedona/spark/SedonaContext.scala
+++ b/spark/common/src/main/scala/org/apache/sedona/spark/SedonaContext.scala
@@ -20,11 +20,12 @@ package org.apache.sedona.spark
import org.apache.sedona.common.utils.TelemetryCollector
import org.apache.sedona.core.serde.SedonaKryoRegistrator
-import org.apache.sedona.sql.{ParserRegistrator, RasterRegistrator}
+import org.apache.sedona.sql.RasterRegistrator
import org.apache.sedona.sql.UDF.UdfRegistrator
import org.apache.sedona.sql.UDT.UdtRegistrator
import org.apache.spark.serializer.KryoSerializer
-import org.apache.spark.sql.sedona_sql.optimization.SpatialFilterPushDownForGeoParquet
+import org.apache.spark.sql.sedona_sql.optimization.{ExtractGeoStatsFunctions, SpatialFilterPushDownForGeoParquet, SpatialTemporalFilterPushDownForStacScan}
+import org.apache.spark.sql.sedona_sql.strategy.geostats.EvalGeoStatsFunctionStrategy
import org.apache.spark.sql.sedona_sql.strategy.join.JoinQueryDetector
import org.apache.spark.sql.{SQLContext, SparkSession}
@@ -36,6 +37,12 @@ class InternalApi(
extends StaticAnnotation
object SedonaContext {
+
+ private def customOptimizationsWithSession(sparkSession: SparkSession) =
+ Seq(
+ new SpatialFilterPushDownForGeoParquet(sparkSession),
+ new SpatialTemporalFilterPushDownForStacScan(sparkSession))
+
def create(sqlContext: SQLContext): SQLContext = {
create(sqlContext.sparkSession)
sqlContext
@@ -56,18 +63,30 @@ object SedonaContext {
if (!sparkSession.experimental.extraStrategies.exists(_.isInstanceOf[JoinQueryDetector])) {
sparkSession.experimental.extraStrategies ++= Seq(new JoinQueryDetector(sparkSession))
}
- if (!sparkSession.experimental.extraOptimizations.exists(
- _.isInstanceOf[SpatialFilterPushDownForGeoParquet])) {
- sparkSession.experimental.extraOptimizations ++= Seq(
- new SpatialFilterPushDownForGeoParquet(sparkSession))
+
+ customOptimizationsWithSession(sparkSession).foreach { opt =>
+ if (!sparkSession.experimental.extraOptimizations.exists {
+ case _: opt.type => true
+ case _ => false
+ }) {
+ sparkSession.experimental.extraOptimizations ++= Seq(opt)
+ }
+ }
+
+ // Support geostats functions
+ if (!sparkSession.experimental.extraOptimizations.contains(ExtractGeoStatsFunctions)) {
+ sparkSession.experimental.extraOptimizations ++= Seq(ExtractGeoStatsFunctions)
+ }
+ if (!sparkSession.experimental.extraStrategies.exists(
+ _.isInstanceOf[EvalGeoStatsFunctionStrategy])) {
+ sparkSession.experimental.extraStrategies ++= Seq(
+ new EvalGeoStatsFunctionStrategy(sparkSession))
}
+
addGeoParquetToSupportNestedFilterSources(sparkSession)
RasterRegistrator.registerAll(sparkSession)
UdtRegistrator.registerAll()
UdfRegistrator.registerAll(sparkSession)
- if (sparkSession.conf.get("spark.sedona.enableParserExtension", "true").toBoolean) {
- ParserRegistrator.register(sparkSession)
- }
sparkSession
}
diff --git a/spark/common/src/main/scala/org/apache/sedona/sql/ParserRegistrator.scala b/spark/common/src/main/scala/org/apache/sedona/sql/ParserRegistrator.scala
deleted file mode 100644
index db3c623a09..0000000000
--- a/spark/common/src/main/scala/org/apache/sedona/sql/ParserRegistrator.scala
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.sedona.sql
-
-import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.catalyst.parser.ParserInterface
-import org.apache.spark.sql.parser.ParserFactory
-
-object ParserRegistrator {
-
- /**
- * Register the custom Sedona Spark parser
- * @param sparkSession
- */
- def register(sparkSession: SparkSession): Unit = {
- // try to register the parser with the new constructor for spark 3.1 and above
- try {
- val parserClassName = "org.apache.sedona.sql.parser.SedonaSqlParser"
- val delegate: ParserInterface = sparkSession.sessionState.sqlParser
-
- val parser = ParserFactory.getParser(parserClassName, delegate)
- val field = sparkSession.sessionState.getClass.getDeclaredField("sqlParser")
- field.setAccessible(true)
- field.set(sparkSession.sessionState, parser)
- return // return if the new constructor is available
- } catch {
- case _: Exception =>
- }
-
- // try to register the parser with the legacy constructor for spark 3.0
- try {
- val parserClassName = "org.apache.sedona.sql.parser.SedonaSqlParser"
- val delegate: ParserInterface = sparkSession.sessionState.sqlParser
-
- val parser =
- ParserFactory.getParser(parserClassName, sparkSession.sessionState.conf, delegate)
- val field = sparkSession.sessionState.getClass.getDeclaredField("sqlParser")
- field.setAccessible(true)
- field.set(sparkSession.sessionState, parser)
- } catch {
- case _: Exception =>
- }
- }
-}
diff --git a/spark/common/src/main/scala/org/apache/sedona/sql/SedonaSqlExtensions.scala b/spark/common/src/main/scala/org/apache/sedona/sql/SedonaSqlExtensions.scala
index be0774ac90..fbc3567192 100644
--- a/spark/common/src/main/scala/org/apache/sedona/sql/SedonaSqlExtensions.scala
+++ b/spark/common/src/main/scala/org/apache/sedona/sql/SedonaSqlExtensions.scala
@@ -19,13 +19,24 @@
package org.apache.sedona.sql
import org.apache.sedona.spark.SedonaContext
+import org.apache.spark.SparkContext
import org.apache.spark.sql.SparkSessionExtensions
+import org.apache.spark.sql.parser.ParserFactory
class SedonaSqlExtensions extends (SparkSessionExtensions => Unit) {
+ private lazy val enableParser =
+ SparkContext.getOrCreate().getConf.get("spark.sedona.enableParserExtension", "true").toBoolean
+
def apply(e: SparkSessionExtensions): Unit = {
e.injectCheckRule(spark => {
SedonaContext.create(spark)
_ => ()
})
+
+ if (enableParser) {
+ e.injectParser { case (_, parser) =>
+ ParserFactory.getParser("org.apache.sedona.sql.parser.SedonaSqlParser", parser)
+ }
+ }
}
}
diff --git a/spark/common/src/main/scala/org/apache/sedona/sql/UDF/Catalog.scala b/spark/common/src/main/scala/org/apache/sedona/sql/UDF/Catalog.scala
index b491375379..0bffa54baf 100644
--- a/spark/common/src/main/scala/org/apache/sedona/sql/UDF/Catalog.scala
+++ b/spark/common/src/main/scala/org/apache/sedona/sql/UDF/Catalog.scala
@@ -38,6 +38,7 @@ object Catalog {
val expressions: Seq[FunctionDescription] = Seq(
// Expression for vectors
function[GeometryType](),
+ function[ST_LabelPoint](),
function[ST_PointFromText](),
function[ST_PointFromWKB](),
function[ST_LineFromWKB](),
@@ -56,6 +57,7 @@ object Catalog {
function[ST_GeomFromKML](),
function[ST_CoordDim](),
function[ST_Perimeter](),
+ function[ST_Perimeter2D](),
function[ST_Point](),
function[ST_Points](),
function[ST_MakeEnvelope](),
@@ -168,6 +170,7 @@ object Catalog {
function[ST_IsPolygonCCW](),
function[ST_ForcePolygonCCW](),
function[ST_FlipCoordinates](),
+ function[ST_LineSegments](),
function[ST_LineSubstring](),
function[ST_LineInterpolatePoint](),
function[ST_LineLocatePoint](),
@@ -338,7 +341,13 @@ object Catalog {
function[RS_Resample](),
function[RS_ReprojectMatch]("nearestneighbor"),
function[RS_FromNetCDF](),
- function[RS_NetCDFInfo]())
+ function[RS_NetCDFInfo](),
+ // geostats functions
+ function[ST_DBSCAN](),
+ function[ST_LocalOutlierFactor](),
+ function[ST_GLocal](),
+ function[ST_BinaryDistanceBandColumn](),
+ function[ST_WeightedDistanceBandColumn]())
// Aggregate functions with Geometry as buffer
val aggregateExpressions: Seq[Aggregator[Geometry, Geometry, Geometry]] =
diff --git a/spark/common/src/main/scala/org/apache/sedona/sql/utils/Adapter.scala b/spark/common/src/main/scala/org/apache/sedona/sql/utils/Adapter.scala
index 9b1067a25a..96aab1287e 100644
--- a/spark/common/src/main/scala/org/apache/sedona/sql/utils/Adapter.scala
+++ b/spark/common/src/main/scala/org/apache/sedona/sql/utils/Adapter.scala
@@ -29,6 +29,13 @@ import org.apache.spark.sql.{DataFrame, Row, SparkSession}
import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema
import org.locationtech.jts.geom.Geometry
+/**
+ * Adapter for converting between DataFrame and SpatialRDD. It provides methods to convert
+ * DataFrame to SpatialRDD and vice versa. The schema information is lost during conversion. It is
+ * different from [[org.apache.spark.sql.sedona_sql.adapters.StructuredAdapter]] which does not
+ * lose the schema information during conversion. This should be used if your data starts as a
+ * SpatialRDD and you want to convert it to DataFrame.
+ */
object Adapter {
/**
diff --git a/spark/common/src/main/scala/org/apache/sedona/sql/utils/GeometrySerializer.scala b/spark/common/src/main/scala/org/apache/sedona/sql/utils/GeometrySerializer.scala
index a13c181aa2..a75a88f7ba 100644
--- a/spark/common/src/main/scala/org/apache/sedona/sql/utils/GeometrySerializer.scala
+++ b/spark/common/src/main/scala/org/apache/sedona/sql/utils/GeometrySerializer.scala
@@ -19,7 +19,7 @@
package org.apache.sedona.sql.utils
import org.apache.sedona.common.geometrySerde
-import org.locationtech.jts.geom.Geometry
+import org.locationtech.jts.geom.{Geometry, GeometryFactory}
/**
* SerDe using the WKB reader and writer objects
@@ -47,6 +47,9 @@ object GeometrySerializer {
* JTS geometry
*/
def deserialize(value: Array[Byte]): Geometry = {
+ if (value == null) {
+ return new GeometryFactory().createGeometryCollection()
+ }
geometrySerde.GeometrySerializer.deserialize(value)
}
}
diff --git a/spark/common/src/main/scala/org/apache/sedona/stats/Weighting.scala b/spark/common/src/main/scala/org/apache/sedona/stats/Weighting.scala
index 6d5a273854..7713674261 100644
--- a/spark/common/src/main/scala/org/apache/sedona/stats/Weighting.scala
+++ b/spark/common/src/main/scala/org/apache/sedona/stats/Weighting.scala
@@ -18,7 +18,7 @@
*/
package org.apache.sedona.stats
-import org.apache.sedona.stats.Util.getGeometryColumnName
+import org.apache.sedona.util.DfUtils.getGeometryColumnName
import org.apache.spark.sql.functions._
import org.apache.spark.sql.sedona_sql.expressions.st_functions.{ST_Distance, ST_DistanceSpheroid}
import org.apache.spark.sql.{Column, DataFrame}
@@ -54,6 +54,10 @@ object Weighting {
* name of the geometry column
* @param useSpheroid
* whether to use a cartesian or spheroidal distance calculation. Default is false
+ * @param savedAttributes
+ * the attributes to save in the neighbor column. Default is all columns.
+ * @param resultName
+ * the name of the resulting column. Default is 'weights'.
* @return
* The input DataFrame with a weight column added containing neighbors and their weights added
* to each row.
@@ -67,13 +71,15 @@ object Weighting {
includeSelf: Boolean = false,
selfWeight: Double = 1.0,
geometry: String = null,
- useSpheroid: Boolean = false): DataFrame = {
+ useSpheroid: Boolean = false,
+ savedAttributes: Seq[String] = null,
+ resultName: String = "weights"): DataFrame = {
require(threshold >= 0, "Threshold must be greater than or equal to 0")
require(alpha < 0, "Alpha must be less than 0")
val geometryColumn = geometry match {
- case null => getGeometryColumnName(dataframe)
+ case null => getGeometryColumnName(dataframe.schema)
case _ =>
require(
dataframe.schema.fields.exists(_.name == geometry),
@@ -81,6 +87,12 @@ object Weighting {
geometry
}
+ // Always include the geometry column in the saved attributes
+ val savedAttributesWithGeom =
+ if (savedAttributes == null) null
+ else if (!savedAttributes.contains(geometryColumn)) savedAttributes :+ geometryColumn
+ else savedAttributes
+
val distanceFunction: (Column, Column) => Column =
if (useSpheroid) ST_DistanceSpheroid else ST_Distance
@@ -96,14 +108,6 @@ object Weighting {
val formattedDataFrame = dataframe.withColumn(ID_COLUMN, sha2(to_json(struct("*")), 256))
- // Since spark 3.0 doesn't support dropFields, we need a work around
- val withoutId = (prefix: String, colFunc: String => Column) => {
- formattedDataFrame.schema.fields
- .map(_.name)
- .filter(name => name != ID_COLUMN)
- .map(x => colFunc(prefix + "." + x).alias(x))
- }
-
formattedDataFrame
.alias("l")
.join(
@@ -116,7 +120,13 @@ object Weighting {
col(s"l.$ID_COLUMN"),
struct("l.*").alias("left_contents"),
struct(
- struct(withoutId("r", col): _*).alias("neighbor"),
+ (
+ savedAttributesWithGeom match {
+ case null => struct(col("r.*")).dropFields(ID_COLUMN)
+ case _ =>
+ struct(savedAttributesWithGeom.map(c => col(s"r.$c")): _*)
+ }
+ ).alias("neighbor"),
if (!binary)
pow(distanceFunction(col(s"l.$geometryColumn"), col(s"r.$geometryColumn")), alpha)
.alias("value")
@@ -127,14 +137,18 @@ object Weighting {
concat(
collect_list(col("weight")),
if (includeSelf)
- array(
- struct(
- struct(withoutId("left_contents", first): _*).alias("neighbor"),
- lit(selfWeight).alias("value")))
- else array()).alias("weights"))
- .select("left_contents.*", "weights")
+ array(struct(
+ (savedAttributesWithGeom match {
+ case null => first("left_contents").dropFields(ID_COLUMN)
+ case _ =>
+ struct(
+ savedAttributesWithGeom.map(c => first(s"left_contents.$c").alias(c)): _*)
+ }).alias("neighbor"),
+ lit(selfWeight).alias("value")))
+ else array()).alias(resultName))
+ .select("left_contents.*", resultName)
.drop(ID_COLUMN)
- .withColumn("weights", filter(col("weights"), _(f"neighbor")(geometryColumn).isNotNull))
+ .withColumn(resultName, filter(col(resultName), _(f"neighbor")(geometryColumn).isNotNull))
}
/**
@@ -158,6 +172,10 @@ object Weighting {
* name of the geometry column
* @param useSpheroid
* whether to use a cartesian or spheroidal distance calculation. Default is false
+ * @param savedAttributes
+ * the attributes to save in the neighbor column. Default is all columns.
+ * @param resultName
+ * the name of the resulting column. Default is 'weights'.
* @return
* The input DataFrame with a weight column added containing neighbors and their weights
* (always 1) added to each row.
@@ -168,13 +186,73 @@ object Weighting {
includeZeroDistanceNeighbors: Boolean = true,
includeSelf: Boolean = false,
geometry: String = null,
- useSpheroid: Boolean = false): DataFrame = addDistanceBandColumn(
+ useSpheroid: Boolean = false,
+ savedAttributes: Seq[String] = null,
+ resultName: String = "weights"): DataFrame = addDistanceBandColumn(
dataframe,
threshold,
binary = true,
includeZeroDistanceNeighbors = includeZeroDistanceNeighbors,
includeSelf = includeSelf,
geometry = geometry,
- useSpheroid = useSpheroid)
+ useSpheroid = useSpheroid,
+ savedAttributes = savedAttributes,
+ resultName = resultName)
+
+ /**
+ * Annotates a dataframe with a weights column for each data record containing the other members
+ * within the threshold and their weight. Weights will be dist^alpha. The dataframe should
+ * contain at least one GeometryType column. Rows must be unique. If one geometry column is
+ * present it will be used automatically. If two are present, the one named 'geometry' will be
+ * used. If more than one are present and neither is named 'geometry', the column name must be
+ * provided. The new column will be named 'cluster'.
+ *
+ * @param dataframe
+ * DataFrame with geometry column
+ * @param threshold
+ * Distance threshold for considering neighbors
+ * @param alpha
+ * alpha to use for inverse distance weights. Computation is dist^alpha. Default is -1.0
+ * @param includeZeroDistanceNeighbors
+ * whether to include neighbors that are 0 distance. If 0 distance neighbors are included and
+ * binary is false, values are infinity as per the floating point spec (divide by 0)
+ * @param includeSelf
+ * whether to include self in the list of neighbors
+ * @param selfWeight
+ * the weight to provide for the self as its own neighbor. Default is 1.0
+ * @param geometry
+ * name of the geometry column
+ * @param useSpheroid
+ * whether to use a cartesian or spheroidal distance calculation. Default is false
+ * @param savedAttributes
+ * the attributes to save in the neighbor column. Default is all columns.
+ * @param resultName
+ * the name of the resulting column. Default is 'weights'.
+ * @return
+ * The input DataFrame with a weight column added containing neighbors and their weights
+ * (dist^alpha) added to each row.
+ */
+ def addWeightedDistanceBandColumn(
+ dataframe: DataFrame,
+ threshold: Double,
+ alpha: Double = -1.0,
+ includeZeroDistanceNeighbors: Boolean = false,
+ includeSelf: Boolean = false,
+ selfWeight: Double = 1.0,
+ geometry: String = null,
+ useSpheroid: Boolean = false,
+ savedAttributes: Seq[String] = null,
+ resultName: String = "weights"): DataFrame = addDistanceBandColumn(
+ dataframe,
+ threshold,
+ alpha = alpha,
+ binary = false,
+ includeZeroDistanceNeighbors = includeZeroDistanceNeighbors,
+ includeSelf = includeSelf,
+ selfWeight = selfWeight,
+ geometry = geometry,
+ useSpheroid = useSpheroid,
+ savedAttributes = savedAttributes,
+ resultName = resultName)
}
diff --git a/spark/common/src/main/scala/org/apache/sedona/stats/clustering/DBSCAN.scala b/spark/common/src/main/scala/org/apache/sedona/stats/clustering/DBSCAN.scala
index e4cd1f90b4..c75291d971 100644
--- a/spark/common/src/main/scala/org/apache/sedona/stats/clustering/DBSCAN.scala
+++ b/spark/common/src/main/scala/org/apache/sedona/stats/clustering/DBSCAN.scala
@@ -18,7 +18,7 @@
*/
package org.apache.sedona.stats.clustering
-import org.apache.sedona.stats.Util.getGeometryColumnName
+import org.apache.sedona.util.DfUtils.getGeometryColumnName
import org.apache.spark.sql.functions._
import org.apache.spark.sql.sedona_sql.UDT.GeometryUDT
import org.apache.spark.sql.sedona_sql.expressions.st_functions.{ST_Distance, ST_DistanceSpheroid}
@@ -48,6 +48,11 @@ object DBSCAN {
* whether to include outliers in the output. Default is false
* @param useSpheroid
* whether to use a cartesian or spheroidal distance calculation. Default is false
+ * @param isCoreColumnName
+ * what the name of the column indicating if this is a core point should be. Default is
+ * "isCore"
+ * @param clusterColumnName
+ * what the name of the column indicating the cluster id should be. Default is "cluster"
* @return
* The input DataFrame with the cluster label added to each row. Outlier will have a cluster
* value of -1 if included.
@@ -58,10 +63,12 @@ object DBSCAN {
minPts: Int,
geometry: String = null,
includeOutliers: Boolean = true,
- useSpheroid: Boolean = false): DataFrame = {
+ useSpheroid: Boolean = false,
+ isCoreColumnName: String = "isCore",
+ clusterColumnName: String = "cluster"): DataFrame = {
val geometryCol = geometry match {
- case null => getGeometryColumnName(dataframe)
+ case null => getGeometryColumnName(dataframe.schema)
case _ => geometry
}
@@ -89,12 +96,12 @@ object DBSCAN {
first(struct("left.*")).alias("leftContents"),
count(col(s"right.id")).alias("neighbors_count"),
collect_list(col(s"right.id")).alias("neighbors"))
- .withColumn("isCore", col("neighbors_count") >= lit(minPts))
- .select("leftContents.*", "neighbors", "isCore")
+ .withColumn(isCoreColumnName, col("neighbors_count") >= lit(minPts))
+ .select("leftContents.*", "neighbors", isCoreColumnName)
.checkpoint()
- val corePointsDF = isCorePointsDF.filter(col("isCore"))
- val borderPointsDF = isCorePointsDF.filter(!col("isCore"))
+ val corePointsDF = isCorePointsDF.filter(col(isCoreColumnName))
+ val borderPointsDF = isCorePointsDF.filter(!col(isCoreColumnName))
val coreEdgesDf = corePointsDF
.select(col("id").alias("src"), explode(col("neighbors")).alias("dst"))
@@ -117,14 +124,14 @@ object DBSCAN {
val outliersDf = idDataframe
.join(clusteredPointsDf, Seq("id"), "left_anti")
- .withColumn("isCore", lit(false))
+ .withColumn(isCoreColumnName, lit(false))
.withColumn("component", lit(-1))
.withColumn("neighbors", array().cast("array"))
val completedDf = (
if (includeOutliers) clusteredPointsDf.unionByName(outliersDf)
else clusteredPointsDf
- ).withColumnRenamed("component", "cluster")
+ ).withColumnRenamed("component", clusterColumnName)
val returnDf = if (hasIdColumn) {
completedDf.drop("neighbors", "id").withColumnRenamed(ID_COLUMN, "id")
diff --git a/spark/common/src/main/scala/org/apache/sedona/stats/outlierDetection/LocalOutlierFactor.scala b/spark/common/src/main/scala/org/apache/sedona/stats/outlierDetection/LocalOutlierFactor.scala
index b98919de25..2595a90852 100644
--- a/spark/common/src/main/scala/org/apache/sedona/stats/outlierDetection/LocalOutlierFactor.scala
+++ b/spark/common/src/main/scala/org/apache/sedona/stats/outlierDetection/LocalOutlierFactor.scala
@@ -18,8 +18,8 @@
*/
package org.apache.sedona.stats.outlierDetection
-import org.apache.sedona.stats.Util.getGeometryColumnName
-import org.apache.spark.sql.sedona_sql.expressions.st_functions.{ST_Distance, ST_DistanceSpheroid}
+import org.apache.sedona.util.DfUtils.getGeometryColumnName
+import org.apache.spark.sql.sedona_sql.expressions.st_functions.{ST_Distance, ST_DistanceSphere}
import org.apache.spark.sql.{Column, DataFrame, SparkSession, functions => f}
object LocalOutlierFactor {
@@ -42,8 +42,10 @@ object LocalOutlierFactor {
* name of the geometry column
* @param handleTies
* whether to handle ties in the k-distance calculation. Default is false
- * @param useSpheroid
+ * @param useSphere
* whether to use a cartesian or spheroidal distance calculation. Default is false
+ * @param resultColumnName
+ * the name of the column containing the lof for each row. Default is "lof"
*
* @return
* A DataFrame containing the lof for each row
@@ -53,7 +55,8 @@ object LocalOutlierFactor {
k: Int = 20,
geometry: String = null,
handleTies: Boolean = false,
- useSpheroid: Boolean = false): DataFrame = {
+ useSphere: Boolean = false,
+ resultColumnName: String = "lof"): DataFrame = {
if (k < 1)
throw new IllegalArgumentException("k must be a positive integer")
@@ -67,10 +70,11 @@ object LocalOutlierFactor {
} else "false" // else case to make compiler happy
val distanceFunction: (Column, Column) => Column =
- if (useSpheroid) ST_DistanceSpheroid else ST_Distance
- val useSpheroidString = if (useSpheroid) "True" else "False" // for the SQL expression
+ if (useSphere) ST_DistanceSphere else ST_Distance
+ val useSpheroidString = if (useSphere) "True" else "False" // for the SQL expression
- val geometryColumn = if (geometry == null) getGeometryColumnName(dataframe) else geometry
+ val geometryColumn =
+ if (geometry == null) getGeometryColumnName(dataframe.schema) else geometry
val KNNFunction = "ST_KNN"
@@ -136,8 +140,8 @@ object LocalOutlierFactor {
.groupBy("a_id")
.agg(
f.first(CONTENTS_COLUMN_NAME).alias(CONTENTS_COLUMN_NAME),
- (f.sum("b_lrd") / (f.count("b_lrd") * f.first("a_lrd"))).alias("lof"))
- .select(f.col(f"$CONTENTS_COLUMN_NAME.*"), f.col("lof"))
+ (f.sum("b_lrd") / (f.count("b_lrd") * f.first("a_lrd"))).alias(resultColumnName))
+ .select(f.col(f"$CONTENTS_COLUMN_NAME.*"), f.col(resultColumnName))
if (handleTies)
SparkSession.getActiveSession.get.conf
diff --git a/spark/common/src/main/scala/org/apache/sedona/stats/Util.scala b/spark/common/src/main/scala/org/apache/sedona/util/DfUtils.scala
similarity index 84%
rename from spark/common/src/main/scala/org/apache/sedona/stats/Util.scala
rename to spark/common/src/main/scala/org/apache/sedona/util/DfUtils.scala
index cdfe5fca23..5b2bea2180 100644
--- a/spark/common/src/main/scala/org/apache/sedona/stats/Util.scala
+++ b/spark/common/src/main/scala/org/apache/sedona/util/DfUtils.scala
@@ -16,14 +16,14 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.sedona.stats
+package org.apache.sedona.util
-import org.apache.spark.sql.DataFrame
import org.apache.spark.sql.sedona_sql.UDT.GeometryUDT
+import org.apache.spark.sql.types.StructType
-private[stats] object Util {
- def getGeometryColumnName(dataframe: DataFrame): String = {
- val geomFields = dataframe.schema.fields.filter(_.dataType == GeometryUDT)
+object DfUtils {
+ def getGeometryColumnName(schema: StructType): String = {
+ val geomFields = schema.fields.filter(_.dataType == GeometryUDT)
if (geomFields.isEmpty)
throw new IllegalArgumentException(
diff --git a/spark/common/src/main/scala/org/apache/spark/sql/execution/datasource/stac/TemporalFilter.scala b/spark/common/src/main/scala/org/apache/spark/sql/execution/datasource/stac/TemporalFilter.scala
new file mode 100644
index 0000000000..e5bfc947ac
--- /dev/null
+++ b/spark/common/src/main/scala/org/apache/spark/sql/execution/datasource/stac/TemporalFilter.scala
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.execution.datasource.stac
+
+import java.time.LocalDateTime
+
+/* A temporal filter that can be pushed down to the STAC data source.
+ * It wraps a [[TemporalFilter]] and provides a simple string representation.
+ * @param temporalFilter
+ */
+trait TemporalFilter {
+ def evaluate(columns: Map[String, LocalDateTime]): Boolean
+ def simpleString: String
+}
+
+object TemporalFilter {
+
+ case class AndFilter(left: TemporalFilter, right: TemporalFilter) extends TemporalFilter {
+ override def evaluate(columns: Map[String, LocalDateTime]): Boolean = {
+ left.evaluate(columns) && right.evaluate(columns)
+ }
+
+ override def simpleString: String = s"(${left.simpleString}) AND (${right.simpleString})"
+ }
+
+ case class OrFilter(left: TemporalFilter, right: TemporalFilter) extends TemporalFilter {
+ override def evaluate(columns: Map[String, LocalDateTime]): Boolean =
+ left.evaluate(columns) || right.evaluate(columns)
+ override def simpleString: String = s"(${left.simpleString}) OR (${right.simpleString})"
+ }
+
+ case class LessThanFilter(columnName: String, value: LocalDateTime) extends TemporalFilter {
+ override def evaluate(columns: Map[String, LocalDateTime]): Boolean = {
+ columns.get(columnName).exists(_ isBefore value)
+ }
+ override def simpleString: String = s"$columnName < $value"
+ }
+
+ case class GreaterThanFilter(columnName: String, value: LocalDateTime) extends TemporalFilter {
+ override def evaluate(columns: Map[String, LocalDateTime]): Boolean = {
+ columns.get(columnName).exists(_ isAfter value)
+ }
+ override def simpleString: String = s"$columnName > $value"
+ }
+
+ case class EqualFilter(columnName: String, value: LocalDateTime) extends TemporalFilter {
+ override def evaluate(columns: Map[String, LocalDateTime]): Boolean = {
+ columns.get(columnName).exists(_ isEqual value)
+ }
+ override def simpleString: String = s"$columnName = $value"
+ }
+}
diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/adapters/StructuredAdapter.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/adapters/StructuredAdapter.scala
new file mode 100644
index 0000000000..70cef8d783
--- /dev/null
+++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/adapters/StructuredAdapter.scala
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.sedona_sql.adapters
+
+import org.apache.sedona.core.spatialRDD.SpatialRDD
+import org.apache.sedona.sql.utils.GeometrySerializer
+import org.apache.sedona.util.DfUtils
+import org.apache.spark.api.java.JavaPairRDD
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types.{DataType, StructType}
+import org.apache.spark.sql.{DataFrame, Row, SparkSession}
+import org.locationtech.jts.geom.Geometry
+import org.slf4j.{Logger, LoggerFactory}
+import org.apache.sedona.core.spatialPartitioning.GenericUniquePartitioner
+
+/**
+ * Adapter for converting between DataFrame and SpatialRDD. It provides methods to convert
+ * DataFrame to SpatialRDD and vice versa without losing schema. It is different from
+ * [[org.apache.sedona.sql.utils.Adapter]] which loses the schema information during conversion.
+ * This should be used if your data starts as a DataFrame and you want to convert it to SpatialRDD
+ */
+object StructuredAdapter {
+ val logger: Logger = LoggerFactory.getLogger(getClass)
+
+ /**
+ * Convert RDD[Row] to SpatialRDD. It puts Row as user data of Geometry.
+ * @param rdd
+ * @param geometryFieldName
+ * @return
+ */
+ def toSpatialRdd(rdd: RDD[Row], geometryFieldName: String): SpatialRDD[Geometry] = {
+ val spatialRDD = new SpatialRDD[Geometry]
+ if (rdd.isEmpty()) {
+ spatialRDD.schema = StructType(Seq())
+ } else spatialRDD.schema = rdd.first().schema
+ spatialRDD.rawSpatialRDD = rdd
+ .map(row => {
+ val geom = row.getAs[Geometry](geometryFieldName)
+ geom.setUserData(row.copy())
+ geom
+ })
+ spatialRDD
+ }
+
+ /**
+ * Convert RDD[Row] to SpatialRDD. It puts Row as user data of Geometry. It auto-detects
+ * geometry column if geometryFieldName is not provided. It uses the first geometry column in
+ * RDD.
+ * @param rdd
+ * @return
+ */
+ def toSpatialRdd(rdd: RDD[Row]): SpatialRDD[Geometry] = {
+ require(rdd.count() > 0, "Input RDD cannot be empty.")
+ toSpatialRdd(rdd, DfUtils.getGeometryColumnName(rdd.first().schema))
+ }
+
+ /**
+ * Convert SpatialRDD to RDD[Row]. It extracts Row from user data of Geometry.
+ * @param spatialRDD
+ * @return
+ */
+ def toRowRdd(spatialRDD: SpatialRDD[Geometry]): RDD[Row] = {
+ spatialRDD.rawSpatialRDD.map(geometry => {
+ val row = geometry.getUserData.asInstanceOf[Row]
+ row
+ })
+ }
+
+ /**
+ * Convert DataFrame to SpatialRDD. It puts InternalRow as user data of Geometry. It allows only
+ * one geometry column.
+ *
+ * @param dataFrame
+ * @param geometryFieldName
+ */
+ def toSpatialRdd(dataFrame: DataFrame, geometryFieldName: String): SpatialRDD[Geometry] = {
+ val spatialRDD = new SpatialRDD[Geometry]
+ spatialRDD.schema = dataFrame.schema
+ val ordinal = spatialRDD.schema.fieldIndex(geometryFieldName)
+ spatialRDD.rawSpatialRDD = dataFrame.queryExecution.toRdd
+ .map(row => {
+ val geom = GeometrySerializer.deserialize(row.getBinary(ordinal))
+ geom.setUserData(row.copy())
+ geom
+ })
+ spatialRDD
+ }
+
+ /**
+ * Convert DataFrame to SpatialRDD. It puts InternalRow as user data of Geometry. It
+ * auto-detects geometry column if geometryFieldName is not provided. It uses the first geometry
+ * column in DataFrame.
+ * @param dataFrame
+ * @return
+ */
+ def toSpatialRdd(dataFrame: DataFrame): SpatialRDD[Geometry] = {
+ toSpatialRdd(dataFrame, DfUtils.getGeometryColumnName(dataFrame.schema))
+ }
+
+ /**
+ * Convert SpatialRDD.rawSpatialRdd to DataFrame
+ * @param spatialRDD
+ * The SpatialRDD to convert. It must have rawSpatialRDD set.
+ * @param sparkSession
+ * @return
+ */
+ def toDf(spatialRDD: SpatialRDD[Geometry], sparkSession: SparkSession): DataFrame = {
+ val rowRdd = spatialRDD.rawSpatialRDD.map(geometry => {
+ val row = geometry.getUserData.asInstanceOf[InternalRow]
+ row
+ })
+ sparkSession.internalCreateDataFrame(rowRdd, spatialRDD.schema)
+ }
+
+ /**
+ * Convert SpatialRDD.spatialPartitionedRDD to DataFrame This is useful when you want to convert
+ * SpatialRDD after spatial partitioning.
+ * @param spatialRDD
+ * The SpatialRDD to convert. It must have spatialPartitionedRDD set. You must call
+ * spatialPartitioning method before calling this method.
+ * @param sparkSession
+ * @return
+ */
+ def toSpatialPartitionedDf(
+ spatialRDD: SpatialRDD[Geometry],
+ sparkSession: SparkSession): DataFrame = {
+ if (spatialRDD.spatialPartitionedRDD == null)
+ throw new RuntimeException(
+ "SpatialRDD is not spatially partitioned. Please call spatialPartitioning method before calling this method.")
+
+ if (!spatialRDD.getPartitioner().isInstanceOf[GenericUniquePartitioner]) {
+ logger.warn(
+ "SpatialPartitionedRDD might have duplicate geometries. Please make sure you are aware of it.")
+ }
+ val rowRdd = spatialRDD.spatialPartitionedRDD.map(geometry => {
+ val row = geometry.getUserData.asInstanceOf[InternalRow]
+ row
+ })
+ sparkSession.internalCreateDataFrame(rowRdd, spatialRDD.schema)
+ }
+
+ /**
+ * Convert JavaPairRDD[Geometry, Geometry] to DataFrame This method is useful when you want to
+ * convert the result of spatial join to DataFrame.
+ * @param spatialPairRDD
+ * The JavaPairRDD to convert.
+ * @param leftSchemaJson
+ * Schema of the left side. In a json format.
+ * @param rightSchemaJson
+ * Schema of the right side. In a json format.
+ * @param sparkSession
+ * @return
+ */
+ def toDf(
+ spatialPairRDD: JavaPairRDD[Geometry, Geometry],
+ leftSchemaJson: String,
+ rightSchemaJson: String,
+ sparkSession: SparkSession): DataFrame = {
+ val leftSchema = DataType.fromJson(leftSchemaJson).asInstanceOf[StructType]
+ val rightSchema = DataType.fromJson(rightSchemaJson).asInstanceOf[StructType]
+ toDf(spatialPairRDD, leftSchema, rightSchema, sparkSession)
+ }
+
+ /**
+ * Convert JavaPairRDD[Geometry, Geometry] to DataFrame This method is useful when you want to
+ * convert the result of spatial join to DataFrame.
+ * @param spatialPairRDD
+ * The JavaPairRDD to convert.
+ * @param leftSchema
+ * The schema of the left side.
+ * @param rightSchema
+ * The schema of the right side.
+ * @param sparkSession
+ * @return
+ */
+ def toDf(
+ spatialPairRDD: JavaPairRDD[Geometry, Geometry],
+ leftSchema: StructType,
+ rightSchema: StructType,
+ sparkSession: SparkSession): DataFrame = {
+ val rowRdd = spatialPairRDD.rdd.map(pair => {
+ val leftRow = pair._1.getUserData.asInstanceOf[InternalRow].toSeq(leftSchema)
+ val rightRow = pair._2.getUserData.asInstanceOf[InternalRow].toSeq(rightSchema)
+ InternalRow.fromSeq(leftRow ++ rightRow)
+ })
+ sparkSession.internalCreateDataFrame(
+ rowRdd,
+ StructType(leftSchema.fields ++ rightSchema.fields))
+ }
+
+ /**
+ * Convert JavaPairRDD[Geometry, Geometry] to DataFrame This method is useful when you want to
+ * convert the result of spatial join to DataFrame.
+ * @param spatialPairRDD
+ * The JavaPairRDD to convert.
+ * @param originalLeftSpatialRdd
+ * The original left SpatialRDD involved in the join. It is used to get the schema of the left
+ * side.
+ * @param originalRightSpatialRdd
+ * The original right SpatialRDD involved in the join. It is used to get the schema of the
+ * right side.
+ * @param sparkSession
+ * @return
+ */
+ def toDf(
+ spatialPairRDD: JavaPairRDD[Geometry, Geometry],
+ originalLeftSpatialRdd: SpatialRDD[Geometry],
+ originalRightSpatialRdd: SpatialRDD[Geometry],
+ sparkSession: SparkSession): DataFrame = {
+ toDf(
+ spatialPairRDD,
+ originalLeftSpatialRdd.schema,
+ originalRightSpatialRdd.schema,
+ sparkSession)
+ }
+}
diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/Functions.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/Functions.scala
index dc8a290b8a..de7e3170ca 100644
--- a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/Functions.scala
+++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/Functions.scala
@@ -35,6 +35,17 @@ import org.apache.spark.sql.sedona_sql.expressions.InferrableFunctionConverter._
import org.apache.spark.unsafe.types.UTF8String
import org.apache.spark.util.Utils
+case class ST_LabelPoint(inputExpressions: Seq[Expression])
+ extends InferredExpression(
+ inferrableFunction1(Functions.labelPoint),
+ inferrableFunction2(Functions.labelPoint),
+ inferrableFunction3(Functions.labelPoint)) {
+
+ protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]) = {
+ copy(inputExpressions = newChildren)
+ }
+}
+
/**
* Return the distance between two geometries.
*
@@ -719,6 +730,15 @@ case class ST_MMax(inputExpressions: Seq[Expression])
}
}
+case class ST_LineSegments(inputExpressions: Seq[Expression])
+ extends InferredExpression(
+ inferrableFunction2(Functions.lineSegments),
+ inferrableFunction1(Functions.lineSegments)) {
+ protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]) = {
+ copy(inputExpressions = newChildren)
+ }
+}
+
/**
* Return a linestring being a substring of the input one starting and ending at the given
* fractions of total 2d length. Second and third arguments are Double values between 0 and 1.
@@ -1004,6 +1024,17 @@ case class ST_Perimeter(inputExpressions: Seq[Expression])
}
}
+case class ST_Perimeter2D(inputExpressions: Seq[Expression])
+ extends InferredExpression(
+ inferrableFunction3(Functions.perimeter),
+ inferrableFunction2(Functions.perimeter),
+ inferrableFunction1(Functions.perimeter)) {
+
+ protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]) = {
+ copy(inputExpressions = newChildren)
+ }
+}
+
case class ST_Points(inputExpressions: Seq[Expression])
extends InferredExpression(Functions.points _) {
diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/GeoStatsFunctions.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/GeoStatsFunctions.scala
new file mode 100644
index 0000000000..8c6b645daf
--- /dev/null
+++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/GeoStatsFunctions.scala
@@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.sedona_sql.expressions
+
+import org.apache.sedona.core.utils.SedonaConf
+import org.apache.sedona.stats.Weighting.{addBinaryDistanceBandColumn, addWeightedDistanceBandColumn}
+import org.apache.sedona.stats.clustering.DBSCAN.dbscan
+import org.apache.sedona.stats.hotspotDetection.GetisOrd.gLocal
+import org.apache.sedona.stats.outlierDetection.LocalOutlierFactor.localOutlierFactor
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, ImplicitCastInputTypes, Literal, ScalarSubquery, Unevaluable}
+import org.apache.spark.sql.execution.{LogicalRDD, SparkPlan}
+import org.apache.spark.sql.functions.{col, struct}
+import org.apache.spark.sql.sedona_sql.UDT.GeometryUDT
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.{DataFrame, Dataset, SparkSession}
+
+import scala.reflect.ClassTag
+
+// We mark ST_GeoStatsFunction as non-deterministic to avoid the filter push-down optimization pass
+// duplicates the ST_GeoStatsFunction when pushing down aliased ST_GeoStatsFunction through a
+// Project operator. This will make ST_GeoStatsFunction being evaluated twice.
+trait ST_GeoStatsFunction
+ extends Expression
+ with ImplicitCastInputTypes
+ with Unevaluable
+ with Serializable {
+
+ final override lazy val deterministic: Boolean = false
+
+ override def nullable: Boolean = true
+
+ private final lazy val sparkSession = SparkSession.getActiveSession.get
+
+ protected final lazy val geometryColumnName = getInputName(0, "geometry")
+
+ protected def getInputName(i: Int, fieldName: String): String = children(i) match {
+ case ref: AttributeReference => ref.name
+ case _ =>
+ throw new IllegalArgumentException(
+ f"$fieldName argument must be a named reference to an existing column")
+ }
+
+ protected def getInputNames(i: Int, fieldName: String): Seq[String] = children(
+ i).dataType match {
+ case StructType(fields) => fields.map(_.name)
+ case _ => throw new IllegalArgumentException(f"$fieldName argument must be a struct")
+ }
+
+ protected def getResultName(resultAttrs: Seq[Attribute]): String = resultAttrs match {
+ case Seq(attr) => attr.name
+ case _ => throw new IllegalArgumentException("resultAttrs must have exactly one attribute")
+ }
+
+ protected def doExecute(dataframe: DataFrame, resultAttrs: Seq[Attribute]): DataFrame
+
+ protected def getScalarValue[T](i: Int, name: String)(implicit ct: ClassTag[T]): T = {
+ children(i) match {
+ case Literal(l: T, _) => l
+ case _: Literal =>
+ throw new IllegalArgumentException(f"$name must be an instance of ${ct.runtimeClass}")
+ case s: ScalarSubquery =>
+ s.eval() match {
+ case t: T => t
+ case _ =>
+ throw new IllegalArgumentException(
+ f"$name must be an instance of ${ct.runtimeClass}")
+ }
+ case _ => throw new IllegalArgumentException(f"$name must be a scalar value")
+ }
+ }
+
+ def execute(plan: SparkPlan, resultAttrs: Seq[Attribute]): RDD[InternalRow] = {
+ val df = doExecute(
+ Dataset.ofRows(sparkSession, LogicalRDD(plan.output, plan.execute())(sparkSession)),
+ resultAttrs)
+ df.queryExecution.toRdd
+ }
+
+}
+
+case class ST_DBSCAN(children: Seq[Expression]) extends ST_GeoStatsFunction {
+
+ override def dataType: DataType = StructType(
+ Seq(StructField("isCore", BooleanType), StructField("cluster", LongType)))
+
+ override def inputTypes: Seq[AbstractDataType] =
+ Seq(GeometryUDT, DoubleType, IntegerType, BooleanType)
+
+ protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Expression =
+ copy(children = newChildren)
+
+ override def doExecute(dataframe: DataFrame, resultAttrs: Seq[Attribute]): DataFrame = {
+ require(
+ !dataframe.columns.contains("__isCore"),
+ "__isCore is a reserved name by the dbscan algorithm. Please rename the columns before calling the ST_DBSCAN function.")
+ require(
+ !dataframe.columns.contains("__cluster"),
+ "__cluster is a reserved name by the dbscan algorithm. Please rename the columns before calling the ST_DBSCAN function.")
+
+ dbscan(
+ dataframe,
+ getScalarValue[Double](1, "epsilon"),
+ getScalarValue[Int](2, "minPts"),
+ geometryColumnName,
+ SedonaConf.fromActiveSession().getDBSCANIncludeOutliers,
+ getScalarValue[Boolean](3, "useSpheroid"),
+ "__isCore",
+ "__cluster")
+ .withColumn(getResultName(resultAttrs), struct(col("__isCore"), col("__cluster")))
+ .drop("__isCore", "__cluster")
+ }
+}
+
+case class ST_LocalOutlierFactor(children: Seq[Expression]) extends ST_GeoStatsFunction {
+
+ override def dataType: DataType = DoubleType
+
+ override def inputTypes: Seq[AbstractDataType] =
+ Seq(GeometryUDT, IntegerType, BooleanType)
+
+ protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Expression =
+ copy(children = newChildren)
+
+ override def doExecute(dataframe: DataFrame, resultAttrs: Seq[Attribute]): DataFrame = {
+ localOutlierFactor(
+ dataframe,
+ getScalarValue[Int](1, "k"),
+ geometryColumnName,
+ SedonaConf.fromActiveSession().isIncludeTieBreakersInKNNJoins,
+ getScalarValue[Boolean](2, "useSphere"),
+ getResultName(resultAttrs))
+ }
+}
+
+case class ST_GLocal(children: Seq[Expression]) extends ST_GeoStatsFunction {
+
+ override def dataType: DataType = StructType(
+ Seq(
+ StructField("G", DoubleType),
+ StructField("EG", DoubleType),
+ StructField("VG", DoubleType),
+ StructField("Z", DoubleType),
+ StructField("P", DoubleType)))
+
+ override def inputTypes: Seq[AbstractDataType] = {
+ val xDataType = children(0).dataType
+ require(xDataType == DoubleType || xDataType == IntegerType, "x must be a numeric value")
+ Seq(
+ xDataType,
+ children(1).dataType, // Array of the weights
+ BooleanType)
+ }
+
+ protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Expression =
+ copy(children = newChildren)
+
+ override def doExecute(dataframe: DataFrame, resultAttrs: Seq[Attribute]): DataFrame = {
+ gLocal(
+ dataframe,
+ getInputName(0, "x"),
+ getInputName(1, "weights"),
+ 0,
+ getScalarValue[Boolean](2, "star"),
+ 0.0)
+ .withColumn(
+ getResultName(resultAttrs),
+ struct(col("G"), col("EG"), col("VG"), col("Z"), col("P")))
+ .drop("G", "EG", "VG", "Z", "P")
+ }
+}
+
+case class ST_BinaryDistanceBandColumn(children: Seq[Expression]) extends ST_GeoStatsFunction {
+ override def dataType: DataType = ArrayType(
+ StructType(
+ Seq(StructField("neighbor", children(5).dataType), StructField("value", DoubleType))))
+
+ override def inputTypes: Seq[AbstractDataType] =
+ Seq(GeometryUDT, DoubleType, BooleanType, BooleanType, BooleanType, children(5).dataType)
+
+ protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Expression =
+ copy(children = newChildren)
+
+ override def doExecute(dataframe: DataFrame, resultAttrs: Seq[Attribute]): DataFrame = {
+ val attributeNames = getInputNames(5, "attributes")
+ require(attributeNames.nonEmpty, "attributes must have at least one column")
+ require(
+ attributeNames.contains(geometryColumnName),
+ "attributes must contain the geometry column")
+
+ addBinaryDistanceBandColumn(
+ dataframe,
+ getScalarValue[Double](1, "threshold"),
+ getScalarValue[Boolean](2, "includeZeroDistanceNeighbors"),
+ getScalarValue[Boolean](3, "includeSelf"),
+ geometryColumnName,
+ getScalarValue[Boolean](4, "useSpheroid"),
+ attributeNames,
+ getResultName(resultAttrs))
+ }
+}
+
+case class ST_WeightedDistanceBandColumn(children: Seq[Expression]) extends ST_GeoStatsFunction {
+
+ override def dataType: DataType = ArrayType(
+ StructType(
+ Seq(StructField("neighbor", children(7).dataType), StructField("value", DoubleType))))
+
+ override def inputTypes: Seq[AbstractDataType] =
+ Seq(
+ GeometryUDT,
+ DoubleType,
+ DoubleType,
+ BooleanType,
+ BooleanType,
+ DoubleType,
+ BooleanType,
+ children(7).dataType)
+
+ protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Expression =
+ copy(children = newChildren)
+
+ override def doExecute(dataframe: DataFrame, resultAttrs: Seq[Attribute]): DataFrame = {
+ val attributeNames = getInputNames(7, "attributes")
+ require(attributeNames.nonEmpty, "attributes must have at least one column")
+ require(
+ attributeNames.contains(geometryColumnName),
+ "attributes must contain the geometry column")
+
+ addWeightedDistanceBandColumn(
+ dataframe,
+ getScalarValue[Double](1, "threshold"),
+ getScalarValue[Double](2, "alpha"),
+ getScalarValue[Boolean](3, "includeZeroDistanceNeighbors"),
+ getScalarValue[Boolean](4, "includeSelf"),
+ getScalarValue[Double](5, "selfWeight"),
+ geometryColumnName,
+ getScalarValue[Boolean](6, "useSpheroid"),
+ attributeNames,
+ getResultName(resultAttrs))
+ }
+}
diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/st_functions.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/st_functions.scala
index 7bb753cc28..84d555ff64 100644
--- a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/st_functions.scala
+++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/st_functions.scala
@@ -45,6 +45,22 @@ object st_functions extends DataFrameAPI {
def ST_AddPoint(lineString: String, point: String, index: Int): Column =
wrapExpression[ST_AddPoint](lineString, point, index)
+ def ST_LabelPoint(geometry: Column): Column =
+ wrapExpression[ST_LabelPoint](geometry)
+ def ST_LabelPoint(geometry: String): Column =
+ wrapExpression[ST_LabelPoint](geometry)
+ def ST_LabelPoint(geometry: Column, gridResolution: Column): Column =
+ wrapExpression[ST_LabelPoint](geometry, gridResolution)
+ def ST_LabelPoint(geometry: String, gridResolution: Integer): Column =
+ wrapExpression[ST_LabelPoint](geometry, gridResolution)
+ def ST_LabelPoint(geometry: Column, gridResolution: Column, goodnessThreshold: Column): Column =
+ wrapExpression[ST_LabelPoint](geometry, gridResolution, goodnessThreshold)
+ def ST_LabelPoint(
+ geometry: String,
+ gridResolution: Integer,
+ goodnessThreshold: Double): Column =
+ wrapExpression[ST_LabelPoint](geometry, gridResolution, goodnessThreshold)
+
def ST_Area(geometry: Column): Column = wrapExpression[ST_Area](geometry)
def ST_Area(geometry: String): Column = wrapExpression[ST_Area](geometry)
@@ -317,6 +333,13 @@ object st_functions extends DataFrameAPI {
def ST_LineMerge(multiLineString: String): Column =
wrapExpression[ST_LineMerge](multiLineString)
+ def ST_LineSegments(geom: Column): Column = wrapExpression[ST_LineSegments](geom)
+ def ST_LineSegments(geom: String): Column = wrapExpression[ST_LineSegments](geom)
+ def ST_LineSegments(geom: Column, lenient: Column): Column =
+ wrapExpression[ST_LineSegments](geom, lenient)
+ def ST_LineSegments(geom: String, lenient: Boolean): Column =
+ wrapExpression[ST_LineSegments](geom, lenient)
+
def ST_LineSubstring(lineString: Column, startFraction: Column, endFraction: Column): Column =
wrapExpression[ST_LineSubstring](lineString, startFraction, endFraction)
def ST_LineSubstring(lineString: String, startFraction: Double, endFraction: Double): Column =
@@ -369,6 +392,17 @@ object st_functions extends DataFrameAPI {
def ST_Perimeter(geom: String, use_spheroid: Boolean, lenient: Boolean): Column =
wrapExpression[ST_Perimeter](geom, use_spheroid, lenient)
+ def ST_Perimeter2D(geom: Column): Column = wrapExpression[ST_Perimeter2D](geom)
+ def ST_Perimeter2D(geom: String): Column = wrapExpression[ST_Perimeter2D](geom)
+ def ST_Perimeter2D(geom: Column, use_spheroid: Column): Column =
+ wrapExpression[ST_Perimeter2D](geom, use_spheroid)
+ def ST_Perimeter2D(geom: String, use_spheroid: Boolean): Column =
+ wrapExpression[ST_Perimeter2D](geom, use_spheroid)
+ def ST_Perimeter2D(geom: Column, use_spheroid: Column, lenient: Column): Column =
+ wrapExpression[ST_Perimeter2D](geom, use_spheroid, lenient)
+ def ST_Perimeter2D(geom: String, use_spheroid: Boolean, lenient: Boolean): Column =
+ wrapExpression[ST_Perimeter2D](geom, use_spheroid, lenient)
+
def ST_Points(geom: Column): Column = wrapExpression[ST_Points](geom)
def ST_Points(geom: String): Column = wrapExpression[ST_Points](geom)
@@ -943,4 +977,46 @@ object st_functions extends DataFrameAPI {
def ST_InterpolatePoint(geom1: String, geom2: String): Column =
wrapExpression[ST_InterpolatePoint](geom1, geom2)
+ def ST_DBSCAN(geom: Column, epsilon: Column, minPoints: Column, useSpheroid: Column): Column =
+ wrapExpression[ST_DBSCAN](geom, epsilon, minPoints, useSpheroid)
+
+ def ST_LocalOutlierFactor(geom: Column, k: Column, useSpheroid: Column): Column =
+ wrapExpression[ST_LocalOutlierFactor](geom, k, useSpheroid)
+
+ def ST_GLocal(x: Column, weights: Column, star: Column): Column =
+ wrapExpression[ST_GLocal](x, weights, star)
+
+ def ST_BinaryDistanceBandColumn(
+ geometry: Column,
+ threshold: Column,
+ includeZeroDistanceNeighbors: Column,
+ includeSelf: Column,
+ useSpheroid: Column,
+ attributes: Column): Column =
+ wrapExpression[ST_BinaryDistanceBandColumn](
+ geometry,
+ threshold,
+ includeZeroDistanceNeighbors,
+ includeSelf,
+ useSpheroid,
+ attributes)
+
+ def ST_WeightedDistanceBandColumn(
+ geometry: Column,
+ threshold: Column,
+ alpha: Column,
+ includeZeroDistanceNeighbors: Column,
+ includeSelf: Column,
+ selfWeight: Column,
+ useSpheroid: Column,
+ attributes: Column): Column =
+ wrapExpression[ST_BinaryDistanceBandColumn](
+ geometry,
+ threshold,
+ alpha,
+ includeZeroDistanceNeighbors,
+ includeSelf,
+ selfWeight,
+ useSpheroid,
+ attributes)
}
diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/stac/StacBatch.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/stac/StacBatch.scala
new file mode 100644
index 0000000000..98cb35ee07
--- /dev/null
+++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/stac/StacBatch.scala
@@ -0,0 +1,264 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.sedona_sql.io.stac
+
+import com.fasterxml.jackson.databind.{JsonNode, ObjectMapper}
+import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory}
+import org.apache.spark.sql.execution.datasource.stac.TemporalFilter
+import org.apache.spark.sql.execution.datasources.parquet.{GeoParquetSpatialFilter, GeometryFieldMetaData}
+import org.apache.spark.sql.sedona_sql.io.stac.StacUtils.getNumPartitions
+import org.apache.spark.sql.types.StructType
+
+import java.time.LocalDateTime
+import java.time.format.DateTimeFormatterBuilder
+import java.time.temporal.ChronoField
+import scala.jdk.CollectionConverters._
+
+/**
+ * The `StacBatch` class represents a batch of partitions for reading data in the SpatioTemporal
+ * Asset Catalog (STAC) data source. It implements the `Batch` interface from Apache Spark's data
+ * source API.
+ *
+ * This class provides methods to plan input partitions and create a partition reader factory,
+ * which are necessary for batch data processing.
+ */
+case class StacBatch(
+ stacCollectionUrl: String,
+ stacCollectionJson: String,
+ schema: StructType,
+ opts: Map[String, String],
+ spatialFilter: Option[GeoParquetSpatialFilter],
+ temporalFilter: Option[TemporalFilter])
+ extends Batch {
+
+ val mapper = new ObjectMapper()
+
+ /**
+ * Plans the input partitions for reading data from the STAC data source.
+ *
+ * @return
+ * An array of input partitions for reading STAC data.
+ */
+ override def planInputPartitions(): Array[InputPartition] = {
+ val stacCollectionBasePath = StacUtils.getStacCollectionBasePath(stacCollectionUrl)
+
+ // Initialize the itemLinks array
+ val itemLinks = scala.collection.mutable.ArrayBuffer[String]()
+
+ // Start the recursive collection of item links
+ collectItemLinks(stacCollectionBasePath, stacCollectionJson, itemLinks)
+
+ // Handle when the number of items is less than 1
+ if (itemLinks.isEmpty) {
+ return Array.empty[InputPartition]
+ }
+
+ val numPartitions = getNumPartitions(
+ itemLinks.length,
+ opts.getOrElse("numPartitions", "-1").toInt,
+ opts.getOrElse("maxPartitionItemFiles", "-1").toInt,
+ opts.getOrElse("defaultParallelism", "1").toInt)
+
+ // Handle when the number of items is less than the number of partitions
+ if (itemLinks.length < numPartitions) {
+ return itemLinks.zipWithIndex.map { case (item, index) =>
+ StacPartition(index, Array(item), new java.util.HashMap[String, String]())
+ }.toArray
+ }
+
+ // Determine how many items to put in each partition
+ val partitionSize = Math.ceil(itemLinks.length.toDouble / numPartitions).toInt
+
+ // Group the item links into partitions
+ itemLinks
+ .grouped(partitionSize)
+ .zipWithIndex
+ .map { case (items, index) =>
+ // Create a StacPartition for each group of items
+ StacPartition(index, items.toArray, new java.util.HashMap[String, String]())
+ }
+ .toArray
+ }
+
+ /**
+ * Recursively processes collections and collects item links.
+ *
+ * @param collectionBasePath
+ * The base path of the STAC collection.
+ * @param collectionJson
+ * The JSON string representation of the STAC collection.
+ * @param itemLinks
+ * The list of item links to populate.
+ */
+ private def collectItemLinks(
+ collectionBasePath: String,
+ collectionJson: String,
+ itemLinks: scala.collection.mutable.ArrayBuffer[String]): Unit = {
+ // Parse the JSON string into a JsonNode (tree representation of JSON)
+ val rootNode: JsonNode = mapper.readTree(collectionJson)
+
+ // Extract item links from the "links" array
+ val linksNode = rootNode.get("links")
+ val iterator = linksNode.elements()
+ while (iterator.hasNext) {
+ val linkNode = iterator.next()
+ val rel = linkNode.get("rel").asText()
+ val href = linkNode.get("href").asText()
+
+ // item links are identified by the "rel" value of "item" or "items"
+ if (rel == "item" || rel == "items") {
+ // need to handle relative paths and local file paths
+ val itemUrl = if (href.startsWith("http") || href.startsWith("file")) {
+ href
+ } else {
+ collectionBasePath + href
+ }
+ itemLinks += itemUrl // Add the item URL to the list
+ } else if (rel == "child") {
+ val childUrl = if (href.startsWith("http") || href.startsWith("file")) {
+ href
+ } else {
+ collectionBasePath + href
+ }
+ // Recursively process the linked collection
+ val linkedCollectionJson = StacUtils.loadStacCollectionToJson(childUrl)
+ val nestedCollectionBasePath = StacUtils.getStacCollectionBasePath(childUrl)
+ val collectionFiltered =
+ filterCollection(linkedCollectionJson, spatialFilter, temporalFilter)
+
+ if (!collectionFiltered) {
+ collectItemLinks(nestedCollectionBasePath, linkedCollectionJson, itemLinks)
+ }
+ }
+ }
+ }
+
+ /**
+ * Filters a collection based on the provided spatial and temporal filters.
+ *
+ * @param collectionJson
+ * The JSON string representation of the STAC collection.
+ * @param spatialFilter
+ * The spatial filter to apply to the collection.
+ * @param temporalFilter
+ * The temporal filter to apply to the collection.
+ * @return
+ * `true` if the collection is filtered out, `false` otherwise.
+ */
+ def filterCollection(
+ collectionJson: String,
+ spatialFilter: Option[GeoParquetSpatialFilter],
+ temporalFilter: Option[TemporalFilter]): Boolean = {
+
+ val mapper = new ObjectMapper()
+ val rootNode: JsonNode = mapper.readTree(collectionJson)
+
+ // Filter based on spatial extent
+ val spatialFiltered = spatialFilter match {
+ case Some(filter) =>
+ val extentNode = rootNode.path("extent").path("spatial").path("bbox")
+ if (extentNode.isMissingNode) {
+ false
+ } else {
+ val bbox = extentNode
+ .elements()
+ .asScala
+ .map { bboxNode =>
+ val minX = bboxNode.get(0).asDouble()
+ val minY = bboxNode.get(1).asDouble()
+ val maxX = bboxNode.get(2).asDouble()
+ val maxY = bboxNode.get(3).asDouble()
+ (minX, minY, maxX, maxY)
+ }
+ .toList
+
+ !bbox.exists { case (minX, minY, maxX, maxY) =>
+ val geometryTypes = Seq("Polygon")
+ val bbox = Seq(minX, minY, maxX, maxY)
+
+ val geometryFieldMetaData = GeometryFieldMetaData(
+ encoding = "WKB",
+ geometryTypes = geometryTypes,
+ bbox = bbox,
+ crs = None,
+ covering = None)
+
+ filter.evaluate(Map("geometry" -> geometryFieldMetaData))
+ }
+ }
+ case None => false
+ }
+
+ // Filter based on temporal extent
+ val temporalFiltered = temporalFilter match {
+ case Some(filter) =>
+ val extentNode = rootNode.path("extent").path("temporal").path("interval")
+ if (extentNode.isMissingNode) {
+ // if extent is missing, we assume the collection is not filtered
+ true
+ } else {
+ // parse the temporal intervals
+ val formatter = new DateTimeFormatterBuilder()
+ .appendPattern("yyyy-MM-dd'T'HH:mm:ss")
+ .optionalStart()
+ .appendFraction(ChronoField.MILLI_OF_SECOND, 0, 3, true)
+ .optionalEnd()
+ .appendPattern("'Z'")
+ .toFormatter()
+
+ val intervals = extentNode
+ .elements()
+ .asScala
+ .map { intervalNode =>
+ val start = LocalDateTime.parse(intervalNode.get(0).asText(), formatter)
+ val end = LocalDateTime.parse(intervalNode.get(1).asText(), formatter)
+ (start, end)
+ }
+ .toList
+
+ // check if the filter evaluates to true for any of the interval start or end times
+ !intervals.exists { case (start, end) =>
+ filter.evaluate(Map("datetime" -> start)) ||
+ filter.evaluate(Map("datetime" -> end))
+ }
+ }
+ // if the collection is not filtered, return false
+ case None => false
+ }
+
+ spatialFiltered || temporalFiltered
+ }
+
+ /**
+ * Creates a partition reader factory for reading data from the STAC data source.
+ *
+ * @return
+ * A partition reader factory for reading STAC data.
+ */
+ override def createReaderFactory(): PartitionReaderFactory = { (partition: InputPartition) =>
+ {
+ new StacPartitionReader(
+ partition.asInstanceOf[StacPartition],
+ schema,
+ opts,
+ spatialFilter,
+ temporalFilter)
+ }
+ }
+}
diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/stac/StacDataSource.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/stac/StacDataSource.scala
new file mode 100644
index 0000000000..ac64b8393b
--- /dev/null
+++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/stac/StacDataSource.scala
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.sedona_sql.io.stac
+
+import StacUtils.{inferStacSchema, updatePropertiesPromotedSchema}
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.sedona_sql.UDT.GeometryUDT
+import org.apache.spark.sql.sedona_sql.io.geojson.GeoJSONUtils
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+import java.util
+import java.util.concurrent.ConcurrentHashMap
+import scala.jdk.CollectionConverters._
+
+/**
+ * The `StacDataSource` class is responsible for enabling the reading of SpatioTemporal Asset
+ * Catalogs (STAC) as tables in Apache Spark. It allows integrating geospatial metadata from local
+ * or remote STAC catalog sources into Spark for processing.
+ *
+ * This class implements Apache Spark's `TableProvider` interface to define how STAC data sources
+ * are converted into Spark tables, and the `DataSourceRegister` interface to provide a custom
+ * short name for easier data source loading.
+ */
+class StacDataSource() extends TableProvider with DataSourceRegister {
+
+ // Cache to store inferred schemas
+ private val schemaCache = new ConcurrentHashMap[Map[String, String], StructType]()
+
+ /**
+ * Returns the short name of this data source, which can be used in Spark SQL queries for
+ * loading the data source. For example:
+ *
+ * `spark.read.format("stac").load(...)`
+ *
+ * @return
+ * The string identifier for this data source, "stac".
+ */
+ override def shortName(): String = "stac"
+
+ /**
+ * Infers and returns the schema of the STAC data source. This implementation checks if a local
+ * cache of the processed STAC collection exists. If not, it processes the STAC collection and
+ * saves it as a GeoJson file. The schema is then inferred from this GeoJson file.
+ *
+ * @param opts
+ * Mapping of data source options, which should include either 'url' or 'service'.
+ * @return
+ * The inferred schema of the STAC data source table.
+ * @throws IllegalArgumentException
+ * If neither 'url' nor 'service' are provided.
+ */
+ override def inferSchema(opts: CaseInsensitiveStringMap): StructType = {
+ val optsMap = opts.asCaseSensitiveMap().asScala.toMap
+
+ // Check if the schema is already cached
+ val fullSchema = schemaCache.computeIfAbsent(optsMap, _ => inferStacSchema(optsMap))
+ val updatedGeometrySchema = GeoJSONUtils.updateGeometrySchema(fullSchema, GeometryUDT)
+ updatePropertiesPromotedSchema(updatedGeometrySchema)
+ }
+
+ /**
+ * Provides a table implementation for the STAC data source based on the input schema and
+ * configuration properties. This method supports loading STAC catalogs either from a local file
+ * system or from a remote HTTP/HTTPS endpoint.
+ *
+ * @param schema
+ * The schema of the table, ignored as the schema is pre-defined.
+ * @param partitioning
+ * Unused, but represents potential transformations (partitioning) in Spark.
+ * @param properties
+ * A map of properties to configure the data source. Must include either "path" for local file
+ * access or "service" for HTTP access.
+ * @return
+ * An instance of `StacTable`, wrapping the parsed STAC catalog JSON data.
+ * @throws IllegalArgumentException
+ * If neither "url" nor "service" are provided.
+ */
+ override def getTable(
+ schema: StructType,
+ partitioning: Array[Transform],
+ properties: util.Map[String, String]): Table = {
+ val opts = new CaseInsensitiveStringMap(properties)
+
+ val optsMap: Map[String, String] = opts.asCaseSensitiveMap().asScala.toMap ++ Map(
+ "sessionLocalTimeZone" -> SparkSession.active.sessionState.conf.sessionLocalTimeZone,
+ "columnNameOfCorruptRecord" -> SparkSession.active.sessionState.conf.columnNameOfCorruptRecord,
+ "defaultParallelism" -> SparkSession.active.sparkContext.defaultParallelism.toString,
+ "maxPartitionItemFiles" -> SparkSession.active.conf
+ .get("spark.wherobots.stac.load.maxPartitionItemFiles", "0"),
+ "numPartitions" -> SparkSession.active.conf
+ .get("spark.wherobots.stac.load.numPartitions", "-1"))
+ val stacCollectionJsonString = StacUtils.loadStacCollectionToJson(optsMap)
+
+ new StacTable(stacCollectionJson = stacCollectionJsonString, opts = optsMap)
+ }
+}
diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/stac/StacPartition.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/stac/StacPartition.scala
new file mode 100644
index 0000000000..5589bd9bad
--- /dev/null
+++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/stac/StacPartition.scala
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.sedona_sql.io.stac
+
+import org.apache.spark.sql.connector.read.InputPartition
+
+case class StacPartition(index: Int, items: Array[String], opts: java.util.Map[String, String])
+ extends InputPartition
diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/stac/StacPartitionReader.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/stac/StacPartitionReader.scala
new file mode 100644
index 0000000000..4929087db5
--- /dev/null
+++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/stac/StacPartitionReader.scala
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.sedona_sql.io.stac
+
+import com.fasterxml.jackson.databind.ObjectMapper
+import org.apache.hadoop.conf.Configuration
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.json.JSONOptionsInRead
+import org.apache.spark.sql.connector.read.PartitionReader
+import org.apache.spark.sql.execution.datasource.stac.TemporalFilter
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.execution.datasources.json.JsonDataSource
+import org.apache.spark.sql.execution.datasources.parquet.GeoParquetSpatialFilter
+import org.apache.spark.sql.sedona_sql.io.geojson.{GeoJSONUtils, SparkCompatUtil}
+import org.apache.spark.sql.sedona_sql.io.stac.StacUtils.{buildOutDbRasterFields, promotePropertiesToTop}
+import org.apache.spark.sql.types.{StringType, StructType}
+
+import java.io.{File, PrintWriter}
+import java.lang.reflect.Constructor
+import scala.io.Source
+
+class StacPartitionReader(
+ partition: StacPartition,
+ schema: StructType,
+ opts: Map[String, String],
+ spatialFilter: Option[GeoParquetSpatialFilter],
+ temporalFilter: Option[TemporalFilter])
+ extends PartitionReader[InternalRow] {
+
+ private val itemsIterator = partition.items.iterator
+ private var currentItem: String = _
+ private var currentFile: File = _
+ private var featureIterator: Iterator[InternalRow] = Iterator.empty
+ private val mapper = new ObjectMapper()
+
+ override def next(): Boolean = {
+ if (featureIterator.hasNext) {
+ true
+ } else if (itemsIterator.hasNext) {
+ currentItem = itemsIterator.next()
+ if (currentItem.startsWith("http://") || currentItem.startsWith("https://") || currentItem
+ .startsWith("file://")) {
+ val url = new java.net.URL(currentItem)
+
+ // Download the file to a local temp file
+ val tempFile = File.createTempFile("stac_item_", ".json")
+ val writer = new PrintWriter(tempFile)
+ try {
+ val fileContent = Source.fromURL(url).mkString
+ val rootNode = mapper.readTree(fileContent)
+ val nodeType = rootNode.get("type").asText()
+
+ nodeType match {
+ case "Feature" =>
+ // Write the content as a single line JSON
+ val content = mapper.writeValueAsString(rootNode)
+ writer.write(content)
+ case "FeatureCollection" =>
+ // Write each feature in the features array to a multi-line JSON file
+ val features = rootNode.get("features")
+ val featureIterator = features.elements()
+ while (featureIterator.hasNext) {
+ val feature = featureIterator.next()
+ val content = mapper.writeValueAsString(feature)
+ writer.write(content)
+ writer.write("\n")
+ }
+ case _ =>
+ throw new IllegalArgumentException(s"Unsupported type for item: $nodeType")
+ }
+
+ } finally {
+ writer.close()
+ }
+ checkAndDeleteTempFile(currentFile)
+ currentFile = tempFile
+ } else {
+ throw new IllegalArgumentException(s"Unsupported protocol for item: $currentItem")
+ }
+
+ // Parse the current file and extract features
+ featureIterator = if (currentFile.exists()) {
+
+ val parsedOptions = new JSONOptionsInRead(
+ opts,
+ opts.getOrElse("sessionLocalTimeZone", "UTC"),
+ opts.getOrElse("columnNameOfCorruptRecord", "_corrupt_record"))
+ val dataSource = JsonDataSource(parsedOptions)
+
+ val alteredSchema = GeoJSONUtils.updateGeometrySchema(schema, StringType)
+
+ val parser = SparkCompatUtil.constructJacksonParser(
+ alteredSchema,
+ parsedOptions,
+ allowArrayAsStructs = true)
+
+ val rows = SparkCompatUtil
+ .readFile(
+ dataSource,
+ new Configuration(),
+ createPartitionedFile(currentFile),
+ parser,
+ schema)
+
+ rows.map(row => {
+ val geometryConvertedRow = GeoJSONUtils.convertGeoJsonToGeometry(row, alteredSchema)
+ val rasterAddedRow = buildOutDbRasterFields(geometryConvertedRow, alteredSchema)
+ val propertiesPromotedRow = promotePropertiesToTop(rasterAddedRow, alteredSchema)
+ propertiesPromotedRow
+ })
+ } else {
+ Iterator.empty
+ }
+
+ next()
+ } else {
+ false
+ }
+ }
+
+ override def get(): InternalRow = {
+ featureIterator.next()
+ }
+
+ override def close(): Unit = {
+ checkAndDeleteTempFile(currentFile)
+ }
+
+ private def checkAndDeleteTempFile(file: File): Unit = {
+ // Delete the local file if it was downloaded to tmp
+ if (file != null && file.exists() && file.getAbsolutePath.startsWith(
+ System.getProperty("java.io.tmpdir"))) {
+ file.delete()
+ }
+ }
+
+ /**
+ * Create a PartitionedFile instance using reflection. The constructor parameters differ between
+ * these versions, so we need to handle both cases. For Spark 3.4 and below, the constructor has
+ * 7 parameters, while for Spark 3.5 and above, it has 8 parameters. Additionally, the type of
+ * the second parameter may be `SparkPath` in some cases, which requires special handling.
+ *
+ * @param currentFile
+ * The file to create the PartitionedFile for.
+ * @return
+ * The created PartitionedFile instance.
+ * @throws NoSuchMethodException
+ * If no suitable constructor is found.
+ */
+ def createPartitionedFile(currentFile: File): PartitionedFile = {
+ val partitionedFileClass =
+ Class.forName("org.apache.spark.sql.execution.datasources.PartitionedFile")
+ val constructors = partitionedFileClass.getConstructors
+ val constructor = constructors
+ .find(_.getParameterCount == 7)
+ .getOrElse(
+ constructors
+ .find(_.getParameterCount == 8)
+ .getOrElse(
+ throw new NoSuchMethodException("No constructor with 7 or 8 parameters found")))
+
+ val params = if (constructor.getParameterCount == 7) {
+ val secondParamType = constructor.getParameterTypes()(1)
+ if (secondParamType.getName == "org.apache.spark.paths.SparkPath") {
+ Array(
+ null,
+ createSparkPath(currentFile.getPath),
+ java.lang.Long.valueOf(0L),
+ java.lang.Long.valueOf(currentFile.length()),
+ Array.empty[String],
+ java.lang.Long.valueOf(0L),
+ java.lang.Long.valueOf(0L))
+ } else {
+ Array(
+ null,
+ currentFile.getPath,
+ java.lang.Long.valueOf(0L),
+ java.lang.Long.valueOf(currentFile.length()),
+ Array.empty[String],
+ java.lang.Long.valueOf(0L),
+ java.lang.Long.valueOf(0L))
+ }
+ } else {
+ Array(
+ null,
+ createSparkPath(currentFile.getPath),
+ java.lang.Long.valueOf(0L),
+ java.lang.Long.valueOf(currentFile.length()),
+ Array.empty[String],
+ java.lang.Long.valueOf(0L),
+ java.lang.Long.valueOf(0L),
+ null)
+ }
+
+ constructor.newInstance(params: _*).asInstanceOf[PartitionedFile]
+ }
+
+ /**
+ * Create a SparkPath instance using reflection. This is needed to support both Spark 3.3 and
+ * below and Spark 3.4 and above.
+ *
+ * @param pathString
+ * The path to create the SparkPath for.
+ * @return
+ * The created SparkPath instance.
+ */
+ def createSparkPath(pathString: String): Object = {
+ val sparkPathClass = Class.forName("org.apache.spark.paths.SparkPath")
+ val constructor: Constructor[_] = sparkPathClass.getDeclaredConstructor(classOf[String])
+ constructor.setAccessible(true) // Make the private constructor accessible
+ constructor.newInstance(pathString).asInstanceOf[Object]
+ }
+}
diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/stac/StacScan.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/stac/StacScan.scala
new file mode 100644
index 0000000000..2edf082912
--- /dev/null
+++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/stac/StacScan.scala
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.sedona_sql.io.stac
+
+import org.apache.spark.sql.connector.read.{Batch, Scan}
+import org.apache.spark.sql.execution.datasource.stac.TemporalFilter
+import org.apache.spark.sql.execution.datasources.parquet.GeoParquetSpatialFilter
+import org.apache.spark.sql.internal.connector.SupportsMetadata
+import org.apache.spark.sql.sedona_sql.io.stac.StacUtils.{getFullCollectionUrl, inferStacSchema}
+import org.apache.spark.sql.types.StructType
+
+class StacScan(stacCollectionJson: String, opts: Map[String, String])
+ extends Scan
+ with SupportsMetadata {
+
+ // The spatial filter to be pushed down to the data source
+ var spatialFilter: Option[GeoParquetSpatialFilter] = None
+
+ // The temporal filter to be pushed down to the data source
+ var temporalFilter: Option[TemporalFilter] = None
+
+ /**
+ * Returns the schema of the data to be read.
+ *
+ * The schema is statically defined in the `StacTable` object.
+ */
+ override def readSchema(): StructType = {
+ val url = opts.get("path")
+ val service = opts.get("service")
+
+ if (url == null && service == null) {
+ throw new IllegalArgumentException("Either 'path' or 'service' must be provided")
+ }
+
+ inferStacSchema(opts)
+ }
+
+ /**
+ * Returns a `Batch` instance for reading the data in batch mode.
+ *
+ * The `StacBatch` class provides the implementation for the batch reading.
+ *
+ * @return
+ * A `Batch` instance for batch-based data processing.
+ */
+ override def toBatch: Batch = {
+ val stacCollectionUrl = getFullCollectionUrl(opts)
+ StacBatch(
+ stacCollectionUrl,
+ stacCollectionJson,
+ readSchema(),
+ opts,
+ spatialFilter,
+ temporalFilter)
+ }
+
+ /**
+ * Sets the spatial predicates to be pushed down to the data source.
+ *
+ * @param combinedSpatialFilter
+ * The combined spatial filter to be pushed down.
+ */
+ def setSpatialPredicates(combinedSpatialFilter: GeoParquetSpatialFilter) = {
+ spatialFilter = Some(combinedSpatialFilter)
+ }
+
+ /**
+ * Sets the temporal predicates to be pushed down to the data source.
+ *
+ * @param combineTemporalFilter
+ * The combined temporal filter to be pushed down.
+ */
+ def setTemporalPredicates(combineTemporalFilter: TemporalFilter) = {
+ temporalFilter = Some(combineTemporalFilter)
+ }
+
+ /**
+ * Returns metadata about the data to be read.
+ *
+ * The metadata includes information about the pushed filters.
+ *
+ * @return
+ * A map of metadata key-value pairs.
+ */
+ override def getMetaData(): Map[String, String] = {
+ Map(
+ "PushedSpatialFilters" -> spatialFilter.map(_.toString).getOrElse("None"),
+ "PushedTemporalFilters" -> temporalFilter.map(_.toString).getOrElse("None"))
+ }
+
+ /**
+ * Returns a description of the data to be read.
+ *
+ * The description includes the metadata information.
+ *
+ * @return
+ * A string description of the data to be read.
+ */
+ override def description(): String = {
+ super.description() + " " + getMetaData().mkString(", ")
+ }
+}
diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/stac/StacScanBuilder.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/stac/StacScanBuilder.scala
new file mode 100644
index 0000000000..ebaab87dda
--- /dev/null
+++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/stac/StacScanBuilder.scala
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.sedona_sql.io.stac
+
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+
+/**
+ * The `StacScanBuilder` class represents the builder for creating a `Scan` instance in the
+ * SpatioTemporal Asset Catalog (STAC) data source.
+ *
+ * This class is responsible for assembling the scan operation for reading STAC data. It acts as a
+ * bridge between Spark's data source API and the specific implementation of the STAC data read
+ * operation.
+ */
+class StacScanBuilder(stacCollectionJson: String, opts: Map[String, String]) extends ScanBuilder {
+
+ /**
+ * Builds and returns a `Scan` instance. The `Scan` defines the schema and batch reading methods
+ * for STAC data.
+ *
+ * @return
+ * A `Scan` instance that defines how to read STAC data.
+ */
+ override def build(): Scan = new StacScan(stacCollectionJson, opts)
+}
diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/stac/StacTable.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/stac/StacTable.scala
new file mode 100644
index 0000000000..bd536f6de6
--- /dev/null
+++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/stac/StacTable.scala
@@ -0,0 +1,261 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.sedona_sql.io.stac
+
+import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.sedona_sql.UDT.GeometryUDT
+import org.apache.spark.sql.sedona_sql.io.geojson.GeoJSONUtils
+import org.apache.spark.sql.sedona_sql.io.stac.StacUtils.{inferStacSchema, updatePropertiesPromotedSchema}
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+import java.util.concurrent.ConcurrentHashMap
+
+/**
+ * The `StacTable` class represents a table in the SpatioTemporal Asset Catalog (STAC) data
+ * source.
+ *
+ * This class implements the `Table` and `SupportsRead` interfaces to integrate with Apache
+ * Spark's data source API, providing support for reading data from STAC.
+ *
+ * @constructor
+ * Creates a new instance of the `StacTable` class.
+ */
+class StacTable(stacCollectionJson: String, opts: Map[String, String])
+ extends Table
+ with SupportsRead {
+
+ // Cache to store inferred schemas
+ private val schemaCache = new ConcurrentHashMap[Map[String, String], StructType]()
+
+ /**
+ * Returns the name of the table.
+ *
+ * @return
+ * The name of the table as a string.
+ */
+ override def name(): String = "stac"
+
+ /**
+ * Defines the schema of the STAC table.
+ *
+ * @return
+ * The schema as a StructType.
+ */
+ override def schema(): StructType = {
+ // Check if the schema is already cached
+ val fullSchema = schemaCache.computeIfAbsent(opts, _ => inferStacSchema(opts))
+ val updatedGeometrySchema = GeoJSONUtils.updateGeometrySchema(fullSchema, GeometryUDT)
+ updatePropertiesPromotedSchema(updatedGeometrySchema)
+ }
+
+ /**
+ * Indicates the capabilities supported by the STAC table, specifically batch read.
+ *
+ * @return
+ * A set of table capabilities.
+ */
+ override def capabilities(): java.util.Set[TableCapability] =
+ java.util.EnumSet.of(TableCapability.BATCH_READ)
+
+ /**
+ * Creates a new scan builder for reading data from the STAC table.
+ *
+ * @param options
+ * The configuration options for the scan.
+ * @return
+ * A new instance of ScanBuilder.
+ */
+ override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder =
+ new StacScanBuilder(stacCollectionJson, opts)
+}
+
+object StacTable {
+
+ /**
+ * Defines the schema of the STAC table, which supports various fields including collection
+ * information, asset details, geometries, and more. The schema is based on the STAC
+ * specification version 1.1.0.
+ */
+ val SCHEMA_V1_1_0: StructType = StructType(
+ Seq(
+ StructField("stac_version", StringType, nullable = false),
+ StructField("stac_extensions", ArrayType(StringType), nullable = true),
+ StructField("type", StringType, nullable = false),
+ StructField("id", StringType, nullable = false),
+ StructField("bbox", ArrayType(DoubleType), nullable = true),
+ StructField(
+ "geometry",
+ StructType(
+ Seq(
+ StructField("type", StringType, nullable = true),
+ StructField("coordinates", ArrayType(ArrayType(DoubleType)), nullable = true))),
+ nullable = true),
+ StructField(
+ "properties",
+ StructType(Seq(
+ StructField("title", StringType, nullable = true),
+ StructField("description", StringType, nullable = true),
+ StructField("datetime", TimestampType, nullable = true),
+ StructField("start_datetime", TimestampType, nullable = true),
+ StructField("end_datetime", TimestampType, nullable = true),
+ StructField("created", TimestampType, nullable = true),
+ StructField("updated", TimestampType, nullable = true),
+ StructField("platform", StringType, nullable = true),
+ StructField("instruments", ArrayType(StringType), nullable = true),
+ StructField("constellation", StringType, nullable = true),
+ StructField("mission", StringType, nullable = true),
+ StructField("gsd", DoubleType, nullable = true))),
+ nullable = false),
+ StructField("collection", StringType, nullable = true),
+ StructField(
+ "links",
+ ArrayType(StructType(Seq(
+ StructField("rel", StringType, nullable = true),
+ StructField("href", StringType, nullable = true),
+ StructField("type", StringType, nullable = true),
+ StructField("title", StringType, nullable = true)))),
+ nullable = false),
+ StructField(
+ "assets",
+ MapType(
+ StringType,
+ StructType(Seq(
+ StructField("href", StringType, nullable = true),
+ StructField("type", StringType, nullable = true),
+ StructField("title", StringType, nullable = true),
+ StructField("roles", ArrayType(StringType), nullable = true)))),
+ nullable = false)))
+
+ /**
+ * Defines the schema of the STAC table, which supports various fields including collection
+ * information, asset details, geometries, and more. The schema is based on the STAC
+ * specification version 1.0.0.
+ */
+ val SCHEMA_V1_0_0: StructType = StructType(
+ Seq(
+ StructField("stac_version", StringType, nullable = false),
+ StructField("stac_extensions", ArrayType(StringType), nullable = true),
+ StructField("type", StringType, nullable = false),
+ StructField("id", StringType, nullable = false),
+ StructField("bbox", ArrayType(DoubleType), nullable = true),
+ StructField(
+ "geometry",
+ StructType(
+ Seq(
+ StructField("type", StringType, nullable = true),
+ StructField("coordinates", ArrayType(ArrayType(DoubleType)), nullable = true))),
+ nullable = true),
+ StructField(
+ "properties",
+ StructType(Seq(
+ StructField("title", StringType, nullable = true),
+ StructField("description", StringType, nullable = true),
+ StructField("datetime", TimestampType, nullable = true),
+ StructField("start_datetime", TimestampType, nullable = true),
+ StructField("end_datetime", TimestampType, nullable = true),
+ StructField("created", TimestampType, nullable = true),
+ StructField("updated", TimestampType, nullable = true),
+ StructField("platform", StringType, nullable = true),
+ StructField("instruments", ArrayType(StringType), nullable = true),
+ StructField("constellation", StringType, nullable = true),
+ StructField("mission", StringType, nullable = true),
+ StructField("gsd", DoubleType, nullable = true))),
+ nullable = true),
+ StructField("collection", StringType, nullable = true),
+ StructField(
+ "links",
+ ArrayType(StructType(Seq(
+ StructField("rel", StringType, nullable = true),
+ StructField("href", StringType, nullable = true),
+ StructField("type", StringType, nullable = true),
+ StructField("title", StringType, nullable = true)))),
+ nullable = true),
+ StructField(
+ "assets",
+ MapType(
+ StringType,
+ StructType(Seq(
+ StructField("href", StringType, nullable = true),
+ StructField("type", StringType, nullable = true),
+ StructField("title", StringType, nullable = true),
+ StructField("roles", ArrayType(StringType), nullable = true)))),
+ nullable = true)))
+
+ val SCHEMA_GEOPARQUET: StructType = StructType(
+ Seq(
+ StructField("stac_version", StringType, nullable = false),
+ StructField("stac_extensions", ArrayType(StringType), nullable = true),
+ StructField("type", StringType, nullable = false),
+ StructField("id", StringType, nullable = false),
+ StructField("bbox", ArrayType(DoubleType), nullable = true),
+ StructField(
+ "geometry",
+ StructType(
+ Seq(
+ StructField("type", StringType, nullable = true),
+ StructField("coordinates", ArrayType(ArrayType(DoubleType)), nullable = true))),
+ nullable = true),
+ StructField("datetime", TimestampType, nullable = true),
+ StructField("collection", StringType, nullable = true),
+ StructField(
+ "links",
+ ArrayType(StructType(Seq(
+ StructField("rel", StringType, nullable = true),
+ StructField("href", StringType, nullable = true),
+ StructField("type", StringType, nullable = true),
+ StructField("title", StringType, nullable = true)))),
+ nullable = false)))
+
+ def addAssetStruct(schema: StructType, name: String): StructType = {
+ val assetStruct = StructType(
+ Seq(
+ StructField("href", StringType, nullable = true),
+ StructField("roles", ArrayType(StringType), nullable = true),
+ StructField("title", StringType, nullable = true),
+ StructField("type", StringType, nullable = true)))
+
+ val updatedFields = schema.fields.map {
+ case StructField("assets", existingStruct: StructType, nullable, metadata) =>
+ StructField(
+ "assets",
+ StructType(existingStruct.fields :+ StructField(name, assetStruct, nullable = true)),
+ nullable,
+ metadata)
+ case other => other
+ }
+
+ if (!schema.fieldNames.contains("assets")) {
+ StructType(
+ updatedFields :+ StructField(
+ "assets",
+ StructType(Seq(StructField(name, assetStruct, nullable = true))),
+ nullable = true))
+ } else {
+ StructType(updatedFields)
+ }
+ }
+
+ def addAssetsStruct(schema: StructType, names: Array[String]): StructType = {
+ names.foldLeft(schema) { (currentSchema, name) =>
+ addAssetStruct(currentSchema, name)
+ }
+ }
+}
diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/stac/StacUtils.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/stac/StacUtils.scala
new file mode 100644
index 0000000000..4e148422bf
--- /dev/null
+++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/io/stac/StacUtils.scala
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.sedona_sql.io.stac
+
+import com.fasterxml.jackson.databind.ObjectMapper
+import com.fasterxml.jackson.module.scala.DefaultScalaModule
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.util.ArrayBasedMapData
+import org.apache.spark.sql.types.{MapType, StringType, StructField, StructType}
+
+import scala.io.Source
+
+object StacUtils {
+
+ // Function to load JSON from URL or service
+ def loadStacCollectionToJson(opts: Map[String, String]): String = {
+ val urlFull: String = getFullCollectionUrl(opts)
+
+ loadStacCollectionToJson(urlFull)
+ }
+
+ def getFullCollectionUrl(opts: Map[String, String]) = {
+ val url = opts.getOrElse(
+ "path",
+ opts.getOrElse(
+ "service",
+ throw new IllegalArgumentException("Either 'path' or 'service' must be provided")))
+ val urlFinal = if (url.matches("^[a-zA-Z][a-zA-Z0-9+.-]*://.*")) url else s"file://$url"
+ urlFinal
+ }
+
+ // Function to load JSON from URL or service
+ def loadStacCollectionToJson(url: String): String = {
+ if (url.startsWith("s3://") || url.startsWith("s3a://")) {
+ SparkSession.active.read.textFile(url).collect().mkString("\n")
+ } else {
+ Source.fromURL(url).mkString
+ }
+ }
+
+ // Function to get the base URL from the collection URL or service
+ def getStacCollectionBasePath(opts: Map[String, String]): String = {
+ val ref = opts.getOrElse(
+ "path",
+ opts.getOrElse(
+ "service",
+ throw new IllegalArgumentException("Either 'path' or 'service' must be provided")))
+ getStacCollectionBasePath(ref)
+ }
+
+ // Function to get the base URL from the collection URL or service
+ def getStacCollectionBasePath(collectionUrl: String): String = {
+ val urlPattern = "(https?://[^/]+/|http://[^/]+/).*".r
+ val filePattern = "(file:///.*/|/.*/).*".r
+
+ collectionUrl match {
+ case urlPattern(baseUrl) => baseUrl
+ case filePattern(basePath) =>
+ if (basePath.startsWith("file://")) basePath else s"file://$basePath"
+ case _ => throw new IllegalArgumentException(s"Invalid URL or file path: $collectionUrl")
+ }
+ }
+
+ /**
+ * Infer the schema of the STAC data source table.
+ *
+ * This method checks if a cached schema exists for the given data source options. If not, it
+ * processes the STAC collection and saves it as a GeoJson file. The schema is then inferred
+ * from this GeoJson file.
+ *
+ * @param opts
+ * Mapping of data source options, which should include either 'url' or 'service'.
+ * @return
+ * The inferred schema of the STAC data source table.
+ * @throws IllegalArgumentException
+ * If neither 'url' nor 'service' are provided.
+ */
+ def inferStacSchema(opts: Map[String, String]): StructType = {
+ val stacCollectionJsonString = loadStacCollectionToJson(opts)
+
+ // Create the ObjectMapper
+ val mapper = new ObjectMapper()
+ mapper.registerModule(DefaultScalaModule)
+
+ // Parse the STAC collection JSON
+ val collection = mapper.readTree(stacCollectionJsonString)
+
+ // Extract the stac_version
+ val stacVersion = collection.get("stac_version").asText()
+
+ // Return the corresponding schema based on the stac_version
+ stacVersion match {
+ case "1.0.0" => StacTable.SCHEMA_V1_0_0
+ case version if version.matches("1\\.[1-9]\\d*\\.\\d*") => StacTable.SCHEMA_V1_1_0
+ // Add more cases here for other versions if needed
+ case _ => throw new IllegalArgumentException(s"Unsupported STAC version: $stacVersion")
+ }
+ }
+
+ /**
+ * Promote the properties field to the top level of the row.
+ */
+ def promotePropertiesToTop(row: InternalRow, schema: StructType): InternalRow = {
+ val propertiesIndex = schema.fieldIndex("properties")
+ val propertiesStruct = schema("properties").dataType.asInstanceOf[StructType]
+ val propertiesRow = row.getStruct(propertiesIndex, propertiesStruct.fields.length)
+
+ val newValues = schema.fields.zipWithIndex.foldLeft(Seq.empty[Any]) {
+ case (acc, (field, index)) if field.name == "properties" =>
+ acc ++ propertiesStruct.fields.zipWithIndex.map { case (propField, propIndex) =>
+ propertiesRow.get(propIndex, propField.dataType)
+ }
+ case (acc, (_, index)) =>
+ acc :+ row.get(index, schema(index).dataType)
+ }
+
+ InternalRow.fromSeq(newValues)
+ }
+
+ def updatePropertiesPromotedSchema(schema: StructType): StructType = {
+ val propertiesIndex = schema.fieldIndex("properties")
+ val propertiesStruct = schema("properties").dataType.asInstanceOf[StructType]
+
+ val newFields = schema.fields.foldLeft(Seq.empty[StructField]) {
+ case (acc, StructField("properties", _, _, _)) =>
+ acc ++ propertiesStruct.fields
+ case (acc, other) =>
+ acc :+ other
+ }
+
+ StructType(newFields)
+ }
+
+ /**
+ * Builds the output row with the raster field in the assets map.
+ *
+ * @param row
+ * The input row.
+ * @param schema
+ * The schema of the input row.
+ * @return
+ * The output row with the raster field in the assets map.
+ */
+ def buildOutDbRasterFields(row: InternalRow, schema: StructType): InternalRow = {
+ val newValues = new Array[Any](schema.fields.length)
+
+ schema.fields.zipWithIndex.foreach {
+ case (StructField("assets", MapType(StringType, valueType: StructType, _), _, _), index) =>
+ val assetsMap = row.getMap(index)
+ if (assetsMap != null) {
+ val updatedAssets = assetsMap
+ .keyArray()
+ .array
+ .zip(assetsMap.valueArray().array)
+ .map { case (key, value) =>
+ val assetRow = value.asInstanceOf[InternalRow]
+ if (assetRow != null) {
+ key -> assetRow
+ } else {
+ key -> null
+ }
+ }
+ .toMap
+ newValues(index) = ArrayBasedMapData(updatedAssets)
+ } else {
+ newValues(index) = null
+ }
+ case (_, index) =>
+ newValues(index) = row.get(index, schema.fields(index).dataType)
+ }
+
+ InternalRow.fromSeq(newValues)
+ }
+
+ /**
+ * Returns the number of partitions to use for reading the data.
+ *
+ * The number of partitions is determined based on the number of items, the number of partitions
+ * requested, the maximum number of item files per partition, and the default parallelism.
+ *
+ * @param itemCount
+ * The number of items in the collection.
+ * @param numPartitions
+ * The number of partitions requested.
+ * @param maxPartitionItemFiles
+ * The maximum number of item files per partition.
+ * @param defaultParallelism
+ * The default parallelism.
+ * @return
+ * The number of partitions to use for reading the data.
+ */
+ def getNumPartitions(
+ itemCount: Int,
+ numPartitions: Int,
+ maxPartitionItemFiles: Int,
+ defaultParallelism: Int): Int = {
+ if (numPartitions > 0) {
+ numPartitions
+ } else {
+ val maxSplitFiles = if (maxPartitionItemFiles > 0) {
+ Math.min(maxPartitionItemFiles, Math.ceil(itemCount.toDouble / defaultParallelism).toInt)
+ } else {
+ Math.ceil(itemCount.toDouble / defaultParallelism).toInt
+ }
+ Math.max(1, Math.ceil(itemCount.toDouble / maxSplitFiles).toInt)
+ }
+ }
+}
diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/optimization/ExtractGeoStatsFunctions.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/optimization/ExtractGeoStatsFunctions.scala
new file mode 100644
index 0000000000..6b4cf9ccea
--- /dev/null
+++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/optimization/ExtractGeoStatsFunctions.scala
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.sedona_sql.optimization
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.sedona_sql.expressions.ST_GeoStatsFunction
+import org.apache.spark.sql.sedona_sql.plans.logical.EvalGeoStatsFunction
+
+import scala.collection.mutable
+
+/**
+ * Extracts GeoStats functions from operators, rewriting the query plan so that the geo-stats
+ * functions can be evaluated alone in its own physical executors.
+ */
+object ExtractGeoStatsFunctions extends Rule[LogicalPlan] {
+ var geoStatsResultCount = 0
+
+ private def collectGeoStatsFunctionsFromExpressions(
+ expressions: Seq[Expression]): Seq[ST_GeoStatsFunction] = {
+ def collectGeoStatsFunctions(expr: Expression): Seq[ST_GeoStatsFunction] = expr match {
+ case expr: ST_GeoStatsFunction => Seq(expr)
+ case e => e.children.flatMap(collectGeoStatsFunctions)
+ }
+ expressions.flatMap(collectGeoStatsFunctions)
+ }
+
+ def apply(plan: LogicalPlan): LogicalPlan = plan match {
+ // SPARK-26293: A subquery will be rewritten into join later, and will go through this rule
+ // eventually. Here we skip subquery, as geo-stats functions only needs to be extracted once.
+ case s: Subquery if s.correlated => plan
+ case _ =>
+ plan.transformUp {
+ case p: EvalGeoStatsFunction => p
+ case plan: LogicalPlan => extract(plan)
+ }
+ }
+
+ private def canonicalizeDeterministic(u: ST_GeoStatsFunction) = {
+ if (u.deterministic) {
+ u.canonicalized.asInstanceOf[ST_GeoStatsFunction]
+ } else {
+ u
+ }
+ }
+
+ /**
+ * Extract all the geo-stats functions from the current operator and evaluate them before the
+ * operator.
+ */
+ private def extract(plan: LogicalPlan): LogicalPlan = {
+ val geoStatsFuncs = plan match {
+ case e: EvalGeoStatsFunction =>
+ collectGeoStatsFunctionsFromExpressions(e.function.children)
+ case _ =>
+ ExpressionSet(collectGeoStatsFunctionsFromExpressions(plan.expressions))
+ // ignore the ST_GeoStatsFunction that come from second/third aggregate, which is not used
+ .filter(func => func.references.subsetOf(plan.inputSet))
+ .filter(func =>
+ plan.children.exists(child => func.references.subsetOf(child.outputSet)))
+ .toSeq
+ .asInstanceOf[Seq[ST_GeoStatsFunction]]
+ }
+
+ if (geoStatsFuncs.isEmpty) {
+ // If there aren't any, we are done.
+ plan
+ } else {
+ // Transform the first geo-stats function we have found. We'll call extract recursively later
+ // to transform the rest.
+ val geoStatsFunc = geoStatsFuncs.head
+
+ val attributeMap = mutable.HashMap[ST_GeoStatsFunction, Expression]()
+ // Rewrite the child that has the input required for the UDF
+ val newChildren = plan.children.map { child =>
+ if (geoStatsFunc.references.subsetOf(child.outputSet)) {
+ geoStatsResultCount += 1
+ val resultAttr =
+ AttributeReference(f"geoStatsResult$geoStatsResultCount", geoStatsFunc.dataType)()
+ val evaluation = EvalGeoStatsFunction(geoStatsFunc, Seq(resultAttr), child)
+ attributeMap += (canonicalizeDeterministic(geoStatsFunc) -> resultAttr)
+ extract(evaluation) // handle nested geo-stats functions
+ } else {
+ child
+ }
+ }
+
+ // Replace the geo stats function call with the newly created geoStatsResult attribute
+ val rewritten = plan.withNewChildren(newChildren).transformExpressions {
+ case p: ST_GeoStatsFunction => attributeMap.getOrElse(canonicalizeDeterministic(p), p)
+ }
+
+ // extract remaining geo-stats functions recursively
+ val newPlan = extract(rewritten)
+ if (newPlan.output != plan.output) {
+ // Trim away the new UDF value if it was only used for filtering or something.
+ Project(plan.output, newPlan)
+ } else {
+ newPlan
+ }
+ }
+ }
+}
diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/optimization/SpatialFilterPushDownForGeoParquet.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/optimization/SpatialFilterPushDownForGeoParquet.scala
index ba0ecf8a40..7ef96ac970 100644
--- a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/optimization/SpatialFilterPushDownForGeoParquet.scala
+++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/optimization/SpatialFilterPushDownForGeoParquet.scala
@@ -92,7 +92,7 @@ class SpatialFilterPushDownForGeoParquet(sparkSession: SparkSession) extends Rul
lr.relation.isInstanceOf[HadoopFsRelation] &&
lr.relation.asInstanceOf[HadoopFsRelation].fileFormat.isInstanceOf[GeoParquetFileFormatBase]
- private def translateToGeoParquetSpatialFilters(
+ def translateToGeoParquetSpatialFilters(
predicates: Seq[Expression]): Seq[GeoParquetSpatialFilter] = {
val pushableColumn = PushableColumn(nestedPredicatePushdownEnabled = false)
predicates.flatMap { predicate =>
diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/optimization/SpatialTemporalFilterPushDownForStacScan.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/optimization/SpatialTemporalFilterPushDownForStacScan.scala
new file mode 100644
index 0000000000..566d368d69
--- /dev/null
+++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/optimization/SpatialTemporalFilterPushDownForStacScan.scala
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.sedona_sql.optimization
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions.{And, EqualTo, Expression, GreaterThan, GreaterThanOrEqual, LessThan, LessThanOrEqual, Literal, Or, SubqueryExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan}
+import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.parseColumnPath
+import org.apache.spark.sql.execution.datasource.stac.TemporalFilter
+import org.apache.spark.sql.execution.datasource.stac.TemporalFilter.{AndFilter => TemporalAndFilter}
+import org.apache.spark.sql.execution.datasources.parquet.GeoParquetSpatialFilter.{AndFilter => SpatialAndFilter}
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, PushableColumn, PushableColumnBase}
+import org.apache.spark.sql.sedona_sql.io.stac.StacScan
+import org.apache.spark.sql.sedona_sql.optimization.ExpressionUtils.splitConjunctivePredicates
+import org.apache.spark.sql.types.TimestampType
+
+import java.time.{Instant, LocalDateTime, ZoneOffset}
+
+/*
+ * This class is responsible for pushing down spatial filters to the STAC data source.
+ * It extends and reuses the `SpatialFilterPushDownForGeoParquet` class, which is responsible for pushing down
+ */
+class SpatialTemporalFilterPushDownForStacScan(sparkSession: SparkSession)
+ extends SpatialFilterPushDownForGeoParquet(sparkSession) {
+
+ /**
+ * Pushes down spatial filters to the STAC data source.
+ *
+ * @param plan
+ * The logical plan to optimize.
+ * @return
+ * The optimized logical plan with spatial filters pushed down to the STAC data source.
+ */
+ override def apply(plan: LogicalPlan): LogicalPlan = {
+ val enableSpatialFilterPushDown =
+ sparkSession.conf.get("spark.sedona.stac.spatialFilterPushDown", "true").toBoolean
+ if (!enableSpatialFilterPushDown) plan
+ else {
+ plan transform {
+ case filter @ Filter(condition, lr: DataSourceV2ScanRelation) if isStacScanRelation(lr) =>
+ val filters = splitConjunctivePredicates(condition)
+ val normalizedFilters = DataSourceStrategy.normalizeExprs(filters, lr.output)
+ val (_, normalizedFiltersWithoutSubquery) =
+ normalizedFilters.partition(SubqueryExpression.hasSubquery)
+ // reuse the `translateToGeoParquetSpatialFilters` method from the `SpatialFilterPushDownForGeoParquet` class
+ val spatialFilters =
+ translateToGeoParquetSpatialFilters(normalizedFiltersWithoutSubquery)
+ if (!spatialFilters.isEmpty) {
+ val combinedSpatialFilter = spatialFilters.reduce(SpatialAndFilter)
+ val scan = lr.scan.asInstanceOf[StacScan]
+ // set the spatial predicates in the STAC scan
+ scan.setSpatialPredicates(combinedSpatialFilter)
+ filter.copy()
+ }
+ val temporalFilters =
+ translateToTemporalFilters(normalizedFiltersWithoutSubquery)
+ if (!temporalFilters.isEmpty) {
+ val combinedTemporalFilter = temporalFilters.reduce(TemporalAndFilter)
+ val scan = lr.scan.asInstanceOf[StacScan]
+ // set the spatial predicates in the STAC scan
+ scan.setTemporalPredicates(combinedTemporalFilter)
+ filter.copy()
+ }
+ filter.copy()
+ }
+ }
+ }
+
+ private def isStacScanRelation(lr: DataSourceV2ScanRelation): Boolean =
+ lr.scan.isInstanceOf[StacScan]
+
+ def translateToTemporalFilters(predicates: Seq[Expression]): Seq[TemporalFilter] = {
+ val pushableColumn = PushableColumn(nestedPredicatePushdownEnabled = true)
+ predicates.flatMap { predicate =>
+ translateToTemporalFilter(predicate, pushableColumn)
+ }
+ }
+
+ private def translateToTemporalFilter(
+ predicate: Expression,
+ pushableColumn: PushableColumnBase): Option[TemporalFilter] = {
+ predicate match {
+ case And(left, right) =>
+ val temporalFilterLeft = translateToTemporalFilter(left, pushableColumn)
+ val temporalFilterRight = translateToTemporalFilter(right, pushableColumn)
+ (temporalFilterLeft, temporalFilterRight) match {
+ case (Some(l), Some(r)) => Some(TemporalFilter.AndFilter(l, r))
+ case (Some(l), None) => Some(l)
+ case (None, Some(r)) => Some(r)
+ case _ => None
+ }
+
+ case Or(left, right) =>
+ for {
+ temporalFilterLeft <- translateToTemporalFilter(left, pushableColumn)
+ temporalFilterRight <- translateToTemporalFilter(right, pushableColumn)
+ } yield TemporalFilter.OrFilter(temporalFilterLeft, temporalFilterRight)
+
+ case LessThan(pushableColumn(name), Literal(v, TimestampType)) =>
+ Some(
+ TemporalFilter
+ .LessThanFilter(
+ unquote(name),
+ LocalDateTime
+ .ofInstant(Instant.ofEpochMilli(v.asInstanceOf[Long] / 1000), ZoneOffset.UTC)))
+
+ case LessThanOrEqual(pushableColumn(name), Literal(v, TimestampType)) =>
+ Some(
+ TemporalFilter
+ .LessThanFilter(
+ unquote(name),
+ LocalDateTime
+ .ofInstant(Instant.ofEpochMilli(v.asInstanceOf[Long] / 1000), ZoneOffset.UTC)))
+
+ case GreaterThan(pushableColumn(name), Literal(v, TimestampType)) =>
+ Some(
+ TemporalFilter
+ .GreaterThanFilter(
+ unquote(name),
+ LocalDateTime
+ .ofInstant(Instant.ofEpochMilli(v.asInstanceOf[Long] / 1000), ZoneOffset.UTC)))
+
+ case GreaterThanOrEqual(pushableColumn(name), Literal(v, TimestampType)) =>
+ Some(
+ TemporalFilter
+ .GreaterThanFilter(
+ unquote(name),
+ LocalDateTime
+ .ofInstant(Instant.ofEpochMilli(v.asInstanceOf[Long] / 1000), ZoneOffset.UTC)))
+
+ case EqualTo(pushableColumn(name), Literal(v, TimestampType)) =>
+ Some(
+ TemporalFilter
+ .EqualFilter(
+ unquote(name),
+ LocalDateTime
+ .ofInstant(Instant.ofEpochMilli(v.asInstanceOf[Long] / 1000), ZoneOffset.UTC)))
+
+ case _ => None
+ }
+ }
+
+ private def unquote(name: String): String = {
+ parseColumnPath(name).mkString(".")
+ }
+}
diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/plans/logical/EvalGeoStatsFunction.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/plans/logical/EvalGeoStatsFunction.scala
new file mode 100644
index 0000000000..8daeb0c304
--- /dev/null
+++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/plans/logical/EvalGeoStatsFunction.scala
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.sedona_sql.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.AttributeSet
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.UnaryNode
+
+case class EvalGeoStatsFunction(
+ function: Expression,
+ resultAttrs: Seq[Attribute],
+ child: LogicalPlan)
+ extends UnaryNode {
+
+ override def output: Seq[Attribute] = child.output ++ resultAttrs
+
+ override def producedAttributes: AttributeSet = AttributeSet(resultAttrs)
+
+ override protected def withNewChildInternal(newChild: LogicalPlan): LogicalPlan =
+ copy(child = newChild)
+}
diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/geostats/EvalGeoStatsFunctionExec.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/geostats/EvalGeoStatsFunctionExec.scala
new file mode 100644
index 0000000000..fbecb69ec4
--- /dev/null
+++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/geostats/EvalGeoStatsFunctionExec.scala
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.sedona_sql.strategy.geostats
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet}
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+import org.apache.spark.sql.sedona_sql.expressions.ST_GeoStatsFunction
+
+case class EvalGeoStatsFunctionExec(
+ function: ST_GeoStatsFunction,
+ child: SparkPlan,
+ resultAttrs: Seq[Attribute])
+ extends UnaryExecNode {
+
+ override protected def doExecute(): RDD[InternalRow] = function.execute(child, resultAttrs)
+
+ override def output: Seq[Attribute] = child.output ++ resultAttrs
+
+ override def producedAttributes: AttributeSet = AttributeSet(resultAttrs)
+
+ override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan =
+ copy(child = newChild)
+}
diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/geostats/EvalGeoStatsFunctionStrategy.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/geostats/EvalGeoStatsFunctionStrategy.scala
new file mode 100644
index 0000000000..4c10b747a6
--- /dev/null
+++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/geostats/EvalGeoStatsFunctionStrategy.scala
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.sedona_sql.strategy.geostats
+
+import org.apache.spark.sql.Strategy
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.sedona_sql.plans.logical.EvalGeoStatsFunction
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.sedona_sql.expressions.ST_GeoStatsFunction
+
+class EvalGeoStatsFunctionStrategy(spark: SparkSession) extends Strategy {
+
+ override def apply(plan: LogicalPlan): Seq[SparkPlan] = {
+ plan match {
+ case EvalGeoStatsFunction(function: ST_GeoStatsFunction, resultAttrs, child) =>
+ EvalGeoStatsFunctionExec(function, planLater(child), resultAttrs) :: Nil
+ case _ => Nil
+ }
+ }
+}
diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/BroadcastObjectSideKNNJoinExec.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/BroadcastObjectSideKNNJoinExec.scala
index 1b21c79e7c..c5777be3c1 100644
--- a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/BroadcastObjectSideKNNJoinExec.scala
+++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/BroadcastObjectSideKNNJoinExec.scala
@@ -120,7 +120,7 @@ case class BroadcastObjectSideKNNJoinExec(
sedonaConf: SedonaConf): Unit = {
require(numPartitions > 0, "The number of partitions must be greater than 0.")
val kValue: Int = this.k.eval().asInstanceOf[Int]
- require(kValue > 0, "The number of neighbors must be greater than 0.")
+ require(kValue >= 1, "The number of neighbors (k) must be equal or greater than 1.")
objectsShapes.setNeighborSampleNumber(kValue)
broadcastJoin = true
}
diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/BroadcastQuerySideKNNJoinExec.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/BroadcastQuerySideKNNJoinExec.scala
index 812bc6e6d6..9ce40c6d42 100644
--- a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/BroadcastQuerySideKNNJoinExec.scala
+++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/BroadcastQuerySideKNNJoinExec.scala
@@ -127,22 +127,13 @@ case class BroadcastQuerySideKNNJoinExec(
sedonaConf: SedonaConf): Unit = {
require(numPartitions > 0, "The number of partitions must be greater than 0.")
val kValue: Int = this.k.eval().asInstanceOf[Int]
- require(kValue > 0, "The number of neighbors must be greater than 0.")
+ require(kValue >= 1, "The number of neighbors (k) must be equal or greater than 1.")
objectsShapes.setNeighborSampleNumber(kValue)
- val joinPartitions: Integer = numPartitions
- broadcastJoin = false
-
- // expand the boundary for partition to include both RDDs
- objectsShapes.analyze()
- queryShapes.analyze()
- objectsShapes.boundaryEnvelope.expandToInclude(queryShapes.boundaryEnvelope)
-
- objectsShapes.spatialPartitioning(GridType.QUADTREE_RTREE, joinPartitions)
- queryShapes.spatialPartitioning(
- objectsShapes.getPartitioner.asInstanceOf[QuadTreeRTPartitioner].nonOverlappedPartitioner())
-
- objectsShapes.buildIndex(IndexType.RTREE, true)
+ // index the objects on regular partitions (not spatial partitions)
+ // this avoids the cost of spatial partitioning
+ objectsShapes.buildIndex(IndexType.RTREE, false)
+ broadcastJoin = true
}
/**
diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/JoinQueryDetector.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/JoinQueryDetector.scala
index 825855b88c..b89b1adeda 100644
--- a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/JoinQueryDetector.scala
+++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/JoinQueryDetector.scala
@@ -582,10 +582,21 @@ class JoinQueryDetector(sparkSession: SparkSession) extends Strategy {
return Nil
}
+ // validate the k value
+ val kValue: Int = distance.eval().asInstanceOf[Int]
+ require(kValue >= 1, "The number of neighbors (k) must be equal or greater than 1.")
+
val leftShape = children.head
val rightShape = children.tail.head
- val querySide = getKNNQuerySide(left, leftShape)
+ val querySide = matchExpressionsToPlans(leftShape, rightShape, left, right) match {
+ case Some((_, _, false)) =>
+ LeftSide
+ case Some((_, _, true)) =>
+ RightSide
+ case None =>
+ Nil
+ }
val objectSidePlan = if (querySide == LeftSide) right else left
checkObjectPlanFilterPushdown(objectSidePlan)
@@ -711,10 +722,21 @@ class JoinQueryDetector(sparkSession: SparkSession) extends Strategy {
if (spatialPredicate == SpatialPredicate.KNN) {
{
+ // validate the k value for KNN join
+ val kValue: Int = distance.get.eval().asInstanceOf[Int]
+ require(kValue >= 1, "The number of neighbors (k) must be equal or greater than 1.")
+
val leftShape = children.head
val rightShape = children.tail.head
- val querySide = getKNNQuerySide(left, leftShape)
+ val querySide = matchExpressionsToPlans(leftShape, rightShape, left, right) match {
+ case Some((_, _, false)) =>
+ LeftSide
+ case Some((_, _, true)) =>
+ RightSide
+ case None =>
+ Nil
+ }
val objectSidePlan = if (querySide == LeftSide) right else left
checkObjectPlanFilterPushdown(objectSidePlan)
@@ -731,7 +753,7 @@ class JoinQueryDetector(sparkSession: SparkSession) extends Strategy {
k = distance.get,
useApproximate = false,
spatialPredicate,
- isGeography = false,
+ isGeography,
condition = null,
extraCondition = None) :: Nil
} else {
@@ -746,7 +768,7 @@ class JoinQueryDetector(sparkSession: SparkSession) extends Strategy {
k = distance.get,
useApproximate = false,
spatialPredicate,
- isGeography = false,
+ isGeography,
condition = null,
extraCondition = None) :: Nil
}
@@ -857,27 +879,6 @@ class JoinQueryDetector(sparkSession: SparkSession) extends Strategy {
}
}
- /**
- * Gets the query and object plans based on the left shape.
- *
- * This method checks if the left shape is part of the left or right plan and returns the query
- * and object plans accordingly.
- *
- * @param leftShape
- * The left shape expression.
- * @return
- * The join side where the left shape is located.
- */
- private def getKNNQuerySide(left: LogicalPlan, leftShape: Expression) = {
- val isLeftQuerySide =
- left.toString().toLowerCase().contains(leftShape.toString().toLowerCase())
- if (isLeftQuerySide) {
- LeftSide
- } else {
- RightSide
- }
- }
-
/**
* Check if the given condition is an equi-join between the given plans. This method basically
* replicates the logic of
diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/KNNJoinExec.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/KNNJoinExec.scala
index 2b9bbfb50b..fdc53d13ce 100644
--- a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/KNNJoinExec.scala
+++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/KNNJoinExec.scala
@@ -162,7 +162,7 @@ case class KNNJoinExec(
sedonaConf: SedonaConf): Unit = {
require(numPartitions > 0, "The number of partitions must be greater than 0.")
val kValue: Int = this.k.eval().asInstanceOf[Int]
- require(kValue > 0, "The number of neighbors must be greater than 0.")
+ require(kValue >= 1, "The number of neighbors (k) must be equal or greater than 1.")
objectsShapes.setNeighborSampleNumber(kValue)
exactSpatialPartitioning(objectsShapes, queryShapes, numPartitions)
diff --git a/spark/common/src/test/java/org/apache/sedona/core/spatialOperator/JoinQueryDeduplicationTest.java b/spark/common/src/test/java/org/apache/sedona/core/spatialOperator/JoinQueryDeduplicationTest.java
index 14b3af49b0..919d8245ac 100644
--- a/spark/common/src/test/java/org/apache/sedona/core/spatialOperator/JoinQueryDeduplicationTest.java
+++ b/spark/common/src/test/java/org/apache/sedona/core/spatialOperator/JoinQueryDeduplicationTest.java
@@ -43,7 +43,10 @@ public static void teardown() {
sc.stop();
}
- /** See https://issues.apache.org/jira/browse/SEDONA-233 */
+ /**
+ * See https://issues.apache.org/jira/browse/SEDONA-233
+ */
@Test
public void testDeduplication() throws Exception {
SpatialRDD leftRDD = new SpatialRDD<>();
diff --git a/spark/common/src/test/java/org/apache/sedona/core/spatialPartitioning/GenericUniquePartitionerTest.java b/spark/common/src/test/java/org/apache/sedona/core/spatialPartitioning/GenericUniquePartitionerTest.java
new file mode 100644
index 0000000000..1df270c0a0
--- /dev/null
+++ b/spark/common/src/test/java/org/apache/sedona/core/spatialPartitioning/GenericUniquePartitionerTest.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.sedona.core.spatialPartitioning;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import org.apache.commons.collections.IteratorUtils;
+import org.junit.Test;
+import org.locationtech.jts.geom.Envelope;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import scala.Tuple2;
+
+public class GenericUniquePartitionerTest {
+ private final GeometryFactory factory = new GeometryFactory();
+
+ @Test
+ public void testUniquePartition() throws Exception {
+ ArrayList grids = new ArrayList();
+ grids.add(new Envelope(0, 10, 0, 10));
+ grids.add(new Envelope(10, 20, 0, 10));
+ grids.add(new Envelope(0, 10, 10, 20));
+ grids.add(new Envelope(10, 20, 10, 20));
+
+ FlatGridPartitioner partitioner = new FlatGridPartitioner(grids);
+ GenericUniquePartitioner uniquePartitioner = new GenericUniquePartitioner(partitioner);
+
+ assertEquals(partitioner.getGridType(), uniquePartitioner.getGridType());
+ assertEquals(partitioner.getGrids(), uniquePartitioner.getGrids());
+
+ Envelope definitelyHasMultiplePartitions = new Envelope(5, 15, 5, 15);
+
+ Iterator> placedWithDuplicates =
+ partitioner.placeObject(factory.toGeometry(definitelyHasMultiplePartitions));
+ // Because the geometry is not completely contained by any of the partitions,
+ // it also gets placed in the overflow partition (hence 5, not 4)
+ assertEquals(5, IteratorUtils.toList(placedWithDuplicates).size());
+
+ Iterator> placedWithoutDuplicates =
+ uniquePartitioner.placeObject(factory.toGeometry(definitelyHasMultiplePartitions));
+ assertEquals(1, IteratorUtils.toList(placedWithoutDuplicates).size());
+ }
+}
diff --git a/spark/common/src/test/java/org/apache/sedona/core/spatialPartitioning/IndexedGridPartitionerTest.java b/spark/common/src/test/java/org/apache/sedona/core/spatialPartitioning/IndexedGridPartitionerTest.java
new file mode 100644
index 0000000000..cedd94eadb
--- /dev/null
+++ b/spark/common/src/test/java/org/apache/sedona/core/spatialPartitioning/IndexedGridPartitionerTest.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.sedona.core.spatialPartitioning;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import junit.framework.TestCase;
+import org.junit.Assert;
+import org.junit.Test;
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.Envelope;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import scala.Tuple2;
+
+public class IndexedGridPartitionerTest extends TestCase {
+
+ private List getGrids() {
+ List grids = new ArrayList<>();
+ grids.add(new Envelope(0, 50, 0, 50));
+ grids.add(new Envelope(50, 100, 0, 50));
+ grids.add(new Envelope(0, 50, 50, 100));
+ grids.add(new Envelope(50, 100, 50, 100));
+ return grids;
+ }
+
+ private IndexedGridPartitioner getPartitioner(Boolean preserveUncontainedGeometries) {
+ return new IndexedGridPartitioner(getGrids(), preserveUncontainedGeometries);
+ }
+
+ public void testPlaceObjectPreserveContainedGeometries() throws Exception {
+ IndexedGridPartitioner partitioner = getPartitioner(true);
+ GeometryFactory geometryFactory = new GeometryFactory();
+ Geometry spatialObject = geometryFactory.createPoint(new Coordinate(25, 25));
+ Iterator> result = partitioner.placeObject(spatialObject);
+
+ List> resultList = new ArrayList<>();
+ result.forEachRemaining(resultList::add);
+
+ Assert.assertFalse(resultList.isEmpty());
+ Assert.assertEquals(1, resultList.size());
+ Assert.assertEquals(0, (int) resultList.get(0)._1());
+ }
+
+ public void testPlaceObjectDoesntPreserveUncontainedGeometries() throws Exception {
+ IndexedGridPartitioner partitioner = getPartitioner(false);
+ GeometryFactory geometryFactory = new GeometryFactory();
+ Geometry spatialObject = geometryFactory.createPoint(new Coordinate(-25, -25));
+ Iterator> result = partitioner.placeObject(spatialObject);
+ Assert.assertFalse(result.hasNext());
+ }
+
+ @Test
+ public void testGetGrids() {
+ IndexedGridPartitioner partitioner = getPartitioner(true);
+ Assert.assertEquals(getGrids(), partitioner.getGrids());
+ }
+
+ @Test
+ public void testNumPartitions() {
+ IndexedGridPartitioner partitioner = getPartitioner(true);
+ Assert.assertEquals(5, partitioner.numPartitions());
+
+ partitioner = getPartitioner(false);
+ Assert.assertEquals(4, partitioner.numPartitions());
+ }
+
+ @Test
+ public void testEquals() {
+ IndexedGridPartitioner partitioner = getPartitioner(true);
+ List grids = new ArrayList<>();
+ grids.add(new Envelope(0, 50, 0, 50));
+ grids.add(new Envelope(50, 100, 0, 50));
+ grids.add(new Envelope(0, 50, 50, 100));
+ grids.add(new Envelope(50, 100, 50, 100));
+ IndexedGridPartitioner otherPartitioner = new IndexedGridPartitioner(grids, true);
+ Assert.assertTrue(partitioner.equals(otherPartitioner));
+ }
+}
diff --git a/spark/common/src/test/resources/.gitignore b/spark/common/src/test/resources/.gitignore
index 764e830895..958c6de423 100644
--- a/spark/common/src/test/resources/.gitignore
+++ b/spark/common/src/test/resources/.gitignore
@@ -1,2 +1,3 @@
*.DS_Store
real-*
+wkb/testSaveAs*
diff --git a/spark/common/src/test/resources/datasource_stac/collection-items.json b/spark/common/src/test/resources/datasource_stac/collection-items.json
new file mode 100644
index 0000000000..fe42d2d4e2
--- /dev/null
+++ b/spark/common/src/test/resources/datasource_stac/collection-items.json
@@ -0,0 +1,7204 @@
+{
+ "type": "FeatureCollection",
+ "stac_version": "1.0.0",
+ "stac_extensions": [],
+ "context": {
+ "limit": 10,
+ "matched": 21436719,
+ "returned": 10
+ },
+ "numberMatched": 21436719,
+ "numberReturned": 10,
+ "features": [
+ {
+ "type": "Feature",
+ "stac_version": "1.0.0",
+ "id": "S2B_T28UEC_20250110T120355_L2A",
+ "properties": {
+ "created": "2025-01-10T16:22:52.618Z",
+ "platform": "sentinel-2b",
+ "constellation": "sentinel-2",
+ "instruments": [
+ "msi"
+ ],
+ "eo:cloud_cover": 26.967385,
+ "proj:epsg": 32628,
+ "proj:centroid": {
+ "lat": 52.31849,
+ "lon": -13.53342
+ },
+ "mgrs:utm_zone": 28,
+ "mgrs:latitude_band": "U",
+ "mgrs:grid_square": "EC",
+ "grid:code": "MGRS-28UEC",
+ "view:azimuth": 118.315984493439,
+ "view:incidence_angle": 3.15096590200837,
+ "view:sun_azimuth": 166.250646673544,
+ "view:sun_elevation": 15.2125487342296,
+ "s2:tile_id": "S2B_OPER_MSI_L2A_TL_2BPS_20250110T154043_A040990_T28UEC_N05.11",
+ "s2:degraded_msi_data_percentage": 0.0045,
+ "s2:nodata_pixel_percentage": 99.030858,
+ "s2:saturated_defective_pixel_percentage": 0,
+ "s2:cloud_shadow_percentage": 0,
+ "s2:vegetation_percentage": 0.000342,
+ "s2:not_vegetated_percentage": 0.000342,
+ "s2:water_percentage": 73.031932,
+ "s2:unclassified_percentage": 0,
+ "s2:medium_proba_clouds_percentage": 13.981807,
+ "s2:high_proba_clouds_percentage": 12.049599,
+ "s2:thin_cirrus_percentage": 0.935978,
+ "s2:snow_ice_percentage": 0,
+ "s2:product_type": "S2MSI2A",
+ "s2:processing_baseline": "05.11",
+ "s2:product_uri": "S2B_MSIL2A_20250110T120359_N0511_R066_T28UEC_20250110T154043.SAFE",
+ "s2:generation_time": "2025-01-10T15:40:43.000000Z",
+ "s2:datatake_id": "GS2B_20250110T120359_040990_N05.11",
+ "s2:datatake_type": "INS-NOBS",
+ "s2:datastrip_id": "S2B_OPER_MSI_L2A_DS_2BPS_20250110T154043_S20250110T120355_N05.11",
+ "s2:reflectance_conversion_factor": 1.03425891111326,
+ "datetime": "2025-01-10T12:07:02.966000Z",
+ "earthsearch:payload_id": "roda-sentinel-2-c1-l2a/workflow-sentinel-2-c1-l2a-to-stac/212b523557c5336ebe1c078fc29a2069",
+ "storage:platform": "AWS",
+ "storage:region": "us-west-2",
+ "storage:requester_pays": false,
+ "processing:software": {
+ "sentinel-2-c1-l2a-to-stac": "v2024.02.01"
+ },
+ "updated": "2025-01-10T16:22:52.618Z"
+ },
+ "geometry": {
+ "type": "Polygon",
+ "coordinates": [
+ [
+ [-13.8282447208263, 52.3446542644649],
+ [-13.7600637715014, 52.3345986073346],
+ [-13.3911071072422, 52.2712256131467],
+ [-13.3886306359678, 52.3394760649279],
+ [-13.8282447208263, 52.3446542644649]
+ ]
+ ]
+ },
+ "links": [
+ {
+ "rel": "self",
+ "type": "application/geo+json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a/items/S2B_T28UEC_20250110T120355_L2A"
+ },
+ {
+ "rel": "canonical",
+ "href": "s3://e84-earth-search-sentinel-data/sentinel-2-c1-l2a/28/U/EC/2025/1/S2B_T28UEC_20250110T120355_L2A/S2B_T28UEC_20250110T120355_L2A.json",
+ "type": "application/json"
+ },
+ {
+ "rel": "via",
+ "href": "s3://sentinel-s2-l2a/tiles/28/U/EC/2025/1/10/0/metadata.xml",
+ "type": "application/xml",
+ "title": "Granule Metadata in Sinergize RODA Archive"
+ },
+ {
+ "rel": "parent",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a"
+ },
+ {
+ "rel": "collection",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a"
+ },
+ {
+ "rel": "root",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1"
+ },
+ {
+ "rel": "thumbnail",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a/items/S2B_T28UEC_20250110T120355_L2A/thumbnail"
+ }
+ ],
+ "assets": {
+ "red": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EC/2025/1/S2B_T28UEC_20250110T120355_L2A/B04.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red - 10m",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 499980, 0, -10, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122032a6cbfd123d742d90e53162ae4660586361a5ef85e13429946aa77953065cf5",
+ "file:size": 2357747,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "green": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EC/2025/1/S2B_T28UEC_20250110T120355_L2A/B03.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Green - 10m",
+ "eo:bands": [
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 499980, 0, -10, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122033b3a77113b9fa9931115939f8c3738d00a9b73b0e499d23e21da0bef4696be7",
+ "file:size": 2371586,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "blue": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EC/2025/1/S2B_T28UEC_20250110T120355_L2A/B02.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Blue - 10m",
+ "eo:bands": [
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 499980, 0, -10, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220f68f82bf174b37fc0e5c195c2bb19ed0a2e7d4aa1117636eace4199f49a83845",
+ "file:size": 2396700,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "visual": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EC/2025/1/S2B_T28UEC_20250110T120355_L2A/TCI.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "True color image",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ },
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ },
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ },
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ },
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 499980, 0, -10, 5800020],
+ "file:checksum": "122083fa02d60e11ee92c40028343320794d6b341bfeb88a50b47b317226c3cd45dc",
+ "file:size": 3097977,
+ "roles": [
+ "visual"
+ ]
+ },
+ "nir": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EC/2025/1/S2B_T28UEC_20250110T120355_L2A/B08.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 1 - 10m",
+ "eo:bands": [
+ {
+ "name": "B08",
+ "common_name": "nir",
+ "center_wavelength": 0.842,
+ "full_width_half_max": 0.145
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 499980, 0, -10, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122063993cadf2b23b6b877c29387a416c9400d2cbe94092c66dfeea6d6f3851f99c",
+ "file:size": 2368644,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "swir22": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EC/2025/1/S2B_T28UEC_20250110T120355_L2A/B12.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "SWIR 2.2μm - 20m",
+ "eo:bands": [
+ {
+ "name": "B12",
+ "common_name": "swir22",
+ "center_wavelength": 2.19,
+ "full_width_half_max": 0.242
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12208c9c4478d4bbb29ce1fd8b233fec7262d91de3535ab7623dca8821d17ce2b92c",
+ "file:size": 629724,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge2": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EC/2025/1/S2B_T28UEC_20250110T120355_L2A/B06.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 2 - 20m",
+ "eo:bands": [
+ {
+ "name": "B06",
+ "common_name": "rededge",
+ "center_wavelength": 0.74,
+ "full_width_half_max": 0.018
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12202676f8075b28a063e89a9d461f31f8ac6e9a53e4c54091e218b681c8e74fa41f",
+ "file:size": 638352,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge3": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EC/2025/1/S2B_T28UEC_20250110T120355_L2A/B07.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 3 - 20m",
+ "eo:bands": [
+ {
+ "name": "B07",
+ "common_name": "rededge",
+ "center_wavelength": 0.783,
+ "full_width_half_max": 0.028
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220d44985e336e4999bc99a3d0d6affd8ea0ce1cf89698fe5727a4495dd43a0664c",
+ "file:size": 639056,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge1": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EC/2025/1/S2B_T28UEC_20250110T120355_L2A/B05.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 1 - 20m",
+ "eo:bands": [
+ {
+ "name": "B05",
+ "common_name": "rededge",
+ "center_wavelength": 0.704,
+ "full_width_half_max": 0.019
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220f4c994b1ae1c5c8c46d95e862e3888199f703615e01f69b86063885f00bfda19",
+ "file:size": 638484,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "swir16": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EC/2025/1/S2B_T28UEC_20250110T120355_L2A/B11.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "SWIR 1.6μm - 20m",
+ "eo:bands": [
+ {
+ "name": "B11",
+ "common_name": "swir16",
+ "center_wavelength": 1.61,
+ "full_width_half_max": 0.143
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12204553e83bb568a301ed89a19976ce706da96e62da25cd98a30bf47c788c63fdff",
+ "file:size": 626909,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "wvp": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EC/2025/1/S2B_T28UEC_20250110T120355_L2A/WVP.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Water Vapour (WVP)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "unit": "cm",
+ "scale": 0.001,
+ "offset": 0
+ }
+ ],
+ "file:checksum": "12204701f12f0a40fbe11da4652e3c66a17355edbe9ede6d7cfb6ccbbf2db9aa95cd",
+ "file:size": 98743,
+ "roles": [
+ "data"
+ ]
+ },
+ "nir08": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EC/2025/1/S2B_T28UEC_20250110T120355_L2A/B8A.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 2 - 20m",
+ "eo:bands": [
+ {
+ "name": "B8A",
+ "common_name": "nir08",
+ "center_wavelength": 0.865,
+ "full_width_half_max": 0.033
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220bf5827bf5efdb2e488c95d707d2f3f56ac524578e6f8b7e0bbff01c2bd051cd0",
+ "file:size": 639283,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "scl": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EC/2025/1/S2B_T28UEC_20250110T120355_L2A/SCL.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Scene classification map (SCL)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "1220cfbbc94375ef0b9d9c2844eb52cae66a47b996aa960e104edfe170fa07d70897",
+ "file:size": 78184,
+ "roles": [
+ "data"
+ ]
+ },
+ "aot": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EC/2025/1/S2B_T28UEC_20250110T120355_L2A/AOT.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Aerosol optical thickness (AOT)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.001,
+ "offset": 0
+ }
+ ],
+ "file:checksum": "122075bb84c0519c39937be4e010d502a7973f907792da5c78f0aa0f19e920ae871b",
+ "file:size": 99487,
+ "roles": [
+ "data"
+ ]
+ },
+ "coastal": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EC/2025/1/S2B_T28UEC_20250110T120355_L2A/B01.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Coastal - 60m",
+ "eo:bands": [
+ {
+ "name": "B01",
+ "common_name": "coastal",
+ "center_wavelength": 0.443,
+ "full_width_half_max": 0.027
+ }
+ ],
+ "gsd": 60,
+ "proj:shape": [1830, 1830],
+ "proj:transform": [60, 0, 499980, 0, -60, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 60,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12205a2e22580d4ee310b6a7ec8b72b7a75b0bfa3b4c46806a7338b3ad03ccdea04a",
+ "file:size": 82752,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "nir09": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EC/2025/1/S2B_T28UEC_20250110T120355_L2A/B09.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 3 - 60m",
+ "eo:bands": [
+ {
+ "name": "B09",
+ "common_name": "nir09",
+ "center_wavelength": 0.945,
+ "full_width_half_max": 0.026
+ }
+ ],
+ "gsd": 60,
+ "proj:shape": [1830, 1830],
+ "proj:transform": [60, 0, 499980, 0, -60, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 60,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12203b837e2b511157325bc14d889e3826661156f93ac206408d0b0e442f17cc388e",
+ "file:size": 84972,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "cloud": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EC/2025/1/S2B_T28UEC_20250110T120355_L2A/CLD_20m.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Cloud Probabilities",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "12206f6d9989064b0614512729c79f4292df9c9355550924b42a34703756474e1cf1",
+ "file:size": 120955,
+ "roles": [
+ "data",
+ "cloud"
+ ]
+ },
+ "snow": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EC/2025/1/S2B_T28UEC_20250110T120355_L2A/SNW_20m.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Snow Probabilities",
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "1220646fd31cc898948d16da2d3fa3672ec809f579a0181870342d5a73f19cf1db37",
+ "file:size": 53931,
+ "roles": [
+ "data",
+ "snow-ice"
+ ]
+ },
+ "preview": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EC/2025/1/S2B_T28UEC_20250110T120355_L2A/L2A_PVI.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "True color preview",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ },
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ },
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "file:checksum": "1220eb97564652f39ae6ab53224b027c7eb549c7de4692cd76cb0805b6c854b28839",
+ "file:size": 8059,
+ "roles": [
+ "overview"
+ ]
+ },
+ "granule_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EC/2025/1/S2B_T28UEC_20250110T120355_L2A/metadata.xml",
+ "type": "application/xml",
+ "file:checksum": "122054cadc787eb9b6700b86b7275aa389d20aa862537769fcd7bd54be08556d3d01",
+ "file:size": 257414,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "tileinfo_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EC/2025/1/S2B_T28UEC_20250110T120355_L2A/tileInfo.json",
+ "type": "application/json",
+ "file:checksum": "12200096a9fdb71ede656661a29442dec78f1269d0c233f5bd8ac1a74f14f3e0e484",
+ "file:size": 1581,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "product_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EC/2025/1/S2B_T28UEC_20250110T120355_L2A/product_metadata.xml",
+ "type": "application/xml",
+ "file:checksum": "1220edf7135e20824122846daebe157b450efaef6c8d92cf2c9a6300f312cd787cca",
+ "file:size": 54686,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "thumbnail": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EC/2025/1/S2B_T28UEC_20250110T120355_L2A/L2A_PVI.jpg",
+ "type": "image/jpeg",
+ "title": "Thumbnail of preview image",
+ "file:checksum": "1220b306fe70acae95183a812673cf9e80ca7056bf626f21271dd471e9cda23dcf26",
+ "file:size": 3156,
+ "roles": [
+ "thumbnail"
+ ]
+ }
+ },
+ "bbox": [-13.828245, 52.271226, -13.388631, 52.344654],
+ "stac_extensions": [
+ "https://stac-extensions.github.io/eo/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/file/v2.1.0/schema.json",
+ "https://stac-extensions.github.io/grid/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/mgrs/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/processing/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/projection/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/raster/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/sentinel-2/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/storage/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/view/v1.0.0/schema.json"
+ ],
+ "collection": "sentinel-2-c1-l2a"
+ },
+ {
+ "type": "Feature",
+ "stac_version": "1.0.0",
+ "id": "S2B_T28UFC_20250110T120355_L2A",
+ "properties": {
+ "created": "2025-01-10T16:24:04.511Z",
+ "platform": "sentinel-2b",
+ "constellation": "sentinel-2",
+ "instruments": [
+ "msi"
+ ],
+ "eo:cloud_cover": 41.692412,
+ "proj:epsg": 32628,
+ "proj:centroid": {
+ "lat": 52.21432,
+ "lon": -12.53367
+ },
+ "mgrs:utm_zone": 28,
+ "mgrs:latitude_band": "U",
+ "mgrs:grid_square": "FC",
+ "grid:code": "MGRS-28UFC",
+ "view:azimuth": 245.523494629745,
+ "view:incidence_angle": 3.70536620607632,
+ "view:sun_azimuth": 167.617359597172,
+ "view:sun_elevation": 15.4317023114617,
+ "s2:tile_id": "S2B_OPER_MSI_L2A_TL_2BPS_20250110T154043_A040990_T28UFC_N05.11",
+ "s2:degraded_msi_data_percentage": 0,
+ "s2:nodata_pixel_percentage": 81.846112,
+ "s2:saturated_defective_pixel_percentage": 0,
+ "s2:cloud_shadow_percentage": 0,
+ "s2:vegetation_percentage": 0.000018,
+ "s2:not_vegetated_percentage": 0,
+ "s2:water_percentage": 58.30757,
+ "s2:unclassified_percentage": 0,
+ "s2:medium_proba_clouds_percentage": 17.302617,
+ "s2:high_proba_clouds_percentage": 22.928596,
+ "s2:thin_cirrus_percentage": 1.4612,
+ "s2:snow_ice_percentage": 0,
+ "s2:product_type": "S2MSI2A",
+ "s2:processing_baseline": "05.11",
+ "s2:product_uri": "S2B_MSIL2A_20250110T120359_N0511_R066_T28UFC_20250110T154043.SAFE",
+ "s2:generation_time": "2025-01-10T15:40:43.000000Z",
+ "s2:datatake_id": "GS2B_20250110T120359_040990_N05.11",
+ "s2:datatake_type": "INS-NOBS",
+ "s2:datastrip_id": "S2B_OPER_MSI_L2A_DS_2BPS_20250110T154043_S20250110T120355_N05.11",
+ "s2:reflectance_conversion_factor": 1.03425891111326,
+ "datetime": "2025-01-10T12:07:01.074000Z",
+ "earthsearch:payload_id": "roda-sentinel-2-c1-l2a/workflow-sentinel-2-c1-l2a-to-stac/8e97d11f28adbcdd059cd9db6b9de885",
+ "storage:platform": "AWS",
+ "storage:region": "us-west-2",
+ "storage:requester_pays": false,
+ "processing:software": {
+ "sentinel-2-c1-l2a-to-stac": "v2024.02.01"
+ },
+ "updated": "2025-01-10T16:24:04.511Z"
+ },
+ "geometry": {
+ "type": "Polygon",
+ "coordinates": [
+ [
+ [-13.5321014711154, 52.3413459269826],
+ [-13.5336220434666, 52.2953748622493],
+ [-11.9440745611458, 51.9931444540663],
+ [-11.9222573449227, 52.3103583189958],
+ [-13.5321014711154, 52.3413459269826]
+ ]
+ ]
+ },
+ "links": [
+ {
+ "rel": "self",
+ "type": "application/geo+json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a/items/S2B_T28UFC_20250110T120355_L2A"
+ },
+ {
+ "rel": "canonical",
+ "href": "s3://e84-earth-search-sentinel-data/sentinel-2-c1-l2a/28/U/FC/2025/1/S2B_T28UFC_20250110T120355_L2A/S2B_T28UFC_20250110T120355_L2A.json",
+ "type": "application/json"
+ },
+ {
+ "rel": "via",
+ "href": "s3://sentinel-s2-l2a/tiles/28/U/FC/2025/1/10/0/metadata.xml",
+ "type": "application/xml",
+ "title": "Granule Metadata in Sinergize RODA Archive"
+ },
+ {
+ "rel": "parent",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a"
+ },
+ {
+ "rel": "collection",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a"
+ },
+ {
+ "rel": "root",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1"
+ },
+ {
+ "rel": "thumbnail",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a/items/S2B_T28UFC_20250110T120355_L2A/thumbnail"
+ }
+ ],
+ "assets": {
+ "red": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/FC/2025/1/S2B_T28UFC_20250110T120355_L2A/B04.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red - 10m",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 600000, 0, -10, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122035c6d5cbc12a70c69fc6525d5fca8037dff9239727a85394e2f6e31cb50e3cef",
+ "file:size": 37808553,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "green": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/FC/2025/1/S2B_T28UFC_20250110T120355_L2A/B03.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Green - 10m",
+ "eo:bands": [
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 600000, 0, -10, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12201bc6e8ad7c6cbf3ed5ff4b3e6d2e63b98cd5940ff0484c7999003388ecdaeeeb",
+ "file:size": 38001328,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "blue": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/FC/2025/1/S2B_T28UFC_20250110T120355_L2A/B02.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Blue - 10m",
+ "eo:bands": [
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 600000, 0, -10, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220b3c02b381f6250bde6d1c40512fb5c248739fab60714b0ed457adfe8bc4b2129",
+ "file:size": 38281234,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "visual": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/FC/2025/1/S2B_T28UFC_20250110T120355_L2A/TCI.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "True color image",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ },
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ },
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ },
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ },
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 600000, 0, -10, 5800020],
+ "file:checksum": "12204ff1bdc12c9c16461e1e3a7e17ed3a47bbb0465a1006d2473c4eb341e4232f37",
+ "file:size": 43688315,
+ "roles": [
+ "visual"
+ ]
+ },
+ "nir": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/FC/2025/1/S2B_T28UFC_20250110T120355_L2A/B08.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 1 - 10m",
+ "eo:bands": [
+ {
+ "name": "B08",
+ "common_name": "nir",
+ "center_wavelength": 0.842,
+ "full_width_half_max": 0.145
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 600000, 0, -10, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220efaaa2b724427b0d9d2a36fde08e20abd94f0dde2c92ebc3cc54b8b818fb3fbd",
+ "file:size": 38093188,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "swir22": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/FC/2025/1/S2B_T28UFC_20250110T120355_L2A/B12.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "SWIR 2.2μm - 20m",
+ "eo:bands": [
+ {
+ "name": "B12",
+ "common_name": "swir22",
+ "center_wavelength": 2.19,
+ "full_width_half_max": 0.242
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 600000, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12208c918c9e07cde5d1a0f7582e66329ed068927ecd5f87359efe22f278a578ecb5",
+ "file:size": 10084992,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge2": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/FC/2025/1/S2B_T28UFC_20250110T120355_L2A/B06.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 2 - 20m",
+ "eo:bands": [
+ {
+ "name": "B06",
+ "common_name": "rededge",
+ "center_wavelength": 0.74,
+ "full_width_half_max": 0.018
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 600000, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220d6ab4c1b664e7e40d8dbc0b589f967c807f9fdc57a26466472f528976c085874",
+ "file:size": 10346975,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge3": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/FC/2025/1/S2B_T28UFC_20250110T120355_L2A/B07.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 3 - 20m",
+ "eo:bands": [
+ {
+ "name": "B07",
+ "common_name": "rededge",
+ "center_wavelength": 0.783,
+ "full_width_half_max": 0.028
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 600000, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220c8ca8579f2636deafb9de4798957c6c29d0fa9f7e27161f9e193082951c88c5a",
+ "file:size": 10344767,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge1": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/FC/2025/1/S2B_T28UFC_20250110T120355_L2A/B05.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 1 - 20m",
+ "eo:bands": [
+ {
+ "name": "B05",
+ "common_name": "rededge",
+ "center_wavelength": 0.704,
+ "full_width_half_max": 0.019
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 600000, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12208ed42d02383e816d27d156887fa8ed671c2da73acc32fb622cc81ef767a5850a",
+ "file:size": 10361731,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "swir16": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/FC/2025/1/S2B_T28UFC_20250110T120355_L2A/B11.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "SWIR 1.6μm - 20m",
+ "eo:bands": [
+ {
+ "name": "B11",
+ "common_name": "swir16",
+ "center_wavelength": 1.61,
+ "full_width_half_max": 0.143
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 600000, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12200ea27f0c2a0293ca9ba18cf9065ed2a3bebe106370a083d0fd5867bbf8069346",
+ "file:size": 10076269,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "wvp": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/FC/2025/1/S2B_T28UFC_20250110T120355_L2A/WVP.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Water Vapour (WVP)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 600000, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "unit": "cm",
+ "scale": 0.001,
+ "offset": 0
+ }
+ ],
+ "file:checksum": "1220e988a55b07c65489fb136ac2e87c754824341aa39435abf2162336def50a3463",
+ "file:size": 120608,
+ "roles": [
+ "data"
+ ]
+ },
+ "nir08": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/FC/2025/1/S2B_T28UFC_20250110T120355_L2A/B8A.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 2 - 20m",
+ "eo:bands": [
+ {
+ "name": "B8A",
+ "common_name": "nir08",
+ "center_wavelength": 0.865,
+ "full_width_half_max": 0.033
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 600000, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122044588afb918873ed2c6a1a7fc302c6227579cec72e71c72444e51ff34089da56",
+ "file:size": 10362876,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "scl": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/FC/2025/1/S2B_T28UFC_20250110T120355_L2A/SCL.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Scene classification map (SCL)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 600000, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "12206ad27bd37d2e7386ba6d0c2d6b5de04cbb9ad06000cff2a8fa81ce353f0b24c4",
+ "file:size": 420519,
+ "roles": [
+ "data"
+ ]
+ },
+ "aot": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/FC/2025/1/S2B_T28UFC_20250110T120355_L2A/AOT.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Aerosol optical thickness (AOT)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 600000, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.001,
+ "offset": 0
+ }
+ ],
+ "file:checksum": "1220db700e3012812070a263beea2838d3647844e4bd4234420280942c68b847e492",
+ "file:size": 142297,
+ "roles": [
+ "data"
+ ]
+ },
+ "coastal": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/FC/2025/1/S2B_T28UFC_20250110T120355_L2A/B01.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Coastal - 60m",
+ "eo:bands": [
+ {
+ "name": "B01",
+ "common_name": "coastal",
+ "center_wavelength": 0.443,
+ "full_width_half_max": 0.027
+ }
+ ],
+ "gsd": 60,
+ "proj:shape": [1830, 1830],
+ "proj:transform": [60, 0, 600000, 0, -60, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 60,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220758926402c8bc702c574e19a3edbdc2a9711017c13e52075afc6cfdddaf2a52e",
+ "file:size": 1206892,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "nir09": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/FC/2025/1/S2B_T28UFC_20250110T120355_L2A/B09.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 3 - 60m",
+ "eo:bands": [
+ {
+ "name": "B09",
+ "common_name": "nir09",
+ "center_wavelength": 0.945,
+ "full_width_half_max": 0.026
+ }
+ ],
+ "gsd": 60,
+ "proj:shape": [1830, 1830],
+ "proj:transform": [60, 0, 600000, 0, -60, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 60,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12202747d0e7b18642c93285c3acdd13780ba63fc3937f3ba1456ed9b857566f7181",
+ "file:size": 1261903,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "cloud": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/FC/2025/1/S2B_T28UFC_20250110T120355_L2A/CLD_20m.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Cloud Probabilities",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 600000, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "12205d6e64c391bbe597e37aa0c9b3d5ab81d80390a2fd66ff8d36fc906069df0984",
+ "file:size": 1501376,
+ "roles": [
+ "data",
+ "cloud"
+ ]
+ },
+ "snow": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/FC/2025/1/S2B_T28UFC_20250110T120355_L2A/SNW_20m.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Snow Probabilities",
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 600000, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "12202a673ea9c389b334d6a5bbcf6700845107fe73852bf22833c50b53642b9dd49c",
+ "file:size": 53931,
+ "roles": [
+ "data",
+ "snow-ice"
+ ]
+ },
+ "preview": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/FC/2025/1/S2B_T28UFC_20250110T120355_L2A/L2A_PVI.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "True color preview",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ },
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ },
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "file:checksum": "1220132208c1e4422dfdde88a04d6f52425c455bef66af0351605e2c05521d1c1b30",
+ "file:size": 57507,
+ "roles": [
+ "overview"
+ ]
+ },
+ "granule_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/FC/2025/1/S2B_T28UFC_20250110T120355_L2A/metadata.xml",
+ "type": "application/xml",
+ "file:checksum": "1220306690a016b2b6caed07114c113a9b989cb29efb92bda3e31e7ff29c51799fcd",
+ "file:size": 502351,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "tileinfo_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/FC/2025/1/S2B_T28UFC_20250110T120355_L2A/tileInfo.json",
+ "type": "application/json",
+ "file:checksum": "122051ce84f2407d3a34458f82d94356c968afd65153d5e826278221962d8bbe9b3d",
+ "file:size": 1674,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "product_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/FC/2025/1/S2B_T28UFC_20250110T120355_L2A/product_metadata.xml",
+ "type": "application/xml",
+ "file:checksum": "1220dce37ab88abf5d29cd31f1753966c4e7bda509523d5ce5cc5e15878cea96c75f",
+ "file:size": 55550,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "thumbnail": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/FC/2025/1/S2B_T28UFC_20250110T120355_L2A/L2A_PVI.jpg",
+ "type": "image/jpeg",
+ "title": "Thumbnail of preview image",
+ "file:checksum": "122094252c5c1668db3d539b946f87167780a67844fe61f32c83baf41c53fae255c8",
+ "file:size": 11406,
+ "roles": [
+ "thumbnail"
+ ]
+ }
+ },
+ "bbox": [-13.533622, 51.993144, -11.922257, 52.341346],
+ "stac_extensions": [
+ "https://stac-extensions.github.io/eo/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/file/v2.1.0/schema.json",
+ "https://stac-extensions.github.io/grid/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/mgrs/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/processing/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/projection/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/raster/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/sentinel-2/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/storage/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/view/v1.0.0/schema.json"
+ ],
+ "collection": "sentinel-2-c1-l2a"
+ },
+ {
+ "type": "Feature",
+ "stac_version": "1.0.0",
+ "id": "S2B_T28UDD_20250110T120355_L2A",
+ "properties": {
+ "created": "2025-01-10T16:23:17.223Z",
+ "platform": "sentinel-2b",
+ "constellation": "sentinel-2",
+ "instruments": [
+ "msi"
+ ],
+ "eo:cloud_cover": 45.573676,
+ "proj:epsg": 32628,
+ "proj:centroid": {
+ "lat": 52.78244,
+ "lon": -14.96778
+ },
+ "mgrs:utm_zone": 28,
+ "mgrs:latitude_band": "U",
+ "mgrs:grid_square": "DD",
+ "grid:code": "MGRS-28UDD",
+ "view:azimuth": 111.363270762582,
+ "view:incidence_angle": 11.2176659476937,
+ "view:sun_azimuth": 164.877643157552,
+ "view:sun_elevation": 14.1102990134553,
+ "s2:tile_id": "S2B_OPER_MSI_L2A_TL_2BPS_20250110T154043_A040990_T28UDD_N05.11",
+ "s2:degraded_msi_data_percentage": 0,
+ "s2:nodata_pixel_percentage": 91.37699,
+ "s2:saturated_defective_pixel_percentage": 0,
+ "s2:cloud_shadow_percentage": 0,
+ "s2:vegetation_percentage": 0.000846,
+ "s2:not_vegetated_percentage": 0,
+ "s2:water_percentage": 54.425478,
+ "s2:unclassified_percentage": 0,
+ "s2:medium_proba_clouds_percentage": 12.064364,
+ "s2:high_proba_clouds_percentage": 32.520658,
+ "s2:thin_cirrus_percentage": 0.988655,
+ "s2:snow_ice_percentage": 0,
+ "s2:product_type": "S2MSI2A",
+ "s2:processing_baseline": "05.11",
+ "s2:product_uri": "S2B_MSIL2A_20250110T120359_N0511_R066_T28UDD_20250110T154043.SAFE",
+ "s2:generation_time": "2025-01-10T15:40:43.000000Z",
+ "s2:datatake_id": "GS2B_20250110T120359_040990_N05.11",
+ "s2:datatake_type": "INS-NOBS",
+ "s2:datastrip_id": "S2B_OPER_MSI_L2A_DS_2BPS_20250110T154043_S20250110T120355_N05.11",
+ "s2:reflectance_conversion_factor": 1.03425891111326,
+ "datetime": "2025-01-10T12:06:59.248000Z",
+ "earthsearch:payload_id": "roda-sentinel-2-c1-l2a/workflow-sentinel-2-c1-l2a-to-stac/af0f2bf0eea44a4ea693a69087331d8e",
+ "storage:platform": "AWS",
+ "storage:region": "us-west-2",
+ "storage:requester_pays": false,
+ "processing:software": {
+ "sentinel-2-c1-l2a-to-stac": "v2024.02.01"
+ },
+ "updated": "2025-01-10T16:23:17.223Z"
+ },
+ "geometry": {
+ "type": "Polygon",
+ "coordinates": [
+ [
+ [-14.883934957153, 53.249560984131],
+ [-15.191183625645, 52.5424009271242],
+ [-14.8562491296129, 52.4966018865447],
+ [-14.8537381455861, 53.2495277635177],
+ [-14.883934957153, 53.249560984131]
+ ]
+ ]
+ },
+ "links": [
+ {
+ "rel": "self",
+ "type": "application/geo+json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a/items/S2B_T28UDD_20250110T120355_L2A"
+ },
+ {
+ "rel": "canonical",
+ "href": "s3://e84-earth-search-sentinel-data/sentinel-2-c1-l2a/28/U/DD/2025/1/S2B_T28UDD_20250110T120355_L2A/S2B_T28UDD_20250110T120355_L2A.json",
+ "type": "application/json"
+ },
+ {
+ "rel": "via",
+ "href": "s3://sentinel-s2-l2a/tiles/28/U/DD/2025/1/10/0/metadata.xml",
+ "type": "application/xml",
+ "title": "Granule Metadata in Sinergize RODA Archive"
+ },
+ {
+ "rel": "parent",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a"
+ },
+ {
+ "rel": "collection",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a"
+ },
+ {
+ "rel": "root",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1"
+ },
+ {
+ "rel": "thumbnail",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a/items/S2B_T28UDD_20250110T120355_L2A/thumbnail"
+ }
+ ],
+ "assets": {
+ "red": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/DD/2025/1/S2B_T28UDD_20250110T120355_L2A/B04.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red - 10m",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 399960, 0, -10, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122054a5a778414a5dcf84cedb25f35f7d742e6f7a3a90a39f9f0efb8aa73997ee3d",
+ "file:size": 17308393,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "green": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/DD/2025/1/S2B_T28UDD_20250110T120355_L2A/B03.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Green - 10m",
+ "eo:bands": [
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 399960, 0, -10, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122057746c62fa7de4da7c715ed1f9a32daf2a756aeffadb091cd885884fb4bbaa2d",
+ "file:size": 17453898,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "blue": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/DD/2025/1/S2B_T28UDD_20250110T120355_L2A/B02.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Blue - 10m",
+ "eo:bands": [
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 399960, 0, -10, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220f66c150b7d0072381550f6daf7bbd2845705d3b7459db07f4c99359e35561598",
+ "file:size": 17687244,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "visual": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/DD/2025/1/S2B_T28UDD_20250110T120355_L2A/TCI.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "True color image",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ },
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ },
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ },
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ },
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 399960, 0, -10, 5900040],
+ "file:checksum": "12201b16ef75a69922ab923ef55e87362bc0e8e12c25aa7131d1db37d20c12978c16",
+ "file:size": 17154534,
+ "roles": [
+ "visual"
+ ]
+ },
+ "nir": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/DD/2025/1/S2B_T28UDD_20250110T120355_L2A/B08.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 1 - 10m",
+ "eo:bands": [
+ {
+ "name": "B08",
+ "common_name": "nir",
+ "center_wavelength": 0.842,
+ "full_width_half_max": 0.145
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 399960, 0, -10, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220d6c16cc1335ba85c3cde38b65161d43120f6f210a29e5ac3bd29992f10db0061",
+ "file:size": 17347684,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "swir22": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/DD/2025/1/S2B_T28UDD_20250110T120355_L2A/B12.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "SWIR 2.2μm - 20m",
+ "eo:bands": [
+ {
+ "name": "B12",
+ "common_name": "swir22",
+ "center_wavelength": 2.19,
+ "full_width_half_max": 0.242
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 399960, 0, -20, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220fadc97c30a1a1471f272b5eb1b62bca1c09a010cc7bf2218a782df05f7844592",
+ "file:size": 4577358,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge2": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/DD/2025/1/S2B_T28UDD_20250110T120355_L2A/B06.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 2 - 20m",
+ "eo:bands": [
+ {
+ "name": "B06",
+ "common_name": "rededge",
+ "center_wavelength": 0.74,
+ "full_width_half_max": 0.018
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 399960, 0, -20, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220a79316d379277821cc95628beee1bb62d33db354f26aa7693126d19237dfbc99",
+ "file:size": 4788245,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge3": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/DD/2025/1/S2B_T28UDD_20250110T120355_L2A/B07.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 3 - 20m",
+ "eo:bands": [
+ {
+ "name": "B07",
+ "common_name": "rededge",
+ "center_wavelength": 0.783,
+ "full_width_half_max": 0.028
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 399960, 0, -20, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220cad9f61dc28368c2db5efb069e2948bdd433476a4ba1e8744035abe07910d4b7",
+ "file:size": 4801599,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge1": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/DD/2025/1/S2B_T28UDD_20250110T120355_L2A/B05.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 1 - 20m",
+ "eo:bands": [
+ {
+ "name": "B05",
+ "common_name": "rededge",
+ "center_wavelength": 0.704,
+ "full_width_half_max": 0.019
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 399960, 0, -20, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220a6c66252a5437bb34de51dcee0552c8c102cbe62839037b7b509bd3d53cb6476",
+ "file:size": 4819787,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "swir16": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/DD/2025/1/S2B_T28UDD_20250110T120355_L2A/B11.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "SWIR 1.6μm - 20m",
+ "eo:bands": [
+ {
+ "name": "B11",
+ "common_name": "swir16",
+ "center_wavelength": 1.61,
+ "full_width_half_max": 0.143
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 399960, 0, -20, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12209d4fb63d60fdbb1a60cbc1e7489affee9c45556fc06cebbf01bf9c82bfd98f34",
+ "file:size": 4580867,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "wvp": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/DD/2025/1/S2B_T28UDD_20250110T120355_L2A/WVP.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Water Vapour (WVP)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 399960, 0, -20, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "unit": "cm",
+ "scale": 0.001,
+ "offset": 0
+ }
+ ],
+ "file:checksum": "122023974cded5bb589228df181ca430c8c9873a298e40e6abeda080b716042c135c",
+ "file:size": 120033,
+ "roles": [
+ "data"
+ ]
+ },
+ "nir08": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/DD/2025/1/S2B_T28UDD_20250110T120355_L2A/B8A.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 2 - 20m",
+ "eo:bands": [
+ {
+ "name": "B8A",
+ "common_name": "nir08",
+ "center_wavelength": 0.865,
+ "full_width_half_max": 0.033
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 399960, 0, -20, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220952b926daeefadbbf02d6abb78c13adb3cb6daf998d0ce3ee3932c51fdfa3c07",
+ "file:size": 4791768,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "scl": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/DD/2025/1/S2B_T28UDD_20250110T120355_L2A/SCL.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Scene classification map (SCL)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 399960, 0, -20, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "1220a6ef3402bad56ffe7bcfe50f81b1ada6ce752623c3b2437896c5f7abaf52f82a",
+ "file:size": 184466,
+ "roles": [
+ "data"
+ ]
+ },
+ "aot": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/DD/2025/1/S2B_T28UDD_20250110T120355_L2A/AOT.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Aerosol optical thickness (AOT)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 399960, 0, -20, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.001,
+ "offset": 0
+ }
+ ],
+ "file:checksum": "12204118b1e0a9d86bfe943e2af1de7e7cbfdde5bce35cd19597d3cad795906ff15a",
+ "file:size": 133584,
+ "roles": [
+ "data"
+ ]
+ },
+ "coastal": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/DD/2025/1/S2B_T28UDD_20250110T120355_L2A/B01.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Coastal - 60m",
+ "eo:bands": [
+ {
+ "name": "B01",
+ "common_name": "coastal",
+ "center_wavelength": 0.443,
+ "full_width_half_max": 0.027
+ }
+ ],
+ "gsd": 60,
+ "proj:shape": [1830, 1830],
+ "proj:transform": [60, 0, 399960, 0, -60, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 60,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220c57b36366bb26cff43ba58f29db4d2e7da27ad9793fe9fb1c37c25f1af4af5e4",
+ "file:size": 562940,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "nir09": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/DD/2025/1/S2B_T28UDD_20250110T120355_L2A/B09.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 3 - 60m",
+ "eo:bands": [
+ {
+ "name": "B09",
+ "common_name": "nir09",
+ "center_wavelength": 0.945,
+ "full_width_half_max": 0.026
+ }
+ ],
+ "gsd": 60,
+ "proj:shape": [1830, 1830],
+ "proj:transform": [60, 0, 399960, 0, -60, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 60,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220eeed3e0cebbb75aa936744142ace9c12dd6a82f5ba3acebcde9c97ddbb9f932a",
+ "file:size": 577302,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "cloud": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/DD/2025/1/S2B_T28UDD_20250110T120355_L2A/CLD_20m.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Cloud Probabilities",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 399960, 0, -20, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "1220410f661fbe8e0a4220db2d2e14d9e8b0d148466c6e42745218e1a82561af8d60",
+ "file:size": 523213,
+ "roles": [
+ "data",
+ "cloud"
+ ]
+ },
+ "snow": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/DD/2025/1/S2B_T28UDD_20250110T120355_L2A/SNW_20m.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Snow Probabilities",
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 399960, 0, -20, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "1220b71d1bd3c5b571cbded0862e4f4785ad1517371019d7a93177f3156c2ecebea7",
+ "file:size": 53931,
+ "roles": [
+ "data",
+ "snow-ice"
+ ]
+ },
+ "preview": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/DD/2025/1/S2B_T28UDD_20250110T120355_L2A/L2A_PVI.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "True color preview",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ },
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ },
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "file:checksum": "1220bfa6a79b1279b8892a2a9297157d630645b78a29e97179329fb723a4c9f47654",
+ "file:size": 25349,
+ "roles": [
+ "overview"
+ ]
+ },
+ "granule_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/DD/2025/1/S2B_T28UDD_20250110T120355_L2A/metadata.xml",
+ "type": "application/xml",
+ "file:checksum": "122011626ced3f3137673cce60b87bcf1a209ad4ac5db04feb55806e90e3ea5f5701",
+ "file:size": 105054,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "tileinfo_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/DD/2025/1/S2B_T28UDD_20250110T120355_L2A/tileInfo.json",
+ "type": "application/json",
+ "file:checksum": "122083498a89e0c8f200b5222497bad130a56c06ef24ef7c3ba3e236267aa963011b",
+ "file:size": 1535,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "product_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/DD/2025/1/S2B_T28UDD_20250110T120355_L2A/product_metadata.xml",
+ "type": "application/xml",
+ "file:checksum": "1220007c793944fcc6a5856a609c3d7a90a7981441c0c68bcc8e42d6da314802db15",
+ "file:size": 54651,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "thumbnail": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/DD/2025/1/S2B_T28UDD_20250110T120355_L2A/L2A_PVI.jpg",
+ "type": "image/jpeg",
+ "title": "Thumbnail of preview image",
+ "file:checksum": "1220e9c0f4836898f69022ec8dc67ae68ac26e772723680385d5606261fcff8d3b9a",
+ "file:size": 5672,
+ "roles": [
+ "thumbnail"
+ ]
+ }
+ },
+ "bbox": [-15.191184, 52.496602, -14.853738, 53.249561],
+ "stac_extensions": [
+ "https://stac-extensions.github.io/eo/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/file/v2.1.0/schema.json",
+ "https://stac-extensions.github.io/grid/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/mgrs/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/processing/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/projection/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/raster/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/sentinel-2/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/storage/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/view/v1.0.0/schema.json"
+ ],
+ "collection": "sentinel-2-c1-l2a"
+ },
+ {
+ "type": "Feature",
+ "stac_version": "1.0.0",
+ "id": "S2B_T28UGC_20250110T120355_L2A",
+ "properties": {
+ "created": "2025-01-10T16:24:15.582Z",
+ "platform": "sentinel-2b",
+ "constellation": "sentinel-2",
+ "instruments": [
+ "msi"
+ ],
+ "eo:cloud_cover": 70.551813,
+ "proj:epsg": 32628,
+ "proj:centroid": {
+ "lat": 52.10021,
+ "lon": -11.44977
+ },
+ "mgrs:utm_zone": 28,
+ "mgrs:latitude_band": "U",
+ "mgrs:grid_square": "GC",
+ "grid:code": "MGRS-28UGC",
+ "view:azimuth": 288.991145822068,
+ "view:incidence_angle": 8.88645376535856,
+ "view:sun_azimuth": 168.984702494377,
+ "view:sun_elevation": 15.6474414959517,
+ "s2:tile_id": "S2B_OPER_MSI_L2A_TL_2BPS_20250110T154043_A040990_T28UGC_N05.11",
+ "s2:degraded_msi_data_percentage": 0.0257,
+ "s2:nodata_pixel_percentage": 70.794016,
+ "s2:saturated_defective_pixel_percentage": 0,
+ "s2:cloud_shadow_percentage": 0,
+ "s2:vegetation_percentage": 0.000011,
+ "s2:not_vegetated_percentage": 0.000125,
+ "s2:water_percentage": 29.44805,
+ "s2:unclassified_percentage": 0,
+ "s2:medium_proba_clouds_percentage": 19.48228,
+ "s2:high_proba_clouds_percentage": 50.357151,
+ "s2:thin_cirrus_percentage": 0.712383,
+ "s2:snow_ice_percentage": 0,
+ "s2:product_type": "S2MSI2A",
+ "s2:processing_baseline": "05.11",
+ "s2:product_uri": "S2B_MSIL2A_20250110T120359_N0511_R066_T28UGC_20250110T154043.SAFE",
+ "s2:generation_time": "2025-01-10T15:40:43.000000Z",
+ "s2:datatake_id": "GS2B_20250110T120359_040990_N05.11",
+ "s2:datatake_type": "INS-NOBS",
+ "s2:datastrip_id": "S2B_OPER_MSI_L2A_DS_2BPS_20250110T154043_S20250110T120355_N05.11",
+ "s2:reflectance_conversion_factor": 1.03425891111326,
+ "datetime": "2025-01-10T12:06:59.032000Z",
+ "earthsearch:payload_id": "roda-sentinel-2-c1-l2a/workflow-sentinel-2-c1-l2a-to-stac/b262a7540f8c30c340c7c4d8c2b08a7b",
+ "storage:platform": "AWS",
+ "storage:region": "us-west-2",
+ "storage:requester_pays": false,
+ "processing:software": {
+ "sentinel-2-c1-l2a-to-stac": "v2024.02.01"
+ },
+ "updated": "2025-01-10T16:24:15.582Z"
+ },
+ "geometry": {
+ "type": "Polygon",
+ "coordinates": [
+ [
+ [-12.0663701781865, 52.3140284614992],
+ [-12.0854674480038, 52.0229565808749],
+ [-11.0571384840821, 51.7940576782695],
+ [-10.8034742021281, 52.2758584353317],
+ [-12.0663701781865, 52.3140284614992]
+ ]
+ ]
+ },
+ "links": [
+ {
+ "rel": "self",
+ "type": "application/geo+json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a/items/S2B_T28UGC_20250110T120355_L2A"
+ },
+ {
+ "rel": "canonical",
+ "href": "s3://e84-earth-search-sentinel-data/sentinel-2-c1-l2a/28/U/GC/2025/1/S2B_T28UGC_20250110T120355_L2A/S2B_T28UGC_20250110T120355_L2A.json",
+ "type": "application/json"
+ },
+ {
+ "rel": "via",
+ "href": "s3://sentinel-s2-l2a/tiles/28/U/GC/2025/1/10/0/metadata.xml",
+ "type": "application/xml",
+ "title": "Granule Metadata in Sinergize RODA Archive"
+ },
+ {
+ "rel": "parent",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a"
+ },
+ {
+ "rel": "collection",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a"
+ },
+ {
+ "rel": "root",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1"
+ },
+ {
+ "rel": "thumbnail",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a/items/S2B_T28UGC_20250110T120355_L2A/thumbnail"
+ }
+ ],
+ "assets": {
+ "red": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/GC/2025/1/S2B_T28UGC_20250110T120355_L2A/B04.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red - 10m",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 699960, 0, -10, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12201cd9c10ddd829fac7b1648dd387ad4fbb1bca16f50c5e3c4fefe23407a9c4011",
+ "file:size": 58207551,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "green": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/GC/2025/1/S2B_T28UGC_20250110T120355_L2A/B03.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Green - 10m",
+ "eo:bands": [
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 699960, 0, -10, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220451b51412527a211547b7f1fb65618ab70152c0b5876a28710d82c7afb07228e",
+ "file:size": 58118240,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "blue": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/GC/2025/1/S2B_T28UGC_20250110T120355_L2A/B02.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Blue - 10m",
+ "eo:bands": [
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 699960, 0, -10, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122029c3cfa284c85df6ebfaa6b0225ca33c147c2666175b9faab17d726468c25684",
+ "file:size": 58042746,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "visual": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/GC/2025/1/S2B_T28UGC_20250110T120355_L2A/TCI.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "True color image",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ },
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ },
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ },
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ },
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 699960, 0, -10, 5800020],
+ "file:checksum": "1220569a22ecc77cb50b68399d0587f0d4b6d7912694d646593166e712e03088fe8a",
+ "file:size": 42064723,
+ "roles": [
+ "visual"
+ ]
+ },
+ "nir": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/GC/2025/1/S2B_T28UGC_20250110T120355_L2A/B08.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 1 - 10m",
+ "eo:bands": [
+ {
+ "name": "B08",
+ "common_name": "nir",
+ "center_wavelength": 0.842,
+ "full_width_half_max": 0.145
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 699960, 0, -10, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122086b8c7c05c080220c58c7837aa5cb177148c998ed3e20bcbc82b5f4483c08de2",
+ "file:size": 58822385,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "swir22": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/GC/2025/1/S2B_T28UGC_20250110T120355_L2A/B12.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "SWIR 2.2μm - 20m",
+ "eo:bands": [
+ {
+ "name": "B12",
+ "common_name": "swir22",
+ "center_wavelength": 2.19,
+ "full_width_half_max": 0.242
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 699960, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220923638898bfc9c9f7146c955a4a57585779d374044ba21a7158e9810629894c4",
+ "file:size": 16095523,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge2": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/GC/2025/1/S2B_T28UGC_20250110T120355_L2A/B06.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 2 - 20m",
+ "eo:bands": [
+ {
+ "name": "B06",
+ "common_name": "rededge",
+ "center_wavelength": 0.74,
+ "full_width_half_max": 0.018
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 699960, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12200d043c3f0dcbee9be9b5e6c1bc5885ecaebb32730b97758c7a058f824b30ae00",
+ "file:size": 17051737,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge3": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/GC/2025/1/S2B_T28UGC_20250110T120355_L2A/B07.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 3 - 20m",
+ "eo:bands": [
+ {
+ "name": "B07",
+ "common_name": "rededge",
+ "center_wavelength": 0.783,
+ "full_width_half_max": 0.028
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 699960, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12201496c6d8c6a667ce1068dcb750ce1ac590966a54524102be8fcfa41c1c97b95d",
+ "file:size": 17021702,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge1": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/GC/2025/1/S2B_T28UGC_20250110T120355_L2A/B05.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 1 - 20m",
+ "eo:bands": [
+ {
+ "name": "B05",
+ "common_name": "rededge",
+ "center_wavelength": 0.704,
+ "full_width_half_max": 0.019
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 699960, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220943299479e9fc7167e046c58922a0cd43468693b4e14a8086ebba2643c7501a1",
+ "file:size": 17061778,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "swir16": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/GC/2025/1/S2B_T28UGC_20250110T120355_L2A/B11.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "SWIR 1.6μm - 20m",
+ "eo:bands": [
+ {
+ "name": "B11",
+ "common_name": "swir16",
+ "center_wavelength": 1.61,
+ "full_width_half_max": 0.143
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 699960, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220fa82c7aa49224096f09453a532e579f12fb83c7b57eb5efb3e01d7f86d87309f",
+ "file:size": 16432588,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "wvp": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/GC/2025/1/S2B_T28UGC_20250110T120355_L2A/WVP.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Water Vapour (WVP)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 699960, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "unit": "cm",
+ "scale": 0.001,
+ "offset": 0
+ }
+ ],
+ "file:checksum": "1220819c7233d673f986b8d77c116ec375ed68ac1f9d11665a3edfe513c73d26805e",
+ "file:size": 133428,
+ "roles": [
+ "data"
+ ]
+ },
+ "nir08": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/GC/2025/1/S2B_T28UGC_20250110T120355_L2A/B8A.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 2 - 20m",
+ "eo:bands": [
+ {
+ "name": "B8A",
+ "common_name": "nir08",
+ "center_wavelength": 0.865,
+ "full_width_half_max": 0.033
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 699960, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122008817be7d0e498c3ca35dc04ec5299db2f686150c71ac4e4d6c2219896656f68",
+ "file:size": 17052752,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "scl": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/GC/2025/1/S2B_T28UGC_20250110T120355_L2A/SCL.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Scene classification map (SCL)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 699960, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "12201be991eef66b3ee9fb9993040702d74dc890ee3bc5f13767c779dfbbe49f3004",
+ "file:size": 645587,
+ "roles": [
+ "data"
+ ]
+ },
+ "aot": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/GC/2025/1/S2B_T28UGC_20250110T120355_L2A/AOT.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Aerosol optical thickness (AOT)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 699960, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.001,
+ "offset": 0
+ }
+ ],
+ "file:checksum": "1220633bfaa1c34b320443dbb216a3479c1e75b634b9785145005de6cf6130d864b9",
+ "file:size": 201548,
+ "roles": [
+ "data"
+ ]
+ },
+ "coastal": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/GC/2025/1/S2B_T28UGC_20250110T120355_L2A/B01.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Coastal - 60m",
+ "eo:bands": [
+ {
+ "name": "B01",
+ "common_name": "coastal",
+ "center_wavelength": 0.443,
+ "full_width_half_max": 0.027
+ }
+ ],
+ "gsd": 60,
+ "proj:shape": [1830, 1830],
+ "proj:transform": [60, 0, 699960, 0, -60, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 60,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12207141b1f48d8faf012d0acb1beebbf23b5c3df7c4b9a167d8d73ef7d15ebff353",
+ "file:size": 2012692,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "nir09": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/GC/2025/1/S2B_T28UGC_20250110T120355_L2A/B09.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 3 - 60m",
+ "eo:bands": [
+ {
+ "name": "B09",
+ "common_name": "nir09",
+ "center_wavelength": 0.945,
+ "full_width_half_max": 0.026
+ }
+ ],
+ "gsd": 60,
+ "proj:shape": [1830, 1830],
+ "proj:transform": [60, 0, 699960, 0, -60, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 60,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12202e70675c4b9ab5a47692cd176acea1f68e5d1c435edda8e5c97c66bb775ddb0d",
+ "file:size": 2032167,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "cloud": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/GC/2025/1/S2B_T28UGC_20250110T120355_L2A/CLD_20m.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Cloud Probabilities",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 699960, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "1220f14c816e5e28cba274f817a6a1a078ca93a33b79f8c6b5d2805f13f302df01a6",
+ "file:size": 2676520,
+ "roles": [
+ "data",
+ "cloud"
+ ]
+ },
+ "snow": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/GC/2025/1/S2B_T28UGC_20250110T120355_L2A/SNW_20m.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Snow Probabilities",
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 699960, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "122074b684a20156d8958e2cc54b9b3740f5217f825466c66d10e8df379e946e5a96",
+ "file:size": 53931,
+ "roles": [
+ "data",
+ "snow-ice"
+ ]
+ },
+ "preview": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/GC/2025/1/S2B_T28UGC_20250110T120355_L2A/L2A_PVI.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "True color preview",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ },
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ },
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "file:checksum": "1220f1f49f585c5c9d162e3fde0c6524393af24d0f98bd94008bf153e0d1f0d84545",
+ "file:size": 66287,
+ "roles": [
+ "overview"
+ ]
+ },
+ "granule_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/GC/2025/1/S2B_T28UGC_20250110T120355_L2A/metadata.xml",
+ "type": "application/xml",
+ "file:checksum": "1220cd75a8a624f29c0f31c90df9f791b916a3e495ea7c087b579e2e09770559a655",
+ "file:size": 351063,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "tileinfo_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/GC/2025/1/S2B_T28UGC_20250110T120355_L2A/tileInfo.json",
+ "type": "application/json",
+ "file:checksum": "1220d1c9a251b350d7c950c3be70977893a5b2c10c2e124fb0b66093ed644cab670e",
+ "file:size": 1607,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "product_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/GC/2025/1/S2B_T28UGC_20250110T120355_L2A/product_metadata.xml",
+ "type": "application/xml",
+ "file:checksum": "12208208152cf7b8ff0461a8f98b958c6b601c8d7e2b0daf2a6da54fcee0188d250d",
+ "file:size": 55288,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "thumbnail": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/GC/2025/1/S2B_T28UGC_20250110T120355_L2A/L2A_PVI.jpg",
+ "type": "image/jpeg",
+ "title": "Thumbnail of preview image",
+ "file:checksum": "12204514017e6fc351907beb978f8bf0d1e782b2f3f48280443825827d0e5218ff1f",
+ "file:size": 13689,
+ "roles": [
+ "thumbnail"
+ ]
+ }
+ },
+ "bbox": [-12.085467, 51.794058, -10.803474, 52.314028],
+ "stac_extensions": [
+ "https://stac-extensions.github.io/eo/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/file/v2.1.0/schema.json",
+ "https://stac-extensions.github.io/grid/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/mgrs/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/processing/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/projection/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/raster/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/sentinel-2/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/storage/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/view/v1.0.0/schema.json"
+ ],
+ "collection": "sentinel-2-c1-l2a"
+ },
+ {
+ "type": "Feature",
+ "stac_version": "1.0.0",
+ "id": "S2B_T29ULT_20250110T120355_L2A",
+ "properties": {
+ "created": "2025-01-10T16:24:35.264Z",
+ "platform": "sentinel-2b",
+ "constellation": "sentinel-2",
+ "instruments": [
+ "msi"
+ ],
+ "eo:cloud_cover": 76.681787,
+ "proj:epsg": 32629,
+ "proj:centroid": {
+ "lat": 52.11257,
+ "lon": -11.37077
+ },
+ "mgrs:utm_zone": 29,
+ "mgrs:latitude_band": "U",
+ "mgrs:grid_square": "LT",
+ "grid:code": "MGRS-29ULT",
+ "view:azimuth": 289.739549819918,
+ "view:incidence_angle": 9.25309699449269,
+ "view:sun_azimuth": 169.170970252047,
+ "view:sun_elevation": 15.6313798177967,
+ "s2:tile_id": "S2B_OPER_MSI_L2A_TL_2BPS_20250110T154043_A040990_T29ULT_N05.11",
+ "s2:degraded_msi_data_percentage": 0.0281,
+ "s2:nodata_pixel_percentage": 71.433568,
+ "s2:saturated_defective_pixel_percentage": 0,
+ "s2:cloud_shadow_percentage": 0,
+ "s2:vegetation_percentage": 0.000012,
+ "s2:not_vegetated_percentage": 0.000186,
+ "s2:water_percentage": 23.318014,
+ "s2:unclassified_percentage": 0,
+ "s2:medium_proba_clouds_percentage": 35.262308,
+ "s2:high_proba_clouds_percentage": 41.088226,
+ "s2:thin_cirrus_percentage": 0.331256,
+ "s2:snow_ice_percentage": 0,
+ "s2:product_type": "S2MSI2A",
+ "s2:processing_baseline": "05.11",
+ "s2:product_uri": "S2B_MSIL2A_20250110T120359_N0511_R066_T29ULT_20250110T154043.SAFE",
+ "s2:generation_time": "2025-01-10T15:40:43.000000Z",
+ "s2:datatake_id": "GS2B_20250110T120359_040990_N05.11",
+ "s2:datatake_type": "INS-NOBS",
+ "s2:datastrip_id": "S2B_OPER_MSI_L2A_DS_2BPS_20250110T154043_S20250110T120355_N05.11",
+ "s2:reflectance_conversion_factor": 1.03425891111326,
+ "datetime": "2025-01-10T12:06:58.560000Z",
+ "earthsearch:payload_id": "roda-sentinel-2-c1-l2a/workflow-sentinel-2-c1-l2a-to-stac/cc5f14a543c20f0fbb2d8a793a17b244",
+ "storage:platform": "AWS",
+ "storage:region": "us-west-2",
+ "storage:requester_pays": false,
+ "processing:software": {
+ "sentinel-2-c1-l2a-to-stac": "v2024.02.01"
+ },
+ "updated": "2025-01-10T16:24:35.264Z"
+ },
+ "geometry": {
+ "type": "Polygon",
+ "coordinates": [
+ [
+ [-11.9341865273641, 52.3140146232143],
+ [-11.9127080253672, 51.9863614962068],
+ [-11.0571957464118, 51.7939912466001],
+ [-10.7711148716862, 52.3371889236931],
+ [-11.9341865273641, 52.3140146232143]
+ ]
+ ]
+ },
+ "links": [
+ {
+ "rel": "self",
+ "type": "application/geo+json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a/items/S2B_T29ULT_20250110T120355_L2A"
+ },
+ {
+ "rel": "canonical",
+ "href": "s3://e84-earth-search-sentinel-data/sentinel-2-c1-l2a/29/U/LT/2025/1/S2B_T29ULT_20250110T120355_L2A/S2B_T29ULT_20250110T120355_L2A.json",
+ "type": "application/json"
+ },
+ {
+ "rel": "via",
+ "href": "s3://sentinel-s2-l2a/tiles/29/U/LT/2025/1/10/0/metadata.xml",
+ "type": "application/xml",
+ "title": "Granule Metadata in Sinergize RODA Archive"
+ },
+ {
+ "rel": "parent",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a"
+ },
+ {
+ "rel": "collection",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a"
+ },
+ {
+ "rel": "root",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1"
+ },
+ {
+ "rel": "thumbnail",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a/items/S2B_T29ULT_20250110T120355_L2A/thumbnail"
+ }
+ ],
+ "assets": {
+ "red": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/LT/2025/1/S2B_T29ULT_20250110T120355_L2A/B04.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red - 10m",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 300000, 0, -10, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12203e227546657330900c61d6557c4750c3262eb5eeb70edd161878f0db44cd933f",
+ "file:size": 56941462,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "green": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/LT/2025/1/S2B_T29ULT_20250110T120355_L2A/B03.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Green - 10m",
+ "eo:bands": [
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 300000, 0, -10, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122025246795e56b26d4f7d9fc31b995680db8d55ba9ad507ebc922be4509769f6c9",
+ "file:size": 56786069,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "blue": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/LT/2025/1/S2B_T29ULT_20250110T120355_L2A/B02.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Blue - 10m",
+ "eo:bands": [
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 300000, 0, -10, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122039d4d6a228946d25de5f8480487f6c492398ed7d8a2c991648d04d3cd85a9935",
+ "file:size": 56623575,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "visual": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/LT/2025/1/S2B_T29ULT_20250110T120355_L2A/TCI.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "True color image",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ },
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ },
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ },
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ },
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 300000, 0, -10, 5800020],
+ "file:checksum": "122010e77d3a354175a4f00e427377412b14cb594c0ef55f2744eee10062fc94494d",
+ "file:size": 36438161,
+ "roles": [
+ "visual"
+ ]
+ },
+ "nir": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/LT/2025/1/S2B_T29ULT_20250110T120355_L2A/B08.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 1 - 10m",
+ "eo:bands": [
+ {
+ "name": "B08",
+ "common_name": "nir",
+ "center_wavelength": 0.842,
+ "full_width_half_max": 0.145
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 300000, 0, -10, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12203493df70a2489aae8e8c51256ab1b149136d0399aeff8bb521944d0a373ffb8e",
+ "file:size": 57557157,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "swir22": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/LT/2025/1/S2B_T29ULT_20250110T120355_L2A/B12.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "SWIR 2.2μm - 20m",
+ "eo:bands": [
+ {
+ "name": "B12",
+ "common_name": "swir22",
+ "center_wavelength": 2.19,
+ "full_width_half_max": 0.242
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 300000, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12207a1e16f696777c391dc53358b0a0892061539000855005f8cffb5ef1a00659c4",
+ "file:size": 15781560,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge2": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/LT/2025/1/S2B_T29ULT_20250110T120355_L2A/B06.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 2 - 20m",
+ "eo:bands": [
+ {
+ "name": "B06",
+ "common_name": "rededge",
+ "center_wavelength": 0.74,
+ "full_width_half_max": 0.018
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 300000, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12203277ff67ac82a6260dd05378d0880c66af3280495382175f61b0b152cb72e671",
+ "file:size": 16820810,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge3": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/LT/2025/1/S2B_T29ULT_20250110T120355_L2A/B07.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 3 - 20m",
+ "eo:bands": [
+ {
+ "name": "B07",
+ "common_name": "rededge",
+ "center_wavelength": 0.783,
+ "full_width_half_max": 0.028
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 300000, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220ce612d9136bc91eb41bc1ac94486ce6c46c6fd80f6a3c2370bb165b54e22d264",
+ "file:size": 16799086,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge1": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/LT/2025/1/S2B_T29ULT_20250110T120355_L2A/B05.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 1 - 20m",
+ "eo:bands": [
+ {
+ "name": "B05",
+ "common_name": "rededge",
+ "center_wavelength": 0.704,
+ "full_width_half_max": 0.019
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 300000, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220f90ef2d43a3d5a38ccfa2dc7bbcebef1241ee73f8cd34f016f9fd1dd4195050c",
+ "file:size": 16823874,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "swir16": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/LT/2025/1/S2B_T29ULT_20250110T120355_L2A/B11.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "SWIR 1.6μm - 20m",
+ "eo:bands": [
+ {
+ "name": "B11",
+ "common_name": "swir16",
+ "center_wavelength": 1.61,
+ "full_width_half_max": 0.143
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 300000, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12206eaa8e87752b95598c4ad90eaead667c44881553ded86b19479eda4aa18b47fc",
+ "file:size": 16192304,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "wvp": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/LT/2025/1/S2B_T29ULT_20250110T120355_L2A/WVP.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Water Vapour (WVP)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 300000, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "unit": "cm",
+ "scale": 0.001,
+ "offset": 0
+ }
+ ],
+ "file:checksum": "1220549c7f1f9556aa87f2e9e2306077af27a4871edd6f23a7b7483a945def3a3eb9",
+ "file:size": 134178,
+ "roles": [
+ "data"
+ ]
+ },
+ "nir08": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/LT/2025/1/S2B_T29ULT_20250110T120355_L2A/B8A.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 2 - 20m",
+ "eo:bands": [
+ {
+ "name": "B8A",
+ "common_name": "nir08",
+ "center_wavelength": 0.865,
+ "full_width_half_max": 0.033
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 300000, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12207d766a2efd3de74d5956711abca5765dd5734ca28672459eb4deb45421228eb7",
+ "file:size": 16836512,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "scl": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/LT/2025/1/S2B_T29ULT_20250110T120355_L2A/SCL.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Scene classification map (SCL)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 300000, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "1220cef0ac90be0d43aa4a3a7f4e32f86304227b58c4e1d6f12398f02ede7d0f1916",
+ "file:size": 661235,
+ "roles": [
+ "data"
+ ]
+ },
+ "aot": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/LT/2025/1/S2B_T29ULT_20250110T120355_L2A/AOT.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Aerosol optical thickness (AOT)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 300000, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.001,
+ "offset": 0
+ }
+ ],
+ "file:checksum": "12206d45dd57516e4491befa01b0fdc88fb8b4ac6672e9b61e302b6f8f42bc5979de",
+ "file:size": 198595,
+ "roles": [
+ "data"
+ ]
+ },
+ "coastal": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/LT/2025/1/S2B_T29ULT_20250110T120355_L2A/B01.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Coastal - 60m",
+ "eo:bands": [
+ {
+ "name": "B01",
+ "common_name": "coastal",
+ "center_wavelength": 0.443,
+ "full_width_half_max": 0.027
+ }
+ ],
+ "gsd": 60,
+ "proj:shape": [1830, 1830],
+ "proj:transform": [60, 0, 300000, 0, -60, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 60,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220ddc842e5e2933d01235e0bb88ea63c867ce228316a0c5277f7cbf6f1b3221769",
+ "file:size": 1991573,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "nir09": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/LT/2025/1/S2B_T29ULT_20250110T120355_L2A/B09.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 3 - 60m",
+ "eo:bands": [
+ {
+ "name": "B09",
+ "common_name": "nir09",
+ "center_wavelength": 0.945,
+ "full_width_half_max": 0.026
+ }
+ ],
+ "gsd": 60,
+ "proj:shape": [1830, 1830],
+ "proj:transform": [60, 0, 300000, 0, -60, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 60,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220225bec7e6941aba12271c19bbe8d9c5c14ea3fc71262b2d4b75880c53ac1ee39",
+ "file:size": 2016382,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "cloud": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/LT/2025/1/S2B_T29ULT_20250110T120355_L2A/CLD_20m.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Cloud Probabilities",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 300000, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "1220a755be67c7393e95e506909fdcfd568f6d43a6fa42cbccfc5232584d02d1eaa4",
+ "file:size": 2162795,
+ "roles": [
+ "data",
+ "cloud"
+ ]
+ },
+ "snow": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/LT/2025/1/S2B_T29ULT_20250110T120355_L2A/SNW_20m.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Snow Probabilities",
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 300000, 0, -20, 5800020],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "122099280d6b49f45f813f5c9a54c08d31cae51069d59d9558d551878d41e7dab826",
+ "file:size": 370705,
+ "roles": [
+ "data",
+ "snow-ice"
+ ]
+ },
+ "preview": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/LT/2025/1/S2B_T29ULT_20250110T120355_L2A/L2A_PVI.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "True color preview",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ },
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ },
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "file:checksum": "12204c088e206ddaf24ab7fcba7afb6b533fab045b82c4272f76aa21f56f3c4af391",
+ "file:size": 60549,
+ "roles": [
+ "overview"
+ ]
+ },
+ "granule_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/LT/2025/1/S2B_T29ULT_20250110T120355_L2A/metadata.xml",
+ "type": "application/xml",
+ "file:checksum": "12201c67cbc583f92626907c855af5e64967e8c9137a7c1c633f260c38900e5b8f6e",
+ "file:size": 350629,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "tileinfo_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/LT/2025/1/S2B_T29ULT_20250110T120355_L2A/tileInfo.json",
+ "type": "application/json",
+ "file:checksum": "12205a7be274328fa33fbbb3aa6c6c482371a65b51770862aaafe3c67fcc4860ab32",
+ "file:size": 1568,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "product_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/LT/2025/1/S2B_T29ULT_20250110T120355_L2A/product_metadata.xml",
+ "type": "application/xml",
+ "file:checksum": "1220d52627f80e6fe4575c7128c6971309760e7f6097a1c1965ffb298cc4bebefb83",
+ "file:size": 55102,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "thumbnail": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/LT/2025/1/S2B_T29ULT_20250110T120355_L2A/L2A_PVI.jpg",
+ "type": "image/jpeg",
+ "title": "Thumbnail of preview image",
+ "file:checksum": "1220360fbeaa6ce13b08877966ad223cdaf36a8bae092899879eba5da47e1dcf3523",
+ "file:size": 13517,
+ "roles": [
+ "thumbnail"
+ ]
+ }
+ },
+ "bbox": [-11.934187, 51.793991, -10.771115, 52.337189],
+ "stac_extensions": [
+ "https://stac-extensions.github.io/eo/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/file/v2.1.0/schema.json",
+ "https://stac-extensions.github.io/grid/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/mgrs/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/processing/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/projection/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/raster/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/sentinel-2/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/storage/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/view/v1.0.0/schema.json"
+ ],
+ "collection": "sentinel-2-c1-l2a"
+ },
+ {
+ "type": "Feature",
+ "stac_version": "1.0.0",
+ "id": "S2B_T29UMU_20250110T120355_L2A",
+ "properties": {
+ "created": "2025-01-10T16:22:10.669Z",
+ "platform": "sentinel-2b",
+ "constellation": "sentinel-2",
+ "instruments": [
+ "msi"
+ ],
+ "eo:cloud_cover": 99.851912,
+ "proj:epsg": 32629,
+ "proj:centroid": {
+ "lat": 53.11473,
+ "lon": -10.41947
+ },
+ "mgrs:utm_zone": 29,
+ "mgrs:latitude_band": "U",
+ "mgrs:grid_square": "MU",
+ "grid:code": "MGRS-29UMU",
+ "view:azimuth": 295.133344244557,
+ "view:incidence_angle": 11.4258216022895,
+ "view:sun_azimuth": 170.514168318609,
+ "view:sun_elevation": 14.876218695379,
+ "s2:tile_id": "S2B_OPER_MSI_L2A_TL_2BPS_20250110T154043_A040990_T29UMU_N05.11",
+ "s2:degraded_msi_data_percentage": 0,
+ "s2:nodata_pixel_percentage": 97.318125,
+ "s2:saturated_defective_pixel_percentage": 0,
+ "s2:cloud_shadow_percentage": 0,
+ "s2:vegetation_percentage": 0,
+ "s2:not_vegetated_percentage": 0.001979,
+ "s2:water_percentage": 0.146106,
+ "s2:unclassified_percentage": 0,
+ "s2:medium_proba_clouds_percentage": 9.566632,
+ "s2:high_proba_clouds_percentage": 90.11963,
+ "s2:thin_cirrus_percentage": 0.165652,
+ "s2:snow_ice_percentage": 0,
+ "s2:product_type": "S2MSI2A",
+ "s2:processing_baseline": "05.11",
+ "s2:product_uri": "S2B_MSIL2A_20250110T120359_N0511_R066_T29UMU_20250110T154043.SAFE",
+ "s2:generation_time": "2025-01-10T15:40:43.000000Z",
+ "s2:datatake_id": "GS2B_20250110T120359_040990_N05.11",
+ "s2:datatake_type": "INS-NOBS",
+ "s2:datastrip_id": "S2B_OPER_MSI_L2A_DS_2BPS_20250110T154043_S20250110T120355_N05.11",
+ "s2:reflectance_conversion_factor": 1.03425891111326,
+ "datetime": "2025-01-10T12:06:38.977000Z",
+ "earthsearch:payload_id": "roda-sentinel-2-c1-l2a/workflow-sentinel-2-c1-l2a-to-stac/d7d4c025f2d4f3b3e51a14d9d98f1575",
+ "storage:platform": "AWS",
+ "storage:region": "us-west-2",
+ "storage:requester_pays": false,
+ "processing:software": {
+ "sentinel-2-c1-l2a-to-stac": "v2024.02.01"
+ },
+ "updated": "2025-01-10T16:22:10.669Z"
+ },
+ "geometry": {
+ "type": "Polygon",
+ "coordinates": [
+ [
+ [-10.4990442948644, 53.2401920169908],
+ [-10.4859475114454, 52.8611868644894],
+ [-10.2734217675276, 53.242816076468],
+ [-10.4990442948644, 53.2401920169908]
+ ]
+ ]
+ },
+ "links": [
+ {
+ "rel": "self",
+ "type": "application/geo+json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a/items/S2B_T29UMU_20250110T120355_L2A"
+ },
+ {
+ "rel": "canonical",
+ "href": "s3://e84-earth-search-sentinel-data/sentinel-2-c1-l2a/29/U/MU/2025/1/S2B_T29UMU_20250110T120355_L2A/S2B_T29UMU_20250110T120355_L2A.json",
+ "type": "application/json"
+ },
+ {
+ "rel": "via",
+ "href": "s3://sentinel-s2-l2a/tiles/29/U/MU/2025/1/10/0/metadata.xml",
+ "type": "application/xml",
+ "title": "Granule Metadata in Sinergize RODA Archive"
+ },
+ {
+ "rel": "parent",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a"
+ },
+ {
+ "rel": "collection",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a"
+ },
+ {
+ "rel": "root",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1"
+ },
+ {
+ "rel": "thumbnail",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a/items/S2B_T29UMU_20250110T120355_L2A/thumbnail"
+ }
+ ],
+ "assets": {
+ "red": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/MU/2025/1/S2B_T29UMU_20250110T120355_L2A/B04.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red - 10m",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 399960, 0, -10, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220d309c47c8b7acb6ae4c1e17ec394013535cc1013c9c9735d1f83e257272bb3aa",
+ "file:size": 5111486,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "green": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/MU/2025/1/S2B_T29UMU_20250110T120355_L2A/B03.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Green - 10m",
+ "eo:bands": [
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 399960, 0, -10, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12207891e5d0515df9add88bbe4bd1fe100fbcaf720e34972134abc213b78c02b076",
+ "file:size": 5100143,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "blue": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/MU/2025/1/S2B_T29UMU_20250110T120355_L2A/B02.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Blue - 10m",
+ "eo:bands": [
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 399960, 0, -10, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12202be80f386072c1c9af35f23f2dad5abb0ea0670c275a50bee9fa47a65a97437b",
+ "file:size": 5092633,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "visual": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/MU/2025/1/S2B_T29UMU_20250110T120355_L2A/TCI.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "True color image",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ },
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ },
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ },
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ },
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 399960, 0, -10, 5900040],
+ "file:checksum": "1220101e3fd7208a1cb7deec2cb6d8d27db55beeefe9e348787feb9cf0d8ddc67da8",
+ "file:size": 562065,
+ "roles": [
+ "visual"
+ ]
+ },
+ "nir": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/MU/2025/1/S2B_T29UMU_20250110T120355_L2A/B08.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 1 - 10m",
+ "eo:bands": [
+ {
+ "name": "B08",
+ "common_name": "nir",
+ "center_wavelength": 0.842,
+ "full_width_half_max": 0.145
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 399960, 0, -10, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122012b1bd512d34f36373af0856da95c7f8a4bd87e3608736484c5244e4159d3012",
+ "file:size": 5187449,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "swir22": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/MU/2025/1/S2B_T29UMU_20250110T120355_L2A/B12.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "SWIR 2.2μm - 20m",
+ "eo:bands": [
+ {
+ "name": "B12",
+ "common_name": "swir22",
+ "center_wavelength": 2.19,
+ "full_width_half_max": 0.242
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 399960, 0, -20, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12208f7d6fe1f69da7d1e94610dece34be45ea32a49e47577c644090eaf0bcb22611",
+ "file:size": 1596928,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge2": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/MU/2025/1/S2B_T29UMU_20250110T120355_L2A/B06.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 2 - 20m",
+ "eo:bands": [
+ {
+ "name": "B06",
+ "common_name": "rededge",
+ "center_wavelength": 0.74,
+ "full_width_half_max": 0.018
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 399960, 0, -20, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220ca6df763e9ae2c033cb78bdcea27ad9d8885ff4bede1413576bb66ff28de366b",
+ "file:size": 1669812,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge3": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/MU/2025/1/S2B_T29UMU_20250110T120355_L2A/B07.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 3 - 20m",
+ "eo:bands": [
+ {
+ "name": "B07",
+ "common_name": "rededge",
+ "center_wavelength": 0.783,
+ "full_width_half_max": 0.028
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 399960, 0, -20, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122050956255cbee8d0801242c316be3c8dca2018b9be3c555a49df3919a991396c4",
+ "file:size": 1666645,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge1": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/MU/2025/1/S2B_T29UMU_20250110T120355_L2A/B05.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 1 - 20m",
+ "eo:bands": [
+ {
+ "name": "B05",
+ "common_name": "rededge",
+ "center_wavelength": 0.704,
+ "full_width_half_max": 0.019
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 399960, 0, -20, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220a29376f79d03afef311addd6ae68b447fec13192e4c96dc19cb6875bf47abc43",
+ "file:size": 1670391,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "swir16": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/MU/2025/1/S2B_T29UMU_20250110T120355_L2A/B11.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "SWIR 1.6μm - 20m",
+ "eo:bands": [
+ {
+ "name": "B11",
+ "common_name": "swir16",
+ "center_wavelength": 1.61,
+ "full_width_half_max": 0.143
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 399960, 0, -20, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220792a6aa646da0a6da7185b9571185672519b6c5619d0184c2c7f33a512cf58fd",
+ "file:size": 1619071,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "wvp": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/MU/2025/1/S2B_T29UMU_20250110T120355_L2A/WVP.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Water Vapour (WVP)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 399960, 0, -20, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "unit": "cm",
+ "scale": 0.001,
+ "offset": 0
+ }
+ ],
+ "file:checksum": "122096a7ec83f826c2d64ddfffdf0de254ab329380fe27a62a5316684905d3d2fb57",
+ "file:size": 105171,
+ "roles": [
+ "data"
+ ]
+ },
+ "nir08": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/MU/2025/1/S2B_T29UMU_20250110T120355_L2A/B8A.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 2 - 20m",
+ "eo:bands": [
+ {
+ "name": "B8A",
+ "common_name": "nir08",
+ "center_wavelength": 0.865,
+ "full_width_half_max": 0.033
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 399960, 0, -20, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12200d824dc3b6cc3b2a25a1f4418439599f9855bf56a45e0743366b5bf79fe6e430",
+ "file:size": 1673352,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "scl": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/MU/2025/1/S2B_T29UMU_20250110T120355_L2A/SCL.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Scene classification map (SCL)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 399960, 0, -20, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "1220b1b8d5243264af9a9a6c20e3c60bbc1cf405376b735d78cf80aa23b526237b68",
+ "file:size": 81560,
+ "roles": [
+ "data"
+ ]
+ },
+ "aot": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/MU/2025/1/S2B_T29UMU_20250110T120355_L2A/AOT.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Aerosol optical thickness (AOT)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 399960, 0, -20, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.001,
+ "offset": 0
+ }
+ ],
+ "file:checksum": "12204f80ce4edb419f177e533dba4ac913145de59cc8e384be1455217e2fed4e25e0",
+ "file:size": 113778,
+ "roles": [
+ "data"
+ ]
+ },
+ "coastal": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/MU/2025/1/S2B_T29UMU_20250110T120355_L2A/B01.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Coastal - 60m",
+ "eo:bands": [
+ {
+ "name": "B01",
+ "common_name": "coastal",
+ "center_wavelength": 0.443,
+ "full_width_half_max": 0.027
+ }
+ ],
+ "gsd": 60,
+ "proj:shape": [1830, 1830],
+ "proj:transform": [60, 0, 399960, 0, -60, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 60,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220708a0c79160b88446637eeeee0bb16b66b7553f6472b29b65275dd5c0944ff14",
+ "file:size": 204521,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "nir09": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/MU/2025/1/S2B_T29UMU_20250110T120355_L2A/B09.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 3 - 60m",
+ "eo:bands": [
+ {
+ "name": "B09",
+ "common_name": "nir09",
+ "center_wavelength": 0.945,
+ "full_width_half_max": 0.026
+ }
+ ],
+ "gsd": 60,
+ "proj:shape": [1830, 1830],
+ "proj:transform": [60, 0, 399960, 0, -60, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 60,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12209c228a60e2ae567cfe920c25213db4ea1d78a90de769f0e8dc1f1d946db2538a",
+ "file:size": 212847,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "cloud": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/MU/2025/1/S2B_T29UMU_20250110T120355_L2A/CLD_20m.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Cloud Probabilities",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 399960, 0, -20, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "1220a3f9f026bd5025f82f3adf0f6c2b419771977a89dd84da2726802de5492beb6f",
+ "file:size": 90369,
+ "roles": [
+ "data",
+ "cloud"
+ ]
+ },
+ "snow": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/MU/2025/1/S2B_T29UMU_20250110T120355_L2A/SNW_20m.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Snow Probabilities",
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 399960, 0, -20, 5900040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "1220a94513ceac58211fe15df962a282ba81af3b7a6c86448fdd158af7e5a72571d7",
+ "file:size": 77916,
+ "roles": [
+ "data",
+ "snow-ice"
+ ]
+ },
+ "preview": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/MU/2025/1/S2B_T29UMU_20250110T120355_L2A/L2A_PVI.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "True color preview",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ },
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ },
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "file:checksum": "1220a918fb0a9713c7662bea5d722e61ea704c8f72df11172c45b6faa37709bf0ba7",
+ "file:size": 5132,
+ "roles": [
+ "overview"
+ ]
+ },
+ "granule_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/MU/2025/1/S2B_T29UMU_20250110T120355_L2A/metadata.xml",
+ "type": "application/xml",
+ "file:checksum": "1220688e99c289650d08535e4526f0b30b9f827f967e201a550181d5ba95cbd06706",
+ "file:size": 101884,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "tileinfo_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/MU/2025/1/S2B_T29UMU_20250110T120355_L2A/tileInfo.json",
+ "type": "application/json",
+ "file:checksum": "12209891f3af981af6621dedc4fb99d9b1e69346d48bb303bd5dd0d3c3c38798b74d",
+ "file:size": 1482,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "product_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/MU/2025/1/S2B_T29UMU_20250110T120355_L2A/product_metadata.xml",
+ "type": "application/xml",
+ "file:checksum": "1220b390185e418500d3d7c46dfd44f6f490cb6771c5c2b11c8ddd909345efa0b488",
+ "file:size": 54497,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "thumbnail": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/MU/2025/1/S2B_T29UMU_20250110T120355_L2A/L2A_PVI.jpg",
+ "type": "image/jpeg",
+ "title": "Thumbnail of preview image",
+ "file:checksum": "1220ad716bb6e2c9c8aeec0e64984b21d504b7fcad9c6fb68ee4c7fbd1e86a76ca48",
+ "file:size": 3202,
+ "roles": [
+ "thumbnail"
+ ]
+ }
+ },
+ "bbox": [-10.499044, 52.861187, -10.273422, 53.242816],
+ "stac_extensions": [
+ "https://stac-extensions.github.io/eo/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/file/v2.1.0/schema.json",
+ "https://stac-extensions.github.io/grid/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/mgrs/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/processing/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/projection/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/raster/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/sentinel-2/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/storage/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/view/v1.0.0/schema.json"
+ ],
+ "collection": "sentinel-2-c1-l2a"
+ },
+ {
+ "type": "Feature",
+ "stac_version": "1.0.0",
+ "id": "S2B_T28UEG_20250110T120355_L2A",
+ "properties": {
+ "created": "2025-01-10T16:24:42.505Z",
+ "platform": "sentinel-2b",
+ "constellation": "sentinel-2",
+ "instruments": [
+ "msi"
+ ],
+ "eo:cloud_cover": 100,
+ "proj:epsg": 32628,
+ "proj:centroid": {
+ "lat": 55.38441,
+ "lon": -13.58014
+ },
+ "mgrs:utm_zone": 28,
+ "mgrs:latitude_band": "U",
+ "mgrs:grid_square": "EG",
+ "grid:code": "MGRS-28UEG",
+ "view:azimuth": 107.206073425202,
+ "view:incidence_angle": 10.2725275481198,
+ "view:sun_azimuth": 166.291906822544,
+ "view:sun_elevation": 11.6956433007712,
+ "s2:tile_id": "S2B_OPER_MSI_L2A_TL_2BPS_20250110T154043_A040990_T28UEG_N05.11",
+ "s2:degraded_msi_data_percentage": 0,
+ "s2:nodata_pixel_percentage": 65.152574,
+ "s2:saturated_defective_pixel_percentage": 0,
+ "s2:cloud_shadow_percentage": 0,
+ "s2:vegetation_percentage": 0,
+ "s2:not_vegetated_percentage": 0,
+ "s2:water_percentage": 0,
+ "s2:unclassified_percentage": 0,
+ "s2:medium_proba_clouds_percentage": 1.233756,
+ "s2:high_proba_clouds_percentage": 98.766243,
+ "s2:thin_cirrus_percentage": 0,
+ "s2:snow_ice_percentage": 0,
+ "s2:product_type": "S2MSI2A",
+ "s2:processing_baseline": "05.11",
+ "s2:product_uri": "S2B_MSIL2A_20250110T120359_N0511_R066_T28UEG_20250110T154043.SAFE",
+ "s2:generation_time": "2025-01-10T15:40:43.000000Z",
+ "s2:datatake_id": "GS2B_20250110T120359_040990_N05.11",
+ "s2:datatake_type": "INS-NOBS",
+ "s2:datastrip_id": "S2B_OPER_MSI_L2A_DS_2BPS_20250110T154043_S20250110T120355_N05.11",
+ "s2:reflectance_conversion_factor": 1.03425891111326,
+ "datetime": "2025-01-10T12:06:13.388000Z",
+ "earthsearch:payload_id": "roda-sentinel-2-c1-l2a/workflow-sentinel-2-c1-l2a-to-stac/3f433637e07c4b2600b19a2e7741e644",
+ "storage:platform": "AWS",
+ "storage:region": "us-west-2",
+ "storage:requester_pays": false,
+ "processing:software": {
+ "sentinel-2-c1-l2a-to-stac": "v2024.02.01"
+ },
+ "updated": "2025-01-10T16:24:42.505Z"
+ },
+ "geometry": {
+ "type": "Polygon",
+ "coordinates": [
+ [
+ [-13.6265322920318, 55.9380706206949],
+ [-14.1026482225707, 54.9557969721861],
+ [-13.2860297382675, 54.9470270219076],
+ [-13.2427407104991, 55.933193717984],
+ [-13.6265322920318, 55.9380706206949]
+ ]
+ ]
+ },
+ "links": [
+ {
+ "rel": "self",
+ "type": "application/geo+json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a/items/S2B_T28UEG_20250110T120355_L2A"
+ },
+ {
+ "rel": "canonical",
+ "href": "s3://e84-earth-search-sentinel-data/sentinel-2-c1-l2a/28/U/EG/2025/1/S2B_T28UEG_20250110T120355_L2A/S2B_T28UEG_20250110T120355_L2A.json",
+ "type": "application/json"
+ },
+ {
+ "rel": "via",
+ "href": "s3://sentinel-s2-l2a/tiles/28/U/EG/2025/1/10/0/metadata.xml",
+ "type": "application/xml",
+ "title": "Granule Metadata in Sinergize RODA Archive"
+ },
+ {
+ "rel": "parent",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a"
+ },
+ {
+ "rel": "collection",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a"
+ },
+ {
+ "rel": "root",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1"
+ },
+ {
+ "rel": "thumbnail",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a/items/S2B_T28UEG_20250110T120355_L2A/thumbnail"
+ }
+ ],
+ "assets": {
+ "red": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EG/2025/1/S2B_T28UEG_20250110T120355_L2A/B04.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red - 10m",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 499980, 0, -10, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220b9d353ff3c5b32a6a6b9d7373a4a23e53977ca4a2a53c331a33f42db253bf1d9",
+ "file:size": 60381704,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "green": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EG/2025/1/S2B_T28UEG_20250110T120355_L2A/B03.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Green - 10m",
+ "eo:bands": [
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 499980, 0, -10, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122069bc8ca3f9e02a5d9cc9ccfaa346b82fbc43d49b5733ef8c0c442cd7c1029e95",
+ "file:size": 60566679,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "blue": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EG/2025/1/S2B_T28UEG_20250110T120355_L2A/B02.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Blue - 10m",
+ "eo:bands": [
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 499980, 0, -10, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220e79da24993985cbba76d10c1921e9e623c42efbe0dfeef64e64541abec1cabc3",
+ "file:size": 61289399,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "visual": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EG/2025/1/S2B_T28UEG_20250110T120355_L2A/TCI.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "True color image",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ },
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ },
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ },
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ },
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 499980, 0, -10, 6200040],
+ "file:checksum": "122088b33cf108ade9b4c26653c73e36729cd15ac949ded66e021c33e23c027e5c0a",
+ "file:size": 2404937,
+ "roles": [
+ "visual"
+ ]
+ },
+ "nir": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EG/2025/1/S2B_T28UEG_20250110T120355_L2A/B08.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 1 - 10m",
+ "eo:bands": [
+ {
+ "name": "B08",
+ "common_name": "nir",
+ "center_wavelength": 0.842,
+ "full_width_half_max": 0.145
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 499980, 0, -10, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220669b38f781482a19c416b172d75902376262b840cd11d4f2c0485c0483264f1a",
+ "file:size": 61404018,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "swir22": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EG/2025/1/S2B_T28UEG_20250110T120355_L2A/B12.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "SWIR 2.2μm - 20m",
+ "eo:bands": [
+ {
+ "name": "B12",
+ "common_name": "swir22",
+ "center_wavelength": 2.19,
+ "full_width_half_max": 0.242
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122007f0ada73a0dfc1ea5c01b7cef273f3a621e567ff8cac3d230b0f9b98dc7d377",
+ "file:size": 17601586,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge2": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EG/2025/1/S2B_T28UEG_20250110T120355_L2A/B06.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 2 - 20m",
+ "eo:bands": [
+ {
+ "name": "B06",
+ "common_name": "rededge",
+ "center_wavelength": 0.74,
+ "full_width_half_max": 0.018
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12200d4476aa9543d440f22ef65eb48baaf15d0246f02061e48ef2bf58b0802ef7ca",
+ "file:size": 19136948,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge3": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EG/2025/1/S2B_T28UEG_20250110T120355_L2A/B07.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 3 - 20m",
+ "eo:bands": [
+ {
+ "name": "B07",
+ "common_name": "rededge",
+ "center_wavelength": 0.783,
+ "full_width_half_max": 0.028
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12202f084ef5aba2af0dd40f435b33d219b893acabbc9e921d86b1a94bf112230669",
+ "file:size": 19261255,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge1": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EG/2025/1/S2B_T28UEG_20250110T120355_L2A/B05.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 1 - 20m",
+ "eo:bands": [
+ {
+ "name": "B05",
+ "common_name": "rededge",
+ "center_wavelength": 0.704,
+ "full_width_half_max": 0.019
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220969140f9d7e9cc58cf32e457f566f4c340273427ffb8dddb2f683d150835ad6c",
+ "file:size": 19065317,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "swir16": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EG/2025/1/S2B_T28UEG_20250110T120355_L2A/B11.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "SWIR 1.6μm - 20m",
+ "eo:bands": [
+ {
+ "name": "B11",
+ "common_name": "swir16",
+ "center_wavelength": 1.61,
+ "full_width_half_max": 0.143
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220914df8e59a4a35b02a50c7a491dc3ca9a1cae1f4ae2d84a7e0255f997be7f78f",
+ "file:size": 18069457,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "wvp": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EG/2025/1/S2B_T28UEG_20250110T120355_L2A/WVP.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Water Vapour (WVP)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "unit": "cm",
+ "scale": 0.001,
+ "offset": 0
+ }
+ ],
+ "file:checksum": "1220737f0c96ba09ebf59029954c78d1a0a402c1053743ebff2525838d83a1511a38",
+ "file:size": 95845,
+ "roles": [
+ "data"
+ ]
+ },
+ "nir08": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EG/2025/1/S2B_T28UEG_20250110T120355_L2A/B8A.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 2 - 20m",
+ "eo:bands": [
+ {
+ "name": "B8A",
+ "common_name": "nir08",
+ "center_wavelength": 0.865,
+ "full_width_half_max": 0.033
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122011dd8b6bdb5f241dd736b2d504b45e34c42749a53368505a15fc6ef2affb2090",
+ "file:size": 19513705,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "scl": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EG/2025/1/S2B_T28UEG_20250110T120355_L2A/SCL.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Scene classification map (SCL)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "1220a5d8882157abc3181bb1a6a46d7c64fc4c9d024b75e6d74fc16ab0e83c904c5c",
+ "file:size": 121275,
+ "roles": [
+ "data"
+ ]
+ },
+ "aot": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EG/2025/1/S2B_T28UEG_20250110T120355_L2A/AOT.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Aerosol optical thickness (AOT)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.001,
+ "offset": 0
+ }
+ ],
+ "file:checksum": "1220c6e200980d8d0b7d7f2e6646d494b790d40a5cbfe0cdc31df7cf1a4576456fa2",
+ "file:size": 173930,
+ "roles": [
+ "data"
+ ]
+ },
+ "coastal": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EG/2025/1/S2B_T28UEG_20250110T120355_L2A/B01.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Coastal - 60m",
+ "eo:bands": [
+ {
+ "name": "B01",
+ "common_name": "coastal",
+ "center_wavelength": 0.443,
+ "full_width_half_max": 0.027
+ }
+ ],
+ "gsd": 60,
+ "proj:shape": [1830, 1830],
+ "proj:transform": [60, 0, 499980, 0, -60, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 60,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122086a4aeabc9e17edd3c8951f13de11c1ed3857bb33fa3cb97e68a9d0100147924",
+ "file:size": 2249862,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "nir09": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EG/2025/1/S2B_T28UEG_20250110T120355_L2A/B09.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 3 - 60m",
+ "eo:bands": [
+ {
+ "name": "B09",
+ "common_name": "nir09",
+ "center_wavelength": 0.945,
+ "full_width_half_max": 0.026
+ }
+ ],
+ "gsd": 60,
+ "proj:shape": [1830, 1830],
+ "proj:transform": [60, 0, 499980, 0, -60, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 60,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220199c59b4af3b328f6aeb6d39fc7ceb285748f594cf5c955cd0a8f4942cba3781",
+ "file:size": 1916080,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "cloud": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EG/2025/1/S2B_T28UEG_20250110T120355_L2A/CLD_20m.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Cloud Probabilities",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "122076cefbddfb96e123b42521059ea8dca0c458e3394688c0f886fb43c33d907402",
+ "file:size": 231288,
+ "roles": [
+ "data",
+ "cloud"
+ ]
+ },
+ "snow": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EG/2025/1/S2B_T28UEG_20250110T120355_L2A/SNW_20m.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Snow Probabilities",
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "12209a39c5a396b02efa2a98f06213d63ee40227e1d292883e792279df61c431e8ae",
+ "file:size": 53931,
+ "roles": [
+ "data",
+ "snow-ice"
+ ]
+ },
+ "preview": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EG/2025/1/S2B_T28UEG_20250110T120355_L2A/L2A_PVI.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "True color preview",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ },
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ },
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "file:checksum": "122075dc35d66b6d9a941a1277a77f576ba3b7bd5c0a325d60641feefa4812bc1124",
+ "file:size": 8431,
+ "roles": [
+ "overview"
+ ]
+ },
+ "granule_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EG/2025/1/S2B_T28UEG_20250110T120355_L2A/metadata.xml",
+ "type": "application/xml",
+ "file:checksum": "1220c4312c4f5ad1f1c757186808c99ec2a552cf2874bc7740e635da2b3538bbfbfb",
+ "file:size": 276748,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "tileinfo_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EG/2025/1/S2B_T28UEG_20250110T120355_L2A/tileInfo.json",
+ "type": "application/json",
+ "file:checksum": "122071fa608aa4046865213a5eac7e5d35d24204e73c393ef5ddaea518918a91e35f",
+ "file:size": 1518,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "product_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EG/2025/1/S2B_T28UEG_20250110T120355_L2A/product_metadata.xml",
+ "type": "application/xml",
+ "file:checksum": "12205fd1335176b336218ed5c4e5f7930ae15d28a2944d592d89b1ca962b84f9b0ee",
+ "file:size": 54707,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "thumbnail": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/U/EG/2025/1/S2B_T28UEG_20250110T120355_L2A/L2A_PVI.jpg",
+ "type": "image/jpeg",
+ "title": "Thumbnail of preview image",
+ "file:checksum": "1220659ac78ac2c72c27ef4b3d2f06306fa35ba128b539ce99a12e601e3b5dba16cd",
+ "file:size": 4633,
+ "roles": [
+ "thumbnail"
+ ]
+ }
+ },
+ "bbox": [-14.102648, 54.947027, -13.242741, 55.938071],
+ "stac_extensions": [
+ "https://stac-extensions.github.io/eo/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/file/v2.1.0/schema.json",
+ "https://stac-extensions.github.io/grid/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/mgrs/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/processing/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/projection/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/raster/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/sentinel-2/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/storage/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/view/v1.0.0/schema.json"
+ ],
+ "collection": "sentinel-2-c1-l2a"
+ },
+ {
+ "type": "Feature",
+ "stac_version": "1.0.0",
+ "id": "S2B_T28VEH_20250110T120355_L2A",
+ "properties": {
+ "created": "2025-01-10T16:23:30.675Z",
+ "platform": "sentinel-2b",
+ "constellation": "sentinel-2",
+ "instruments": [
+ "msi"
+ ],
+ "eo:cloud_cover": 99.301517,
+ "proj:epsg": 32628,
+ "proj:centroid": {
+ "lat": 56.15644,
+ "lon": -13.37486
+ },
+ "mgrs:utm_zone": 28,
+ "mgrs:latitude_band": "V",
+ "mgrs:grid_square": "EH",
+ "grid:code": "MGRS-28VEH",
+ "view:azimuth": 111.841329557221,
+ "view:incidence_angle": 11.1233547024194,
+ "view:sun_azimuth": 166.296128019016,
+ "view:sun_elevation": 10.8177386822595,
+ "s2:tile_id": "S2B_OPER_MSI_L2A_TL_2BPS_20250110T154043_A040990_T28VEH_N05.11",
+ "s2:degraded_msi_data_percentage": 0,
+ "s2:nodata_pixel_percentage": 88.483912,
+ "s2:saturated_defective_pixel_percentage": 0,
+ "s2:cloud_shadow_percentage": 0,
+ "s2:vegetation_percentage": 0,
+ "s2:not_vegetated_percentage": 0,
+ "s2:water_percentage": 0.698481,
+ "s2:unclassified_percentage": 0,
+ "s2:medium_proba_clouds_percentage": 42.502564,
+ "s2:high_proba_clouds_percentage": 44.748831,
+ "s2:thin_cirrus_percentage": 12.050124,
+ "s2:snow_ice_percentage": 0,
+ "s2:product_type": "S2MSI2A",
+ "s2:processing_baseline": "05.11",
+ "s2:product_uri": "S2B_MSIL2A_20250110T120359_N0511_R066_T28VEH_20250110T154043.SAFE",
+ "s2:generation_time": "2025-01-10T15:40:43.000000Z",
+ "s2:datatake_id": "GS2B_20250110T120359_040990_N05.11",
+ "s2:datatake_type": "INS-NOBS",
+ "s2:datastrip_id": "S2B_OPER_MSI_L2A_DS_2BPS_20250110T154043_S20250110T120355_N05.11",
+ "s2:reflectance_conversion_factor": 1.03425891111326,
+ "datetime": "2025-01-10T12:06:00.902000Z",
+ "earthsearch:payload_id": "roda-sentinel-2-c1-l2a/workflow-sentinel-2-c1-l2a-to-stac/11834873512e6300f7681a47b13878fd",
+ "storage:platform": "AWS",
+ "storage:region": "us-west-2",
+ "storage:requester_pays": false,
+ "processing:software": {
+ "sentinel-2-c1-l2a-to-stac": "v2024.02.01"
+ },
+ "updated": "2025-01-10T16:23:30.675Z"
+ },
+ "geometry": {
+ "type": "Polygon",
+ "coordinates": [
+ [
+ [-13.2483259551958, 55.8448624735234],
+ [-13.206223668266, 56.7743145359432],
+ [-13.6700421061974, 55.8501437572837],
+ [-13.2483259551958, 55.8448624735234]
+ ]
+ ]
+ },
+ "links": [
+ {
+ "rel": "self",
+ "type": "application/geo+json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a/items/S2B_T28VEH_20250110T120355_L2A"
+ },
+ {
+ "rel": "canonical",
+ "href": "s3://e84-earth-search-sentinel-data/sentinel-2-c1-l2a/28/V/EH/2025/1/S2B_T28VEH_20250110T120355_L2A/S2B_T28VEH_20250110T120355_L2A.json",
+ "type": "application/json"
+ },
+ {
+ "rel": "via",
+ "href": "s3://sentinel-s2-l2a/tiles/28/V/EH/2025/1/10/0/metadata.xml",
+ "type": "application/xml",
+ "title": "Granule Metadata in Sinergize RODA Archive"
+ },
+ {
+ "rel": "parent",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a"
+ },
+ {
+ "rel": "collection",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a"
+ },
+ {
+ "rel": "root",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1"
+ },
+ {
+ "rel": "thumbnail",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a/items/S2B_T28VEH_20250110T120355_L2A/thumbnail"
+ }
+ ],
+ "assets": {
+ "red": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/V/EH/2025/1/S2B_T28VEH_20250110T120355_L2A/B04.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red - 10m",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 499980, 0, -10, 6300000],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12209a078d936edd1a7e1bcdf766e8fcf55431b60f65328965b08f0af871af8464eb",
+ "file:size": 23470320,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "green": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/V/EH/2025/1/S2B_T28VEH_20250110T120355_L2A/B03.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Green - 10m",
+ "eo:bands": [
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 499980, 0, -10, 6300000],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220d798375df0fc4cd2f16f037408fecb24b72f372f1658c733d3a7d522dd3eac07",
+ "file:size": 23549627,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "blue": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/V/EH/2025/1/S2B_T28VEH_20250110T120355_L2A/B02.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Blue - 10m",
+ "eo:bands": [
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 499980, 0, -10, 6300000],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12207470ebca173e6c56435c6e9a2da0013603294d304b7e2a12803a97029c1269b0",
+ "file:size": 23446317,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "visual": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/V/EH/2025/1/S2B_T28VEH_20250110T120355_L2A/TCI.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "True color image",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ },
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ },
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ },
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ },
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 499980, 0, -10, 6300000],
+ "file:checksum": "12207b390449b17d0cebbbd6b2de54203a94e925ac96ce54aa5b34583674c0113e89",
+ "file:size": 21770593,
+ "roles": [
+ "visual"
+ ]
+ },
+ "nir": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/V/EH/2025/1/S2B_T28VEH_20250110T120355_L2A/B08.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 1 - 10m",
+ "eo:bands": [
+ {
+ "name": "B08",
+ "common_name": "nir",
+ "center_wavelength": 0.842,
+ "full_width_half_max": 0.145
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 499980, 0, -10, 6300000],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12205b740b77c08f48ce13ee908590a25d361e72d47ade71fa055c9d268534e04223",
+ "file:size": 23641591,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "swir22": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/V/EH/2025/1/S2B_T28VEH_20250110T120355_L2A/B12.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "SWIR 2.2μm - 20m",
+ "eo:bands": [
+ {
+ "name": "B12",
+ "common_name": "swir22",
+ "center_wavelength": 2.19,
+ "full_width_half_max": 0.242
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6300000],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122085f33db31d5fbed707c061c7df0cc929dcc7b861df99561166b57bd6600e84ab",
+ "file:size": 6241149,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge2": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/V/EH/2025/1/S2B_T28VEH_20250110T120355_L2A/B06.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 2 - 20m",
+ "eo:bands": [
+ {
+ "name": "B06",
+ "common_name": "rededge",
+ "center_wavelength": 0.74,
+ "full_width_half_max": 0.018
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6300000],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220eb01170f6a76786fd032101911244c117c5ca2a43fb2a8162327246afaccfe09",
+ "file:size": 6636232,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge3": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/V/EH/2025/1/S2B_T28VEH_20250110T120355_L2A/B07.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 3 - 20m",
+ "eo:bands": [
+ {
+ "name": "B07",
+ "common_name": "rededge",
+ "center_wavelength": 0.783,
+ "full_width_half_max": 0.028
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6300000],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220577312552117447a506afcc312f45302506b7b184048c7da1c7e4c0dd514db9c",
+ "file:size": 6676498,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge1": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/V/EH/2025/1/S2B_T28VEH_20250110T120355_L2A/B05.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 1 - 20m",
+ "eo:bands": [
+ {
+ "name": "B05",
+ "common_name": "rededge",
+ "center_wavelength": 0.704,
+ "full_width_half_max": 0.019
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6300000],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220c33490a3ec481d86db0f85ecea1d741794cc12424dba366d2f606f8e5b23588a",
+ "file:size": 6631667,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "swir16": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/V/EH/2025/1/S2B_T28VEH_20250110T120355_L2A/B11.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "SWIR 1.6μm - 20m",
+ "eo:bands": [
+ {
+ "name": "B11",
+ "common_name": "swir16",
+ "center_wavelength": 1.61,
+ "full_width_half_max": 0.143
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6300000],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122072b1ed47a63d6dc0bf1ac50432268e594262a2d2b451397c0ebbdabbadd88cff",
+ "file:size": 6317373,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "wvp": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/V/EH/2025/1/S2B_T28VEH_20250110T120355_L2A/WVP.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Water Vapour (WVP)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6300000],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "unit": "cm",
+ "scale": 0.001,
+ "offset": 0
+ }
+ ],
+ "file:checksum": "12209a8982ff57262f14ebd82f32666904914fb550aabe6a6ddf554350fac7c2136d",
+ "file:size": 95845,
+ "roles": [
+ "data"
+ ]
+ },
+ "nir08": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/V/EH/2025/1/S2B_T28VEH_20250110T120355_L2A/B8A.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 2 - 20m",
+ "eo:bands": [
+ {
+ "name": "B8A",
+ "common_name": "nir08",
+ "center_wavelength": 0.865,
+ "full_width_half_max": 0.033
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6300000],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220efe682a3aa1a20332f77390e4f53fbcd1b16f32f694fb3b352f8410919e5955b",
+ "file:size": 6716668,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "scl": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/V/EH/2025/1/S2B_T28VEH_20250110T120355_L2A/SCL.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Scene classification map (SCL)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6300000],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "122017b19cbd43088542594dd2b1d2f70313da1a2ecfb5a4e29f20bd1bc3da1f7815",
+ "file:size": 222714,
+ "roles": [
+ "data"
+ ]
+ },
+ "aot": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/V/EH/2025/1/S2B_T28VEH_20250110T120355_L2A/AOT.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Aerosol optical thickness (AOT)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6300000],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.001,
+ "offset": 0
+ }
+ ],
+ "file:checksum": "1220246803d9849a422405163a49896d7106184d4e9ec6691fc49ac83816517f0509",
+ "file:size": 138722,
+ "roles": [
+ "data"
+ ]
+ },
+ "coastal": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/V/EH/2025/1/S2B_T28VEH_20250110T120355_L2A/B01.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Coastal - 60m",
+ "eo:bands": [
+ {
+ "name": "B01",
+ "common_name": "coastal",
+ "center_wavelength": 0.443,
+ "full_width_half_max": 0.027
+ }
+ ],
+ "gsd": 60,
+ "proj:shape": [1830, 1830],
+ "proj:transform": [60, 0, 499980, 0, -60, 6300000],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 60,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122090dcd8bdb5db4777e23fbd9a863b3475caccea2632e6f082c1a40e7c0d7c615f",
+ "file:size": 801606,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "nir09": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/V/EH/2025/1/S2B_T28VEH_20250110T120355_L2A/B09.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 3 - 60m",
+ "eo:bands": [
+ {
+ "name": "B09",
+ "common_name": "nir09",
+ "center_wavelength": 0.945,
+ "full_width_half_max": 0.026
+ }
+ ],
+ "gsd": 60,
+ "proj:shape": [1830, 1830],
+ "proj:transform": [60, 0, 499980, 0, -60, 6300000],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 60,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220cb7d707d19dfdd20a6619f08f7f55e7becb8c164c73fc1ba0924ac0f50518bf6",
+ "file:size": 716309,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "cloud": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/V/EH/2025/1/S2B_T28VEH_20250110T120355_L2A/CLD_20m.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Cloud Probabilities",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6300000],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "12201cea1fb6f01083885b59b8c36f635d80e32062c28e214bc65e25e272e2eb73d8",
+ "file:size": 603026,
+ "roles": [
+ "data",
+ "cloud"
+ ]
+ },
+ "snow": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/V/EH/2025/1/S2B_T28VEH_20250110T120355_L2A/SNW_20m.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Snow Probabilities",
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6300000],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "12206d5fa18554952bea897b5e8ffe90c64ae6ba98d31744de65c4e269ca0e913bbd",
+ "file:size": 53931,
+ "roles": [
+ "data",
+ "snow-ice"
+ ]
+ },
+ "preview": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/V/EH/2025/1/S2B_T28VEH_20250110T120355_L2A/L2A_PVI.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "True color preview",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ },
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ },
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "file:checksum": "12205adc705f4f9892cbca80a870037a9ab732ec8c93ab91639a7c9477e000d1764a",
+ "file:size": 31910,
+ "roles": [
+ "overview"
+ ]
+ },
+ "granule_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/V/EH/2025/1/S2B_T28VEH_20250110T120355_L2A/metadata.xml",
+ "type": "application/xml",
+ "file:checksum": "1220e635711170fce5974513ea062f6739b962681828dd8d1db23b026072abb6064e",
+ "file:size": 185550,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "tileinfo_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/V/EH/2025/1/S2B_T28VEH_20250110T120355_L2A/tileInfo.json",
+ "type": "application/json",
+ "file:checksum": "1220d7a2e8b29cf19d297e4c685b7cd4a8a82399f0e8d6b063c1d40b9978a734f008",
+ "file:size": 1510,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "product_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/V/EH/2025/1/S2B_T28VEH_20250110T120355_L2A/product_metadata.xml",
+ "type": "application/xml",
+ "file:checksum": "12200362284af62323155e23089028e143ef902c4f19f040a4341fb1ab723056e132",
+ "file:size": 54677,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "thumbnail": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/28/V/EH/2025/1/S2B_T28VEH_20250110T120355_L2A/L2A_PVI.jpg",
+ "type": "image/jpeg",
+ "title": "Thumbnail of preview image",
+ "file:checksum": "1220d058912bbb7875f8b023701f44852a6a18f09d9062bb2a0476acb44e06e62ab1",
+ "file:size": 7597,
+ "roles": [
+ "thumbnail"
+ ]
+ }
+ },
+ "bbox": [-13.670042, 55.844862, -13.206224, 56.774315],
+ "stac_extensions": [
+ "https://stac-extensions.github.io/eo/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/file/v2.1.0/schema.json",
+ "https://stac-extensions.github.io/grid/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/mgrs/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/processing/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/projection/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/raster/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/sentinel-2/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/storage/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/view/v1.0.0/schema.json"
+ ],
+ "collection": "sentinel-2-c1-l2a"
+ },
+ {
+ "type": "Feature",
+ "stac_version": "1.0.0",
+ "id": "S2B_T29UNB_20250110T120355_L2A",
+ "properties": {
+ "created": "2025-01-10T16:22:47.191Z",
+ "platform": "sentinel-2b",
+ "constellation": "sentinel-2",
+ "instruments": [
+ "msi"
+ ],
+ "eo:cloud_cover": 58.027476,
+ "proj:epsg": 32629,
+ "proj:centroid": {
+ "lat": 55.76895,
+ "lon": -8.89008
+ },
+ "mgrs:utm_zone": 29,
+ "mgrs:latitude_band": "U",
+ "mgrs:grid_square": "NB",
+ "grid:code": "MGRS-29UNB",
+ "view:azimuth": 296.213391314926,
+ "view:incidence_angle": 11.2885323247034,
+ "view:sun_azimuth": 171.888242896347,
+ "view:sun_elevation": 12.3330462371241,
+ "s2:tile_id": "S2B_OPER_MSI_L2A_TL_2BPS_20250110T154043_A040990_T29UNB_N05.11",
+ "s2:degraded_msi_data_percentage": 0,
+ "s2:nodata_pixel_percentage": 94.879043,
+ "s2:saturated_defective_pixel_percentage": 0,
+ "s2:cloud_shadow_percentage": 0,
+ "s2:vegetation_percentage": 0,
+ "s2:not_vegetated_percentage": 0.000454,
+ "s2:water_percentage": 41.972074,
+ "s2:unclassified_percentage": 0,
+ "s2:medium_proba_clouds_percentage": 19.25049,
+ "s2:high_proba_clouds_percentage": 1.270002,
+ "s2:thin_cirrus_percentage": 37.50698,
+ "s2:snow_ice_percentage": 0,
+ "s2:product_type": "S2MSI2A",
+ "s2:processing_baseline": "05.11",
+ "s2:product_uri": "S2B_MSIL2A_20250110T120359_N0511_R066_T29UNB_20250110T154043.SAFE",
+ "s2:generation_time": "2025-01-10T15:40:43.000000Z",
+ "s2:datatake_id": "GS2B_20250110T120359_040990_N05.11",
+ "s2:datatake_type": "INS-NOBS",
+ "s2:datastrip_id": "S2B_OPER_MSI_L2A_DS_2BPS_20250110T154043_S20250110T120355_N05.11",
+ "s2:reflectance_conversion_factor": 1.03425891111326,
+ "datetime": "2025-01-10T12:05:52.609000Z",
+ "earthsearch:payload_id": "roda-sentinel-2-c1-l2a/workflow-sentinel-2-c1-l2a-to-stac/b708c4badc2109a13b89da40305c2ef8",
+ "storage:platform": "AWS",
+ "storage:region": "us-west-2",
+ "storage:requester_pays": false,
+ "processing:software": {
+ "sentinel-2-c1-l2a-to-stac": "v2024.02.01"
+ },
+ "updated": "2025-01-10T16:22:47.191Z"
+ },
+ "geometry": {
+ "type": "Polygon",
+ "coordinates": [
+ [
+ [-9.00030421811058, 55.9457254173272],
+ [-9.00030013314029, 55.4158504628607],
+ [-8.66964406830726, 55.9452826258813],
+ [-9.00030421811058, 55.9457254173272]
+ ]
+ ]
+ },
+ "links": [
+ {
+ "rel": "self",
+ "type": "application/geo+json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a/items/S2B_T29UNB_20250110T120355_L2A"
+ },
+ {
+ "rel": "canonical",
+ "href": "s3://e84-earth-search-sentinel-data/sentinel-2-c1-l2a/29/U/NB/2025/1/S2B_T29UNB_20250110T120355_L2A/S2B_T29UNB_20250110T120355_L2A.json",
+ "type": "application/json"
+ },
+ {
+ "rel": "via",
+ "href": "s3://sentinel-s2-l2a/tiles/29/U/NB/2025/1/10/0/metadata.xml",
+ "type": "application/xml",
+ "title": "Granule Metadata in Sinergize RODA Archive"
+ },
+ {
+ "rel": "parent",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a"
+ },
+ {
+ "rel": "collection",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a"
+ },
+ {
+ "rel": "root",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1"
+ },
+ {
+ "rel": "thumbnail",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a/items/S2B_T29UNB_20250110T120355_L2A/thumbnail"
+ }
+ ],
+ "assets": {
+ "red": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/NB/2025/1/S2B_T29UNB_20250110T120355_L2A/B04.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red - 10m",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 499980, 0, -10, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220e789936a2387f8dc7e6aabb0968bae1a6e9f84523186412e9929641effdaa361",
+ "file:size": 10027685,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "green": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/NB/2025/1/S2B_T29UNB_20250110T120355_L2A/B03.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Green - 10m",
+ "eo:bands": [
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 499980, 0, -10, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122063893492f1d7288fc4035b468681f4553f2887f3c74655a4c84aaf6516a37f00",
+ "file:size": 10190327,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "blue": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/NB/2025/1/S2B_T29UNB_20250110T120355_L2A/B02.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Blue - 10m",
+ "eo:bands": [
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 499980, 0, -10, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122058f51b3925cec6f1e94cb2e1ba55cac8ae0b1c98fe7df627e0f0843da4b1ecd4",
+ "file:size": 10515581,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "visual": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/NB/2025/1/S2B_T29UNB_20250110T120355_L2A/TCI.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "True color image",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ },
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ },
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ },
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ },
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 499980, 0, -10, 6200040],
+ "file:checksum": "1220bad96fa28f2414eccd6c5b23831d905d7df22fd3bd24267446eaf3630d3972ea",
+ "file:size": 14154510,
+ "roles": [
+ "visual"
+ ]
+ },
+ "nir": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/NB/2025/1/S2B_T29UNB_20250110T120355_L2A/B08.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 1 - 10m",
+ "eo:bands": [
+ {
+ "name": "B08",
+ "common_name": "nir",
+ "center_wavelength": 0.842,
+ "full_width_half_max": 0.145
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 499980, 0, -10, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220e5b6a275b9c649ab2288243d5824eeebda0808e51f6a8a40a69903c690b2fbde",
+ "file:size": 10111553,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "swir22": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/NB/2025/1/S2B_T29UNB_20250110T120355_L2A/B12.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "SWIR 2.2μm - 20m",
+ "eo:bands": [
+ {
+ "name": "B12",
+ "common_name": "swir22",
+ "center_wavelength": 2.19,
+ "full_width_half_max": 0.242
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122026ec8ebcfa8c2d83b76bef0e7b3496ed3c1d39f87fab9e11a8ed939b4c01636c",
+ "file:size": 2437631,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge2": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/NB/2025/1/S2B_T29UNB_20250110T120355_L2A/B06.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 2 - 20m",
+ "eo:bands": [
+ {
+ "name": "B06",
+ "common_name": "rededge",
+ "center_wavelength": 0.74,
+ "full_width_half_max": 0.018
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220dd58b9912f585b50907e6526958402a7283bdf9420db9b9efb8407d6b0d36da1",
+ "file:size": 2593728,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge3": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/NB/2025/1/S2B_T29UNB_20250110T120355_L2A/B07.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 3 - 20m",
+ "eo:bands": [
+ {
+ "name": "B07",
+ "common_name": "rededge",
+ "center_wavelength": 0.783,
+ "full_width_half_max": 0.028
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122009281ca55d8aa3bccfd8fc14d334e3fda9ed2d0083bf9e6e1c9a9799c23ab1eb",
+ "file:size": 2605950,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge1": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/NB/2025/1/S2B_T29UNB_20250110T120355_L2A/B05.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 1 - 20m",
+ "eo:bands": [
+ {
+ "name": "B05",
+ "common_name": "rededge",
+ "center_wavelength": 0.704,
+ "full_width_half_max": 0.019
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220e4b4d1744e36c8113ac5a208d79b122c4d41d6484fe43a060a306c633949d678",
+ "file:size": 2596301,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "swir16": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/NB/2025/1/S2B_T29UNB_20250110T120355_L2A/B11.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "SWIR 1.6μm - 20m",
+ "eo:bands": [
+ {
+ "name": "B11",
+ "common_name": "swir16",
+ "center_wavelength": 1.61,
+ "full_width_half_max": 0.143
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12203738cfcacf2d49ff3ee15807d2d85fc84b3074196aa3bd7cd5ff4ec5906143b1",
+ "file:size": 2403501,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "wvp": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/NB/2025/1/S2B_T29UNB_20250110T120355_L2A/WVP.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Water Vapour (WVP)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "unit": "cm",
+ "scale": 0.001,
+ "offset": 0
+ }
+ ],
+ "file:checksum": "1220b4f370a63b84c6d603d295d249fcb26089f11d09a85567594f2f12507533daed",
+ "file:size": 109472,
+ "roles": [
+ "data"
+ ]
+ },
+ "nir08": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/NB/2025/1/S2B_T29UNB_20250110T120355_L2A/B8A.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 2 - 20m",
+ "eo:bands": [
+ {
+ "name": "B8A",
+ "common_name": "nir08",
+ "center_wavelength": 0.865,
+ "full_width_half_max": 0.033
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12208c9a6417984eccfdd754b8dc7d6e5041c475412fa297ce8650d58394cace8190",
+ "file:size": 2590780,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "scl": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/NB/2025/1/S2B_T29UNB_20250110T120355_L2A/SCL.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Scene classification map (SCL)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "1220c31980a5d07ba12b22daec798def163f4c63a5e2ff830a44c4ed49091da75d35",
+ "file:size": 112713,
+ "roles": [
+ "data"
+ ]
+ },
+ "aot": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/NB/2025/1/S2B_T29UNB_20250110T120355_L2A/AOT.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Aerosol optical thickness (AOT)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.001,
+ "offset": 0
+ }
+ ],
+ "file:checksum": "1220d0ded0020d1500a07a3ceb31efefb0b7e06474d74223f4a30d714bfdfa26faf7",
+ "file:size": 123669,
+ "roles": [
+ "data"
+ ]
+ },
+ "coastal": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/NB/2025/1/S2B_T29UNB_20250110T120355_L2A/B01.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Coastal - 60m",
+ "eo:bands": [
+ {
+ "name": "B01",
+ "common_name": "coastal",
+ "center_wavelength": 0.443,
+ "full_width_half_max": 0.027
+ }
+ ],
+ "gsd": 60,
+ "proj:shape": [1830, 1830],
+ "proj:transform": [60, 0, 499980, 0, -60, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 60,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122002f9d96b4938ef7381d908139deffbc922903f7e59cba1c43f746f55dca19b66",
+ "file:size": 306561,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "nir09": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/NB/2025/1/S2B_T29UNB_20250110T120355_L2A/B09.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 3 - 60m",
+ "eo:bands": [
+ {
+ "name": "B09",
+ "common_name": "nir09",
+ "center_wavelength": 0.945,
+ "full_width_half_max": 0.026
+ }
+ ],
+ "gsd": 60,
+ "proj:shape": [1830, 1830],
+ "proj:transform": [60, 0, 499980, 0, -60, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 60,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220df4b77e33fe71a118c587ddde0e8b1db4a2507dc246c3c7678330e2a40b9acd9",
+ "file:size": 322261,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "cloud": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/NB/2025/1/S2B_T29UNB_20250110T120355_L2A/CLD_20m.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Cloud Probabilities",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "1220a3d40b111608fa4ca822a1eaf45d94e7c122e942b8378374484cdd907401f755",
+ "file:size": 116319,
+ "roles": [
+ "data",
+ "cloud"
+ ]
+ },
+ "snow": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/NB/2025/1/S2B_T29UNB_20250110T120355_L2A/SNW_20m.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Snow Probabilities",
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 499980, 0, -20, 6200040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "1220ad48781d7c7fd26e277f85a5cb6e9c786d4f4231a494a77954790b7c993e4a2a",
+ "file:size": 53931,
+ "roles": [
+ "data",
+ "snow-ice"
+ ]
+ },
+ "preview": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/NB/2025/1/S2B_T29UNB_20250110T120355_L2A/L2A_PVI.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "True color preview",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ },
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ },
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "file:checksum": "1220fe64e947665671aa18aaaa003c9aee5c29fe8cc23ecb611255003623f6d33aac",
+ "file:size": 19984,
+ "roles": [
+ "overview"
+ ]
+ },
+ "granule_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/NB/2025/1/S2B_T29UNB_20250110T120355_L2A/metadata.xml",
+ "type": "application/xml",
+ "file:checksum": "1220c3421f3b8734d926d99797954bdbb5043e5e2a8f985f84c3a1f8a66cbc0db831",
+ "file:size": 103367,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "tileinfo_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/NB/2025/1/S2B_T29UNB_20250110T120355_L2A/tileInfo.json",
+ "type": "application/json",
+ "file:checksum": "12208512e1fe6787eaaad288dae43ad3c0c5cc17ce96eca14a5a7038c7b592215753",
+ "file:size": 1483,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "product_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/NB/2025/1/S2B_T29UNB_20250110T120355_L2A/product_metadata.xml",
+ "type": "application/xml",
+ "file:checksum": "1220e04a251e328f357ce7c5c94372db4eb36ade7254870d997547795ed3087b7c3d",
+ "file:size": 54565,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "thumbnail": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/U/NB/2025/1/S2B_T29UNB_20250110T120355_L2A/L2A_PVI.jpg",
+ "type": "image/jpeg",
+ "title": "Thumbnail of preview image",
+ "file:checksum": "1220df8bda6af78d169612f7fb7dff03dea50a780a19c124b033df1ed036db158db4",
+ "file:size": 4219,
+ "roles": [
+ "thumbnail"
+ ]
+ }
+ },
+ "bbox": [-9.000304, 55.41585, -8.669644, 55.945725],
+ "stac_extensions": [
+ "https://stac-extensions.github.io/eo/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/file/v2.1.0/schema.json",
+ "https://stac-extensions.github.io/grid/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/mgrs/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/processing/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/projection/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/raster/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/sentinel-2/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/storage/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/view/v1.0.0/schema.json"
+ ],
+ "collection": "sentinel-2-c1-l2a"
+ },
+ {
+ "type": "Feature",
+ "stac_version": "1.0.0",
+ "id": "S2B_T29VPE_20250110T120355_L2A",
+ "properties": {
+ "created": "2025-01-10T16:23:15.779Z",
+ "platform": "sentinel-2b",
+ "constellation": "sentinel-2",
+ "instruments": [
+ "msi"
+ ],
+ "eo:cloud_cover": 11.435952,
+ "proj:epsg": 32629,
+ "proj:centroid": {
+ "lat": 58.41245,
+ "lon": -7.14589
+ },
+ "mgrs:utm_zone": 29,
+ "mgrs:latitude_band": "V",
+ "mgrs:grid_square": "PE",
+ "grid:code": "MGRS-29VPE",
+ "view:azimuth": 297.497570529651,
+ "view:incidence_angle": 11.1788806985056,
+ "view:sun_azimuth": 173.432089910917,
+ "view:sun_elevation": 9.7935341997534,
+ "s2:tile_id": "S2B_OPER_MSI_L2A_TL_2BPS_20250110T154043_A040990_T29VPE_N05.11",
+ "s2:degraded_msi_data_percentage": 0,
+ "s2:nodata_pixel_percentage": 92.532736,
+ "s2:saturated_defective_pixel_percentage": 0,
+ "s2:cloud_shadow_percentage": 0,
+ "s2:vegetation_percentage": 0,
+ "s2:not_vegetated_percentage": 0.000044,
+ "s2:water_percentage": 88.564003,
+ "s2:unclassified_percentage": 0,
+ "s2:medium_proba_clouds_percentage": 1.015978,
+ "s2:high_proba_clouds_percentage": 0.477642,
+ "s2:thin_cirrus_percentage": 9.942332,
+ "s2:snow_ice_percentage": 0,
+ "s2:product_type": "S2MSI2A",
+ "s2:processing_baseline": "05.11",
+ "s2:product_uri": "S2B_MSIL2A_20250110T120359_N0511_R066_T29VPE_20250110T154043.SAFE",
+ "s2:generation_time": "2025-01-10T15:40:43.000000Z",
+ "s2:datatake_id": "GS2B_20250110T120359_040990_N05.11",
+ "s2:datatake_type": "INS-NOBS",
+ "s2:datastrip_id": "S2B_OPER_MSI_L2A_DS_2BPS_20250110T154043_S20250110T120355_N05.11",
+ "s2:reflectance_conversion_factor": 1.03425891111326,
+ "datetime": "2025-01-10T12:05:06.060000Z",
+ "earthsearch:payload_id": "roda-sentinel-2-c1-l2a/workflow-sentinel-2-c1-l2a-to-stac/e2bcd39fde0f241cb4b72e32d0fab57c",
+ "storage:platform": "AWS",
+ "storage:region": "us-west-2",
+ "storage:requester_pays": false,
+ "processing:software": {
+ "sentinel-2-c1-l2a-to-stac": "v2024.02.01"
+ },
+ "updated": "2025-01-10T16:23:15.779Z"
+ },
+ "geometry": {
+ "type": "Polygon",
+ "coordinates": [
+ [
+ [-7.27777230636626, 58.629120925364],
+ [-7.30877351638107, 57.985540520195],
+ [-6.85111741280027, 58.6227000712756],
+ [-7.27777230636626, 58.629120925364]
+ ]
+ ]
+ },
+ "links": [
+ {
+ "rel": "self",
+ "type": "application/geo+json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a/items/S2B_T29VPE_20250110T120355_L2A"
+ },
+ {
+ "rel": "canonical",
+ "href": "s3://e84-earth-search-sentinel-data/sentinel-2-c1-l2a/29/V/PE/2025/1/S2B_T29VPE_20250110T120355_L2A/S2B_T29VPE_20250110T120355_L2A.json",
+ "type": "application/json"
+ },
+ {
+ "rel": "via",
+ "href": "s3://sentinel-s2-l2a/tiles/29/V/PE/2025/1/10/0/metadata.xml",
+ "type": "application/xml",
+ "title": "Granule Metadata in Sinergize RODA Archive"
+ },
+ {
+ "rel": "parent",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a"
+ },
+ {
+ "rel": "collection",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a"
+ },
+ {
+ "rel": "root",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1"
+ },
+ {
+ "rel": "thumbnail",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a/items/S2B_T29VPE_20250110T120355_L2A/thumbnail"
+ }
+ ],
+ "assets": {
+ "red": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/V/PE/2025/1/S2B_T29VPE_20250110T120355_L2A/B04.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red - 10m",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 600000, 0, -10, 6500040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220afcf1c0cb6d8ba2a05279e4a67c5c3e15915f1840af95899154c282e389af636",
+ "file:size": 13978420,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "green": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/V/PE/2025/1/S2B_T29VPE_20250110T120355_L2A/B03.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Green - 10m",
+ "eo:bands": [
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 600000, 0, -10, 6500040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122089e64eb2d9546aff1732cd91675186f1d02cc709b24f1541e2be3c1890aca965",
+ "file:size": 14234068,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "blue": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/V/PE/2025/1/S2B_T29VPE_20250110T120355_L2A/B02.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Blue - 10m",
+ "eo:bands": [
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 600000, 0, -10, 6500040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220cf37736e3df661528f6bbe9ca0a835cf8d2781c2549e65057fcfaa47b6c07c8b",
+ "file:size": 13792914,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "visual": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/V/PE/2025/1/S2B_T29VPE_20250110T120355_L2A/TCI.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "True color image",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ },
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ },
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ },
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ },
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 10
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 600000, 0, -10, 6500040],
+ "file:checksum": "1220e1a8602049d4fb2d7e3682a0510a08b0b15e43f0ce758de8d594011941a7f255",
+ "file:size": 20039038,
+ "roles": [
+ "visual"
+ ]
+ },
+ "nir": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/V/PE/2025/1/S2B_T29VPE_20250110T120355_L2A/B08.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 1 - 10m",
+ "eo:bands": [
+ {
+ "name": "B08",
+ "common_name": "nir",
+ "center_wavelength": 0.842,
+ "full_width_half_max": 0.145
+ }
+ ],
+ "gsd": 10,
+ "proj:shape": [10980, 10980],
+ "proj:transform": [10, 0, 600000, 0, -10, 6500040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 10,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220e00f1a4bc6109c2c0d887962fd3f7cd4b0a83739f1ebba147e40de73ba6e541c",
+ "file:size": 14086891,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "swir22": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/V/PE/2025/1/S2B_T29VPE_20250110T120355_L2A/B12.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "SWIR 2.2μm - 20m",
+ "eo:bands": [
+ {
+ "name": "B12",
+ "common_name": "swir22",
+ "center_wavelength": 2.19,
+ "full_width_half_max": 0.242
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 600000, 0, -20, 6500040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220ad59be9557ef198922b6fb613b6ed28f469a57a2719868cd6b498cb7a9119a36",
+ "file:size": 3440114,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge2": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/V/PE/2025/1/S2B_T29VPE_20250110T120355_L2A/B06.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 2 - 20m",
+ "eo:bands": [
+ {
+ "name": "B06",
+ "common_name": "rededge",
+ "center_wavelength": 0.74,
+ "full_width_half_max": 0.018
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 600000, 0, -20, 6500040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "122015fe200e3e96f760b07dd5a085148560c4822f9a9907784f397e1a8f9c7840f9",
+ "file:size": 3555878,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge3": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/V/PE/2025/1/S2B_T29VPE_20250110T120355_L2A/B07.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 3 - 20m",
+ "eo:bands": [
+ {
+ "name": "B07",
+ "common_name": "rededge",
+ "center_wavelength": 0.783,
+ "full_width_half_max": 0.028
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 600000, 0, -20, 6500040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220e5beec733f2675a692cdb0a1c4029712965030cfac0afa9ef2a9cb23184210f8",
+ "file:size": 3620180,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "rededge1": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/V/PE/2025/1/S2B_T29VPE_20250110T120355_L2A/B05.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Red Edge 1 - 20m",
+ "eo:bands": [
+ {
+ "name": "B05",
+ "common_name": "rededge",
+ "center_wavelength": 0.704,
+ "full_width_half_max": 0.019
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 600000, 0, -20, 6500040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12201da5a52ba45a38f0e61f58490bfa7a9e650d5afa08b7f8350850715edb74a35b",
+ "file:size": 3561439,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "swir16": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/V/PE/2025/1/S2B_T29VPE_20250110T120355_L2A/B11.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "SWIR 1.6μm - 20m",
+ "eo:bands": [
+ {
+ "name": "B11",
+ "common_name": "swir16",
+ "center_wavelength": 1.61,
+ "full_width_half_max": 0.143
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 600000, 0, -20, 6500040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12203f8f3d0c9672452094c2de1bc2d045b16216074ef10c8c50f2de4cf85fd62f20",
+ "file:size": 3339112,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "wvp": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/V/PE/2025/1/S2B_T29VPE_20250110T120355_L2A/WVP.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Water Vapour (WVP)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 600000, 0, -20, 6500040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "unit": "cm",
+ "scale": 0.001,
+ "offset": 0
+ }
+ ],
+ "file:checksum": "1220cfc4488c1c1f04f077f62aab5a1f1aed5c8246e32e31c15bffbcdf8a575aaefe",
+ "file:size": 113489,
+ "roles": [
+ "data"
+ ]
+ },
+ "nir08": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/V/PE/2025/1/S2B_T29VPE_20250110T120355_L2A/B8A.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 2 - 20m",
+ "eo:bands": [
+ {
+ "name": "B8A",
+ "common_name": "nir08",
+ "center_wavelength": 0.865,
+ "full_width_half_max": 0.033
+ }
+ ],
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 600000, 0, -20, 6500040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12207fa89e2f2e5509cb3bb46a5798bf3d10bfc77e0db99b4099705e7ba414a673d4",
+ "file:size": 3571061,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "scl": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/V/PE/2025/1/S2B_T29VPE_20250110T120355_L2A/SCL.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Scene classification map (SCL)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 600000, 0, -20, 6500040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "12207e6a0165b890c165ff479833371ffd58e090f76ab592ad6eee4315671ae49481",
+ "file:size": 134774,
+ "roles": [
+ "data"
+ ]
+ },
+ "aot": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/V/PE/2025/1/S2B_T29VPE_20250110T120355_L2A/AOT.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Aerosol optical thickness (AOT)",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 600000, 0, -20, 6500040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 20,
+ "scale": 0.001,
+ "offset": 0
+ }
+ ],
+ "file:checksum": "12203c6ac06eb7a3d9d51ec7681002b16eae54e57856d7b28f14770cbf31fb882782",
+ "file:size": 118913,
+ "roles": [
+ "data"
+ ]
+ },
+ "coastal": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/V/PE/2025/1/S2B_T29VPE_20250110T120355_L2A/B01.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Coastal - 60m",
+ "eo:bands": [
+ {
+ "name": "B01",
+ "common_name": "coastal",
+ "center_wavelength": 0.443,
+ "full_width_half_max": 0.027
+ }
+ ],
+ "gsd": 60,
+ "proj:shape": [1830, 1830],
+ "proj:transform": [60, 0, 600000, 0, -60, 6500040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 60,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "12204dcef553ab0c337169780220b31ec9b1a24d778438cc90c9eae23a08dc7a1fb3",
+ "file:size": 371951,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "nir09": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/V/PE/2025/1/S2B_T29VPE_20250110T120355_L2A/B09.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "NIR 3 - 60m",
+ "eo:bands": [
+ {
+ "name": "B09",
+ "common_name": "nir09",
+ "center_wavelength": 0.945,
+ "full_width_half_max": 0.026
+ }
+ ],
+ "gsd": 60,
+ "proj:shape": [1830, 1830],
+ "proj:transform": [60, 0, 600000, 0, -60, 6500040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint16",
+ "spatial_resolution": 60,
+ "scale": 0.0001,
+ "offset": -0.1
+ }
+ ],
+ "file:checksum": "1220fd8f1fb2de99745bc4fbda52099ee12f21cc147f02b6ff946c6422a88bf0fedd",
+ "file:size": 360501,
+ "roles": [
+ "data",
+ "reflectance"
+ ]
+ },
+ "cloud": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/V/PE/2025/1/S2B_T29VPE_20250110T120355_L2A/CLD_20m.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Cloud Probabilities",
+ "gsd": 20,
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 600000, 0, -20, 6500040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "122054a47012552753b4e55a873394377da8c2e1264c613cdb9fcf92e8a4dfc7fbc7",
+ "file:size": 89971,
+ "roles": [
+ "data",
+ "cloud"
+ ]
+ },
+ "snow": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/V/PE/2025/1/S2B_T29VPE_20250110T120355_L2A/SNW_20m.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "Snow Probabilities",
+ "proj:shape": [5490, 5490],
+ "proj:transform": [20, 0, 600000, 0, -20, 6500040],
+ "raster:bands": [
+ {
+ "nodata": 0,
+ "data_type": "uint8",
+ "spatial_resolution": 20
+ }
+ ],
+ "file:checksum": "1220bd2dd2207c60d18109116b1a1725ac526fd6101b1ce16e8c352b0c58b0d89b22",
+ "file:size": 53931,
+ "roles": [
+ "data",
+ "snow-ice"
+ ]
+ },
+ "preview": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/V/PE/2025/1/S2B_T29VPE_20250110T120355_L2A/L2A_PVI.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "True color preview",
+ "eo:bands": [
+ {
+ "name": "B04",
+ "common_name": "red",
+ "center_wavelength": 0.665,
+ "full_width_half_max": 0.038
+ },
+ {
+ "name": "B03",
+ "common_name": "green",
+ "center_wavelength": 0.56,
+ "full_width_half_max": 0.045
+ },
+ {
+ "name": "B02",
+ "common_name": "blue",
+ "center_wavelength": 0.49,
+ "full_width_half_max": 0.098
+ }
+ ],
+ "file:checksum": "122065d3085676567c18174d812abb82f2823a98b2442d3013b3a2ab05af3ae5d892",
+ "file:size": 23139,
+ "roles": [
+ "overview"
+ ]
+ },
+ "granule_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/V/PE/2025/1/S2B_T29VPE_20250110T120355_L2A/metadata.xml",
+ "type": "application/xml",
+ "file:checksum": "1220c057f7cfd0855e31ead555fe7f3adc705757a2ed7ba69b6ec90f392c6d9afb77",
+ "file:size": 104646,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "tileinfo_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/V/PE/2025/1/S2B_T29VPE_20250110T120355_L2A/tileInfo.json",
+ "type": "application/json",
+ "file:checksum": "122085c2343946f211d76226554f92b456be62783d29a427f4e9f81fd47d9baa492e",
+ "file:size": 1481,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "product_metadata": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/V/PE/2025/1/S2B_T29VPE_20250110T120355_L2A/product_metadata.xml",
+ "type": "application/xml",
+ "file:checksum": "1220ec3afeb78300267b27f5a3707163883d208fc20b23bb0feacc85f1ad69b9a5d3",
+ "file:size": 54561,
+ "roles": [
+ "metadata"
+ ]
+ },
+ "thumbnail": {
+ "href": "https://e84-earth-search-sentinel-data.s3.us-west-2.amazonaws.com/sentinel-2-c1-l2a/29/V/PE/2025/1/S2B_T29VPE_20250110T120355_L2A/L2A_PVI.jpg",
+ "type": "image/jpeg",
+ "title": "Thumbnail of preview image",
+ "file:checksum": "1220e2bff46670175ca4680698e6f65659767c331ed35c51865fdc46267bd4096960",
+ "file:size": 3710,
+ "roles": [
+ "thumbnail"
+ ]
+ }
+ },
+ "bbox": [-7.308774, 57.985541, -6.851117, 58.629121],
+ "stac_extensions": [
+ "https://stac-extensions.github.io/eo/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/file/v2.1.0/schema.json",
+ "https://stac-extensions.github.io/grid/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/mgrs/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/processing/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/projection/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/raster/v1.1.0/schema.json",
+ "https://stac-extensions.github.io/sentinel-2/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/storage/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/view/v1.0.0/schema.json"
+ ],
+ "collection": "sentinel-2-c1-l2a"
+ }
+ ],
+ "links": [
+ {
+ "rel": "next",
+ "title": "Next page of Items",
+ "method": "GET",
+ "type": "application/geo+json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a/items?collections=sentinel-2-c1-l2a&next=2025-01-10T12%3A05%3A06.060000Z%2CS2B_T29VPE_20250110T120355_L2A%2Csentinel-2-c1-l2a"
+ },
+ {
+ "rel": "root",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1"
+ },
+ {
+ "rel": "self",
+ "type": "application/geo+json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a/items"
+ },
+ {
+ "rel": "collection",
+ "type": "application/json",
+ "href": "https://earth-search.aws.element84.com/v1/collections/sentinel-2-c1-l2a"
+ }
+ ]
+}
diff --git a/spark/common/src/test/resources/datasource_stac/collection.json b/spark/common/src/test/resources/datasource_stac/collection.json
new file mode 100644
index 0000000000..4fbe98d46b
--- /dev/null
+++ b/spark/common/src/test/resources/datasource_stac/collection.json
@@ -0,0 +1,142 @@
+{
+ "id": "simple-collection",
+ "type": "Collection",
+ "stac_extensions": [
+ "https://stac-extensions.github.io/eo/v2.0.0/schema.json",
+ "https://stac-extensions.github.io/projection/v2.0.0/schema.json",
+ "https://stac-extensions.github.io/view/v1.0.0/schema.json"
+ ],
+ "stac_version": "1.1.0",
+ "description": "A simple collection demonstrating core catalog fields with links to a couple of items",
+ "title": "Simple Example Collection",
+ "keywords": [
+ "simple",
+ "example",
+ "collection"
+ ],
+ "providers": [
+ {
+ "name": "Remote Data, Inc",
+ "description": "Producers of awesome spatiotemporal assets",
+ "roles": [
+ "producer",
+ "processor"
+ ],
+ "url": "http://remotedata.io"
+ }
+ ],
+ "extent": {
+ "spatial": {
+ "bbox": [
+ [
+ 172.91173669923782,
+ 1.3438851951615003,
+ 172.95469614953714,
+ 1.3690476620161975
+ ]
+ ]
+ },
+ "temporal": {
+ "interval": [
+ [
+ "2020-12-11T22:38:32.125Z",
+ "2020-12-14T18:02:31.437Z"
+ ]
+ ]
+ }
+ },
+ "license": "CC-BY-4.0",
+ "summaries": {
+ "platform": [
+ "cool_sat1",
+ "cool_sat2"
+ ],
+ "constellation": [
+ "ion"
+ ],
+ "instruments": [
+ "cool_sensor_v1",
+ "cool_sensor_v2"
+ ],
+ "gsd": {
+ "minimum": 0.512,
+ "maximum": 0.66
+ },
+ "eo:cloud_cover": {
+ "minimum": 1.2,
+ "maximum": 1.2
+ },
+ "proj:cpde": [
+ "EPSG:32659"
+ ],
+ "view:sun_elevation": {
+ "minimum": 54.9,
+ "maximum": 54.9
+ },
+ "view:off_nadir": {
+ "minimum": 3.8,
+ "maximum": 3.8
+ },
+ "view:sun_azimuth": {
+ "minimum": 135.7,
+ "maximum": 135.7
+ },
+ "statistics": {
+ "type": "object",
+ "properties": {
+ "vegetation": {
+ "description": "Percentage of pixels that are detected as vegetation, e.g. forests, grasslands, etc.",
+ "minimum": 0,
+ "maximum": 100
+ },
+ "water": {
+ "description": "Percentage of pixels that are detected as water, e.g. rivers, oceans and ponds.",
+ "minimum": 0,
+ "maximum": 100
+ },
+ "urban": {
+ "description": "Percentage of pixels that detected as urban, e.g. roads and buildings.",
+ "minimum": 0,
+ "maximum": 100
+ }
+ }
+ }
+ },
+ "links": [
+ {
+ "rel": "root",
+ "href": "./collection.json",
+ "type": "application/json",
+ "title": "Simple Example Collection"
+ },
+ {
+ "rel": "item",
+ "href": "./simple-item.json",
+ "type": "application/geo+json",
+ "title": "Simple Item"
+ },
+ {
+ "rel": "item",
+ "href": "./core-item.json",
+ "type": "application/geo+json",
+ "title": "Core Item"
+ },
+ {
+ "rel": "item",
+ "href": "./extended-item.json",
+ "type": "application/geo+json",
+ "title": "Extended Item"
+ },
+ {
+ "rel": "self",
+ "href": "https://raw.githubusercontent.com/radiantearth/stac-spec/v1.1.0/examples/collection.json",
+ "type": "application/json"
+ },
+ {
+ "rel": "child",
+ "href": "./nested/nested-collection.json",
+ "type": "application/json",
+ "title": "Nested Collection"
+ }
+ ]
+}
diff --git a/spark/common/src/test/resources/datasource_stac/core-item.json b/spark/common/src/test/resources/datasource_stac/core-item.json
new file mode 100644
index 0000000000..e151b1353c
--- /dev/null
+++ b/spark/common/src/test/resources/datasource_stac/core-item.json
@@ -0,0 +1,125 @@
+{
+ "stac_version": "1.1.0",
+ "stac_extensions": [],
+ "type": "Feature",
+ "id": "20201211_223832_CS2",
+ "bbox": [
+ 172.91173669923782,
+ 1.3438851951615003,
+ 172.95469614953714,
+ 1.3690476620161975
+ ],
+ "geometry": {
+ "type": "Polygon",
+ "coordinates": [
+ [
+ [
+ 172.91173669923782,
+ 1.3438851951615003
+ ],
+ [
+ 172.95469614953714,
+ 1.3438851951615003
+ ],
+ [
+ 172.95469614953714,
+ 1.3690476620161975
+ ],
+ [
+ 172.91173669923782,
+ 1.3690476620161975
+ ],
+ [
+ 172.91173669923782,
+ 1.3438851951615003
+ ]
+ ]
+ ]
+ },
+ "properties": {
+ "title": "Core Item",
+ "description": "A sample STAC Item that includes examples of all common metadata",
+ "datetime": null,
+ "start_datetime": "2020-12-11T22:38:32.125Z",
+ "end_datetime": "2020-12-11T22:38:32.327Z",
+ "created": "2020-12-12T01:48:13.725Z",
+ "updated": "2020-12-12T01:48:13.725Z",
+ "platform": "cool_sat1",
+ "instruments": [
+ "cool_sensor_v1"
+ ],
+ "constellation": "ion",
+ "mission": "collection 5624",
+ "gsd": 0.512
+ },
+ "collection": "simple-collection",
+ "links": [
+ {
+ "rel": "collection",
+ "href": "./collection.json",
+ "type": "application/json",
+ "title": "Simple Example Collection"
+ },
+ {
+ "rel": "root",
+ "href": "./collection.json",
+ "type": "application/json",
+ "title": "Simple Example Collection"
+ },
+ {
+ "rel": "parent",
+ "href": "./collection.json",
+ "type": "application/json",
+ "title": "Simple Example Collection"
+ },
+ {
+ "rel": "alternate",
+ "type": "text/html",
+ "href": "http://remotedata.io/catalog/20201211_223832_CS2/index.html",
+ "title": "HTML version of this STAC Item"
+ }
+ ],
+ "assets": {
+ "analytic": {
+ "href": "https://storage.googleapis.com/open-cogs/stac-examples/20201211_223832_CS2_analytic.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "4-Band Analytic",
+ "roles": [
+ "data"
+ ]
+ },
+ "thumbnail": {
+ "href": "https://storage.googleapis.com/open-cogs/stac-examples/20201211_223832_CS2.jpg",
+ "title": "Thumbnail",
+ "type": "image/png",
+ "roles": [
+ "thumbnail"
+ ]
+ },
+ "visual": {
+ "href": "https://storage.googleapis.com/open-cogs/stac-examples/20201211_223832_CS2.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "3-Band Visual",
+ "roles": [
+ "visual"
+ ]
+ },
+ "udm": {
+ "href": "https://storage.googleapis.com/open-cogs/stac-examples/20201211_223832_CS2_analytic_udm.tif",
+ "title": "Unusable Data Mask",
+ "type": "image/tiff; application=geotiff"
+ },
+ "json-metadata": {
+ "href": "http://remotedata.io/catalog/20201211_223832_CS2/extended-metadata.json",
+ "title": "Extended Metadata",
+ "type": "application/json",
+ "roles": [
+ "metadata"
+ ]
+ },
+ "ephemeris": {
+ "href": "http://cool-sat.com/catalog/20201211_223832_CS2/20201211_223832_CS2.EPH",
+ "title": "Satellite Ephemeris Metadata"
+ }
+ }
+}
diff --git a/spark/common/src/test/resources/datasource_stac/extended-item.json b/spark/common/src/test/resources/datasource_stac/extended-item.json
new file mode 100644
index 0000000000..b5f3a0a9df
--- /dev/null
+++ b/spark/common/src/test/resources/datasource_stac/extended-item.json
@@ -0,0 +1,210 @@
+{
+ "stac_version": "1.1.0",
+ "stac_extensions": [
+ "https://stac-extensions.github.io/eo/v2.0.0/schema.json",
+ "https://stac-extensions.github.io/projection/v2.0.0/schema.json",
+ "https://stac-extensions.github.io/scientific/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/view/v1.0.0/schema.json",
+ "https://stac-extensions.github.io/remote-data/v1.0.0/schema.json"
+ ],
+ "type": "Feature",
+ "id": "20201211_223832_CS2",
+ "bbox": [
+ 172.91173669923782,
+ 1.3438851951615003,
+ 172.95469614953714,
+ 1.3690476620161975
+ ],
+ "geometry": {
+ "type": "Polygon",
+ "coordinates": [
+ [
+ [
+ 172.91173669923782,
+ 1.3438851951615003
+ ],
+ [
+ 172.95469614953714,
+ 1.3438851951615003
+ ],
+ [
+ 172.95469614953714,
+ 1.3690476620161975
+ ],
+ [
+ 172.91173669923782,
+ 1.3690476620161975
+ ],
+ [
+ 172.91173669923782,
+ 1.3438851951615003
+ ]
+ ]
+ ]
+ },
+ "properties": {
+ "title": "Extended Item",
+ "description": "A sample STAC Item that includes a variety of examples from the stable extensions",
+ "keywords": [
+ "extended",
+ "example",
+ "item"
+ ],
+ "datetime": "2020-12-14T18:02:31.437000Z",
+ "created": "2020-12-15T01:48:13.725Z",
+ "updated": "2020-12-15T01:48:13.725Z",
+ "platform": "cool_sat2",
+ "instruments": [
+ "cool_sensor_v2"
+ ],
+ "gsd": 0.66,
+ "eo:cloud_cover": 1.2,
+ "eo:snow_cover": 0,
+ "statistics": {
+ "vegetation": 12.57,
+ "water": 1.23,
+ "urban": 26.2
+ },
+ "proj:code": "EPSG:32659",
+ "proj:shape": [
+ 5558,
+ 9559
+ ],
+ "proj:transform": [
+ 0.5,
+ 0,
+ 712710,
+ 0,
+ -0.5,
+ 151406,
+ 0,
+ 0,
+ 1
+ ],
+ "view:sun_elevation": 54.9,
+ "view:off_nadir": 3.8,
+ "view:sun_azimuth": 135.7,
+ "rd:type": "scene",
+ "rd:anomalous_pixels": 0.14,
+ "rd:earth_sun_distance": 1.014156,
+ "rd:sat_id": "cool_sat2",
+ "rd:product_level": "LV3A",
+ "sci:doi": "10.5061/dryad.s2v81.2/27.2"
+ },
+ "collection": "simple-collection",
+ "links": [
+ {
+ "rel": "collection",
+ "href": "./collection.json",
+ "type": "application/json",
+ "title": "Simple Example Collection"
+ },
+ {
+ "rel": "root",
+ "href": "./collection.json",
+ "type": "application/json",
+ "title": "Simple Example Collection"
+ },
+ {
+ "rel": "parent",
+ "href": "./collection.json",
+ "type": "application/json",
+ "title": "Simple Example Collection"
+ },
+ {
+ "rel": "alternate",
+ "type": "text/html",
+ "href": "http://remotedata.io/catalog/20201211_223832_CS2/index.html",
+ "title": "HTML version of this STAC Item"
+ }
+ ],
+ "assets": {
+ "analytic": {
+ "href": "https://storage.googleapis.com/open-cogs/stac-examples/20201211_223832_CS2_analytic.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "4-Band Analytic",
+ "roles": [
+ "data"
+ ],
+ "bands": [
+ {
+ "name": "band1",
+ "eo:common_name": "blue",
+ "eo:center_wavelength": 0.47,
+ "eo:full_width_half_max": 70
+ },
+ {
+ "name": "band2",
+ "eo:common_name": "green",
+ "eo:center_wavelength": 0.56,
+ "eo:full_width_half_max": 80
+ },
+ {
+ "name": "band3",
+ "eo:common_name": "red",
+ "eo:center_wavelength": 0.645,
+ "eo:full_width_half_max": 90
+ },
+ {
+ "name": "band4",
+ "eo:common_name": "nir",
+ "eo:center_wavelength": 0.8,
+ "eo:full_width_half_max": 152
+ }
+ ]
+ },
+ "thumbnail": {
+ "href": "https://storage.googleapis.com/open-cogs/stac-examples/20201211_223832_CS2.jpg",
+ "title": "Thumbnail",
+ "type": "image/png",
+ "roles": [
+ "thumbnail"
+ ]
+ },
+ "visual": {
+ "href": "https://storage.googleapis.com/open-cogs/stac-examples/20201211_223832_CS2.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "3-Band Visual",
+ "roles": [
+ "visual"
+ ],
+ "bands": [
+ {
+ "name": "band3",
+ "eo:common_name": "red",
+ "eo:center_wavelength": 0.645,
+ "eo:full_width_half_max": 90
+ },
+ {
+ "name": "band2",
+ "eo:common_name": "green",
+ "eo:center_wavelength": 0.56,
+ "eo:full_width_half_max": 80
+ },
+ {
+ "name": "band1",
+ "eo:common_name": "blue",
+ "eo:center_wavelength": 0.47,
+ "eo:full_width_half_max": 70
+ }
+ ]
+ },
+ "udm": {
+ "href": "https://storage.googleapis.com/open-cogs/stac-examples/20201211_223832_CS2_analytic_udm.tif",
+ "title": "Unusable Data Mask",
+ "type": "image/tiff; application=geotiff"
+ },
+ "json-metadata": {
+ "href": "http://remotedata.io/catalog/20201211_223832_CS2/extended-metadata.json",
+ "title": "Extended Metadata",
+ "type": "application/json",
+ "roles": [
+ "metadata"
+ ]
+ },
+ "ephemeris": {
+ "href": "http://cool-sat.com/catalog/20201211_223832_CS2/20201211_223832_CS2.EPH",
+ "title": "Satellite Ephemeris Metadata"
+ }
+ }
+}
\ No newline at end of file
diff --git a/spark/common/src/test/resources/datasource_stac/nested/nested-collection.json b/spark/common/src/test/resources/datasource_stac/nested/nested-collection.json
new file mode 100644
index 0000000000..eac6a0ea84
--- /dev/null
+++ b/spark/common/src/test/resources/datasource_stac/nested/nested-collection.json
@@ -0,0 +1,130 @@
+{
+ "id": "nested-collection",
+ "type": "Collection",
+ "stac_extensions": [
+ "https://stac-extensions.github.io/eo/v2.0.0/schema.json",
+ "https://stac-extensions.github.io/projection/v2.0.0/schema.json",
+ "https://stac-extensions.github.io/view/v1.0.0/schema.json"
+ ],
+ "stac_version": "1.1.0",
+ "description": "A nested collection demonstrating core catalog fields with links to an item and items",
+ "title": "Nested Example Collection",
+ "keywords": [
+ "nested",
+ "example",
+ "collection"
+ ],
+ "providers": [
+ {
+ "name": "Remote Data, Inc",
+ "description": "Producers of awesome spatiotemporal assets",
+ "roles": [
+ "producer",
+ "processor"
+ ],
+ "url": "http://remotedata.io"
+ }
+ ],
+ "extent": {
+ "spatial": {
+ "bbox": [
+ [
+ 17.91173669923782,
+ 10.3438851951615003,
+ 17.95469614953714,
+ 10.3690476620161975
+ ]
+ ]
+ },
+ "temporal": {
+ "interval": [
+ [
+ "2020-12-11T22:38:32.125Z",
+ "2020-12-14T18:02:31.437Z"
+ ]
+ ]
+ }
+ },
+ "license": "CC-BY-4.0",
+ "summaries": {
+ "platform": [
+ "cool_sat1",
+ "cool_sat2"
+ ],
+ "constellation": [
+ "ion"
+ ],
+ "instruments": [
+ "cool_sensor_v1",
+ "cool_sensor_v2"
+ ],
+ "gsd": {
+ "minimum": 0.512,
+ "maximum": 0.66
+ },
+ "eo:cloud_cover": {
+ "minimum": 1.2,
+ "maximum": 1.2
+ },
+ "proj:cpde": [
+ "EPSG:32659"
+ ],
+ "view:sun_elevation": {
+ "minimum": 54.9,
+ "maximum": 54.9
+ },
+ "view:off_nadir": {
+ "minimum": 3.8,
+ "maximum": 3.8
+ },
+ "view:sun_azimuth": {
+ "minimum": 135.7,
+ "maximum": 135.7
+ },
+ "statistics": {
+ "type": "object",
+ "properties": {
+ "vegetation": {
+ "description": "Percentage of pixels that are detected as vegetation, e.g. forests, grasslands, etc.",
+ "minimum": 0,
+ "maximum": 100
+ },
+ "water": {
+ "description": "Percentage of pixels that are detected as water, e.g. rivers, oceans and ponds.",
+ "minimum": 0,
+ "maximum": 100
+ },
+ "urban": {
+ "description": "Percentage of pixels that detected as urban, e.g. roads and buildings.",
+ "minimum": 0,
+ "maximum": 100
+ }
+ }
+ }
+ },
+ "links": [
+ {
+ "rel": "root",
+ "href": "./nested-collection.json",
+ "type": "application/json",
+ "title": "Nested Example Collection"
+ },
+ {
+ "rel": "item",
+ "href": "./nested-item.json",
+ "type": "application/geo+json",
+ "title": "Nested Item"
+ },
+ {
+ "rel": "items",
+ "href": "./nested-items.json",
+ "type": "application/geo+json",
+ "title": "Nested Items"
+ },
+ {
+ "rel": "self",
+ "href": "https://raw.githubusercontent.com/radiantearth/stac-spec/v1.1.0/examples/nested-collection.json",
+ "type": "application/json"
+ }
+ ]
+}
diff --git a/spark/common/src/test/resources/datasource_stac/nested/nested-item.json b/spark/common/src/test/resources/datasource_stac/nested/nested-item.json
new file mode 100644
index 0000000000..1a26b4f400
--- /dev/null
+++ b/spark/common/src/test/resources/datasource_stac/nested/nested-item.json
@@ -0,0 +1,55 @@
+{
+ "id": "nested-item",
+ "type": "Feature",
+ "stac_version": "1.1.0",
+ "stac_extensions": [
+ "https://stac-extensions.github.io/eo/v2.0.0/schema.json",
+ "https://stac-extensions.github.io/projection/v2.0.0/schema.json",
+ "https://stac-extensions.github.io/view/v1.0.0/schema.json"
+ ],
+ "bbox": [
+ 17.91173669923782,
+ 10.3438851951615003,
+ 17.95469614953714,
+ 10.3690476620161975
+ ],
+ "geometry": {
+ "type": "Polygon",
+ "coordinates": [
+ [
+ [17.91173669923782, 10.3438851951615003],
+ [17.95469614953714, 10.3438851951615003],
+ [17.95469614953714, 10.3690476620161975],
+ [17.91173669923782, 10.3690476620161975],
+ [17.91173669923782, 10.3438851951615003]
+ ]
+ ]
+ },
+ "properties": {
+ "title": "Nested Item",
+ "description": "A sample STAC nested Item that includes examples of all common metadata",
+ "datetime": "2020-12-12T00:00:00Z",
+ "eo:cloud_cover": 1.2,
+ "proj:epsg": 32659,
+ "view:sun_elevation": 54.9,
+ "view:off_nadir": 3.8,
+ "view:sun_azimuth": 135.7
+ },
+ "assets": {
+ "visual": {
+ "href": "https://e84-earth-search-sentinel-data.s3/example/visual.tif",
+ "title": "Visual asset",
+ "type": "image/tiff; application=geotiff",
+ "roles": ["visual"]
+ }
+ },
+ "links": [
+ {
+ "rel": "collection",
+ "href": "./nested-collection.json",
+ "type": "application/json",
+ "title": "Nested Example Collection"
+ }
+ ]
+}
+
diff --git a/spark/common/src/test/resources/datasource_stac/nested/nested-items.json b/spark/common/src/test/resources/datasource_stac/nested/nested-items.json
new file mode 100644
index 0000000000..1c05033775
--- /dev/null
+++ b/spark/common/src/test/resources/datasource_stac/nested/nested-items.json
@@ -0,0 +1,110 @@
+{
+ "type": "FeatureCollection",
+ "stac_version": "1.1.0",
+ "stac_extensions": [
+ "https://stac-extensions.github.io/eo/v2.0.0/schema.json",
+ "https://stac-extensions.github.io/projection/v2.0.0/schema.json",
+ "https://stac-extensions.github.io/view/v1.0.0/schema.json"
+ ],
+ "type": "FeatureCollection",
+ "features": [
+ {
+ "stac_version": "1.1.0",
+ "id": "nested-item-1",
+ "type": "Feature",
+ "bbox": [
+ 17.91173669923782,
+ 10.3438851951615003,
+ 17.95469614953714,
+ 10.3690476620161975
+ ],
+ "geometry": {
+ "type": "Polygon",
+ "coordinates": [
+ [
+ [17.91173669923782, 10.3438851951615003],
+ [17.95469614953714, 10.3438851951615003],
+ [17.95469614953714, 10.3690476620161975],
+ [17.91173669923782, 10.3690476620161975],
+ [17.91173669923782, 10.3438851951615003]
+ ]
+ ]
+ },
+ "properties": {
+ "title": "Nested Item 1",
+ "description": "A sample STAC nested Item that includes examples of all common metadata",
+ "datetime": "2020-12-12T00:00:00Z",
+ "eo:cloud_cover": 1.2,
+ "proj:epsg": 32659,
+ "view:sun_elevation": 54.9,
+ "view:off_nadir": 3.8,
+ "view:sun_azimuth": 135.7
+ },
+ "assets": {
+ "visual": {
+ "href": "http://e84-earth-search-sentinel-data.s3/example/visual1.tif",
+ "title": "Visual asset 1",
+ "type": "image/tiff; application=geotiff",
+ "roles": ["visual"]
+ }
+ },
+ "links": [
+ {
+ "rel": "collection",
+ "href": "./nested-collection.json",
+ "type": "application/json",
+ "title": "Nested Example Collection"
+ }
+ ]
+ },
+ {
+ "stac_version": "1.1.0",
+ "id": "nested-item-2",
+ "type": "Feature",
+ "bbox": [
+ 17.91173669923782,
+ 10.3438851951615003,
+ 17.95469614953714,
+ 10.3690476620161975
+ ],
+ "geometry": {
+ "type": "Polygon",
+ "coordinates": [
+ [
+ [17.91173669923782, 10.3438851951615003],
+ [17.95469614953714, 10.3438851951615003],
+ [17.95469614953714, 10.3690476620161975],
+ [17.91173669923782, 10.3690476620161975],
+ [17.91173669923782, 10.3438851951615003]
+ ]
+ ]
+ },
+ "properties": {
+ "title": "Nested Item 2",
+ "description": "A sample STAC nested Item that includes examples of all common metadata",
+ "datetime": "2020-12-13T00:00:00Z",
+ "eo:cloud_cover": 1.2,
+ "proj:epsg": 32659,
+ "view:sun_elevation": 54.9,
+ "view:off_nadir": 3.8,
+ "view:sun_azimuth": 135.7
+ },
+ "assets": {
+ "visual": {
+ "href": "http://e84-earth-search-sentinel-data.s3/example/visual2.tif",
+ "title": "Visual asset 2",
+ "type": "image/tiff; application=geotiff",
+ "roles": ["visual"]
+ }
+ },
+ "links": [
+ {
+ "rel": "collection",
+ "href": "./nested-collection.json",
+ "type": "application/json",
+ "title": "Nested Example Collection"
+ }
+ ]
+ }
+ ]
+}
diff --git a/spark/common/src/test/resources/datasource_stac/simple-item.json b/spark/common/src/test/resources/datasource_stac/simple-item.json
new file mode 100644
index 0000000000..277b973462
--- /dev/null
+++ b/spark/common/src/test/resources/datasource_stac/simple-item.json
@@ -0,0 +1,83 @@
+{
+ "stac_version": "1.1.0",
+ "stac_extensions": [],
+ "type": "Feature",
+ "id": "20201211_223832_CS2",
+ "bbox": [
+ 172.91173669923782,
+ 1.3438851951615003,
+ 172.95469614953714,
+ 1.3690476620161975
+ ],
+ "geometry": {
+ "type": "Polygon",
+ "coordinates": [
+ [
+ [
+ 172.91173669923782,
+ 1.3438851951615003
+ ],
+ [
+ 172.95469614953714,
+ 1.3438851951615003
+ ],
+ [
+ 172.95469614953714,
+ 1.3690476620161975
+ ],
+ [
+ 172.91173669923782,
+ 1.3690476620161975
+ ],
+ [
+ 172.91173669923782,
+ 1.3438851951615003
+ ]
+ ]
+ ]
+ },
+ "properties": {
+ "title": "Simple Item ",
+ "description": "A sample STAC nested Item that includes examples of some metadata",
+ "datetime": "2020-12-11T22:38:32.125000Z"
+ },
+ "collection": "simple-collection",
+ "links": [
+ {
+ "rel": "collection",
+ "href": "./collection.json",
+ "type": "application/json",
+ "title": "Simple Example Collection"
+ },
+ {
+ "rel": "root",
+ "href": "./collection.json",
+ "type": "application/json",
+ "title": "Simple Example Collection"
+ },
+ {
+ "rel": "parent",
+ "href": "./collection.json",
+ "type": "application/json",
+ "title": "Simple Example Collection"
+ }
+ ],
+ "assets": {
+ "visual": {
+ "href": "https://storage.googleapis.com/open-cogs/stac-examples/20201211_223832_CS2.tif",
+ "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ "title": "3-Band Visual",
+ "roles": [
+ "visual"
+ ]
+ },
+ "thumbnail": {
+ "href": "https://storage.googleapis.com/open-cogs/stac-examples/20201211_223832_CS2.jpg",
+ "title": "Thumbnail",
+ "type": "image/jpeg",
+ "roles": [
+ "thumbnail"
+ ]
+ }
+ }
+}
diff --git a/spark/common/src/test/scala/org/apache/sedona/sql/GeoStatsSuite.scala b/spark/common/src/test/scala/org/apache/sedona/sql/GeoStatsSuite.scala
new file mode 100644
index 0000000000..9567dcc95f
--- /dev/null
+++ b/spark/common/src/test/scala/org/apache/sedona/sql/GeoStatsSuite.scala
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.sedona.sql
+
+import org.apache.sedona.stats.Weighting.{addBinaryDistanceBandColumn, addWeightedDistanceBandColumn}
+import org.apache.sedona.stats.clustering.DBSCAN.dbscan
+import org.apache.sedona.stats.hotspotDetection.GetisOrd.gLocal
+import org.apache.sedona.stats.outlierDetection.LocalOutlierFactor.localOutlierFactor
+import org.apache.spark.sql.DataFrame
+import org.apache.spark.sql.functions.{col, expr, lit}
+import org.apache.spark.sql.sedona_sql.expressions.st_functions.{ST_DBSCAN, ST_LocalOutlierFactor}
+
+class GeoStatsSuite extends TestBaseScala {
+ private val spark = sparkSession
+
+ case class Record(id: Int, x: Double, y: Double)
+
+ def getData: DataFrame = {
+ spark
+ .createDataFrame(
+ Seq(
+ Record(10, 1.0, 1.8),
+ Record(11, 1.0, 1.9),
+ Record(12, 1.0, 2.0),
+ Record(13, 1.0, 2.1),
+ Record(14, 2.0, 2.0),
+ Record(15, 3.0, 1.9),
+ Record(16, 3.0, 2.0),
+ Record(17, 3.0, 2.1),
+ Record(18, 3.0, 2.2)))
+ .withColumn("geometry", expr("ST_Point(x, y)"))
+ }
+
+ it("test dbscan function") {
+ dbscan(getData.withColumn("sql_results", expr("ST_DBSCAN(geometry, 1.0, 4, false)")), 1.0, 4)
+ .where("sql_results.cluster = cluster and sql_results.isCore = isCore")
+ .count() == getData.count()
+ }
+
+ it("test dbscan function df method") {
+ dbscan(
+ getData.withColumn("sql_results", ST_DBSCAN(col("geometry"), lit(1.0), lit(4), lit(false))),
+ 1.0,
+ 4)
+ .where("sql_results.cluster = cluster and sql_results.isCore = isCore")
+ .count() == getData.count()
+ }
+
+ it("test dbscan function with distance column") {
+ dbscan(
+ getData.withColumn("sql_results", expr("ST_DBSCAN(geometry, 1.0, 4, true)")),
+ 1.0,
+ 4,
+ useSpheroid = true)
+ .where("sql_results.cluster = cluster and sql_results.isCore = isCore")
+ .count() == getData.count()
+ }
+
+ it("test dbscan function with scalar subquery") {
+ dbscan(
+ getData.withColumn(
+ "sql_results",
+ expr("ST_DBSCAN(geometry, (SELECT ARRAY(1.0, 2.0)[0]), 4, false)")),
+ 1.0,
+ 4)
+ .where("sql_results.cluster = cluster and sql_results.isCore = isCore")
+ .count() == getData.count()
+ }
+
+ it("test dbscan with geom literal") {
+ val error = intercept[IllegalArgumentException] {
+ spark.sql("SELECT ST_DBSCAN(ST_GeomFromWKT('POINT(0.0 1.1)'), 1.0, 4, false)").collect()
+ }
+ assert(
+ error
+ .asInstanceOf[IllegalArgumentException]
+ .getMessage == "geometry argument must be a named reference to an existing column")
+ }
+
+ it("test dbscan with minPts variable") {
+ val error = intercept[IllegalArgumentException] {
+ getData
+ .withColumn("result", ST_DBSCAN(col("geometry"), lit(1.0), col("id"), lit(false)))
+ .collect()
+ }
+
+ assert(
+ error
+ .asInstanceOf[IllegalArgumentException]
+ .getMessage
+ .contains("minPts must be a scalar value"))
+ }
+
+ it("test lof") {
+ localOutlierFactor(
+ getData.withColumn("sql_result", expr("ST_LocalOutlierFactor(geometry, 4, false)")),
+ 4)
+ .where("sql_result = lof")
+ .count() == getData.count()
+ }
+
+ it("test lof with dataframe method") {
+ localOutlierFactor(
+ getData.withColumn(
+ "sql_result",
+ ST_LocalOutlierFactor(col("geometry"), lit(4), lit(false))),
+ 4)
+ .where("sql_result = lof")
+ .count() == getData.count()
+ }
+
+ it("test geostats function in another function") {
+ getData
+ .withColumn("sql_result", expr("SQRT(ST_LocalOutlierFactor(geometry, 4, false))"))
+ .collect()
+ }
+
+ it("test DBSCAN with a column named __isCore in input df") {
+ val exception = intercept[IllegalArgumentException] {
+ getData
+ .withColumn("__isCore", lit(1))
+ .withColumn("sql_result", expr("ST_DBSCAN(geometry, 0.1, 4, false)"))
+ .collect()
+ }
+ assert(
+ exception.getMessage == "requirement failed: __isCore is a reserved name by the dbscan algorithm. Please rename the columns before calling the ST_DBSCAN function.")
+ }
+
+ it("test ST_BinaryDistanceBandColumn") {
+ val weightedDf = getData
+ .withColumn(
+ "someWeights",
+ expr(
+ "array_sort(ST_BinaryDistanceBandColumn(geometry, 1.0, true, true, false, struct(id, geometry)))"))
+
+ val resultsDf = addBinaryDistanceBandColumn(
+ weightedDf,
+ 1.0,
+ true,
+ true,
+ savedAttributes = Seq("id", "geometry"))
+ .withColumn("weights", expr("array_sort(weights)"))
+ .where("someWeights = weights")
+
+ assert(resultsDf.count == weightedDf.count())
+ }
+
+ it("test ST_WeightedDistanceBandColumn") {
+ val weightedDf = getData
+ .withColumn(
+ "someWeights",
+ expr(
+ "array_sort(ST_WeightedDistanceBandColumn(geometry, 1.0, -1.0, true, true, 1.0, false, struct(id, geometry)))"))
+
+ val resultsDf = addWeightedDistanceBandColumn(
+ weightedDf,
+ 1.0,
+ -1.0,
+ true,
+ true,
+ savedAttributes = Seq("id", "geometry"),
+ selfWeight = 1.0)
+ .withColumn("weights", expr("array_sort(weights)"))
+ .where("someWeights = weights")
+
+ assert(resultsDf.count == weightedDf.count())
+ }
+
+ it("test GI with ST_BinaryDistanceBandColumn") {
+ val weightedDf = getData
+ .withColumn(
+ "someWeights",
+ expr(
+ "ST_BinaryDistanceBandColumn(geometry, 1.0, true, true, false, struct(id, geometry))"))
+
+ val giDf = weightedDf
+ .withColumn("gi", expr("ST_GLocal(id, someWeights, true)"))
+ assert(
+ gLocal(giDf, "id", weights = "someWeights", star = true)
+ .where("G = gi.G")
+ .count() == weightedDf.count())
+ }
+
+ it("test nested ST_Geostats calls with getis ord") {
+ getData
+ .withColumn(
+ "GI",
+ expr(
+ "ST_GLocal(id, ST_BinaryDistanceBandColumn(geometry, 1.0, true, true, false, struct(id, geometry)), true)"))
+ .collect()
+ }
+
+ it("test ST_Geostats with string column") {
+ getData
+ .withColumn("someString", lit("test"))
+ .withColumn("sql_results", expr("ST_DBSCAN(geometry, 1.0, 4, false)"))
+ .collect()
+ }
+}
diff --git a/spark/common/src/test/scala/org/apache/sedona/sql/KnnJoinSuite.scala b/spark/common/src/test/scala/org/apache/sedona/sql/KnnJoinSuite.scala
index 1d6119d02d..ab2c64898a 100644
--- a/spark/common/src/test/scala/org/apache/sedona/sql/KnnJoinSuite.scala
+++ b/spark/common/src/test/scala/org/apache/sedona/sql/KnnJoinSuite.scala
@@ -209,6 +209,22 @@ class KnnJoinSuite extends TestBaseScala with TableDrivenPropertyChecks {
"[1,3][1,6][1,13][1,16][2,1][2,5][2,11][2,15][3,3][3,9][3,13][3,19]")
}
+ it("KNN Join should verify the correct parameter k is passed to the join function") {
+ val df = sparkSession
+ .range(0, 1)
+ .toDF("id")
+ .withColumn("geom", expr("ST_Point(id, id)"))
+ .repartition(1)
+ df.createOrReplaceTempView("df1")
+ val exception = intercept[IllegalArgumentException] {
+ sparkSession
+ .sql(s"SELECT A.ID, B.ID FROM df1 A JOIN df1 B ON ST_KNN(A.GEOM, B.GEOM, 0, false)")
+ .collect()
+ }
+ exception.getMessage should include(
+ "The number of neighbors (k) must be equal or greater than 1.")
+ }
+
it("KNN Join with exact algorithms with additional join conditions on id") {
val df = sparkSession.sql(
s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_KNN(QUERIES.GEOM, OBJECTS.GEOM, 4, false) AND QUERIES.ID > 1")
@@ -425,6 +441,23 @@ class KnnJoinSuite extends TestBaseScala with TableDrivenPropertyChecks {
resultAll.mkString should be("[0,6][0,7]")
}
}
+
+ it("KNN Join with exact algorithms should not fail with null geometries") {
+ val df1 = sparkSession.sql(
+ "SELECT ST_GeomFromText(col1) as geom1 from values ('POINT (0.0 0.0)'), (null)")
+ val df2 = sparkSession.sql("SELECT ST_Point(0.0, 0.0) as geom2")
+ df1.cache()
+ df2.cache()
+ df1.join(df2, expr("ST_KNN(geom1, geom2, 1)")).count() should be(1)
+ }
+
+ it("KNN Join with exact algorithms should not fail with empty geometries") {
+ val df1 = sparkSession.sql("SELECT ST_GeomFromText('POINT EMPTY') as geom1")
+ val df2 = sparkSession.sql("SELECT ST_Point(0.0, 0.0) as geom2")
+ df1.cache()
+ df2.cache()
+ df1.join(df2, expr("ST_KNN(geom1, geom2, 1)")).count() should be(0)
+ }
}
private def withOptimizationMode(mode: String)(body: => Unit): Unit = {
diff --git a/spark/common/src/test/scala/org/apache/sedona/sql/PreserveSRIDSuite.scala b/spark/common/src/test/scala/org/apache/sedona/sql/PreserveSRIDSuite.scala
index ea6092629f..02fdb3149b 100644
--- a/spark/common/src/test/scala/org/apache/sedona/sql/PreserveSRIDSuite.scala
+++ b/spark/common/src/test/scala/org/apache/sedona/sql/PreserveSRIDSuite.scala
@@ -54,10 +54,12 @@ class PreserveSRIDSuite extends TestBaseScala with TableDrivenPropertyChecks {
("ST_SimplifyVW(geom1, 0.1)", 1000),
("ST_SimplifyPolygonHull(geom1, 0.5)", 1000),
("ST_SetSRID(geom1, 2000)", 2000),
+ ("ST_LabelPoint(geom1)", 1000),
("ST_LineMerge(geom2)", 1000),
("ST_StartPoint(geom3)", 1000),
("ST_Snap(geom3, geom3, 0.1)", 1000),
("ST_Boundary(geom1)", 1000),
+ ("ST_LineSegments(geom3)[0]", 1000),
("ST_LineSubstring(geom3, 0.1, 0.9)", 1000),
("ST_LineInterpolatePoint(geom3, 0.1)", 1000),
("ST_EndPoint(geom3)", 1000),
diff --git a/spark/common/src/test/scala/org/apache/sedona/sql/dataFrameAPITestScala.scala b/spark/common/src/test/scala/org/apache/sedona/sql/dataFrameAPITestScala.scala
index cf9b8a0f7a..a89af2355d 100644
--- a/spark/common/src/test/scala/org/apache/sedona/sql/dataFrameAPITestScala.scala
+++ b/spark/common/src/test/scala/org/apache/sedona/sql/dataFrameAPITestScala.scala
@@ -411,6 +411,38 @@ class dataFrameAPITestScala extends TestBaseScala {
}
// functions
+ it("Passed ST_LabelPoint") {
+ var geomDf = sparkSession.sql(
+ "SELECT ST_GeomFromWKT('POLYGON ((-112.637484 33.440546, -112.546852 33.477209, -112.489177 33.550488, -112.41777 33.751684, -111.956371 33.719707, -111.766868 33.616843, -111.775107 33.527595, -111.640533 33.504695, -111.440044 33.463462, -111.415326 33.374055, -111.514197 33.309809, -111.643279 33.222542, -111.893203 33.174278, -111.96461 33.250109, -112.123903 33.261593, -112.252985 33.35341, -112.406784 33.346527, -112.667694 33.316695, -112.637484 33.440546))') AS geom, 2.0 AS gridResolution, 0.2 AS goodnessThreshold")
+ var result = geomDf.select(
+ ST_LabelPoint(col("geom"), col("gridResolution"), col("goodnessThreshold")).as("geom"))
+ var actualResult = result.take(1)(0).get(0).asInstanceOf[Geometry].toText()
+ var expected = "POINT (-112.04278737349767 33.46420809489905)"
+ assertEquals(expected, actualResult)
+
+ geomDf = sparkSession.sql(
+ "SELECT ST_GeomFromWKT('GEOMETRYCOLLECTION(POLYGON ((-112.840785 33.435962, -112.840785 33.708284, -112.409597 33.708284, -112.409597 33.435962, -112.840785 33.435962)), POLYGON ((-112.309264 33.398167, -112.309264 33.746007, -111.787444 33.746007, -111.787444 33.398167, -112.309264 33.398167)))') AS geom")
+ geomDf.createOrReplaceTempView("geomDf")
+ result = geomDf.select(ST_LabelPoint(col("geom"), lit(1)).as("geom"))
+ actualResult = result.take(1)(0).get(0).asInstanceOf[Geometry].toText()
+ expected = "POINT (-112.04835399999999 33.57208699999999)"
+ assertEquals(expected, actualResult)
+
+ geomDf = sparkSession.sql(
+ "SELECT ST_GeomFromWKT('POLYGON ((-112.654072 33.114485, -112.313516 33.653431, -111.63515 33.314399, -111.497829 33.874913, -111.692825 33.431378, -112.376684 33.788215, -112.654072 33.114485))') AS geom, 0.01 AS goodnessThreshold")
+ geomDf.createOrReplaceTempView("geomDf")
+ result =
+ geomDf.select(ST_LabelPoint(col("geom"), lit(2), col("goodnessThreshold")).as("geom"))
+ actualResult = result.take(1)(0).get(0).asInstanceOf[Geometry].toText()
+ expected = "POINT (-112.0722602222832 33.53914975012836)"
+ assertEquals(expected, actualResult)
+
+ result = geomDf.select(ST_LabelPoint(col("geom")).as("geom"))
+ actualResult = result.take(1)(0).get(0).asInstanceOf[Geometry].toText()
+ expected = "POINT (-112.0722602222832 33.53914975012836)"
+ assertEquals(expected, actualResult)
+ }
+
it("Passed ST_ConcaveHull") {
val baseDF = sparkSession.sql(
"SELECT ST_GeomFromWKT('Polygon ((0 0, 1 2, 2 2, 3 2, 5 0, 4 0, 3 1, 2 1, 1 0, 0 0))') as mline")
@@ -779,6 +811,25 @@ class dataFrameAPITestScala extends TestBaseScala {
assertEquals(expected, actual)
}
+ it("Passed ST_Perimeter2D") {
+ var baseDf = sparkSession.sql(
+ "SELECT ST_GeomFromWKT('POLYGON((743238 2967416,743238 2967450,743265 2967450,743265.625 2967416,743238 2967416))') AS geom")
+ var actual = baseDf.select(ST_Perimeter2D("geom")).first().get(0)
+ var expected = 122.63074400009504
+ assertEquals(expected, actual)
+
+ baseDf = sparkSession.sql(
+ "SELECT ST_GeomFromWKT('POLYGON ((0 0, 0 1, 1 1, 1 0, 0 0))', 4326) AS geom")
+ actual = baseDf.select(ST_Perimeter2D("geom", use_spheroid = true)).first().get(0)
+ expected = 443770.91724830196
+ assertEquals(expected, actual)
+
+ actual =
+ baseDf.select(ST_Perimeter2D("geom", use_spheroid = true, lenient = false)).first().get(0)
+ expected = 443770.91724830196
+ assertEquals(expected, actual)
+ }
+
it("Passed ST_Project") {
val baseDf = sparkSession.sql(
"SELECT ST_GeomFromWKT('POINT(0 0)') as point, ST_MakeEnvelope(0, 1, 2, 0) as poly")
@@ -1399,6 +1450,22 @@ class dataFrameAPITestScala extends TestBaseScala {
assert(actualRadius == expectedRadius)
}
+ it("Passed ST_LineSegments") {
+ val baseDf = sparkSession.sql(
+ "SELECT ST_GeomFromWKT('LINESTRING(120 140, 60 120, 30 20)') AS line, ST_GeomFromWKT('POLYGON ((0 0, 0 1, 1 0, 0 0))') AS poly")
+ var resultSize = baseDf
+ .select(ST_LineSegments("line", false))
+ .first()
+ .getAs[WrappedArray[Geometry]](0)
+ .length
+ val expected = 2
+ assertEquals(expected, resultSize)
+
+ resultSize =
+ baseDf.select(ST_LineSegments("poly")).first().getAs[WrappedArray[Geometry]](0).length
+ assertEquals(0, resultSize)
+ }
+
it("Passed ST_LineSubstring") {
val baseDf = sparkSession.sql("SELECT ST_GeomFromWKT('LINESTRING (0 0, 2 0)') AS line")
val df = baseDf.select(ST_LineSubstring("line", 0.5, 1.0))
diff --git a/spark/common/src/test/scala/org/apache/sedona/sql/functionTestScala.scala b/spark/common/src/test/scala/org/apache/sedona/sql/functionTestScala.scala
index 82dbff82c1..84770157c7 100644
--- a/spark/common/src/test/scala/org/apache/sedona/sql/functionTestScala.scala
+++ b/spark/common/src/test/scala/org/apache/sedona/sql/functionTestScala.scala
@@ -49,6 +49,36 @@ class functionTestScala
describe("Sedona-SQL Function Test") {
+ it("Passed ST_LabelPoint") {
+ var geomDf = sparkSession.sql(
+ "SELECT ST_GeomFromWKT('POLYGON ((-112.637484 33.440546, -112.546852 33.477209, -112.489177 33.550488, -112.41777 33.751684, -111.956371 33.719707, -111.766868 33.616843, -111.775107 33.527595, -111.640533 33.504695, -111.440044 33.463462, -111.415326 33.374055, -111.514197 33.309809, -111.643279 33.222542, -111.893203 33.174278, -111.96461 33.250109, -112.123903 33.261593, -112.252985 33.35341, -112.406784 33.346527, -112.667694 33.316695, -112.637484 33.440546))') AS geom, 2 AS gridResolution, 0.2 AS GoodnessThreshold")
+ geomDf.createOrReplaceTempView("geomDf")
+ var result =
+ sparkSession.sql(
+ "SELECT ST_AsEWKT(ST_LabelPoint(geom, gridResolution, goodnessThreshold)) FROM geomDf")
+ var expected = "POINT (-112.04278737349767 33.46420809489905)"
+ assertEquals(expected, result.take(1)(0).get(0).asInstanceOf[String])
+
+ geomDf = sparkSession.sql(
+ "SELECT ST_GeomFromWKT('GEOMETRYCOLLECTION(POLYGON ((-112.840785 33.435962, -112.840785 33.708284, -112.409597 33.708284, -112.409597 33.435962, -112.840785 33.435962)), POLYGON ((-112.309264 33.398167, -112.309264 33.746007, -111.787444 33.746007, -111.787444 33.398167, -112.309264 33.398167)))') AS geom")
+ geomDf.createOrReplaceTempView("geomDf")
+ result = sparkSession.sql("SELECT ST_AsEWKT(ST_LabelPoint(geom, 1)) FROM geomDf")
+ expected = "POINT (-112.04835399999999 33.57208699999999)"
+ assertEquals(expected, result.take(1)(0).get(0).asInstanceOf[String])
+
+ geomDf = sparkSession.sql(
+ "SELECT ST_GeomFromWKT('POLYGON ((-112.654072 33.114485, -112.313516 33.653431, -111.63515 33.314399, -111.497829 33.874913, -111.692825 33.431378, -112.376684 33.788215, -112.654072 33.114485))') AS geom, 0.01 AS goodnessThreshold")
+ geomDf.createOrReplaceTempView("geomDf")
+ result = sparkSession.sql(
+ "SELECT ST_AsEWKT(ST_LabelPoint(geom, 2, goodnessThreshold)) FROM geomDf")
+ expected = "POINT (-112.0722602222832 33.53914975012836)"
+ assertEquals(expected, result.take(1)(0).get(0).asInstanceOf[String])
+
+ result = sparkSession.sql("SELECT ST_AsEWKT(ST_LabelPoint(geom)) FROM geomDf")
+ expected = "POINT (-112.0722602222832 33.53914975012836)"
+ assertEquals(expected, result.take(1)(0).get(0).asInstanceOf[String])
+ }
+
it("Passed ST_ConcaveHull") {
var polygonWktDf = sparkSession.read
.format("csv")
@@ -613,6 +643,24 @@ class functionTestScala
assertEquals(expected, actual)
}
+ it("Passed ST_Perimeter2D") {
+ var baseDf = sparkSession.sql(
+ "SELECT ST_GeomFromWKT('POLYGON((743238 2967416,743238 2967450,743265 2967450,743265.625 2967416,743238 2967416))') AS geom")
+ var actual = baseDf.selectExpr("ST_Perimeter2D(geom)").first().get(0)
+ var expected = 122.63074400009504
+ assertEquals(expected, actual)
+
+ baseDf = sparkSession.sql(
+ "SELECT ST_GeomFromWKT('POLYGON ((0 0, 0 1, 1 1, 1 0, 0 0))', 4326) AS geom")
+ actual = baseDf.selectExpr("ST_Perimeter2D(geom, true)").first().get(0)
+ expected = 443770.91724830196
+ assertEquals(expected, actual)
+
+ actual = baseDf.selectExpr("ST_Perimeter2D(geom, true, false)").first().get(0)
+ expected = 443770.91724830196
+ assertEquals(expected, actual)
+ }
+
it("Passed ST_Points") {
val testtable = sparkSession.sql(
@@ -1900,6 +1948,13 @@ class functionTestScala
.get(0)
expected = "LINESTRING (0 0, 5 5, 2 2)"
assertEquals(expected, actual)
+
+ actual = sparkSession
+ .sql("SELECT ST_AsText(ST_RemoveRepeatedPoints(ST_GeomFromWKT('POLYGON ((40 40, 70 70, 70 70, 40 40))')))")
+ .first()
+ .get(0)
+ expected = "POLYGON ((40 40, 70 70, 70 70, 40 40))"
+ assertEquals(expected, actual)
}
it("Should correctly set using ST_SetPoint") {
@@ -2293,6 +2348,17 @@ class functionTestScala
.toList should contain theSameElementsAs List(0, 1, 1)
}
+ it("Should pass ST_LineSegments") {
+ val baseDf = sparkSession.sql(
+ "SELECT ST_GeomFromWKT('LINESTRING(120 140, 60 120, 30 20)') AS line, ST_GeomFromWKT('POLYGON ((0 0, 0 1, 1 0, 0 0))') AS poly")
+ var resultSize = baseDf.selectExpr("array_size(ST_LineSegments(line, false))").first().get(0)
+ val expected = 2
+ assertEquals(expected, resultSize)
+
+ resultSize = baseDf.selectExpr("array_size(ST_LineSegments(poly))").first().get(0)
+ assertEquals(0, resultSize)
+ }
+
it("Should pass ST_LineSubstring") {
Given("Sample geometry dataframe")
val geometryTable = Seq("LINESTRING(25 50, 100 125, 150 190)")
diff --git a/spark/common/src/test/scala/org/apache/sedona/sql/structuredAdapterTestScala.scala b/spark/common/src/test/scala/org/apache/sedona/sql/structuredAdapterTestScala.scala
new file mode 100644
index 0000000000..d258ce3b40
--- /dev/null
+++ b/spark/common/src/test/scala/org/apache/sedona/sql/structuredAdapterTestScala.scala
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.sedona.sql
+
+import org.apache.sedona.core.enums.{GridType, IndexType}
+import org.apache.sedona.core.spatialOperator.{JoinQuery, SpatialPredicate}
+import org.apache.sedona.core.spatialRDD.CircleRDD
+import org.apache.spark.sql.functions.spark_partition_id
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.sedona_sql.adapters.StructuredAdapter
+import org.junit.Assert.assertEquals
+import org.scalatest.GivenWhenThen
+
+class structuredAdapterTestScala extends TestBaseScala with GivenWhenThen {
+
+ describe("Structured Adapter") {
+ it("Should convert DataFrame to SpatialRDD and back") {
+ val seq = generateTestData()
+ val geom1 = seq.head._3
+ val dfOrigin = sparkSession.createDataFrame(seq)
+ val rdd = StructuredAdapter.toSpatialRdd(dfOrigin, "_3")
+ assertGeometryEquals(geom1, rdd.rawSpatialRDD.take(1).get(0))
+ val dfConverted = StructuredAdapter.toDf(rdd, sparkSession)
+ intercept[RuntimeException] {
+ StructuredAdapter.toSpatialPartitionedDf(rdd, sparkSession)
+ }
+ assertEquals(seq.size, dfConverted.count())
+ }
+
+ it("Should convert DataFrame to SpatialRDD and back, without specifying geometry column") {
+ val seq = generateTestData()
+ val geom1 = seq.head._3
+ val dfOrigin = sparkSession.createDataFrame(seq)
+ val rdd = StructuredAdapter.toSpatialRdd(dfOrigin)
+ assertGeometryEquals(geom1, rdd.rawSpatialRDD.take(1).get(0))
+ val dfConverted = StructuredAdapter.toDf(rdd, sparkSession)
+ intercept[RuntimeException] {
+ StructuredAdapter.toSpatialPartitionedDf(rdd, sparkSession)
+ }
+ assertEquals(seq.size, dfConverted.count())
+ }
+
+ it("Should convert to Rdd and do spatial partitioning") {
+ val seq = generateTestData()
+ val dfOrigin = sparkSession.createDataFrame(seq)
+ val rdd = StructuredAdapter.toSpatialRdd(dfOrigin, "_3")
+ rdd.analyze()
+ rdd.spatialPartitioning(GridType.KDBTREE, 10)
+ val dfConverted = StructuredAdapter.toSpatialPartitionedDf(rdd, sparkSession)
+ assertEquals(seq.size, dfConverted.count())
+ }
+
+ it("Should convert a spatial join result back to DataFrame") {
+ val pointRdd =
+ StructuredAdapter.toSpatialRdd(sparkSession.createDataFrame(generateTestData()))
+ val circleRDD = new CircleRDD(pointRdd, 0.0001)
+ circleRDD.analyze()
+ pointRdd.analyze()
+ circleRDD.spatialPartitioning(GridType.KDBTREE)
+ pointRdd.spatialPartitioning(circleRDD.getPartitioner)
+ circleRDD.buildIndex(IndexType.QUADTREE, true)
+ val pairRdd =
+ JoinQuery.DistanceJoinQueryFlat(pointRdd, circleRDD, true, SpatialPredicate.INTERSECTS)
+ var resultDf =
+ StructuredAdapter.toDf(pairRdd, pointRdd.schema, pointRdd.schema, sparkSession)
+ assertEquals(pointRdd.rawSpatialRDD.count(), resultDf.count())
+ resultDf =
+ StructuredAdapter.toDf(pairRdd, pointRdd.schema.json, pointRdd.schema.json, sparkSession)
+ assertEquals(pointRdd.rawSpatialRDD.count(), resultDf.count())
+ }
+
+ it("Should convert a SpatialRdd to RowRdd and back") {
+ val seq = generateTestData()
+ val dfOrigin = sparkSession.createDataFrame(seq)
+ val spatialRdd = StructuredAdapter.toSpatialRdd(dfOrigin.rdd)
+ val rowRdd = StructuredAdapter.toRowRdd(spatialRdd)
+ assertEquals(seq.size, StructuredAdapter.toSpatialRdd(rowRdd).rawSpatialRDD.count())
+ }
+
+ it("Should not be able to convert an empty Row RDD to SpatialRDD if schema is not provided") {
+ val rdd = sparkSession.sparkContext.parallelize(Seq.empty[Row])
+ intercept[IllegalArgumentException] {
+ StructuredAdapter.toSpatialRdd(rdd)
+ }
+ }
+
+ it("Should convert an empty Row RDD to SpatialRDD if schema is provided") {
+ val rdd = sparkSession.sparkContext.parallelize(Seq.empty[Row])
+ val spatialRdd = StructuredAdapter.toSpatialRdd(rdd, null)
+ assertEquals(0, spatialRdd.rawSpatialRDD.count())
+ assertEquals(0, spatialRdd.schema.size)
+ }
+
+ it("can convert spatial RDD to Dataframe preserving spatial partitioning") {
+ var pointCsvDF = sparkSession.read
+ .format("csv")
+ .option("delimiter", ",")
+ .option("header", "false")
+ .load(csvPointInputLocation)
+ pointCsvDF.createOrReplaceTempView("pointtable")
+ var pointDf = sparkSession.sql(
+ "select ST_Point(cast(pointtable._c0 as Decimal(24,20)), cast(pointtable._c1 as Decimal(24,20))) as arealandmark from pointtable")
+ var srcRdd = StructuredAdapter.toSpatialRdd(pointDf, "arealandmark")
+ srcRdd.analyze()
+ srcRdd.spatialPartitioning(GridType.KDBTREE, 16)
+ var numSpatialPartitions = srcRdd.spatialPartitionedRDD.getNumPartitions
+ assert(numSpatialPartitions >= 16)
+
+ var partitionedDF = StructuredAdapter.toSpatialPartitionedDf(srcRdd, sparkSession)
+ val dfPartitions: Long = partitionedDF.select(spark_partition_id).distinct().count()
+ assert(dfPartitions == numSpatialPartitions)
+ }
+ }
+}
diff --git a/spark/common/src/test/scala/org/apache/spark/sql/sedona_sql/io/stac/StacBatchTest.scala b/spark/common/src/test/scala/org/apache/spark/sql/sedona_sql/io/stac/StacBatchTest.scala
new file mode 100644
index 0000000000..0765b3950f
--- /dev/null
+++ b/spark/common/src/test/scala/org/apache/spark/sql/sedona_sql/io/stac/StacBatchTest.scala
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.sedona_sql.io.stac
+
+import org.apache.sedona.sql.TestBaseScala
+import org.apache.spark.sql.connector.read.InputPartition
+import org.apache.spark.sql.types.StructType
+
+import scala.io.Source
+import scala.collection.mutable
+
+class StacBatchTest extends TestBaseScala {
+
+ def loadJsonFromResource(resourceFilePath: String): String = {
+ Source.fromResource(resourceFilePath).getLines().mkString("\n")
+ }
+
+ def getAbsolutePathOfResource(resourceFilePath: String): String = {
+ val resourceUrl = getClass.getClassLoader.getResource(resourceFilePath)
+ if (resourceUrl != null) {
+ resourceUrl.getPath
+ } else {
+ throw new IllegalArgumentException(s"Resource not found: $resourceFilePath")
+ }
+ }
+
+ it("planInputPartitions should create correct number of partitions") {
+ val stacCollectionJson =
+ """
+ |{
+ | "stac_version": "1.0.0",
+ | "id": "sample-collection",
+ | "description": "A sample STAC collection",
+ | "links": [
+ | {"rel": "item", "href": "https://path/to/item1.json"},
+ | {"rel": "item", "href": "https://path/to/item2.json"},
+ | {"rel": "item", "href": "https://path/to/item3.json"}
+ | ]
+ |}
+ """.stripMargin
+
+ val opts = mutable.Map("numPartitions" -> "2").toMap
+ val collectionUrl = "https://path/to/collection.json"
+
+ val stacBatch =
+ StacBatch(collectionUrl, stacCollectionJson, StructType(Seq()), opts, None, None)
+ val partitions: Array[InputPartition] = stacBatch.planInputPartitions()
+
+ assert(partitions.length == 2)
+ assert(partitions(0).asInstanceOf[StacPartition].items.length == 2)
+ assert(partitions(1).asInstanceOf[StacPartition].items.length == 1)
+ }
+
+ it("planInputPartitions should handle empty links array") {
+ val stacCollectionJson =
+ """
+ |{
+ | "links": []
+ |}
+ """.stripMargin
+
+ val opts = mutable.Map("numPartitions" -> "2").toMap
+ val collectionUrl = "https://path/to/collection.json"
+
+ val stacBatch =
+ StacBatch(collectionUrl, stacCollectionJson, StructType(Seq()), opts, None, None)
+ val partitions: Array[InputPartition] = stacBatch.planInputPartitions()
+
+ assert(partitions.isEmpty)
+ }
+
+ it("planInputPartitions should create correct number of partitions with real collection.json") {
+ val rootJsonFile = "datasource_stac/collection.json"
+ val stacCollectionJson = loadJsonFromResource(rootJsonFile)
+ val opts = mutable.Map("numPartitions" -> "3").toMap
+ val collectionUrl = getAbsolutePathOfResource(rootJsonFile)
+
+ val stacBatch =
+ StacBatch(collectionUrl, stacCollectionJson, StructType(Seq()), opts, None, None)
+ val partitions: Array[InputPartition] = stacBatch.planInputPartitions()
+
+ assert(partitions.length == 3)
+ assert(partitions(0).asInstanceOf[StacPartition].items.length == 2)
+ assert(partitions(1).asInstanceOf[StacPartition].items.length == 2)
+ assert(partitions(2).asInstanceOf[StacPartition].items.length == 1)
+ }
+}
diff --git a/spark/common/src/test/scala/org/apache/spark/sql/sedona_sql/io/stac/StacDataSourceTest.scala b/spark/common/src/test/scala/org/apache/spark/sql/sedona_sql/io/stac/StacDataSourceTest.scala
new file mode 100644
index 0000000000..a1234ffa11
--- /dev/null
+++ b/spark/common/src/test/scala/org/apache/spark/sql/sedona_sql/io/stac/StacDataSourceTest.scala
@@ -0,0 +1,209 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.sedona_sql.io.stac
+
+import org.apache.sedona.sql.TestBaseScala
+import org.apache.spark.sql.sedona_sql.UDT.{GeometryUDT, RasterUDT}
+import org.apache.spark.sql.types.{ArrayType, DoubleType, MapType, StringType, StructField, StructType, TimestampType}
+import org.scalatest.BeforeAndAfterAll
+
+import java.util.TimeZone
+
+class StacDataSourceTest extends TestBaseScala {
+
+ val STAC_COLLECTION_LOCAL: String = resourceFolder + "datasource_stac/collection.json"
+ val STAC_ITEM_LOCAL: String = resourceFolder + "geojson/core-item.json"
+
+ val STAC_COLLECTION_REMOTE: List[String] = List(
+ "https://earth-search.aws.element84.com/v1/collections/sentinel-2-pre-c1-l2a",
+ "https://storage.googleapis.com/cfo-public/vegetation/collection.json",
+ "https://storage.googleapis.com/cfo-public/wildfire/collection.json",
+ "https://earthdatahub.destine.eu/api/stac/v1/collections/copernicus-dem",
+ "https://planetarycomputer.microsoft.com/api/stac/v1/collections/naip")
+
+ it("basic df load from local file should work") {
+ val dfStac = sparkSession.read.format("stac").load(STAC_COLLECTION_LOCAL)
+ val rowCount = dfStac.count()
+ assert(rowCount > 0)
+ }
+
+ it("basic df load from remote service endpoints should work") {
+ STAC_COLLECTION_REMOTE.foreach { endpoint =>
+ val dfStac = sparkSession.read.format("stac").load(endpoint)
+ assertSchema(dfStac.schema)
+ }
+ }
+
+ it("normal select SQL without any filter") {
+ val dfStac = sparkSession.read.format("stac").load(STAC_COLLECTION_LOCAL)
+ dfStac.createOrReplaceTempView("STACTBL")
+
+ val dfSelect =
+ sparkSession.sql("SELECT id, datetime as dt, geometry, bbox FROM STACTBL")
+
+ assert(dfSelect.schema.fieldNames.contains("id"))
+ assert(dfSelect.schema.fieldNames.contains("dt"))
+ assert(dfSelect.schema.fieldNames.contains("geometry"))
+ assert(dfSelect.schema.fieldNames.contains("bbox"))
+
+ val rowCount = dfSelect.count()
+ assert(rowCount == 6)
+ }
+
+ it("select SQL with filter on datetime") {
+ val dfStac = sparkSession.read.format("stac").load(STAC_COLLECTION_LOCAL)
+ dfStac.createOrReplaceTempView("STACTBL")
+
+ val dfSelect = sparkSession.sql(
+ "SELECT id, datetime as dt, geometry, bbox " +
+ "FROM STACTBL " +
+ "WHERE datetime BETWEEN '2020-01-01T00:00:00Z' AND '2020-12-13T00:00:00Z'")
+
+ val physicalPlan = dfSelect.queryExecution.executedPlan.toString()
+ assert(physicalPlan.contains(
+ "PushedTemporalFilters -> AndFilter(GreaterThanFilter(datetime,2020-01-01T00:00),LessThanFilter(datetime,2020-12-13T00:00))"))
+
+ val rowCount = dfSelect.count()
+ assert(rowCount == 4)
+ }
+
+ it("select SQL with spatial filter") {
+ val dfStac = sparkSession.read.format("stac").load(STAC_COLLECTION_LOCAL)
+ dfStac.createOrReplaceTempView("STACTBL")
+
+ val dfSelect = sparkSession.sql(
+ "SELECT id, geometry " +
+ "FROM STACTBL " +
+ "WHERE st_contains(ST_GeomFromText('POLYGON((17 10, 18 10, 18 11, 17 11, 17 10))'), geometry)")
+
+ val physicalPlan = dfSelect.queryExecution.executedPlan.toString()
+ assert(physicalPlan.contains(
+ "PushedSpatialFilters -> LeafFilter(geometry,INTERSECTS,POLYGON ((17 10, 18 10, 18 11, 17 11, 17 10)))"))
+
+ val rowCount = dfSelect.count()
+ assert(rowCount == 3)
+ }
+
+ it("select SQL with both spatial and temporal filters") {
+ val dfStac = sparkSession.read.format("stac").load(STAC_COLLECTION_LOCAL)
+ dfStac.createOrReplaceTempView("STACTBL")
+
+ val dfSelect = sparkSession.sql("SELECT id, datetime as dt, geometry, bbox " +
+ "FROM STACTBL " +
+ "WHERE datetime BETWEEN '2020-01-01T00:00:00Z' AND '2020-12-13T00:00:00Z' " +
+ "AND st_contains(ST_GeomFromText('POLYGON((17 10, 18 10, 18 11, 17 11, 17 10))'), geometry)")
+
+ val physicalPlan = dfSelect.queryExecution.executedPlan.toString()
+ assert(physicalPlan.contains(
+ "PushedSpatialFilters -> LeafFilter(geometry,INTERSECTS,POLYGON ((17 10, 18 10, 18 11, 17 11, 17 10)))"))
+ assert(physicalPlan.contains(
+ "PushedTemporalFilters -> AndFilter(GreaterThanFilter(datetime,2020-01-01T00:00),LessThanFilter(datetime,2020-12-13T00:00))"))
+
+ val rowCount = dfSelect.count()
+ assert(rowCount == 3)
+ }
+
+ it("select SQL with regular filter on id") {
+ val dfStac = sparkSession.read.format("stac").load(STAC_COLLECTION_LOCAL)
+ dfStac.createOrReplaceTempView("STACTBL")
+
+ val dfSelect = sparkSession.sql(
+ "SELECT id, datetime as dt, geometry, bbox " +
+ "FROM STACTBL " +
+ "WHERE id = 'some-id'")
+
+ val physicalPlan = dfSelect.queryExecution.executedPlan.toString()
+ assert(physicalPlan.contains("PushedSpatialFilters -> None, PushedTemporalFilters -> None"))
+
+ val rowCount = dfSelect.count()
+ assert(rowCount == 0)
+ }
+
+ it("select SQL with regular, spatial, and temporal filters") {
+ val dfStac = sparkSession.read.format("stac").load(STAC_COLLECTION_LOCAL)
+ dfStac.createOrReplaceTempView("STACTBL")
+
+ val dfSelect = sparkSession.sql("SELECT id, datetime as dt, geometry, bbox " +
+ "FROM STACTBL " +
+ "WHERE id = 'some-id' " +
+ "AND datetime BETWEEN '2020-01-01T00:00:00Z' AND '2020-12-13T00:00:00Z' " +
+ "AND st_contains(ST_GeomFromText('POLYGON((17 10, 18 10, 18 11, 17 11, 17 10))'), geometry)")
+
+ val physicalPlan = dfSelect.queryExecution.executedPlan.toString()
+ assert(physicalPlan.contains(
+ "PushedSpatialFilters -> LeafFilter(geometry,INTERSECTS,POLYGON ((17 10, 18 10, 18 11, 17 11, 17 10)))"))
+ assert(physicalPlan.contains(
+ "PushedTemporalFilters -> AndFilter(GreaterThanFilter(datetime,2020-01-01T00:00),LessThanFilter(datetime,2020-12-13T00:00))"))
+
+ val rowCount = dfSelect.count()
+ assert(rowCount == 0)
+ }
+
+ def assertSchema(actualSchema: StructType): Unit = {
+ val expectedSchema = StructType(
+ Seq(
+ StructField("stac_version", StringType, nullable = false),
+ StructField(
+ "stac_extensions",
+ ArrayType(StringType, containsNull = true),
+ nullable = true),
+ StructField("type", StringType, nullable = false),
+ StructField("id", StringType, nullable = false),
+ StructField("bbox", ArrayType(DoubleType, containsNull = true), nullable = true),
+ StructField("geometry", new GeometryUDT(), nullable = true),
+ StructField("title", StringType, nullable = true),
+ StructField("description", StringType, nullable = true),
+ StructField("datetime", TimestampType, nullable = true),
+ StructField("start_datetime", TimestampType, nullable = true),
+ StructField("end_datetime", TimestampType, nullable = true),
+ StructField("created", TimestampType, nullable = true),
+ StructField("updated", TimestampType, nullable = true),
+ StructField("platform", StringType, nullable = true),
+ StructField("instruments", ArrayType(StringType, containsNull = true), nullable = true),
+ StructField("constellation", StringType, nullable = true),
+ StructField("mission", StringType, nullable = true),
+ StructField("gsd", DoubleType, nullable = true),
+ StructField("collection", StringType, nullable = true),
+ StructField(
+ "links",
+ ArrayType(
+ StructType(Seq(
+ StructField("rel", StringType, nullable = true),
+ StructField("href", StringType, nullable = true),
+ StructField("type", StringType, nullable = true),
+ StructField("title", StringType, nullable = true))),
+ containsNull = true),
+ nullable = true),
+ StructField(
+ "assets",
+ MapType(
+ StringType,
+ StructType(Seq(
+ StructField("href", StringType, nullable = true),
+ StructField("type", StringType, nullable = true),
+ StructField("title", StringType, nullable = true),
+ StructField("roles", ArrayType(StringType, containsNull = true), nullable = true))),
+ valueContainsNull = true),
+ nullable = true)))
+
+ assert(
+ actualSchema == expectedSchema,
+ s"Schema does not match. Expected: $expectedSchema, Actual: $actualSchema")
+ }
+}
diff --git a/spark/common/src/test/scala/org/apache/spark/sql/sedona_sql/io/stac/StacPartitionReaderTest.scala b/spark/common/src/test/scala/org/apache/spark/sql/sedona_sql/io/stac/StacPartitionReaderTest.scala
new file mode 100644
index 0000000000..fc6f8dcfdc
--- /dev/null
+++ b/spark/common/src/test/scala/org/apache/spark/sql/sedona_sql/io/stac/StacPartitionReaderTest.scala
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.sedona_sql.io.stac
+
+import org.apache.sedona.sql.TestBaseScala
+import org.apache.spark.sql.catalyst.InternalRow
+
+import scala.jdk.CollectionConverters._
+
+class StacPartitionReaderTest extends TestBaseScala {
+
+ val TEST_DATA_FOLDER: String =
+ System.getProperty("user.dir") + "/src/test/resources/datasource_stac"
+ val JSON_STAC_ITEM_SIMPLE: String = s"file://$TEST_DATA_FOLDER/simple-item.json"
+ val JSON_STAC_ITEM_CORE: String = s"file://$TEST_DATA_FOLDER/core-item.json"
+ val JSON_STAC_ITEM_EXTENDED: String = s"file://$TEST_DATA_FOLDER/extended-item.json"
+ val JSON_STAC_ITEM_FEATURES: String = s"file://$TEST_DATA_FOLDER/collection-items.json"
+ val HTTPS_STAC_ITEM_FEATURES: String =
+ "https://earth-search.aws.element84.com/v1/collections/sentinel-2-pre-c1-l2a/items"
+
+ it("StacPartitionReader should read feature files from local files") {
+ val jsonFiles =
+ Seq(JSON_STAC_ITEM_SIMPLE, JSON_STAC_ITEM_CORE, JSON_STAC_ITEM_EXTENDED).toArray
+ val partition = StacPartition(0, jsonFiles, Map.empty[String, String].asJava)
+ val reader =
+ new StacPartitionReader(
+ partition,
+ StacTable.SCHEMA_V1_1_0,
+ Map.empty[String, String],
+ None,
+ None)
+
+ assert(reader.next())
+ (1 to 3).foreach { i =>
+ val row: InternalRow = reader.get()
+ assert(row != null)
+ assert(reader.next() == (i < 3))
+ }
+
+ reader.close()
+ }
+
+ it("StacPartitionReader should read features collection file from local files") {
+ val jsonFiles = Seq(JSON_STAC_ITEM_FEATURES).toArray
+ val partition = StacPartition(0, jsonFiles, Map.empty[String, String].asJava)
+ val reader =
+ new StacPartitionReader(
+ partition,
+ StacTable.SCHEMA_V1_1_0,
+ Map.empty[String, String],
+ None,
+ None)
+
+ assert(reader.next())
+ (1 to 10).foreach { i =>
+ val row: InternalRow = reader.get()
+ assert(row != null)
+ assert(reader.next() == (i < 10))
+ }
+
+ reader.close()
+ }
+
+ it("StacPartitionReader should read features collection file from https endpoint") {
+ val jsonFiles = Seq(HTTPS_STAC_ITEM_FEATURES).toArray
+ val partition = StacPartition(0, jsonFiles, Map.empty[String, String].asJava)
+ val reader =
+ new StacPartitionReader(
+ partition,
+ StacTable.SCHEMA_V1_1_0,
+ Map.empty[String, String],
+ None,
+ None)
+
+ assert(reader.next())
+ (1 to 10).foreach { i =>
+ val row: InternalRow = reader.get()
+ assert(row != null)
+ assert(reader.next() == (i < 10))
+ }
+
+ reader.close()
+ }
+}
diff --git a/spark/common/src/test/scala/org/apache/spark/sql/sedona_sql/io/stac/StacTableTest.scala b/spark/common/src/test/scala/org/apache/spark/sql/sedona_sql/io/stac/StacTableTest.scala
new file mode 100644
index 0000000000..eca7768733
--- /dev/null
+++ b/spark/common/src/test/scala/org/apache/spark/sql/sedona_sql/io/stac/StacTableTest.scala
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.sedona_sql.io.stac
+
+import org.apache.spark.sql.sedona_sql.io.stac.StacTable.{SCHEMA_GEOPARQUET, addAssetStruct, addAssetsStruct}
+import org.apache.spark.sql.types.{ArrayType, MapType, StringType, StructField, StructType}
+import org.scalatest.funsuite.AnyFunSuite
+
+class StacTableTest extends AnyFunSuite {
+
+ test("addAssetStruct should add a new asset to an existing assets struct") {
+ val initialSchema = StructType(
+ Seq(
+ StructField("id", StringType, nullable = false),
+ StructField(
+ "assets",
+ StructType(Seq(StructField(
+ "image",
+ StructType(Seq(
+ StructField("href", StringType, nullable = true),
+ StructField("roles", ArrayType(StringType), nullable = true),
+ StructField("title", StringType, nullable = true),
+ StructField("type", StringType, nullable = true))),
+ nullable = true))),
+ nullable = true)))
+
+ val updatedSchema = addAssetStruct(initialSchema, "thumbnail")
+
+ assert(updatedSchema.fieldNames.contains("assets"))
+ val assetsField = updatedSchema("assets").dataType.asInstanceOf[StructType]
+ assert(assetsField.fieldNames.contains("thumbnail"))
+ }
+
+ test("addAssetStruct should create assets struct if it doesn't exist") {
+ val initialSchema = StructType(Seq(StructField("id", StringType, nullable = false)))
+
+ val updatedSchema1 = addAssetStruct(initialSchema, "image")
+ val updatedSchema2 = addAssetStruct(updatedSchema1, "rast")
+
+ assert(updatedSchema2.fieldNames.contains("assets"))
+ val assetsField = updatedSchema2("assets").dataType.asInstanceOf[StructType]
+ assert(assetsField.fieldNames.contains("image"))
+ assert(assetsField.fieldNames.contains("rast"))
+ }
+
+ test("addAssetStruct should not modify other fields") {
+ val initialSchema = SCHEMA_GEOPARQUET
+ val updatedSchema = addAssetsStruct(initialSchema, Array("thumbnail", "preview"))
+
+ assert(updatedSchema.fieldNames.contains("id"))
+ assert(updatedSchema.fieldNames.contains("stac_version"))
+ assert(updatedSchema.fieldNames.contains("stac_extensions"))
+ assert(updatedSchema.fieldNames.contains("bbox"))
+ assert(updatedSchema.fieldNames.contains("geometry"))
+ assert(updatedSchema.fieldNames.contains("assets"))
+ }
+}
diff --git a/spark/common/src/test/scala/org/apache/spark/sql/sedona_sql/io/stac/StacUtilsTest.scala b/spark/common/src/test/scala/org/apache/spark/sql/sedona_sql/io/stac/StacUtilsTest.scala
new file mode 100644
index 0000000000..75542c760d
--- /dev/null
+++ b/spark/common/src/test/scala/org/apache/spark/sql/sedona_sql/io/stac/StacUtilsTest.scala
@@ -0,0 +1,594 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.sedona_sql.io.stac
+
+import com.fasterxml.jackson.databind.{JsonNode, ObjectMapper}
+import com.fasterxml.jackson.module.scala.DefaultScalaModule
+import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.sedona_sql.io.stac.StacUtils.getNumPartitions
+import org.scalatest.funsuite.AnyFunSuite
+
+import java.io.{File, PrintWriter}
+import scala.io.Source
+import scala.jdk.CollectionConverters._
+
+class StacUtilsTest extends AnyFunSuite {
+
+ test("getStacCollectionBasePath should return base URL for HTTP URL") {
+ val opts = Map("path" -> "https://service_url/collections/collection.json")
+ val result = StacUtils.getStacCollectionBasePath(opts)
+ assert(result == "https://service_url/")
+ }
+
+ test("getStacCollectionBasePath should return base URL for HTTPS URL") {
+ val opts = Map("path" -> "https://service_url/collections/collection.json")
+ val result = StacUtils.getStacCollectionBasePath(opts)
+ assert(result == "https://service_url/")
+ }
+
+ test("getStacCollectionBasePath should return base path for file URL") {
+ val opts = Map("path" -> "file:///usr/opt/collection.json")
+ val result = StacUtils.getStacCollectionBasePath(opts)
+ assert(result == "file:///usr/opt/")
+ }
+
+ test("getStacCollectionBasePath should return base path for local file path") {
+ val opts = Map("path" -> "/usr/opt/collection.json")
+ val result = StacUtils.getStacCollectionBasePath(opts)
+ assert(result == "file:///usr/opt/")
+ }
+
+ test(
+ "getStacCollectionBasePath should throw IllegalArgumentException if neither url nor service is provided") {
+ val opts = Map.empty[String, String]
+ assertThrows[IllegalArgumentException] {
+ StacUtils.getStacCollectionBasePath(opts)
+ }
+ }
+
+ test(
+ "getStacCollectionBasePath should throw IllegalArgumentException for invalid URL or file path") {
+ val opts = Map("path" -> "invalid_path")
+ assertThrows[IllegalArgumentException] {
+ StacUtils.getStacCollectionBasePath(opts)
+ }
+ }
+
+ test("getNumPartitions should return numPartitions if it is greater than 0") {
+ assert(
+ getNumPartitions(
+ itemCount = 100,
+ numPartitions = 5,
+ maxPartitionItemFiles = 10,
+ defaultParallelism = 4) == 5)
+ }
+
+ test(
+ "getNumPartitions should calculate partitions based on maxPartitionItemFiles and defaultParallelism") {
+ assert(
+ getNumPartitions(
+ itemCount = 100,
+ numPartitions = 0,
+ maxPartitionItemFiles = 10,
+ defaultParallelism = 4) == 10)
+ }
+
+ test(
+ "getNumPartitions should handle case when maxPartitionItemFiles is less than sum of files / defaultParallelism") {
+ assert(
+ getNumPartitions(
+ itemCount = 100,
+ numPartitions = 0,
+ maxPartitionItemFiles = 5,
+ defaultParallelism = 4) == 20)
+ }
+
+ test("getNumPartitions should handle case when maxPartitionItemFiles is 0") {
+ assert(
+ getNumPartitions(
+ itemCount = 100,
+ numPartitions = 0,
+ maxPartitionItemFiles = 0,
+ defaultParallelism = 4) == 4)
+ }
+
+ test("getNumPartitions should handle case when defaultParallelism is 1") {
+ assert(
+ getNumPartitions(
+ itemCount = 100,
+ numPartitions = 0,
+ maxPartitionItemFiles = 10,
+ defaultParallelism = 1) == 10)
+ }
+
+ test("getNumPartitions should return at least 1 partition") {
+ assert(
+ getNumPartitions(
+ itemCount = 0,
+ numPartitions = 0,
+ maxPartitionItemFiles = 10,
+ defaultParallelism = 4) == 1)
+ }
+
+ test(
+ "processStacCollection should process STAC collection from JSON string and save features to output file") {
+ val spark = SparkSession.builder().master("local").appName("StacUtilsTest").getOrCreate()
+ val hadoopConf = spark.sparkContext.hadoopConfiguration
+ val fs = FileSystem.get(hadoopConf)
+
+ // Create a temporary STAC collection JSON file
+ val stacCollectionJson =
+ """
+ |{
+ | "stac_version": "1.0.0",
+ | "id": "sample-collection",
+ | "description": "A sample STAC collection",
+ | "links": [
+ | {"rel": "item", "href": "file:///tmp/item1.json"},
+ | {"rel": "item", "href": "file:///tmp/item2.json"}
+ | ]
+ |}
+ """.stripMargin
+ val stacCollectionPath = new Path("/tmp/collection.json")
+ val stacCollectionWriter = new PrintWriter(new File(stacCollectionPath.toString))
+ stacCollectionWriter.write(stacCollectionJson)
+ stacCollectionWriter.close()
+
+ // Create temporary item JSON files
+ val item1Json =
+ """
+ |{
+ | "stac_version": "1.1.0",
+ | "stac_extensions": [],
+ | "type": "Feature",
+ | "id": "20201211_223832_CS2_item1",
+ | "bbox": [
+ | 172.91173669923782,
+ | 1.3438851951615003,
+ | 172.95469614953714,
+ | 1.3690476620161975
+ | ],
+ | "geometry": {
+ | "type": "Polygon",
+ | "coordinates": [
+ | [
+ | [
+ | 172.91173669923782,
+ | 1.3438851951615003
+ | ],
+ | [
+ | 172.95469614953714,
+ | 1.3438851951615003
+ | ],
+ | [
+ | 172.95469614953714,
+ | 1.3690476620161975
+ | ],
+ | [
+ | 172.91173669923782,
+ | 1.3690476620161975
+ | ],
+ | [
+ | 172.91173669923782,
+ | 1.3438851951615003
+ | ]
+ | ]
+ | ]
+ | },
+ | "properties": {
+ | "title": "Item 1",
+ | "description": "A sample STAC Item 1 that includes examples of all common metadata",
+ | "datetime": null,
+ | "start_datetime": "2020-12-11T22:38:32.125Z",
+ | "end_datetime": "2020-12-11T22:38:32.327Z",
+ | "created": "2020-12-12T01:48:13.725Z",
+ | "updated": "2020-12-12T01:48:13.725Z",
+ | "platform": "cool_sat1",
+ | "instruments": [
+ | "cool_sensor_v1"
+ | ],
+ | "constellation": "ion",
+ | "mission": "collection 5624",
+ | "gsd": 0.512
+ | },
+ | "collection": "simple-collection",
+ | "links": [
+ | {
+ | "rel": "collection",
+ | "href": "./collection.json",
+ | "type": "application/json",
+ | "title": "Simple Example Collection"
+ | },
+ | {
+ | "rel": "root",
+ | "href": "./collection.json",
+ | "type": "application/json",
+ | "title": "Simple Example Collection"
+ | },
+ | {
+ | "rel": "parent",
+ | "href": "./collection.json",
+ | "type": "application/json",
+ | "title": "Simple Example Collection"
+ | },
+ | {
+ | "rel": "alternate",
+ | "type": "text/html",
+ | "href": "https://remotedata.io/catalog/20201211_223832_CS2_item1/index.html",
+ | "title": "HTML version of this STAC Item"
+ | }
+ | ],
+ | "assets": {
+ | "analytic": {
+ | "href": "https://storage.googleapis.com/open-cogs/stac-examples/20201211_223832_CS2_item1_analytic.tif",
+ | "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ | "title": "4-Band Analytic",
+ | "roles": [
+ | "data"
+ | ]
+ | },
+ | "thumbnail": {
+ | "href": "https://storage.googleapis.com/open-cogs/stac-examples/20201211_223832_CS2_item1.jpg",
+ | "title": "Thumbnail",
+ | "type": "image/png",
+ | "roles": [
+ | "thumbnail"
+ | ]
+ | },
+ | "visual": {
+ | "href": "https://storage.googleapis.com/open-cogs/stac-examples/20201211_223832_CS2_item1.tif",
+ | "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ | "title": "3-Band Visual",
+ | "roles": [
+ | "visual"
+ | ]
+ | },
+ | "udm": {
+ | "href": "https://storage.googleapis.com/open-cogs/stac-examples/20201211_223832_CS2_item1_analytic_udm.tif",
+ | "title": "Unusable Data Mask",
+ | "type": "image/tiff; application=geotiff"
+ | },
+ | "json-metadata": {
+ | "href": "https://remotedata.io/catalog/20201211_223832_CS2_item1/extended-metadata.json",
+ | "title": "Extended Metadata",
+ | "type": "application/json",
+ | "roles": [
+ | "metadata"
+ | ]
+ | },
+ | "ephemeris": {
+ | "href": "https://cool-sat.com/catalog/20201211_223832_CS2_item1/20201211_223832_CS2_item1.EPH",
+ | "title": "Satellite Ephemeris Metadata"
+ | }
+ | }
+ |}
+ """.stripMargin
+ val item1Path = new Path("/tmp/item1.json")
+ val item1Writer = new PrintWriter(new File(item1Path.toString))
+ item1Writer.write(item1Json)
+ item1Writer.close()
+
+ val item2Json =
+ """
+ |{
+ | "stac_version": "1.1.0",
+ | "stac_extensions": [],
+ | "type": "Feature",
+ | "id": "20201211_223832_CS2_item2",
+ | "bbox": [
+ | 173.91173669923782,
+ | 2.3438851951615003,
+ | 173.95469614953714,
+ | 2.3690476620161975
+ | ],
+ | "geometry": {
+ | "type": "Polygon",
+ | "coordinates": [
+ | [
+ | [
+ | 173.91173669923782,
+ | 2.3438851951615003
+ | ],
+ | [
+ | 173.95469614953714,
+ | 2.3438851951615003
+ | ],
+ | [
+ | 173.95469614953714,
+ | 2.3690476620161975
+ | ],
+ | [
+ | 173.91173669923782,
+ | 2.3690476620161975
+ | ],
+ | [
+ | 173.91173669923782,
+ | 2.3438851951615003
+ | ]
+ | ]
+ | ]
+ | },
+ | "properties": {
+ | "title": "Item 2",
+ | "description": "A different sample STAC Item 2 that includes examples of all common metadata",
+ | "datetime": null,
+ | "start_datetime": "2020-12-12T22:38:32.125Z",
+ | "end_datetime": "2020-12-12T22:38:32.327Z",
+ | "created": "2020-12-13T01:48:13.725Z",
+ | "updated": "2020-12-13T01:48:13.725Z",
+ | "platform": "cool_sat2",
+ | "instruments": [
+ | "cool_sensor_v2"
+ | ],
+ | "constellation": "ion",
+ | "mission": "collection 5625",
+ | "gsd": 0.512
+ | },
+ | "collection": "simple-collection",
+ | "links": [
+ | {
+ | "rel": "collection",
+ | "href": "./collection.json",
+ | "type": "application/json",
+ | "title": "Simple Example Collection"
+ | },
+ | {
+ | "rel": "root",
+ | "href": "./collection.json",
+ | "type": "application/json",
+ | "title": "Simple Example Collection"
+ | },
+ | {
+ | "rel": "parent",
+ | "href": "./collection.json",
+ | "type": "application/json",
+ | "title": "Simple Example Collection"
+ | },
+ | {
+ | "rel": "alternate",
+ | "type": "text/html",
+ | "href": "https://remotedata.io/catalog/20201211_223832_CS2_item2/index.html",
+ | "title": "HTML version of this STAC Item"
+ | }
+ | ],
+ | "assets": {
+ | "analytic": {
+ | "href": "https://storage.googleapis.com/open-cogs/stac-examples/20201211_223832_CS2_item2_analytic.tif",
+ | "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ | "title": "4-Band Analytic",
+ | "roles": [
+ | "data"
+ | ]
+ | },
+ | "thumbnail": {
+ | "href": "https://storage.googleapis.com/open-cogs/stac-examples/20201211_223832_CS2_item2.jpg",
+ | "title": "Thumbnail",
+ | "type": "image/png",
+ | "roles": [
+ | "thumbnail"
+ | ]
+ | },
+ | "visual": {
+ | "href": "https://storage.googleapis.com/open-cogs/stac-examples/20201211_223832_CS2_item2.tif",
+ | "type": "image/tiff; application=geotiff; profile=cloud-optimized",
+ | "title": "3-Band Visual",
+ | "roles": [
+ | "visual"
+ | ]
+ | },
+ | "udm": {
+ | "href": "https://storage.googleapis.com/open-cogs/stac-examples/20201211_223832_CS2_item2_analytic_udm.tif",
+ | "title": "Unusable Data Mask",
+ | "type": "image/tiff; application=geotiff"
+ | },
+ | "json-metadata": {
+ | "href": "https://remotedata.io/catalog/20201211_223832_CS2_item2/extended-metadata.json",
+ | "title": "Extended Metadata",
+ | "type": "application/json",
+ | "roles": [
+ | "metadata"
+ | ]
+ | },
+ | "ephemeris": {
+ | "href": "https://cool-sat.com/catalog/20201211_223832_CS2_item2/20201211_223832_CS2_item2.EPH",
+ | "title": "Satellite Ephemeris Metadata"
+ | }
+ | }
+ |}
+ """.stripMargin
+ val item2Path = new Path("/tmp/item2.json")
+ val item2Writer = new PrintWriter(new File(item2Path.toString))
+ item2Writer.write(item2Json)
+ item2Writer.close()
+
+ // Load the STAC collection JSON
+ val opts = Map("path" -> "file:///tmp/collection.json")
+ val stacCollectionJsonString = StacUtils.loadStacCollectionToJson(opts)
+ val outputPath = "/tmp/output.json"
+
+ // Call the function to process the STAC collection
+ saveStacCollection(stacCollectionJsonString, outputPath)
+
+ // Verify the output file
+ val outputFile = new File(outputPath)
+ assert(outputFile.exists())
+
+ val outputContent = Source.fromFile(outputFile).getLines().mkString("\n")
+ assert(outputContent.contains("item1"))
+ assert(outputContent.contains("item2"))
+
+ // Clean up temporary files
+ fs.delete(stacCollectionPath, false)
+ fs.delete(item1Path, false)
+ fs.delete(item2Path, false)
+ outputFile.delete()
+ }
+
+ test(
+ "processStacCollection should process STAC collection with mixed 'item' and 'items' rels and save features to output file") {
+ val spark = SparkSession.builder().master("local").appName("StacUtilsTest").getOrCreate()
+ val hadoopConf = spark.sparkContext.hadoopConfiguration
+ val fs = FileSystem.get(hadoopConf)
+
+ // Create a temporary STAC collection JSON file
+ val stacCollectionJson =
+ """
+ |{
+ | "stac_version": "1.0.0",
+ | "id": "sample-collection",
+ | "description": "A sample STAC collection",
+ | "links": [
+ | {"rel": "item", "href": "file:///tmp/item1.json"},
+ | {"rel": "items", "href": "file:///tmp/items.json"}
+ | ]
+ |}
+ """.stripMargin
+ val stacCollectionPath = new Path("/tmp/collection.json")
+ val stacCollectionWriter = new PrintWriter(new File(stacCollectionPath.toString))
+ stacCollectionWriter.write(stacCollectionJson)
+ stacCollectionWriter.close()
+
+ // Create temporary item JSON files
+ val item1Json =
+ """
+ |{
+ | "type": "Feature",
+ | "id": "item1",
+ | "geometry": {
+ | "type": "Point",
+ | "coordinates": [100.0, 0.0]
+ | },
+ | "properties": {
+ | "title": "Item 1"
+ | }
+ |}
+ """.stripMargin
+ val item1Path = new Path("/tmp/item1.json")
+ val item1Writer = new PrintWriter(new File(item1Path.toString))
+ item1Writer.write(item1Json)
+ item1Writer.close()
+
+ val itemsJson =
+ """
+ |{
+ | "type": "FeatureCollection",
+ | "features": [
+ | {
+ | "type": "Feature",
+ | "id": "item2",
+ | "geometry": {
+ | "type": "Point",
+ | "coordinates": [101.0, 1.0]
+ | },
+ | "properties": {
+ | "title": "Item 2"
+ | }
+ | },
+ | {
+ | "type": "Feature",
+ | "id": "item3",
+ | "geometry": {
+ | "type": "Point",
+ | "coordinates": [102.0, 2.0]
+ | },
+ | "properties": {
+ | "title": "Item 3"
+ | }
+ | }
+ | ]
+ |}
+ """.stripMargin
+ val itemsPath = new Path("/tmp/items.json")
+ val itemsWriter = new PrintWriter(new File(itemsPath.toString))
+ itemsWriter.write(itemsJson)
+ itemsWriter.close()
+
+ // Load the STAC collection JSON
+ val opts = Map("path" -> "file:///tmp/collection.json")
+ val stacCollectionJsonString = StacUtils.loadStacCollectionToJson(opts)
+ val outputPath = "/tmp/output.json"
+
+ // Call the function to process the STAC collection
+ saveStacCollection(stacCollectionJsonString, outputPath)
+
+ // Verify the output file
+ val outputFile = new File(outputPath)
+ assert(outputFile.exists())
+
+ val outputContent = Source.fromFile(outputFile).getLines().mkString("\n")
+ assert(outputContent.contains("item1"))
+ assert(outputContent.contains("item2"))
+ assert(outputContent.contains("item3"))
+
+ // Clean up temporary files
+ fs.delete(stacCollectionPath, false)
+ fs.delete(item1Path, false)
+ fs.delete(itemsPath, false)
+ outputFile.delete()
+ }
+
+ // Function to process STAC collection
+ def saveStacCollection(stacCollectionJson: String, outputPath: String): Unit = {
+ // Create the ObjectMapper
+ val mapper = new ObjectMapper()
+ mapper.registerModule(DefaultScalaModule)
+
+ // Parse the STAC collection JSON
+ val collection: JsonNode = mapper.readTree(stacCollectionJson)
+
+ // Extract item and items links
+ val itemLinks = collection.get("links").elements().asScala.filter { link =>
+ val rel = link.get("rel").asText()
+ rel == "item" || rel == "items"
+ }
+
+ // Open a writer for the output multiline JSON file
+ val writer = new PrintWriter(new File(outputPath))
+
+ try {
+ // Iterate over each item link
+ itemLinks.foreach { link =>
+ val itemUrl = link.get("href").asText()
+
+ // Fetch the item JSON
+ val itemJson = Source.fromURL(itemUrl).mkString
+
+ // Parse the item JSON
+ val itemCollection: JsonNode = mapper.readTree(itemJson)
+
+ // Check if the link is of type "items"
+ if (link.get("rel").asText() == "items") {
+ // Iterate over each feature in the item collection
+ val features = itemCollection.get("features").elements().asScala
+ features.foreach { feature =>
+ // Write each feature JSON as a single line in the output file
+ writer.println(mapper.writeValueAsString(feature))
+ }
+ } else {
+ // Write the item JSON as a single line in the output file
+ writer.println(mapper.writeValueAsString(itemCollection))
+ }
+ }
+ } finally {
+ // Close the writer
+ writer.close()
+ }
+ }
+}
diff --git a/spark/spark-3.3/src/main/scala/org/apache/sedona/sql/parser/SedonaSqlParser.scala b/spark/spark-3.3/src/main/scala/org/apache/sedona/sql/parser/SedonaSqlParser.scala
index 6c70419122..56c27ba76b 100644
--- a/spark/spark-3.3/src/main/scala/org/apache/sedona/sql/parser/SedonaSqlParser.scala
+++ b/spark/spark-3.3/src/main/scala/org/apache/sedona/sql/parser/SedonaSqlParser.scala
@@ -35,12 +35,8 @@ class SedonaSqlParser(delegate: ParserInterface) extends SparkSqlParser {
override def parsePlan(sqlText: String): LogicalPlan =
try {
parse(sqlText) { parser =>
- parserBuilder.visit(parser.singleStatement()) match {
- case plan: LogicalPlan => plan
- case _ =>
- delegate.parsePlan(sqlText)
- }
- }
+ parserBuilder.visit(parser.singleStatement())
+ }.asInstanceOf[LogicalPlan]
} catch {
case _: Exception =>
delegate.parsePlan(sqlText)
diff --git a/spark/spark-3.3/src/main/scala/org/apache/spark/sql/execution/datasources/v2/geoparquet/metadata/GeoParquetMetadataPartitionReaderFactory.scala b/spark/spark-3.3/src/main/scala/org/apache/spark/sql/execution/datasources/v2/geoparquet/metadata/GeoParquetMetadataPartitionReaderFactory.scala
index 1fe2faa2e0..a101206fbf 100644
--- a/spark/spark-3.3/src/main/scala/org/apache/spark/sql/execution/datasources/v2/geoparquet/metadata/GeoParquetMetadataPartitionReaderFactory.scala
+++ b/spark/spark-3.3/src/main/scala/org/apache/spark/sql/execution/datasources/v2/geoparquet/metadata/GeoParquetMetadataPartitionReaderFactory.scala
@@ -20,6 +20,8 @@ package org.apache.spark.sql.execution.datasources.v2.geoparquet.metadata
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
+import org.apache.parquet.ParquetReadOptions
+import org.apache.parquet.format.converter.ParquetMetadataConverter
import org.apache.parquet.hadoop.ParquetFileReader
import org.apache.parquet.hadoop.util.HadoopInputFile
import org.apache.spark.broadcast.Broadcast
@@ -67,11 +69,11 @@ object GeoParquetMetadataPartitionReaderFactory {
partitionedFile: PartitionedFile,
readDataSchema: StructType): Iterator[InternalRow] = {
val filePath = partitionedFile.filePath
- val metadata = ParquetFileReader
- .open(HadoopInputFile.fromPath(new Path(filePath), configuration))
- .getFooter
- .getFileMetaData
- .getKeyValueMetaData
+
+ val footer = ParquetFileReader
+ .readFooter(configuration, new Path(filePath), ParquetMetadataConverter.NO_FILTER)
+
+ val metadata = footer.getFileMetaData.getKeyValueMetaData
val row = GeoParquetMetaData.parseKeyValueMetaData(metadata) match {
case Some(geo) =>
val geoColumnsMap = geo.columns.map { case (columnName, columnMetadata) =>
diff --git a/spark/spark-3.3/src/test/scala/org/apache/sedona/sql/SQLSyntaxTestScala.scala b/spark/spark-3.3/src/test/scala/org/apache/sedona/sql/SQLSyntaxTestScala.scala
index 72680aacd4..6f873d0a08 100644
--- a/spark/spark-3.3/src/test/scala/org/apache/sedona/sql/SQLSyntaxTestScala.scala
+++ b/spark/spark-3.3/src/test/scala/org/apache/sedona/sql/SQLSyntaxTestScala.scala
@@ -44,14 +44,29 @@ class SQLSyntaxTestScala extends TestBaseScala with TableDrivenPropertyChecks {
it(
"should be able to create a regular table with geometry column should work without a workaround") {
- sparkSession.sql("CREATE TABLE T_TEST_EXPLICIT_GEOMETRY (GEO_COL GEOMETRY)")
- sparkSession.catalog.tableExists("T_TEST_EXPLICIT_GEOMETRY") should be(true)
+ try {
+ sparkSession.sql("CREATE TABLE T_TEST_EXPLICIT_GEOMETRY (GEO_COL GEOMETRY)")
+ sparkSession.catalog.tableExists("T_TEST_EXPLICIT_GEOMETRY") should be(true)
+ sparkSession.sparkContext.getConf.get(keyParserExtension) should be("true")
+ } catch {
+ case ex: Exception =>
+ ex.getClass.getName.endsWith("ParseException") should be(true)
+ sparkSession.sparkContext.getConf.get(keyParserExtension) should be("false")
+ }
}
it(
"should be able to create a regular table with regular and geometry column should work without a workaround") {
- sparkSession.sql("CREATE TABLE T_TEST_EXPLICIT_GEOMETRY_2 (INT_COL INT, GEO_COL GEOMETRY)")
- sparkSession.catalog.tableExists("T_TEST_EXPLICIT_GEOMETRY_2") should be(true)
+ try {
+ sparkSession.sql(
+ "CREATE TABLE T_TEST_EXPLICIT_GEOMETRY_2 (INT_COL INT, GEO_COL GEOMETRY)")
+ sparkSession.catalog.tableExists("T_TEST_EXPLICIT_GEOMETRY_2") should be(true)
+ sparkSession.sparkContext.getConf.get(keyParserExtension) should be("true")
+ } catch {
+ case ex: Exception =>
+ ex.getClass.getName.endsWith("ParseException") should be(true)
+ sparkSession.sparkContext.getConf.get(keyParserExtension) should be("false")
+ }
}
}
}
diff --git a/spark/spark-3.3/src/test/scala/org/apache/sedona/sql/TestBaseScala.scala b/spark/spark-3.3/src/test/scala/org/apache/sedona/sql/TestBaseScala.scala
index f629648b29..8d13f6138d 100644
--- a/spark/spark-3.3/src/test/scala/org/apache/sedona/sql/TestBaseScala.scala
+++ b/spark/spark-3.3/src/test/scala/org/apache/sedona/sql/TestBaseScala.scala
@@ -23,6 +23,8 @@ import org.apache.sedona.spark.SedonaContext
import org.apache.spark.sql.DataFrame
import org.scalatest.{BeforeAndAfterAll, FunSpec}
+import java.util.concurrent.ThreadLocalRandom
+
trait TestBaseScala extends FunSpec with BeforeAndAfterAll {
Logger.getRootLogger().setLevel(Level.WARN)
Logger.getLogger("org.apache").setLevel(Level.WARN)
@@ -30,6 +32,7 @@ trait TestBaseScala extends FunSpec with BeforeAndAfterAll {
Logger.getLogger("akka").setLevel(Level.WARN)
Logger.getLogger("org.apache.sedona.core").setLevel(Level.WARN)
+ val keyParserExtension = "spark.sedona.enableParserExtension"
val warehouseLocation = System.getProperty("user.dir") + "/target/"
val sparkSession = SedonaContext
.builder()
@@ -38,6 +41,8 @@ trait TestBaseScala extends FunSpec with BeforeAndAfterAll {
.config("spark.sql.warehouse.dir", warehouseLocation)
.config("sedona.join.autoBroadcastJoinThreshold", "-1")
.config("spark.sql.session.timeZone", "UTC")
+ .config("spark.sql.extensions", "org.apache.sedona.sql.SedonaSqlExtensions")
+ .config(keyParserExtension, ThreadLocalRandom.current().nextBoolean())
.getOrCreate()
val sparkSessionMinio = SedonaContext
diff --git a/spark/spark-3.4/src/main/scala/org/apache/sedona/sql/parser/SedonaSqlParser.scala b/spark/spark-3.4/src/main/scala/org/apache/sedona/sql/parser/SedonaSqlParser.scala
index 6c70419122..56c27ba76b 100644
--- a/spark/spark-3.4/src/main/scala/org/apache/sedona/sql/parser/SedonaSqlParser.scala
+++ b/spark/spark-3.4/src/main/scala/org/apache/sedona/sql/parser/SedonaSqlParser.scala
@@ -35,12 +35,8 @@ class SedonaSqlParser(delegate: ParserInterface) extends SparkSqlParser {
override def parsePlan(sqlText: String): LogicalPlan =
try {
parse(sqlText) { parser =>
- parserBuilder.visit(parser.singleStatement()) match {
- case plan: LogicalPlan => plan
- case _ =>
- delegate.parsePlan(sqlText)
- }
- }
+ parserBuilder.visit(parser.singleStatement())
+ }.asInstanceOf[LogicalPlan]
} catch {
case _: Exception =>
delegate.parsePlan(sqlText)
diff --git a/spark/spark-3.4/src/main/scala/org/apache/spark/sql/execution/datasources/v2/geoparquet/metadata/GeoParquetMetadataPartitionReaderFactory.scala b/spark/spark-3.4/src/main/scala/org/apache/spark/sql/execution/datasources/v2/geoparquet/metadata/GeoParquetMetadataPartitionReaderFactory.scala
index 2a5e70624c..e4ca35992b 100644
--- a/spark/spark-3.4/src/main/scala/org/apache/spark/sql/execution/datasources/v2/geoparquet/metadata/GeoParquetMetadataPartitionReaderFactory.scala
+++ b/spark/spark-3.4/src/main/scala/org/apache/spark/sql/execution/datasources/v2/geoparquet/metadata/GeoParquetMetadataPartitionReaderFactory.scala
@@ -19,6 +19,7 @@
package org.apache.spark.sql.execution.datasources.v2.geoparquet.metadata
import org.apache.hadoop.conf.Configuration
+import org.apache.parquet.ParquetReadOptions
import org.apache.parquet.hadoop.ParquetFileReader
import org.apache.parquet.hadoop.util.HadoopInputFile
import org.apache.spark.broadcast.Broadcast
@@ -66,12 +67,14 @@ object GeoParquetMetadataPartitionReaderFactory {
configuration: Configuration,
partitionedFile: PartitionedFile,
readDataSchema: StructType): Iterator[InternalRow] = {
+ val inputFile = HadoopInputFile.fromPath(partitionedFile.toPath, configuration)
+ val inputStream = inputFile.newStream()
+
+ val footer = ParquetFileReader
+ .readFooter(inputFile, ParquetReadOptions.builder().build(), inputStream)
+
val filePath = partitionedFile.toPath.toString
- val metadata = ParquetFileReader
- .open(HadoopInputFile.fromPath(partitionedFile.toPath, configuration))
- .getFooter
- .getFileMetaData
- .getKeyValueMetaData
+ val metadata = footer.getFileMetaData.getKeyValueMetaData
val row = GeoParquetMetaData.parseKeyValueMetaData(metadata) match {
case Some(geo) =>
val geoColumnsMap = geo.columns.map { case (columnName, columnMetadata) =>
diff --git a/spark/spark-3.4/src/test/scala/org/apache/sedona/sql/SQLSyntaxTestScala.scala b/spark/spark-3.4/src/test/scala/org/apache/sedona/sql/SQLSyntaxTestScala.scala
index 72680aacd4..6f873d0a08 100644
--- a/spark/spark-3.4/src/test/scala/org/apache/sedona/sql/SQLSyntaxTestScala.scala
+++ b/spark/spark-3.4/src/test/scala/org/apache/sedona/sql/SQLSyntaxTestScala.scala
@@ -44,14 +44,29 @@ class SQLSyntaxTestScala extends TestBaseScala with TableDrivenPropertyChecks {
it(
"should be able to create a regular table with geometry column should work without a workaround") {
- sparkSession.sql("CREATE TABLE T_TEST_EXPLICIT_GEOMETRY (GEO_COL GEOMETRY)")
- sparkSession.catalog.tableExists("T_TEST_EXPLICIT_GEOMETRY") should be(true)
+ try {
+ sparkSession.sql("CREATE TABLE T_TEST_EXPLICIT_GEOMETRY (GEO_COL GEOMETRY)")
+ sparkSession.catalog.tableExists("T_TEST_EXPLICIT_GEOMETRY") should be(true)
+ sparkSession.sparkContext.getConf.get(keyParserExtension) should be("true")
+ } catch {
+ case ex: Exception =>
+ ex.getClass.getName.endsWith("ParseException") should be(true)
+ sparkSession.sparkContext.getConf.get(keyParserExtension) should be("false")
+ }
}
it(
"should be able to create a regular table with regular and geometry column should work without a workaround") {
- sparkSession.sql("CREATE TABLE T_TEST_EXPLICIT_GEOMETRY_2 (INT_COL INT, GEO_COL GEOMETRY)")
- sparkSession.catalog.tableExists("T_TEST_EXPLICIT_GEOMETRY_2") should be(true)
+ try {
+ sparkSession.sql(
+ "CREATE TABLE T_TEST_EXPLICIT_GEOMETRY_2 (INT_COL INT, GEO_COL GEOMETRY)")
+ sparkSession.catalog.tableExists("T_TEST_EXPLICIT_GEOMETRY_2") should be(true)
+ sparkSession.sparkContext.getConf.get(keyParserExtension) should be("true")
+ } catch {
+ case ex: Exception =>
+ ex.getClass.getName.endsWith("ParseException") should be(true)
+ sparkSession.sparkContext.getConf.get(keyParserExtension) should be("false")
+ }
}
}
}
diff --git a/spark/spark-3.4/src/test/scala/org/apache/sedona/sql/TestBaseScala.scala b/spark/spark-3.4/src/test/scala/org/apache/sedona/sql/TestBaseScala.scala
index 34746d0b28..ae1ed5d091 100644
--- a/spark/spark-3.4/src/test/scala/org/apache/sedona/sql/TestBaseScala.scala
+++ b/spark/spark-3.4/src/test/scala/org/apache/sedona/sql/TestBaseScala.scala
@@ -23,6 +23,8 @@ import org.apache.sedona.spark.SedonaContext
import org.apache.spark.sql.DataFrame
import org.scalatest.{BeforeAndAfterAll, FunSpec}
+import java.util.concurrent.ThreadLocalRandom
+
trait TestBaseScala extends FunSpec with BeforeAndAfterAll {
Logger.getRootLogger().setLevel(Level.WARN)
Logger.getLogger("org.apache").setLevel(Level.WARN)
@@ -30,6 +32,7 @@ trait TestBaseScala extends FunSpec with BeforeAndAfterAll {
Logger.getLogger("akka").setLevel(Level.WARN)
Logger.getLogger("org.apache.sedona.core").setLevel(Level.WARN)
+ val keyParserExtension = "spark.sedona.enableParserExtension"
val warehouseLocation = System.getProperty("user.dir") + "/target/"
val sparkSession = SedonaContext
.builder()
@@ -38,6 +41,8 @@ trait TestBaseScala extends FunSpec with BeforeAndAfterAll {
.config("spark.sql.warehouse.dir", warehouseLocation)
// We need to be explicit about broadcasting in tests.
.config("sedona.join.autoBroadcastJoinThreshold", "-1")
+ .config("spark.sql.extensions", "org.apache.sedona.sql.SedonaSqlExtensions")
+ .config(keyParserExtension, ThreadLocalRandom.current().nextBoolean())
.getOrCreate()
val sparkSessionMinio = SedonaContext
diff --git a/spark/spark-3.5/src/main/scala/org/apache/sedona/sql/datasources/geopackage/GeoPackageScanBuilder.scala b/spark/spark-3.5/src/main/scala/org/apache/sedona/sql/datasources/geopackage/GeoPackageScanBuilder.scala
index a9674395b4..829bd9c220 100644
--- a/spark/spark-3.5/src/main/scala/org/apache/sedona/sql/datasources/geopackage/GeoPackageScanBuilder.scala
+++ b/spark/spark-3.5/src/main/scala/org/apache/sedona/sql/datasources/geopackage/GeoPackageScanBuilder.scala
@@ -37,13 +37,11 @@ class GeoPackageScanBuilder(
extends FileScanBuilder(sparkSession, fileIndex, dataSchema) {
override def build(): Scan = {
- val paths = fileIndex.allFiles().map(_.getPath.toString)
-
val fileIndexAdjusted =
if (loadOptions.showMetadata)
new InMemoryFileIndex(
sparkSession,
- paths.slice(0, 1).map(new org.apache.hadoop.fs.Path(_)),
+ fileIndex.inputFiles.slice(0, 1).map(new org.apache.hadoop.fs.Path(_)),
options.asCaseSensitiveMap.asScala.toMap,
userDefinedSchema)
else fileIndex
diff --git a/spark/spark-3.5/src/main/scala/org/apache/sedona/sql/parser/SedonaSqlParser.scala b/spark/spark-3.5/src/main/scala/org/apache/sedona/sql/parser/SedonaSqlParser.scala
index 6c70419122..56c27ba76b 100644
--- a/spark/spark-3.5/src/main/scala/org/apache/sedona/sql/parser/SedonaSqlParser.scala
+++ b/spark/spark-3.5/src/main/scala/org/apache/sedona/sql/parser/SedonaSqlParser.scala
@@ -35,12 +35,8 @@ class SedonaSqlParser(delegate: ParserInterface) extends SparkSqlParser {
override def parsePlan(sqlText: String): LogicalPlan =
try {
parse(sqlText) { parser =>
- parserBuilder.visit(parser.singleStatement()) match {
- case plan: LogicalPlan => plan
- case _ =>
- delegate.parsePlan(sqlText)
- }
- }
+ parserBuilder.visit(parser.singleStatement())
+ }.asInstanceOf[LogicalPlan]
} catch {
case _: Exception =>
delegate.parsePlan(sqlText)
diff --git a/spark/spark-3.5/src/main/scala/org/apache/spark/sql/execution/datasources/v2/geoparquet/metadata/GeoParquetMetadataPartitionReaderFactory.scala b/spark/spark-3.5/src/main/scala/org/apache/spark/sql/execution/datasources/v2/geoparquet/metadata/GeoParquetMetadataPartitionReaderFactory.scala
index 2a5e70624c..e1234e79d8 100644
--- a/spark/spark-3.5/src/main/scala/org/apache/spark/sql/execution/datasources/v2/geoparquet/metadata/GeoParquetMetadataPartitionReaderFactory.scala
+++ b/spark/spark-3.5/src/main/scala/org/apache/spark/sql/execution/datasources/v2/geoparquet/metadata/GeoParquetMetadataPartitionReaderFactory.scala
@@ -19,6 +19,7 @@
package org.apache.spark.sql.execution.datasources.v2.geoparquet.metadata
import org.apache.hadoop.conf.Configuration
+import org.apache.parquet.ParquetReadOptions
import org.apache.parquet.hadoop.ParquetFileReader
import org.apache.parquet.hadoop.util.HadoopInputFile
import org.apache.spark.broadcast.Broadcast
@@ -66,12 +67,15 @@ object GeoParquetMetadataPartitionReaderFactory {
configuration: Configuration,
partitionedFile: PartitionedFile,
readDataSchema: StructType): Iterator[InternalRow] = {
+
+ val inputFile = HadoopInputFile.fromPath(partitionedFile.toPath, configuration)
+ val inputStream = inputFile.newStream()
+
+ val footer = ParquetFileReader
+ .readFooter(inputFile, ParquetReadOptions.builder().build(), inputStream)
+
val filePath = partitionedFile.toPath.toString
- val metadata = ParquetFileReader
- .open(HadoopInputFile.fromPath(partitionedFile.toPath, configuration))
- .getFooter
- .getFileMetaData
- .getKeyValueMetaData
+ val metadata = footer.getFileMetaData.getKeyValueMetaData
val row = GeoParquetMetaData.parseKeyValueMetaData(metadata) match {
case Some(geo) =>
val geoColumnsMap = geo.columns.map { case (columnName, columnMetadata) =>
diff --git a/spark/spark-3.5/src/test/scala/org/apache/sedona/sql/SQLSyntaxTestScala.scala b/spark/spark-3.5/src/test/scala/org/apache/sedona/sql/SQLSyntaxTestScala.scala
index 72680aacd4..6f873d0a08 100644
--- a/spark/spark-3.5/src/test/scala/org/apache/sedona/sql/SQLSyntaxTestScala.scala
+++ b/spark/spark-3.5/src/test/scala/org/apache/sedona/sql/SQLSyntaxTestScala.scala
@@ -44,14 +44,29 @@ class SQLSyntaxTestScala extends TestBaseScala with TableDrivenPropertyChecks {
it(
"should be able to create a regular table with geometry column should work without a workaround") {
- sparkSession.sql("CREATE TABLE T_TEST_EXPLICIT_GEOMETRY (GEO_COL GEOMETRY)")
- sparkSession.catalog.tableExists("T_TEST_EXPLICIT_GEOMETRY") should be(true)
+ try {
+ sparkSession.sql("CREATE TABLE T_TEST_EXPLICIT_GEOMETRY (GEO_COL GEOMETRY)")
+ sparkSession.catalog.tableExists("T_TEST_EXPLICIT_GEOMETRY") should be(true)
+ sparkSession.sparkContext.getConf.get(keyParserExtension) should be("true")
+ } catch {
+ case ex: Exception =>
+ ex.getClass.getName.endsWith("ParseException") should be(true)
+ sparkSession.sparkContext.getConf.get(keyParserExtension) should be("false")
+ }
}
it(
"should be able to create a regular table with regular and geometry column should work without a workaround") {
- sparkSession.sql("CREATE TABLE T_TEST_EXPLICIT_GEOMETRY_2 (INT_COL INT, GEO_COL GEOMETRY)")
- sparkSession.catalog.tableExists("T_TEST_EXPLICIT_GEOMETRY_2") should be(true)
+ try {
+ sparkSession.sql(
+ "CREATE TABLE T_TEST_EXPLICIT_GEOMETRY_2 (INT_COL INT, GEO_COL GEOMETRY)")
+ sparkSession.catalog.tableExists("T_TEST_EXPLICIT_GEOMETRY_2") should be(true)
+ sparkSession.sparkContext.getConf.get(keyParserExtension) should be("true")
+ } catch {
+ case ex: Exception =>
+ ex.getClass.getName.endsWith("ParseException") should be(true)
+ sparkSession.sparkContext.getConf.get(keyParserExtension) should be("false")
+ }
}
}
}
diff --git a/spark/spark-3.5/src/test/scala/org/apache/sedona/sql/TestBaseScala.scala b/spark/spark-3.5/src/test/scala/org/apache/sedona/sql/TestBaseScala.scala
index 34746d0b28..ae1ed5d091 100644
--- a/spark/spark-3.5/src/test/scala/org/apache/sedona/sql/TestBaseScala.scala
+++ b/spark/spark-3.5/src/test/scala/org/apache/sedona/sql/TestBaseScala.scala
@@ -23,6 +23,8 @@ import org.apache.sedona.spark.SedonaContext
import org.apache.spark.sql.DataFrame
import org.scalatest.{BeforeAndAfterAll, FunSpec}
+import java.util.concurrent.ThreadLocalRandom
+
trait TestBaseScala extends FunSpec with BeforeAndAfterAll {
Logger.getRootLogger().setLevel(Level.WARN)
Logger.getLogger("org.apache").setLevel(Level.WARN)
@@ -30,6 +32,7 @@ trait TestBaseScala extends FunSpec with BeforeAndAfterAll {
Logger.getLogger("akka").setLevel(Level.WARN)
Logger.getLogger("org.apache.sedona.core").setLevel(Level.WARN)
+ val keyParserExtension = "spark.sedona.enableParserExtension"
val warehouseLocation = System.getProperty("user.dir") + "/target/"
val sparkSession = SedonaContext
.builder()
@@ -38,6 +41,8 @@ trait TestBaseScala extends FunSpec with BeforeAndAfterAll {
.config("spark.sql.warehouse.dir", warehouseLocation)
// We need to be explicit about broadcasting in tests.
.config("sedona.join.autoBroadcastJoinThreshold", "-1")
+ .config("spark.sql.extensions", "org.apache.sedona.sql.SedonaSqlExtensions")
+ .config(keyParserExtension, ThreadLocalRandom.current().nextBoolean())
.getOrCreate()
val sparkSessionMinio = SedonaContext