You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sedona.apache.org by ji...@apache.org on 2022/09/06 04:48:02 UTC
[incubator-sedona] branch master updated: [SEDONA-162] Add ST_Envelope to the Flink API (#682)
This is an automated email from the ASF dual-hosted git repository.
jiayu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-sedona.git
The following commit(s) were added to refs/heads/master by this push:
new abd142dc [SEDONA-162] Add ST_Envelope to the Flink API (#682)
abd142dc is described below
commit abd142dc930990c0deeb4c50b06895bf55e01fcf
Author: Kengo Seki <se...@apache.org>
AuthorDate: Tue Sep 6 13:47:57 2022 +0900
[SEDONA-162] Add ST_Envelope to the Flink API (#682)
---
.../src/main/java/org/apache/sedona/common/Functions.java | 4 ++++
docs/api/flink/Function.md | 15 +++++++++++++++
flink/src/main/java/org/apache/sedona/flink/Catalog.java | 1 +
.../org/apache/sedona/flink/expressions/Functions.java | 8 ++++++++
.../test/java/org/apache/sedona/flink/FunctionTest.java | 7 +++++++
.../spark/sql/sedona_sql/expressions/Functions.scala | 15 ++-------------
6 files changed, 37 insertions(+), 13 deletions(-)
diff --git a/common/src/main/java/org/apache/sedona/common/Functions.java b/common/src/main/java/org/apache/sedona/common/Functions.java
index 4e5b631b..3c729bc9 100644
--- a/common/src/main/java/org/apache/sedona/common/Functions.java
+++ b/common/src/main/java/org/apache/sedona/common/Functions.java
@@ -62,6 +62,10 @@ public class Functions {
return geometry.buffer(radius);
}
+ public static Geometry envelope(Geometry geometry) {
+ return geometry.getEnvelope();
+ }
+
public static double distance(Geometry left, Geometry right) {
return left.distance(right);
}
diff --git a/docs/api/flink/Function.md b/docs/api/flink/Function.md
index 04583b68..1396cfda 100644
--- a/docs/api/flink/Function.md
+++ b/docs/api/flink/Function.md
@@ -214,6 +214,21 @@ SELECT ST_Distance(polygondf.countyshape, polygondf.countyshape)
FROM polygondf
```
+## ST_Envelope
+
+Introduction: Return the envelop boundary of A
+
+Format: `ST_Envelope (A:geometry)`
+
+Since: `v1.3.0`
+
+Example:
+
+```SQL
+SELECT ST_Envelope(polygondf.countyshape)
+FROM polygondf
+```
+
## ST_ExteriorRing
Introduction: Returns a LINESTRING representing the exterior ring (shell) of a POLYGON. Returns NULL if the geometry is not a polygon.
diff --git a/flink/src/main/java/org/apache/sedona/flink/Catalog.java b/flink/src/main/java/org/apache/sedona/flink/Catalog.java
index 62807f70..48e9a7b5 100644
--- a/flink/src/main/java/org/apache/sedona/flink/Catalog.java
+++ b/flink/src/main/java/org/apache/sedona/flink/Catalog.java
@@ -38,6 +38,7 @@ public class Catalog {
new Functions.ST_Azimuth(),
new Functions.ST_Boundary(),
new Functions.ST_Buffer(),
+ new Functions.ST_Envelope(),
new Functions.ST_Distance(),
new Functions.ST_3DDistance(),
new Functions.ST_Length(),
diff --git a/flink/src/main/java/org/apache/sedona/flink/expressions/Functions.java b/flink/src/main/java/org/apache/sedona/flink/expressions/Functions.java
index 25d8ecdd..798065f7 100644
--- a/flink/src/main/java/org/apache/sedona/flink/expressions/Functions.java
+++ b/flink/src/main/java/org/apache/sedona/flink/expressions/Functions.java
@@ -55,6 +55,14 @@ public class Functions {
}
}
+ public static class ST_Envelope extends ScalarFunction {
+ @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class)
+ public Geometry eval(@DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o) {
+ Geometry geom = (Geometry) o;
+ return org.apache.sedona.common.Functions.envelope(geom);
+ }
+ }
+
public static class ST_Distance extends ScalarFunction {
@DataTypeHint("Double")
public Double eval(@DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o1,
diff --git a/flink/src/test/java/org/apache/sedona/flink/FunctionTest.java b/flink/src/test/java/org/apache/sedona/flink/FunctionTest.java
index 6869c601..a78e6ad1 100644
--- a/flink/src/test/java/org/apache/sedona/flink/FunctionTest.java
+++ b/flink/src/test/java/org/apache/sedona/flink/FunctionTest.java
@@ -68,6 +68,13 @@ public class FunctionTest extends TestBase{
assert(result instanceof Polygon);
}
+ @Test
+ public void testEnvelope() {
+ Table linestringTable = createLineStringTable(1);
+ linestringTable = linestringTable.select(call(Functions.ST_Envelope.class.getSimpleName(), $(linestringColNames[0])));
+ assertEquals("POLYGON ((-0.5 -0.5, -0.5 0.5, 0.5 0.5, 0.5 -0.5, -0.5 -0.5))", first(linestringTable).getField(0).toString());
+ }
+
@Test
public void testFlipCoordinates() {
Table pointTable = createPointTable_real(testDataSize);
diff --git a/sql/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/Functions.scala b/sql/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/Functions.scala
index 67d1eac7..450ea530 100644
--- a/sql/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/Functions.scala
+++ b/sql/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/Functions.scala
@@ -22,7 +22,7 @@ import org.apache.sedona.common.Functions
import org.apache.sedona.sql.utils.GeometrySerializer
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
-import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.{Expression, Generator}
import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData}
import org.apache.spark.sql.sedona_sql.UDT.GeometryUDT
import org.apache.spark.sql.sedona_sql.expressions.collect.Collect
@@ -38,8 +38,6 @@ import org.locationtech.jts.operation.buffer.BufferParameters
import org.locationtech.jts.operation.linemerge.LineMerger
import org.locationtech.jts.precision.GeometryPrecisionReducer
import org.locationtech.jts.simplify.TopologyPreservingSimplifier
-import org.locationtech.jts.geom.Geometry
-import org.locationtech.jts.geom.Coordinate
import scala.util.{Failure, Success, Try}
@@ -144,16 +142,7 @@ case class ST_Buffer(inputExpressions: Seq[Expression])
* @param inputExpressions
*/
case class ST_Envelope(inputExpressions: Seq[Expression])
- extends UnaryGeometryExpression with CodegenFallback {
- assert(inputExpressions.length == 1)
-
- override def nullSafeEval(geometry: Geometry): Any = {
- new GenericArrayData(GeometrySerializer.serialize(geometry.getEnvelope()))
- }
-
- override def dataType: DataType = GeometryUDT
-
- override def children: Seq[Expression] = inputExpressions
+ extends InferredUnaryExpression(Functions.envelope) {
protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]) = {
copy(inputExpressions = newChildren)