You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sedona.apache.org by ma...@apache.org on 2022/03/20 20:36:08 UTC

[incubator-sedona] 01/01: Add ST_GeomFromGeoJSON to flink API

This is an automated email from the ASF dual-hosted git repository.

malka pushed a commit to branch flinkGeomFromGeoJSON
in repository https://gitbox.apache.org/repos/asf/incubator-sedona.git

commit 8630653e7d27f1187a9099ad6c77aa861f7ce59b
Author: Netanel Malka <ne...@gmail.com>
AuthorDate: Sun Mar 20 22:35:54 2022 +0200

    Add ST_GeomFromGeoJSON to flink API
---
 docs/api/flink/Constructor.md                      | 14 +++++++++
 .../main/java/org/apache/sedona/flink/Catalog.java |  1 +
 .../sedona/flink/expressions/Constructors.java     |  8 +++++
 .../org/apache/sedona/flink/ConstructorTest.java   | 20 ++++++++++++
 .../java/org/apache/sedona/flink/TestBase.java     | 36 ++++++++++++++++++++--
 5 files changed, 76 insertions(+), 3 deletions(-)

diff --git a/docs/api/flink/Constructor.md b/docs/api/flink/Constructor.md
index 275bdc9..d241939 100644
--- a/docs/api/flink/Constructor.md
+++ b/docs/api/flink/Constructor.md
@@ -27,6 +27,20 @@ SELECT ST_GeomFromWKB(polygontable._c0) AS polygonshape
 FROM polygontable
 ```
 
+## ST_GeomFromGeoJSON
+
+Introduction: Construct a Geometry from GeoJson
+
+Format: `ST_GeomFromGeoJSON (GeoJson:string)`
+
+Since: `v1.2.0`
+
+SQL example:
+```SQL
+SELECT ST_GeomFromGeoJSON(polygontable._c0) AS polygonshape
+FROM polygontable
+```
+
 ## ST_PointFromText
 
 Introduction: Construct a Point from Text, delimited by Delimiter
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 0ab4607..440f4ad 100644
--- a/flink/src/main/java/org/apache/sedona/flink/Catalog.java
+++ b/flink/src/main/java/org/apache/sedona/flink/Catalog.java
@@ -25,6 +25,7 @@ public class Catalog {
                 new Constructors.ST_PolygonFromEnvelope(),
                 new Constructors.ST_GeomFromWKT(),
                 new Constructors.ST_GeomFromWKB(),
+                new Constructors.ST_GeomFromGeoJSON(),
                 new Functions.ST_Buffer(),
                 new Functions.ST_Distance(),
                 new Functions.ST_Transform(),
diff --git a/flink/src/main/java/org/apache/sedona/flink/expressions/Constructors.java b/flink/src/main/java/org/apache/sedona/flink/expressions/Constructors.java
index b337104..6eb55b3 100644
--- a/flink/src/main/java/org/apache/sedona/flink/expressions/Constructors.java
+++ b/flink/src/main/java/org/apache/sedona/flink/expressions/Constructors.java
@@ -83,4 +83,12 @@ public class Constructors {
             return formatUtils.readGeometry(wkbString);
         }
     }
+
+    public static class ST_GeomFromGeoJSON extends ScalarFunction {
+        @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class)
+        public Geometry eval(@DataTypeHint("String") String geoJson) throws ParseException {
+            FormatUtils formatUtils = new FormatUtils(FileDataSplitter.GEOJSON, false);
+            return formatUtils.readGeometry(geoJson);
+        }
+    }
 }
\ No newline at end of file
diff --git a/flink/src/test/java/org/apache/sedona/flink/ConstructorTest.java b/flink/src/test/java/org/apache/sedona/flink/ConstructorTest.java
index af14c99..b0bc790 100644
--- a/flink/src/test/java/org/apache/sedona/flink/ConstructorTest.java
+++ b/flink/src/test/java/org/apache/sedona/flink/ConstructorTest.java
@@ -21,6 +21,7 @@ import org.junit.Test;
 import org.locationtech.jts.geom.Coordinate;
 import org.locationtech.jts.geom.Geometry;
 import org.locationtech.jts.geom.GeometryFactory;
+import org.wololo.jts2geojson.GeoJSONReader;
 
 import java.util.List;
 
@@ -80,4 +81,23 @@ public class ConstructorTest extends TestBase{
                 .getField(0).toString());
 
     }
+
+    @Test
+    public void testGeomFromGeoJSON() {
+        List<Row> data = createPolygonGeoJSON(testDataSize);
+        Table geojsonTable = createTextTable(data, polygonColNames);
+        Table geomTable = geojsonTable
+                .select(call(Constructors.ST_GeomFromGeoJSON.class.getSimpleName(), $(polygonColNames[0]))
+                        .as(polygonColNames[0]), $(polygonColNames[1]));
+        String result = last(geomTable)
+                .getFieldAs(0)
+                .toString();
+
+        GeoJSONReader reader = new GeoJSONReader();
+        String expectedGeoJSON = data.get(data.size() - 1)
+                .getFieldAs(0);
+        String expectedGeom = reader.read(expectedGeoJSON).toText();
+
+        assertEquals(result, expectedGeom);
+    }
 }
diff --git a/flink/src/test/java/org/apache/sedona/flink/TestBase.java b/flink/src/test/java/org/apache/sedona/flink/TestBase.java
index 97f7fe8..9623d23 100644
--- a/flink/src/test/java/org/apache/sedona/flink/TestBase.java
+++ b/flink/src/test/java/org/apache/sedona/flink/TestBase.java
@@ -27,12 +27,15 @@ import org.apache.flink.util.CloseableIterator;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.apache.sedona.flink.expressions.Constructors;
-import org.locationtech.jts.geom.Coordinate;
-import org.locationtech.jts.geom.GeometryFactory;
-import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.*;
+import org.locationtech.jts.geom.impl.CoordinateArraySequence;
+import org.wololo.geojson.Feature;
+import org.wololo.jts2geojson.GeoJSONWriter;
 
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import static org.apache.flink.table.api.Expressions.$;
 import static org.apache.flink.table.api.Expressions.call;
@@ -143,6 +146,33 @@ public class TestBase {
         return data;
     }
 
+    static List<Row> createPolygonGeoJSON(int size) {
+        List<Row> data = new ArrayList<>();
+        GeometryFactory geometryFactory = new GeometryFactory();
+        GeoJSONWriter writer = new GeoJSONWriter();
+
+        Coordinate[] points = new Coordinate[5];
+        for (int i = 0; i < size; i++) {
+            double minX = i - 0.5;
+            double minY = i - 0.5;
+            double maxX = i + 0.5;
+            double maxY = i + 0.5;
+
+            points[0] = new Coordinate(minX, minY);
+            points[1] = new Coordinate(minX, maxY);
+            points[2] = new Coordinate(maxX, maxY);
+            points[3] = new Coordinate(maxX, minY);
+            points[4] = new Coordinate(minX, minY);
+
+            Geometry polygon = geometryFactory.createPolygon(points);
+
+            String geoJson = writer.write(polygon).toString();
+            data.add(Row.of(geoJson, "polygon" + i));
+        }
+
+        return data;
+    }
+
     static Table createTextTable(List<Row> data, String[] colNames){
         TypeInformation<?>[] colTypes = {
                 BasicTypeInfo.STRING_TYPE_INFO,