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/04/29 20:12:02 UTC
[incubator-sedona] branch master updated: [SEDONA-110] Add STGeomFromGeohash to Flink API (#625)
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 21cfa42e [SEDONA-110] Add STGeomFromGeohash to Flink API (#625)
21cfa42e is described below
commit 21cfa42e34e769e2dcc326d98bbcfd964f540d74
Author: Netanel Malka <ne...@gmail.com>
AuthorDate: Fri Apr 29 23:11:56 2022 +0300
[SEDONA-110] Add STGeomFromGeohash to Flink API (#625)
---
docs/api/flink/Constructor.md | 13 ++++++++
.../main/java/org/apache/sedona/flink/Catalog.java | 1 +
.../sedona/flink/expressions/Constructors.java | 16 +++++++++
.../org/apache/sedona/flink/ConstructorTest.java | 38 ++++++++++++++++++++++
4 files changed, 68 insertions(+)
diff --git a/docs/api/flink/Constructor.md b/docs/api/flink/Constructor.md
index d241939e..05b7d5ef 100644
--- a/docs/api/flink/Constructor.md
+++ b/docs/api/flink/Constructor.md
@@ -81,3 +81,16 @@ SELECT *
FROM pointdf
WHERE ST_Contains(ST_PolygonFromEnvelope(1.0,100.0,1000.0,1100.0), pointdf.pointshape)
```
+
+## ST_GeomFromGeoHash
+
+Introduction: Create Geometry from geohash string and optional precision
+
+Format: `ST_GeomFromGeoHash(geohash: string, precision: int)`
+
+Since: `v1.2.1`
+
+SQL example:
+```SQL
+SELECT ST_GeomFromGeoHash('s00twy01mt', 4) AS geom
+```
\ 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 f5628ee5..940b984b 100644
--- a/flink/src/main/java/org/apache/sedona/flink/Catalog.java
+++ b/flink/src/main/java/org/apache/sedona/flink/Catalog.java
@@ -26,6 +26,7 @@ public class Catalog {
new Constructors.ST_GeomFromWKT(),
new Constructors.ST_GeomFromWKB(),
new Constructors.ST_GeomFromGeoJSON(),
+ new Constructors.ST_GeomFromGeoHash(),
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 6eb55b3e..1a9fcb41 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
@@ -18,6 +18,7 @@ import org.apache.flink.table.functions.ScalarFunction;
import org.apache.sedona.core.enums.FileDataSplitter;
import org.apache.sedona.core.enums.GeometryType;
import org.apache.sedona.core.formatMapper.FormatUtils;
+import org.apache.spark.sql.sedona_sql.expressions.geohash.GeoHashDecoder;
import org.locationtech.jts.geom.Coordinate;
import org.locationtech.jts.geom.Geometry;
import org.locationtech.jts.geom.GeometryFactory;
@@ -91,4 +92,19 @@ public class Constructors {
return formatUtils.readGeometry(geoJson);
}
}
+
+ public static class ST_GeomFromGeoHash extends ScalarFunction {
+ @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class)
+ public Geometry eval(@DataTypeHint("String") String value,
+ @DataTypeHint("Int") Integer precision) throws ParseException {
+ // The default precision is the geohash length. Otherwise, use the precision given by the user
+ scala.Option<Object> optionPrecision = scala.Option.apply(precision);
+ return GeoHashDecoder.decode(value, optionPrecision);
+ }
+
+ @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class)
+ public Geometry eval(@DataTypeHint("String") String value) throws ParseException {
+ return eval(value, null);
+ }
+ }
}
\ 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 b0bc7908..e046679d 100644
--- a/flink/src/test/java/org/apache/sedona/flink/ConstructorTest.java
+++ b/flink/src/test/java/org/apache/sedona/flink/ConstructorTest.java
@@ -23,6 +23,7 @@ import org.locationtech.jts.geom.Geometry;
import org.locationtech.jts.geom.GeometryFactory;
import org.wololo.jts2geojson.GeoJSONReader;
+import java.util.ArrayList;
import java.util.List;
import static org.apache.flink.table.api.Expressions.$;
@@ -100,4 +101,41 @@ public class ConstructorTest extends TestBase{
assertEquals(result, expectedGeom);
}
+
+ @Test
+ public void testGeomFromGeoHash() {
+ Integer precision = 2;
+ List<Row> data = new ArrayList<>();
+ data.add(Row.of("2131s12fd", "polygon"));
+
+ Table geohashTable = createTextTable(data, polygonColNames);
+ Table geomTable = geohashTable
+ .select(call(Constructors.ST_GeomFromGeoHash.class.getSimpleName(),
+ $(polygonColNames[0]), precision)
+ .as(polygonColNames[0]), $(polygonColNames[1]));
+ String result = first(geomTable)
+ .getFieldAs(0)
+ .toString();
+ String expectedGeom = "POLYGON ((-180 -39.375, -180 -33.75, -168.75 -33.75, -168.75 -39.375, -180 -39.375))";
+
+ assertEquals(result, expectedGeom);
+ }
+
+ @Test
+ public void testGeomFromGeoHashNullPrecision() {
+ List<Row> data = new ArrayList<>();
+ data.add(Row.of("2131s12fd", "polygon"));
+
+ Table geohashTable = createTextTable(data, polygonColNames);
+ Table geomTable = geohashTable
+ .select(call(Constructors.ST_GeomFromGeoHash.class.getSimpleName(),
+ $(polygonColNames[0]))
+ .as(polygonColNames[0]), $(polygonColNames[1]));
+ String result = first(geomTable)
+ .getFieldAs(0)
+ .toString();
+ String expectedGeom = "POLYGON ((-178.4168529510498 -37.69778251647949, -178.4168529510498 -37.697739601135254, -178.41681003570557 -37.697739601135254, -178.41681003570557 -37.69778251647949, -178.4168529510498 -37.69778251647949))";
+
+ assertEquals(result, expectedGeom);
+ }
}