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/sedona.git
The following commit(s) were added to refs/heads/master by this push:
new e69a872bf [SEDONA-563] Add ST_GeomFromEWKB (#1433)
e69a872bf is described below
commit e69a872bf97de5cd4683aa6fb12472b42017bfce
Author: Jia Yu <[email protected]>
AuthorDate: Tue May 28 08:59:06 2024 -0700
[SEDONA-563] Add ST_GeomFromEWKB (#1433)
* [1] add ST_GeomFromEWKB (#139)
* feat: add ST_GeomFromEWKB
* docs: fix hyperlinks
* docs: remove version number
* Update version
---------
Co-authored-by: Furqaan Khan <[email protected]>
---
docs/api/flink/Constructor.md | 36 ++++++++++++++++++++++
docs/api/snowflake/vector-data/Constructor.md | 34 ++++++++++++++++++++
docs/api/sql/Constructor.md | 36 ++++++++++++++++++++++
.../main/java/org/apache/sedona/flink/Catalog.java | 1 +
.../sedona/flink/expressions/Constructors.java | 14 +++++++++
.../org/apache/sedona/flink/ConstructorTest.java | 24 +++++++++++++++
python/sedona/sql/st_constructors.py | 11 +++++++
python/tests/sql/test_constructor_test.py | 12 ++++++++
python/tests/sql/test_dataframe_api.py | 2 ++
.../sedona/snowflake/snowsql/TestConstructors.java | 8 +++++
.../org/apache/sedona/snowflake/snowsql/UDFs.java | 5 +++
.../scala/org/apache/sedona/sql/UDF/Catalog.scala | 1 +
.../sql/sedona_sql/expressions/Constructors.scala | 32 +++++++++++++++++++
.../sedona_sql/expressions/st_constructors.scala | 3 ++
.../apache/sedona/sql/constructorTestScala.scala | 22 +++++++++++++
.../apache/sedona/sql/dataFrameAPITestScala.scala | 8 +++++
16 files changed, 249 insertions(+)
diff --git a/docs/api/flink/Constructor.md b/docs/api/flink/Constructor.md
index 428ad5957..c63a8f84c 100644
--- a/docs/api/flink/Constructor.md
+++ b/docs/api/flink/Constructor.md
@@ -186,6 +186,42 @@ Output:
POINT(40.7128 -74.006)
```
+## ST_GeomFromEWKB
+
+Introduction: Construct a Geometry from EWKB string or Binary. This function
is an alias of [ST_GeomFromWKB](#st_geomfromwkb).
+
+Format:
+
+`ST_GeomFromEWKB (Wkb: String)`
+
+`ST_GeomFromEWKB (Wkb: Binary)`
+
+Since: `v1.6.1`
+
+SQL Example
+
+```sql
+SELECT ST_GeomFromEWKB([01 02 00 00 00 02 00 00 00 00 00 00 00 84 D6 00 C0 00
00 00 00 80 B5 D6 BF 00 00 00 60 E1 EF F7 BF 00 00 00 80 07 5D E5 BF])
+```
+
+Output:
+
+```
+LINESTRING (-2.1047439575195312 -0.354827880859375, -1.49606454372406
-0.6676061153411865)
+```
+
+SQL Example
+
+```sql
+SELECT
ST_asEWKT(ST_GeomFromEWKB('01010000a0e6100000000000000000f03f000000000000f03f000000000000f03f'))
+```
+
+Output:
+
+```
+SRID=4326;POINT Z(1 1 1)
+```
+
## ST_GeomFromWKB
Introduction: Construct a Geometry from WKB string or Binary. This function
also supports EWKB format.
diff --git a/docs/api/snowflake/vector-data/Constructor.md
b/docs/api/snowflake/vector-data/Constructor.md
index 954a05c2d..e7b98f9dc 100644
--- a/docs/api/snowflake/vector-data/Constructor.md
+++ b/docs/api/snowflake/vector-data/Constructor.md
@@ -178,6 +178,40 @@ Output:
POINT(40.7128 -74.006)
```
+## ST_GeomFromEWKB
+
+Introduction: Construct a Geometry from EWKB string or Binary. This function
is an alias of [ST_GeomFromWKB](#st_geomfromwkb).
+
+Format:
+
+`ST_GeomFromEWKB (Wkb: String)`
+
+`ST_GeomFromEWKB (Wkb: Binary)`
+
+SQL Example
+
+```sql
+SELECT ST_GeomFromEWKB([01 02 00 00 00 02 00 00 00 00 00 00 00 84 D6 00 C0 00
00 00 00 80 B5 D6 BF 00 00 00 60 E1 EF F7 BF 00 00 00 80 07 5D E5 BF])
+```
+
+Output:
+
+```
+LINESTRING (-2.1047439575195312 -0.354827880859375, -1.49606454372406
-0.6676061153411865)
+```
+
+SQL Example
+
+```sql
+SELECT
ST_asEWKT(ST_GeomFromEWKB('01010000a0e6100000000000000000f03f000000000000f03f000000000000f03f'))
+```
+
+Output:
+
+```
+SRID=4326;POINT Z(1 1 1)
+```
+
## ST_GeomFromWKB
Introduction: Construct a Geometry from WKB string or Binary. This function
also supports EWKB format.
diff --git a/docs/api/sql/Constructor.md b/docs/api/sql/Constructor.md
index efe6e62ca..9738f7024 100644
--- a/docs/api/sql/Constructor.md
+++ b/docs/api/sql/Constructor.md
@@ -243,6 +243,42 @@ Output:
POINT(40.7128 -74.006)
```
+## ST_GeomFromEWKB
+
+Introduction: Construct a Geometry from EWKB string or Binary. This function
is an alias of [ST_GeomFromWKB](#st_geomfromwkb).
+
+Format:
+
+`ST_GeomFromEWKB (Wkb: String)`
+
+`ST_GeomFromEWKB (Wkb: Binary)`
+
+Since: `v1.6.1`
+
+SQL Example
+
+```sql
+SELECT ST_GeomFromEWKB([01 02 00 00 00 02 00 00 00 00 00 00 00 84 D6 00 C0 00
00 00 00 80 B5 D6 BF 00 00 00 60 E1 EF F7 BF 00 00 00 80 07 5D E5 BF])
+```
+
+Output:
+
+```
+LINESTRING (-2.1047439575195312 -0.354827880859375, -1.49606454372406
-0.6676061153411865)
+```
+
+SQL Example
+
+```sql
+SELECT
ST_asEWKT(ST_GeomFromEWKB('01010000a0e6100000000000000000f03f000000000000f03f000000000000f03f'))
+```
+
+Output:
+
+```
+SRID=4326;POINT Z(1 1 1)
+```
+
## ST_GeomFromWKB
Introduction: Construct a Geometry from WKB string or Binary. This function
also supports EWKB format.
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 efd07c3ca..ad2c9a022 100644
--- a/flink/src/main/java/org/apache/sedona/flink/Catalog.java
+++ b/flink/src/main/java/org/apache/sedona/flink/Catalog.java
@@ -34,6 +34,7 @@ public class Catalog {
new Constructors.ST_GeomFromEWKT(),
new Constructors.ST_GeomFromText(),
new Constructors.ST_GeomFromWKB(),
+ new Constructors.ST_GeomFromEWKB(),
new Constructors.ST_GeomFromGeoJSON(),
new Constructors.ST_GeomFromGeoHash(),
new Constructors.ST_GeomFromGML(),
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 827088728..7054e32d5 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
@@ -179,6 +179,20 @@ public class Constructors {
}
+ public static class ST_GeomFromEWKB extends ScalarFunction {
+ @DataTypeHint(value = "RAW", bridgedTo =
org.locationtech.jts.geom.Geometry.class)
+ public Geometry eval(@DataTypeHint("String") String wkbString) throws
ParseException {
+ return getGeometryByFileData(wkbString, FileDataSplitter.WKB);
+ }
+
+ @DataTypeHint(value = "RAW", bridgedTo =
org.locationtech.jts.geom.Geometry.class)
+ public Geometry eval(@DataTypeHint("Bytes") byte[] wkb) throws
ParseException {
+ WKBReader wkbReader = new WKBReader();
+ return wkbReader.read(wkb);
+ }
+
+ }
+
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 {
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 d7b621484..5470f96d0 100644
--- a/flink/src/test/java/org/apache/sedona/flink/ConstructorTest.java
+++ b/flink/src/test/java/org/apache/sedona/flink/ConstructorTest.java
@@ -268,6 +268,30 @@ public class ConstructorTest extends TestBase{
}
+ @Test
+ public void testGeomFromEWKB()
+ {
+ byte[] wkb = new byte[]{1, 2, 0, 0, 0, 2, 0, 0, 0, 0, 0, 0, 0, -124,
-42, 0, -64, 0, 0, 0, 0, -128, -75, -42, -65, 0, 0, 0, 96, -31, -17, -9, -65,
0, 0, 0, -128, 7, 93, -27, -65};
+ List<Row> data = new ArrayList<>();
+ data.add(Row.of(wkb, "polygon"));
+ TypeInformation<?>[] colTypes = {
+ PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO,
+ BasicTypeInfo.STRING_TYPE_INFO};
+ RowTypeInfo typeInfo = new RowTypeInfo(colTypes,
Arrays.copyOfRange(polygonColNames, 0, 2));
+ DataStream<Row> wkbDS = env.fromCollection(data).returns(typeInfo);
+ Table wkbTable = tableEnv.fromDataStream(wkbDS, $(polygonColNames[0]),
$(polygonColNames[1]));
+
+ Table geomTable = wkbTable.select(
+ call(Constructors.ST_GeomFromEWKB.class.getSimpleName(),
$(polygonColNames[0])).
+ as(polygonColNames[0]), $(polygonColNames[1]));
+ String result = first(geomTable).
+ getFieldAs(0).toString();
+
+ String expectedGeom = "LINESTRING (-2.1047439575195312
-0.354827880859375, -1.49606454372406 -0.6676061153411865)";
+
+ assertEquals(expectedGeom, result);
+ }
+
@Test
public void testGeomFromGeoHash() {
Integer precision = 2;
diff --git a/python/sedona/sql/st_constructors.py
b/python/sedona/sql/st_constructors.py
index ae9237abc..11b561bee 100644
--- a/python/sedona/sql/st_constructors.py
+++ b/python/sedona/sql/st_constructors.py
@@ -109,6 +109,17 @@ def ST_GeomFromWKB(wkb: ColumnOrName) -> Column:
"""
return _call_constructor_function("ST_GeomFromWKB", wkb)
+@validate_argument_types
+def ST_GeomFromEWKB(wkb: ColumnOrName) -> Column:
+ """Generate a geometry column from a Well-Known Binary (WKB) binary column.
+
+ :param wkb: WKB binary column to generate from.
+ :type wkb: ColumnOrName
+ :return: Geometry column representing the WKB binary.
+ :rtype: Column
+ """
+ return _call_constructor_function("ST_GeomFromEWKB", wkb)
+
@validate_argument_types
def ST_GeomFromWKT(wkt: ColumnOrName, srid: Optional[ColumnOrNameOrNumber] =
None) -> Column:
diff --git a/python/tests/sql/test_constructor_test.py
b/python/tests/sql/test_constructor_test.py
index 9d0d52bac..2180972cd 100644
--- a/python/tests/sql/test_constructor_test.py
+++ b/python/tests/sql/test_constructor_test.py
@@ -123,6 +123,18 @@ class TestConstructors(TestBase):
polygon_df.show(10)
assert polygon_df.count() == 100
+ def test_st_geom_from_ewkb(self):
+ polygon_wkb_df = self.spark.read.format("csv"). \
+ option("delimiter", "\t"). \
+ option("header", "false"). \
+ load(mixed_wkb_geometry_input_location)
+
+ polygon_wkb_df.createOrReplaceTempView("polygontable")
+ polygon_wkb_df.show()
+ polygon_df = self.spark.sql("select ST_GeomFromEWKB(polygontable._c0)
as countyshape from polygontable")
+ polygon_df.show(10)
+ assert polygon_df.count() == 100
+
def test_st_geom_from_geojson(self):
polygon_json_df = self.spark.read.format("csv").\
option("delimiter", "\t").\
diff --git a/python/tests/sql/test_dataframe_api.py
b/python/tests/sql/test_dataframe_api.py
index 56ec8212a..cabe4b005 100644
--- a/python/tests/sql/test_dataframe_api.py
+++ b/python/tests/sql/test_dataframe_api.py
@@ -45,6 +45,7 @@ test_configurations = [
(stc.ST_GeomFromText, ("wkt",), "linestring_wkt", "", "LINESTRING (1 2, 3
4)"),
(stc.ST_GeomFromText, ("wkt",4326), "linestring_wkt", "", "LINESTRING (1
2, 3 4)"),
(stc.ST_GeomFromWKB, ("wkb",), "constructor", "ST_ReducePrecision(geom,
2)", "LINESTRING (-2.1 -0.35, -1.5 -0.67)"),
+ (stc.ST_GeomFromEWKB, ("wkb",), "constructor", "ST_ReducePrecision(geom,
2)", "LINESTRING (-2.1 -0.35, -1.5 -0.67)"),
(stc.ST_GeomFromWKT, ("wkt",), "linestring_wkt", "", "LINESTRING (1 2, 3
4)"),
(stc.ST_GeomFromWKT, ("wkt",4326), "linestring_wkt", "", "LINESTRING (1 2,
3 4)"),
(stc.ST_GeomFromEWKT, ("ewkt",), "linestring_ewkt", "", "LINESTRING (1 2,
3 4)"),
@@ -224,6 +225,7 @@ wrong_type_configurations = [
(stc.ST_GeomFromKML, (None,)),
(stc.ST_GeomFromText, (None,)),
(stc.ST_GeomFromWKB, (None,)),
+ (stc.ST_GeomFromEWKB, (None,)),
(stc.ST_GeomFromWKT, (None,)),
(stc.ST_LineFromText, (None,)),
(stc.ST_LineStringFromText, (None, "")),
diff --git
a/snowflake-tester/src/test/java/org/apache/sedona/snowflake/snowsql/TestConstructors.java
b/snowflake-tester/src/test/java/org/apache/sedona/snowflake/snowsql/TestConstructors.java
index 7fa950c7e..477f5f0c2 100644
---
a/snowflake-tester/src/test/java/org/apache/sedona/snowflake/snowsql/TestConstructors.java
+++
b/snowflake-tester/src/test/java/org/apache/sedona/snowflake/snowsql/TestConstructors.java
@@ -90,6 +90,14 @@ public class TestConstructors extends TestBase{
);
}
@Test
+ public void test_ST_GeomFromEWKB() {
+ registerUDF("ST_GeomFromEWKB", byte[].class);
+ verifySqlSingleRes(
+ "select
sedona.ST_AsText(sedona.ST_GeomFromEWKB(ST_ASWKB(to_geometry('POINT (0.0
1.0)'))))",
+ "POINT (0 1)"
+ );
+ }
+ @Test
public void test_ST_LineFromText() {
registerUDF("ST_LineFromText", String.class);
verifySqlSingleRes(
diff --git
a/snowflake/src/main/java/org/apache/sedona/snowflake/snowsql/UDFs.java
b/snowflake/src/main/java/org/apache/sedona/snowflake/snowsql/UDFs.java
index bad3f0c89..0342778bf 100644
--- a/snowflake/src/main/java/org/apache/sedona/snowflake/snowsql/UDFs.java
+++ b/snowflake/src/main/java/org/apache/sedona/snowflake/snowsql/UDFs.java
@@ -497,6 +497,11 @@ public class UDFs {
return wkb;
}
+ @UDFAnnotations.ParamMeta(argNames = {"wkb"})
+ public static byte[] ST_GeomFromEWKB(byte[] wkb) throws ParseException {
+ return wkb;
+ }
+
@UDFAnnotations.ParamMeta(argNames = {"wkt", "srid"})
public static byte[] ST_GeomFromWKT(String wkt, int srid) throws
ParseException {
return GeometrySerde.serialize(
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 bd2e801ad..a0b34d78e 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
@@ -45,6 +45,7 @@ object Catalog {
function[ST_GeomFromWKT](0),
function[ST_GeomFromEWKT](),
function[ST_GeomFromWKB](),
+ function[ST_GeomFromEWKB](),
function[ST_GeomFromGeoJSON](),
function[ST_GeomFromGML](),
function[ST_GeomFromKML](),
diff --git
a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/Constructors.scala
b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/Constructors.scala
index dfeb0370c..6ed6d7335 100644
---
a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/Constructors.scala
+++
b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/Constructors.scala
@@ -156,6 +156,38 @@ case class ST_GeomFromWKB(inputExpressions:
Seq[Expression])
}
}
+case class ST_GeomFromEWKB(inputExpressions: Seq[Expression])
+ extends Expression with FoldableExpression with ImplicitCastInputTypes with
CodegenFallback with UserDataGeneratator {
+ // This is an expression which takes one input expressions
+ assert(inputExpressions.length == 1)
+
+ override def nullable: Boolean = true
+
+ override def eval(inputRow: InternalRow): Any = {
+ (inputExpressions.head.eval(inputRow)) match {
+ case (geomString: UTF8String) => {
+ // Parse UTF-8 encoded wkb string
+ Constructors.geomFromText(geomString.toString,
FileDataSplitter.WKB).toGenericArrayData
+ }
+ case (wkb: Array[Byte]) => {
+ // convert raw wkb byte array to geometry
+ Constructors.geomFromWKB(wkb).toGenericArrayData
+ }
+ case null => null
+ }
+ }
+
+ override def dataType: DataType = GeometryUDT
+
+ override def inputTypes: Seq[AbstractDataType] =
Seq(TypeCollection(StringType, BinaryType))
+
+ override def children: Seq[Expression] = inputExpressions
+
+ protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]) =
{
+ copy(inputExpressions = newChildren)
+ }
+}
+
/**
* Return a Geometry from a GeoJSON string
*
diff --git
a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/st_constructors.scala
b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/st_constructors.scala
index ffca69ef1..02d055a1f 100644
---
a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/st_constructors.scala
+++
b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/st_constructors.scala
@@ -48,6 +48,9 @@ object st_constructors extends DataFrameAPI {
def ST_GeomFromWKB(wkb: Column): Column = wrapExpression[ST_GeomFromWKB](wkb)
def ST_GeomFromWKB(wkb: String): Column = wrapExpression[ST_GeomFromWKB](wkb)
+ def ST_GeomFromEWKB(wkb: Column): Column =
wrapExpression[ST_GeomFromEWKB](wkb)
+ def ST_GeomFromEWKB(wkb: String): Column =
wrapExpression[ST_GeomFromEWKB](wkb)
+
def ST_GeomFromWKT(wkt: Column): Column =
wrapExpression[ST_GeomFromWKT](wkt, 0)
def ST_GeomFromWKT(wkt: String): Column =
wrapExpression[ST_GeomFromWKT](wkt, 0)
diff --git
a/spark/common/src/test/scala/org/apache/sedona/sql/constructorTestScala.scala
b/spark/common/src/test/scala/org/apache/sedona/sql/constructorTestScala.scala
index 3caf0e88f..9d6b416bd 100644
---
a/spark/common/src/test/scala/org/apache/sedona/sql/constructorTestScala.scala
+++
b/spark/common/src/test/scala/org/apache/sedona/sql/constructorTestScala.scala
@@ -210,6 +210,28 @@ class constructorTestScala extends TestBaseScala {
}
}
+ it("Passed ST_GeomFromEWKB") {
+ // UTF-8 encoded WKB String
+ val polygonWkbDf = sparkSession.read.format("csv").option("delimiter",
"\t").option("header", "false").load(mixedWkbGeometryInputLocation)
+ polygonWkbDf.createOrReplaceTempView("polygontable")
+ val polygonDf = sparkSession.sql("select
ST_GeomFromEWKB(polygontable._c0) as countyshape from polygontable")
+ assert(polygonDf.count() == 100)
+ // RAW binary array
+ val wkbSeq = Seq[Array[Byte]](Array[Byte](1, 2, 0, 0, 0, 2, 0, 0, 0, 0,
0, 0, 0, -124, -42, 0, -64, 0, 0, 0, 0, -128, -75, -42, -65, 0, 0, 0, 96, -31,
-17, -9, -65, 0, 0, 0, -128, 7, 93, -27, -65))
+ val rawWkbDf = wkbSeq.toDF("wkb")
+ rawWkbDf.createOrReplaceTempView("rawWKBTable")
+ val geometries = sparkSession.sql("SELECT
ST_GeomFromEWKB(rawWKBTable.wkb) as countyshape from rawWKBTable")
+ val expectedGeom = "LINESTRING (-2.1047439575195312 -0.354827880859375,
-1.49606454372406 -0.6676061153411865)";
+
assert(geometries.first().getAs[Geometry](0).toString.equals(expectedGeom))
+ // null input
+ val nullGeom = sparkSession.sql("SELECT ST_GeomFromEWKB(null)")
+ assert(nullGeom.first().isNullAt(0))
+ // Fail on wrong input type
+ intercept[Exception] {
+ sparkSession.sql("SELECT ST_GeomFromEWKB(0)").collect()
+ }
+ }
+
it("Passed ST_GeomFromGeoJSON") {
val polygonJsonDf = sparkSession.read.format("csv").option("delimiter",
"\t").option("header", "false").load(geojsonInputLocation)
polygonJsonDf.createOrReplaceTempView("polygontable")
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 1ff67eeeb..da4584abf 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
@@ -130,6 +130,14 @@ class dataFrameAPITestScala extends TestBaseScala {
assert(actualResult == expectedResult)
}
+ it("passed st_geomfromewkb") {
+ val wkbSeq = Seq[Array[Byte]](Array[Byte](1, 2, 0, 0, 0, 2, 0, 0, 0, 0,
0, 0, 0, -124, -42, 0, -64, 0, 0, 0, 0, -128, -75, -42, -65, 0, 0, 0, 96, -31,
-17, -9, -65, 0, 0, 0, -128, 7, 93, -27, -65))
+ val df = wkbSeq.toDF("wkb").select(ST_GeomFromEWKB("wkb"))
+ val actualResult = df.take(1)(0).get(0).asInstanceOf[Geometry].toText()
+ val expectedResult = "LINESTRING (-2.1047439575195312
-0.354827880859375, -1.49606454372406 -0.6676061153411865)"
+ assert(actualResult == expectedResult)
+ }
+
it("passed st_geomfromgeojson") {
val geojson = "{ \"type\": \"Feature\", \"properties\": { \"prop\":
\"01\" }, \"geometry\": { \"type\": \"Point\", \"coordinates\": [ 0.0, 1.0 ]
}},"
val df =
Seq[String](geojson).toDF("geojson").select(ST_GeomFromGeoJSON("geojson"))