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 5874e1b2 [SEDONA-97] Add ST_Force_2D function to Apache Sedona (#618)
5874e1b2 is described below

commit 5874e1b2f616123de8dd433c9798ddbe3f009f41
Author: tmodi7 <[email protected]>
AuthorDate: Fri Apr 22 22:25:48 2022 -0700

    [SEDONA-97] Add ST_Force_2D function to Apache Sedona (#618)
---
 .../org/apache/sedona/core/utils/GeomUtils.java    | 26 ++++++++++++++++++----
 docs/api/flink/Function.md                         | 19 ++++++++++++++++
 docs/api/sql/Function.md                           | 26 ++++++++++++++++++++++
 .../main/java/org/apache/sedona/flink/Catalog.java |  3 ++-
 .../apache/sedona/flink/expressions/Functions.java | 15 ++++++++++++-
 .../java/org/apache/sedona/flink/FunctionTest.java | 15 ++++++++++++-
 python/tests/sql/test_function.py                  | 15 ++++++++++++-
 .../scala/org/apache/sedona/sql/UDF/Catalog.scala  |  1 +
 .../sql/sedona_sql/expressions/Functions.scala     | 22 ++++++++++++++++++
 .../org/apache/sedona/sql/functionTestScala.scala  | 19 ++++++++++++++++
 10 files changed, 153 insertions(+), 8 deletions(-)

diff --git a/core/src/main/java/org/apache/sedona/core/utils/GeomUtils.java 
b/core/src/main/java/org/apache/sedona/core/utils/GeomUtils.java
index 0eafcd6f..c28a8eed 100644
--- a/core/src/main/java/org/apache/sedona/core/utils/GeomUtils.java
+++ b/core/src/main/java/org/apache/sedona/core/utils/GeomUtils.java
@@ -13,12 +13,12 @@
  */
 package org.apache.sedona.core.utils;
 
-import org.locationtech.jts.geom.CoordinateSequence;
-import org.locationtech.jts.geom.CoordinateSequenceFilter;
-import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.*;
 import org.locationtech.jts.io.WKTWriter;
 import java.util.Objects;
 
+import static org.locationtech.jts.geom.Coordinate.NULL_ORDINATE;
+
 public class GeomUtils
 {
     public static String printGeom(Geometry geom) {
@@ -94,4 +94,22 @@ public class GeomUtils
 
         return sridString + new WKTWriter().write(geometry);
     }
-}
+
+    public static Geometry get2dGeom(Geometry geom) {
+        Coordinate[] coordinates = geom.getCoordinates();
+        GeometryFactory geometryFactory = new GeometryFactory();
+        CoordinateSequence sequence = 
geometryFactory.getCoordinateSequenceFactory().create(coordinates);
+        if(sequence.getDimension() > 2) {
+            for (int i = 0; i < coordinates.length; i++) {
+                sequence.setOrdinate(i, 2, NULL_ORDINATE);
+            }
+            if(sequence.getDimension() == 4) {
+                for (int i = 0; i < coordinates.length; i++) {
+                    sequence.setOrdinate(i, 3, NULL_ORDINATE);
+                }
+            }
+        }
+        geom.geometryChanged();
+        return geom;
+    }
+}
\ No newline at end of file
diff --git a/docs/api/flink/Function.md b/docs/api/flink/Function.md
index 3140dbd4..d3796fd3 100644
--- a/docs/api/flink/Function.md
+++ b/docs/api/flink/Function.md
@@ -152,6 +152,25 @@ Input: `POLYGON ((-0.5 -0.5, -0.5 0.5, 0.5 0.5, 0.5 -0.5, 
-0.5 -0.5))`
 
 Output: `POLYGON ((-0.5 -0.5, 0.5 -0.5, 0.5 0.5, -0.5 0.5, -0.5 -0.5))`
 
+## ST_Force_2D
+
+Introduction: Forces the geometries into a "2-dimensional mode" so that all 
output representations will only have the X and Y coordinates
+
+Format: `ST_Force_2D (A:geometry)`
+
+Since: `v1.2.1`
+
+Example:
+
+```SQL
+SELECT ST_Force_2D(df.geometry) AS geom
+FROM df
+```
+
+Input: `POLYGON((0 0 2,0 5 2,5 0 2,0 0 2),(1 1 2,3 1 2,1 3 2,1 1 2))`
+
+Output: `POLYGON((0 0,0 5,5 0,0 0),(1 1,3 1,1 3,1 1))`
+
 ## ST_AsEWKT
 
 Introduction: Return the Extended Well-Known Text representation of a geometry.
diff --git a/docs/api/sql/Function.md b/docs/api/sql/Function.md
index 04086a42..fa9b5353 100644
--- a/docs/api/sql/Function.md
+++ b/docs/api/sql/Function.md
@@ -1105,4 +1105,30 @@ Result:
 +---------------------------------------------------------------+
 |LINESTRING (3 6, 2 4, 1 2, 0 0)                                |
 +---------------------------------------------------------------+
+```
+
+## ST_Force_2D
+
+Introduction: Forces the geometries into a "2-dimensional mode" so that all 
output representations will only have the X and Y coordinates
+
+Format: `ST_Force_2D (A:geometry)`
+
+Since: `v1.2.1`
+
+Example:
+
+```SQL
+SELECT ST_AsText(
+    ST_Force_2D(ST_GeomFromText('POLYGON((0 0 2,0 5 2,5 0 2,0 0 2),(1 1 2,3 1 
2,1 3 2,1 1 2))'))
+) AS geom
+```
+
+Result:
+
+```
++---------------------------------------------------------------+
+|geom                                                           |
++---------------------------------------------------------------+
+|POLYGON((0 0,0 5,5 0,0 0),(1 1,3 1,1 3,1 1))                                |
++---------------------------------------------------------------+
 ```
\ No newline at end of file
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 725b7857..d48d8835 100644
--- a/flink/src/main/java/org/apache/sedona/flink/Catalog.java
+++ b/flink/src/main/java/org/apache/sedona/flink/Catalog.java
@@ -33,7 +33,8 @@ public class Catalog {
                 new Functions.ST_GeoHash(),
                 new Functions.ST_PointOnSurface(),
                 new Functions.ST_Reverse(),
-                new Functions.ST_AsEWKT()
+                new Functions.ST_AsEWKT(),
+                new Functions.ST_Force_2D()
         };
     }
 
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 9cec63e6..689fd10a 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
@@ -21,7 +21,10 @@ import 
org.apache.spark.sql.sedona_sql.expressions.geohash.GeometryGeoHashEncode
 import org.apache.spark.sql.sedona_sql.expressions.geohash.PointGeoHashEncoder;
 import org.geotools.geometry.jts.JTS;
 import org.geotools.referencing.CRS;
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.CoordinateSequence;
 import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
 import org.opengis.referencing.FactoryException;
 import org.opengis.referencing.crs.CoordinateReferenceSystem;
 import org.opengis.referencing.operation.MathTransform;
@@ -30,6 +33,8 @@ import scala.Option;
 
 import java.util.Optional;
 
+import static org.locationtech.jts.geom.Coordinate.NULL_ORDINATE;
+
 public class Functions {
     public static class ST_Buffer extends ScalarFunction {
         @DataTypeHint(value = "RAW", bridgedTo = 
org.locationtech.jts.geom.Geometry.class)
@@ -110,4 +115,12 @@ public class Functions {
             return GeomUtils.getEWKT(geom);
         }
     }
-}
+
+    public static class ST_Force_2D 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 GeomUtils.get2dGeom(geom);
+        }
+    }
+}
\ No newline at end of file
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 0780ea91..5134b06e 100644
--- a/flink/src/test/java/org/apache/sedona/flink/FunctionTest.java
+++ b/flink/src/test/java/org/apache/sedona/flink/FunctionTest.java
@@ -14,11 +14,16 @@
 package org.apache.sedona.flink;
 
 import org.apache.flink.table.api.Table;
+import org.apache.flink.types.Row;
+import org.apache.sedona.flink.expressions.Constructors;
 import org.apache.sedona.flink.expressions.Functions;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.locationtech.jts.geom.Coordinate;
 import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
 
+import java.util.List;
 import java.util.Optional;
 
 import static org.apache.flink.table.api.Expressions.$;
@@ -88,4 +93,12 @@ public class FunctionTest extends TestBase{
         String result = (String) first(polygonTable).getField(0);
         assertEquals("POLYGON ((-0.5 -0.5, -0.5 0.5, 0.5 0.5, 0.5 -0.5, -0.5 
-0.5))", result);
     }
-}
+
+    @Test
+    public void testForce2D() {
+        Table polygonTable = createPolygonTable(1);
+        Table Forced2DTable = 
polygonTable.select(call(Functions.ST_Force_2D.class.getSimpleName(), 
$(polygonColNames[0])));
+        Geometry result = (Geometry) first(Forced2DTable).getField(0);
+        assertEquals("POLYGON ((-0.5 -0.5, -0.5 0.5, 0.5 0.5, 0.5 -0.5, -0.5 
-0.5))", result.toString());
+    }
+}
\ No newline at end of file
diff --git a/python/tests/sql/test_function.py 
b/python/tests/sql/test_function.py
index e6250a2c..adc5259c 100644
--- a/python/tests/sql/test_function.py
+++ b/python/tests/sql/test_function.py
@@ -956,4 +956,17 @@ class TestPredicateJoin(TestBase):
         for input_geom, expected_geom in tests2.items():
             pointOnSurface = self.spark.sql("select 
ST_AsEWKT(ST_PointOnSurface(ST_GeomFromEWKT({})))".format(input_geom))
             assert pointOnSurface.take(1)[0][0] == expected_geom
-        '''
\ No newline at end of file
+        '''
+
+    def test_st_force2d(self):
+        tests1 = {
+            "'POINT(0 5)'": "POINT (0 5)",
+            "'POLYGON((0 0 2, 0 5 2, 5 0 2, 0 0 2), (1 1 2, 3 1 2, 1 3 2, 1 1 
2))'":
+                "POLYGON ((0 0, 0 5, 5 0, 0 0), (1 1, 3 1, 1 3, 1 1))",
+            "'LINESTRING(0 5 1, 0 0 1, 0 10 2)'": "LINESTRING (0 5, 0 0, 0 10)"
+        }
+
+        for input_geom, expected_geom in tests1.items():
+            geom_2d = self.spark.sql(
+                "select 
ST_AsText(ST_Force_2D(ST_GeomFromText({})))".format(input_geom))
+            assert geom_2d.take(1)[0][0] == expected_geom
\ No newline at end of file
diff --git a/sql/src/main/scala/org/apache/sedona/sql/UDF/Catalog.scala 
b/sql/src/main/scala/org/apache/sedona/sql/UDF/Catalog.scala
index d9fac6f5..32af95f6 100644
--- a/sql/src/main/scala/org/apache/sedona/sql/UDF/Catalog.scala
+++ b/sql/src/main/scala/org/apache/sedona/sql/UDF/Catalog.scala
@@ -106,6 +106,7 @@ object Catalog {
     ST_PointOnSurface,
     ST_Reverse,
     ST_AsEWKT,
+    ST_Force_2D,
     // Expression for rasters
     RS_NormalizedDifference,
     RS_Mean,
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 0cc3e1d1..52d53323 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
@@ -1573,6 +1573,28 @@ case class ST_Reverse(inputExpressions: Seq[Expression])
   }
 }
 
+/**
+ * Forces the geometries into a "2-dimensional mode" so that all output 
representations will only have the X and Y coordinates.
+ *
+ * @param inputExpressions
+ */
+case class ST_Force_2D(inputExpressions: Seq[Expression])
+  extends UnaryGeometryExpression with CodegenFallback {
+  assert(inputExpressions.length == 1)
+
+  override protected def nullSafeEval(geometry: Geometry): Any = {
+    new 
GenericArrayData(GeometrySerializer.serialize(GeomUtils.get2dGeom(geometry)))
+  }
+
+  override def dataType: DataType = GeometryUDT
+
+  override def children: Seq[Expression] = inputExpressions
+
+  protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]) = 
{
+    copy(inputExpressions = newChildren)
+  }
+}
+
 /**
  * Returns the geometry in EWKT format
  *
diff --git a/sql/src/test/scala/org/apache/sedona/sql/functionTestScala.scala 
b/sql/src/test/scala/org/apache/sedona/sql/functionTestScala.scala
index 2f7d0845..70df33d0 100644
--- a/sql/src/test/scala/org/apache/sedona/sql/functionTestScala.scala
+++ b/sql/src/test/scala/org/apache/sedona/sql/functionTestScala.scala
@@ -1344,6 +1344,23 @@ class functionTestScala extends TestBaseScala with 
Matchers with GeometrySample
     assert(df.first().get(0).asInstanceOf[String] == s)
   }
 
+  it ("Should pass ST_Force_2D") {
+    val geomTestCases1 = Map(
+      "'POINT(0 5)'"
+        -> "POINT (0 5)",
+      "'POLYGON((0 0 2, 0 5 2, 5 0 2, 0 0 2), (1 1 2, 3 1 2, 1 3 2, 1 1 2))'"
+        ->"POLYGON ((0 0, 0 5, 5 0, 0 0), (1 1, 3 1, 1 3, 1 1))",
+      "'LINESTRING(0 5 1, 0 0 1, 0 10 2)'"
+        ->"LINESTRING (0 5, 0 0, 0 10)"
+    )
+
+    for((inputGeom, expectedGeom) <- geomTestCases1) {
+      var df = sparkSession.sql(s"select 
ST_AsText(ST_Force_2D(ST_GeomFromText($inputGeom)))")
+      var result = df.collect()
+      assert(result.head.get(0).asInstanceOf[String]==expectedGeom)
+    }
+  }
+
   it("handles nulls") {
     var functionDf: DataFrame = null
     functionDf = sparkSession.sql("select ST_Distance(null, null)")
@@ -1456,5 +1473,7 @@ class functionTestScala extends TestBaseScala with 
Matchers with GeometrySample
     assert(functionDf.first().get(0) == null)
     functionDf = sparkSession.sql("select ST_AsEWKT(null)")
     assert(functionDf.first().get(0) == null)
+    functionDf = sparkSession.sql("select ST_Force_2D(null)")
+    assert(functionDf.first().get(0) == null)
   }
 }

Reply via email to