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 d9153d297 [SEDONA-533] Add ST_Polygonize (#1342)
d9153d297 is described below

commit d9153d297158d97cec716e6d37905d41de3084e6
Author: Jia Yu <[email protected]>
AuthorDate: Wed Apr 17 23:23:18 2024 -0700

    [SEDONA-533] Add ST_Polygonize (#1342)
    
    * [TASK-19] Add ST_Polygonize (#142)
    
    * add ST_Polygonize
    
    * add docs
    
    * Fix python test
    
    * Fix tests
    
    * fix python test
    
    * fix python test
    
    * fix tests
    
    * Update docs
    
    * fix test
    
    * fix test
    
    * fix test
    
    * fix test
    
    * fix tests
    
    * fix tests
    
    * fix test
    
    * fix test
    
    * fix test
    
    * fix test
    
    * Fix test
    
    * Fix typo
    
    * Fix docs
    
    ---------
    
    Co-authored-by: Jia Yu <[email protected]>
    
    * Update ST_Polygonize version to OSS Sedona 1.6.0
    
    ---------
    
    Co-authored-by: Pranav Toggi <[email protected]>
---
 .../java/org/apache/sedona/common/Functions.java   | 28 +++++++++++++
 .../org/apache/sedona/common/FunctionsTest.java    | 48 ++++++++++++++++++++++
 docs/api/flink/Function.md                         | 23 +++++++++++
 docs/api/snowflake/vector-data/Function.md         | 21 ++++++++++
 docs/api/sql/Function.md                           | 23 +++++++++++
 .../main/java/org/apache/sedona/flink/Catalog.java |  1 +
 .../apache/sedona/flink/expressions/Functions.java |  8 ++++
 .../java/org/apache/sedona/flink/FunctionTest.java | 12 ++++++
 python/sedona/sql/st_functions.py                  | 12 ++++++
 python/tests/sql/test_dataframe_api.py             |  3 ++
 python/tests/sql/test_function.py                  | 24 +++++++++++
 .../sedona/snowflake/snowsql/TestFunctions.java    | 10 +++++
 .../sedona/snowflake/snowsql/TestFunctionsV2.java  | 10 +++++
 .../org/apache/sedona/snowflake/snowsql/UDFs.java  |  9 ++++
 .../apache/sedona/snowflake/snowsql/UDFsV2.java    |  9 ++++
 .../scala/org/apache/sedona/sql/UDF/Catalog.scala  |  1 +
 .../sql/sedona_sql/expressions/Functions.scala     |  8 ++++
 .../sql/sedona_sql/expressions/st_functions.scala  |  3 ++
 .../apache/sedona/sql/dataFrameAPITestScala.scala  |  9 ++++
 .../org/apache/sedona/sql/functionTestScala.scala  | 12 ++++++
 20 files changed, 274 insertions(+)

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 2f24f27e8..5cec2a42d 100644
--- a/common/src/main/java/org/apache/sedona/common/Functions.java
+++ b/common/src/main/java/org/apache/sedona/common/Functions.java
@@ -41,6 +41,7 @@ import org.locationtech.jts.operation.distance.DistanceOp;
 import org.locationtech.jts.operation.distance3d.Distance3DOp;
 import org.locationtech.jts.operation.linemerge.LineMerger;
 import org.locationtech.jts.operation.overlay.snap.GeometrySnapper;
+import org.locationtech.jts.operation.polygonize.Polygonizer;
 import org.locationtech.jts.operation.valid.IsSimpleOp;
 import org.locationtech.jts.operation.valid.IsValidOp;
 import org.locationtech.jts.operation.valid.TopologyValidationError;
@@ -1661,4 +1662,31 @@ public class Functions {
             return error.toString();
         }
     }
+
+    /**
+     * This method takes a GeometryCollection and returns another 
GeometryCollection
+     * containing the polygons formed by the linework of a set of geometries.
+     * @param geometry A collection of Geometry objects that should contain 
only linestrings.
+     * @return A GeometryCollection containing the resultant polygons.
+     */
+    public static Geometry polygonize(Geometry geometry) {
+        if (geometry == null || geometry.isEmpty()) {
+            return GEOMETRY_FACTORY.createGeometryCollection(null);
+        }
+
+        if (geometry instanceof GeometryCollection) {
+            Polygonizer polygonizer = new Polygonizer();
+
+            for (int i = 0; i < geometry.getNumGeometries(); i++) {
+                polygonizer.add(geometry.getGeometryN(i));
+            }
+
+            Collection polygons = polygonizer.getPolygons();
+            Geometry[] polyArray = (Geometry[]) polygons.toArray(new 
Geometry[0]);
+
+            return GEOMETRY_FACTORY.createGeometryCollection(polyArray);
+        } else {
+            return GEOMETRY_FACTORY.createGeometryCollection(null);
+        }
+    }
 }
diff --git a/common/src/test/java/org/apache/sedona/common/FunctionsTest.java 
b/common/src/test/java/org/apache/sedona/common/FunctionsTest.java
index ed176190b..701a24594 100644
--- a/common/src/test/java/org/apache/sedona/common/FunctionsTest.java
+++ b/common/src/test/java/org/apache/sedona/common/FunctionsTest.java
@@ -1018,6 +1018,54 @@ public class FunctionsTest extends TestBase {
         assertEquals(4.036497016235249E11, Spheroid.area(geometryCollection), 
0.1);
     }
 
+    @Test
+    public void pologonize() throws ParseException {
+        LineString line1 = GEOMETRY_FACTORY.createLineString(coordArray(180, 
40, 30, 20, 20, 90));
+        LineString line2 = GEOMETRY_FACTORY.createLineString(coordArray(180, 
40, 160, 160));
+        LineString line3 = GEOMETRY_FACTORY.createLineString(coordArray(80, 
60, 120, 130, 150, 80));
+        LineString line4 = GEOMETRY_FACTORY.createLineString(coordArray(80, 
60, 150, 80));
+        LineString line5 = GEOMETRY_FACTORY.createLineString(coordArray(20, 
90, 70, 70, 80, 130));
+        LineString line6 = GEOMETRY_FACTORY.createLineString(coordArray(80, 
130, 160, 160));
+        LineString line7 = GEOMETRY_FACTORY.createLineString(coordArray(20, 
90, 20, 160, 70 ,190));
+        LineString line8 = GEOMETRY_FACTORY.createLineString(coordArray(70, 
190, 80, 130));
+        LineString line9 = GEOMETRY_FACTORY.createLineString(coordArray(70, 
190, 160, 160));
+
+        LineString line10 = 
GEOMETRY_FACTORY.createLineString(coordArray(0,0,0,1,0,2));
+        LineString line11 = 
GEOMETRY_FACTORY.createLineString(coordArray(4,2,4,1,4,0));
+        LineString line12 = 
GEOMETRY_FACTORY.createLineString(coordArray(4,0,3,0,2,0,1,0,0,0));
+        LineString line13 = 
GEOMETRY_FACTORY.createLineString(coordArray(2,0,2,1,2,2));
+        LineString line14 = 
GEOMETRY_FACTORY.createLineString(coordArray(2,2,2,3,2,4));
+        LineString line15 = 
GEOMETRY_FACTORY.createLineString(coordArray(0,2,1,2,2,2));
+        LineString line16 = 
GEOMETRY_FACTORY.createLineString(coordArray(2,2,3,2,4,2));
+        LineString line17 = 
GEOMETRY_FACTORY.createLineString(coordArray(0,2,1,3,2,4));
+        LineString line18 = 
GEOMETRY_FACTORY.createLineString(coordArray(2,4,3,3,4,2));
+
+        GeometryCollection geometryCollection1 = 
GEOMETRY_FACTORY.createGeometryCollection(new Geometry[] {line1, line2, line3, 
line4, line5, line6, line7, line8, line9});
+        GeometryCollection geometryCollection2 = 
GEOMETRY_FACTORY.createGeometryCollection(new Geometry[] {line10, line11, 
line12, line13, line14, line15, line16, line17, line18});
+        GeometryCollection geometryCollection3 = 
GEOMETRY_FACTORY.createGeometryCollection(new Geometry[] {line10, line11, 
line12, line13, line15, line16});
+        GeometryCollection geometryCollection4 = 
GEOMETRY_FACTORY.createGeometryCollection(new Geometry[] {line13, line14, 
line15, line16, line17, line18});
+
+        Geometry expected1 = geomFromEWKT("GEOMETRYCOLLECTION (POLYGON ((20 
90, 20 160, 70 190, 80 130, 70 70, 20 90)), POLYGON ((20 90, 70 70, 80 130, 160 
160, 180 40, 30 20, 20 90), (80 60, 150 80, 120 130, 80 60)), POLYGON ((70 190, 
160 160, 80 130, 70 190)), POLYGON ((80 60, 120 130, 150 80, 80 60)))");
+        Geometry result1 = Functions.polygonize(geometryCollection1);
+        result1.normalize();
+        assertEquals(expected1, result1);
+
+        Geometry expected2 = geomFromEWKT("GEOMETRYCOLLECTION (POLYGON ((0 0, 
0 1, 0 2, 1 2, 2 2, 3 2, 4 2, 4 1, 4 0, 3 0, 2 0, 1 0, 0 0)), POLYGON ((0 2, 1 
3, 2 4, 2 3, 2 2, 1 2, 0 2)), POLYGON ((2 2, 2 3, 2 4, 3 3, 4 2, 3 2, 2 2)))");
+        Geometry result2 = Functions.polygonize(geometryCollection2);
+        result2.normalize();
+        assertEquals(expected2, result2);
+
+        Geometry expected3 = geomFromEWKT("GEOMETRYCOLLECTION (POLYGON ((0 0, 
0 1, 0 2, 1 2, 2 2, 3 2, 4 2, 4 1, 4 0, 3 0, 2 0, 1 0, 0 0)))");
+        Geometry result3 = Functions.polygonize(geometryCollection3);
+        result3.normalize();
+        assertEquals(expected3, result3);
+
+        Geometry expected4 = geomFromEWKT("GEOMETRYCOLLECTION (POLYGON ((0 2, 
1 3, 2 4, 2 3, 2 2, 1 2, 0 2)), POLYGON ((2 2, 2 3, 2 4, 3 3, 4 2, 3 2, 2 
2)))");
+        Geometry result4 = Functions.polygonize(geometryCollection4);
+        result4.normalize();
+        assertEquals(expected4, result4);
+    }
+
     @Test
     public void spheroidLength() {
         Point point = GEOMETRY_FACTORY.createPoint(new Coordinate(90, 0));
diff --git a/docs/api/flink/Function.md b/docs/api/flink/Function.md
index 2eae89966..41b42f47e 100644
--- a/docs/api/flink/Function.md
+++ b/docs/api/flink/Function.md
@@ -2405,6 +2405,29 @@ Output:
 POLYGON((75 29 1, 77 29 2, 77 29 3, 75 29 1))
 ```
 
+## ST_Polygonize
+
+Introduction: Generates a GeometryCollection composed of polygons that are 
formed from the linework of an input GeometryCollection. When the input does 
not contain any linework that forms a polygon, the function will return an 
empty GeometryCollection.
+
+!!!note
+    `ST_Polygonize` function assumes that the input geometries form a valid 
and simple closed linestring that can be turned into a polygon. If the input 
geometries are not noded or do not form such linestrings, the resulting 
GeometryCollection may be empty or may not contain the expected polygons.
+
+Format: `ST_Polygonize(geom: Geometry)`
+
+Since: `v1.6.0`
+
+Example:
+
+```sql
+SELECT ST_AsText(ST_Polygonize(ST_GeomFromEWKT('GEOMETRYCOLLECTION (LINESTRING 
(2 0, 2 1, 2 2), LINESTRING (2 2, 2 3, 2 4), LINESTRING (0 2, 1 2, 2 2), 
LINESTRING (2 2, 3 2, 4 2), LINESTRING (0 2, 1 3, 2 4), LINESTRING (2 4, 3 3, 4 
2))')));
+```
+
+Output:
+
+```
+GEOMETRYCOLLECTION (POLYGON ((0 2, 1 3, 2 4, 2 3, 2 2, 1 2, 0 2)), POLYGON ((2 
2, 2 3, 2 4, 3 3, 4 2, 3 2, 2 2)))
+```
+
 ## ST_ReducePrecision
 
 Introduction: Reduce the decimals places in the coordinates of the geometry to 
the given number of decimal places. The last decimal place will be rounded.
diff --git a/docs/api/snowflake/vector-data/Function.md 
b/docs/api/snowflake/vector-data/Function.md
index 4c4cede1e..d9834bacc 100644
--- a/docs/api/snowflake/vector-data/Function.md
+++ b/docs/api/snowflake/vector-data/Function.md
@@ -1867,6 +1867,27 @@ Output:
 POLYGON((75 29 1, 77 29 2, 77 29 3, 75 29 1))
 ```
 
+## ST_Polygonize
+
+Introduction: Generates a GeometryCollection composed of polygons that are 
formed from the linework of an input GeometryCollection. When the input does 
not contain any linework that forms a polygon, the function will return an 
empty GeometryCollection.
+
+!!!note
+    `ST_Polygonize` function assumes that the input geometries form a valid 
and simple closed linestring that can be turned into a polygon. If the input 
geometries are not noded or do not form such linestrings, the resulting 
GeometryCollection may be empty or may not contain the expected polygons.
+
+Format: `ST_Polygonize(geom: Geometry)`
+
+Example:
+
+```sql
+SELECT ST_AsText(ST_Polygonize(ST_GeomFromEWKT('GEOMETRYCOLLECTION (LINESTRING 
(2 0, 2 1, 2 2), LINESTRING (2 2, 2 3, 2 4), LINESTRING (0 2, 1 2, 2 2), 
LINESTRING (2 2, 3 2, 4 2), LINESTRING (0 2, 1 3, 2 4), LINESTRING (2 4, 3 3, 4 
2))')));
+```
+
+Output:
+
+```
+GEOMETRYCOLLECTION (POLYGON ((0 2, 1 3, 2 4, 2 3, 2 2, 1 2, 0 2)), POLYGON ((2 
2, 2 3, 2 4, 3 3, 4 2, 3 2, 2 2)))
+```
+
 ## ST_ReducePrecision
 
 Introduction: Reduce the decimals places in the coordinates of the geometry to 
the given number of decimal places. The last decimal place will be rounded. 
This function was called ST_PrecisionReduce in versions prior to v1.5.0.
diff --git a/docs/api/sql/Function.md b/docs/api/sql/Function.md
index 6bcd15f0a..a61708d29 100644
--- a/docs/api/sql/Function.md
+++ b/docs/api/sql/Function.md
@@ -2413,6 +2413,29 @@ Output:
 POLYGON((75 29 1, 77 29 2, 77 29 3, 75 29 1))
 ```
 
+## ST_Polygonize
+
+Introduction: Generates a GeometryCollection composed of polygons that are 
formed from the linework of an input GeometryCollection. When the input does 
not contain any linework that forms a polygon, the function will return an 
empty GeometryCollection.
+
+!!!note
+    `ST_Polygonize` function assumes that the input geometries form a valid 
and simple closed linestring that can be turned into a polygon. If the input 
geometries are not noded or do not form such linestrings, the resulting 
GeometryCollection may be empty or may not contain the expected polygons.
+
+Format: `ST_Polygonize(geom: Geometry)`
+
+Since: `v1.6.0`
+
+Example:
+
+```sql
+SELECT ST_AsText(ST_Polygonize(ST_GeomFromEWKT('GEOMETRYCOLLECTION (LINESTRING 
(2 0, 2 1, 2 2), LINESTRING (2 2, 2 3, 2 4), LINESTRING (0 2, 1 2, 2 2), 
LINESTRING (2 2, 3 2, 4 2), LINESTRING (0 2, 1 3, 2 4), LINESTRING (2 4, 3 3, 4 
2))')));
+```
+
+Output:
+
+```
+GEOMETRYCOLLECTION (POLYGON ((0 2, 1 3, 2 4, 2 3, 2 2, 1 2, 0 2)), POLYGON ((2 
2, 2 3, 2 4, 3 3, 4 2, 3 2, 2 2)))
+```
+
 ## ST_ReducePrecision
 
 Introduction: Reduce the decimals places in the coordinates of the geometry to 
the given number of decimal places. The last decimal place will be rounded. 
This function was called ST_PrecisionReduce in versions prior to v1.5.0.
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 fa27fa43d..efd07c3ca 100644
--- a/flink/src/main/java/org/apache/sedona/flink/Catalog.java
+++ b/flink/src/main/java/org/apache/sedona/flink/Catalog.java
@@ -123,6 +123,7 @@ public class Catalog {
                 new Functions.ST_LineSubstring(),
                 new Functions.ST_MakeLine(),
                 new Functions.ST_Polygon(),
+                new Functions.ST_Polygonize(),
                 new Functions.ST_MakePolygon(),
                 new Functions.ST_MakeValid(),
                 new Functions.ST_MinimumBoundingCircle(),
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 dd913d14f..013fb7241 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
@@ -783,6 +783,14 @@ public class Functions {
         }
     }
 
+    public static class ST_Polygonize 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 o1) {
+            Geometry geom = (Geometry) o1;
+            return org.apache.sedona.common.Functions.polygonize(geom);
+        }
+    }
+
     public static class ST_MakeValid 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,
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 524f868f8..0306e5b52 100644
--- a/flink/src/test/java/org/apache/sedona/flink/FunctionTest.java
+++ b/flink/src/test/java/org/apache/sedona/flink/FunctionTest.java
@@ -17,6 +17,7 @@ import org.apache.commons.codec.binary.Hex;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.flink.table.api.Table;
 import org.apache.flink.table.planner.expressions.In;
+import org.apache.sedona.flink.expressions.Constructors;
 import org.apache.sedona.flink.expressions.Functions;
 import org.apache.sedona.flink.expressions.FunctionsGeoTools;
 import org.geotools.referencing.CRS;
@@ -28,6 +29,7 @@ import org.locationtech.jts.io.ParseException;
 import org.locationtech.jts.operation.buffer.BufferParameters;
 import org.opengis.referencing.FactoryException;
 import org.opengis.referencing.crs.CoordinateReferenceSystem;
+import scala.collection.immutable.Stream;
 
 import java.util.Arrays;
 import java.util.List;
@@ -935,6 +937,16 @@ public class FunctionTest extends TestBase{
         assertEquals(4236, result.getSRID());
     }
 
+    @Test
+    public void testPolygonize() {
+        Table table = tableEnv.sqlQuery("SELECT 
ST_GeomFromEWKT('GEOMETRYCOLLECTION (LINESTRING (180 40, 30 20, 20 90), 
LINESTRING (180 40, 160 160), LINESTRING (80 60, 120 130, 150 80), LINESTRING 
(80 60, 150 80), LINESTRING (20 90, 70 70, 80 130), LINESTRING (80 130, 160 
160), LINESTRING (20 90, 20 160, 70 190), LINESTRING (70 190, 80 130), 
LINESTRING (70 190, 160 160))') AS geom");
+        table = 
table.select(call(Functions.ST_Polygonize.class.getSimpleName(), $("geom")));
+        Geometry result = (Geometry) first(table).getField(0);
+        result.normalize();
+        String expected = "GEOMETRYCOLLECTION (POLYGON ((20 90, 20 160, 70 
190, 80 130, 70 70, 20 90)), POLYGON ((20 90, 70 70, 80 130, 160 160, 180 40, 
30 20, 20 90), (80 60, 150 80, 120 130, 80 60)), POLYGON ((70 190, 160 160, 80 
130, 70 190)), POLYGON ((80 60, 120 130, 150 80, 80 60)))";
+        assertEquals(expected, result.toString());
+    }
+
     @Test
     public void testMakePolygon() {
         Table table = tableEnv.sqlQuery("SELECT ST_GeomFromWKT('LINESTRING (0 
0, 1 0, 1 1, 0 0)') AS line");
diff --git a/python/sedona/sql/st_functions.py 
b/python/sedona/sql/st_functions.py
index c5386fe96..3e9cb21f0 100644
--- a/python/sedona/sql/st_functions.py
+++ b/python/sedona/sql/st_functions.py
@@ -86,6 +86,7 @@ __all__ = [
     "ST_LineSubstring",
     "ST_MakeLine",
     "ST_Polygon"
+    "ST_Polygonize"
     "ST_MakePolygon",
     "ST_MakeValid",
     "ST_MinimumBoundingCircle",
@@ -996,6 +997,17 @@ def ST_Polygon(line_string: ColumnOrName, srid: 
ColumnOrNameOrNumber) -> Column:
     """
     return _call_st_function("ST_Polygon", (line_string, srid))
 
+@validate_argument_types
+def ST_Polygonize(geometry: ColumnOrName) -> Column:
+    """Generates a GeometryCollection composed of polygons that are formed 
from the linework of a set of input geometries.
+
+    :param geometry: input geometry of type GeometryCollection
+    :type geometry: ColumnOrName
+    :return: GeometryCollection geometry column created from the input 
geometry.
+    :rtype: Column
+    """
+    return _call_st_function("ST_Polygonize", (geometry))
+
 @validate_argument_types
 def ST_MakePolygon(line_string: ColumnOrName, holes: Optional[ColumnOrName] = 
None) -> Column:
     """Create a polygon geometry from a linestring describing the exterior 
ring as well as an array of linestrings describing holes.
diff --git a/python/tests/sql/test_dataframe_api.py 
b/python/tests/sql/test_dataframe_api.py
index c28cdf767..ca319ab34 100644
--- a/python/tests/sql/test_dataframe_api.py
+++ b/python/tests/sql/test_dataframe_api.py
@@ -131,6 +131,7 @@ test_configurations = [
     (stf.ST_MakeValid, ("geom",), "invalid_geom", "", "MULTIPOLYGON (((1 5, 3 
3, 1 1, 1 5)), ((5 3, 7 5, 7 1, 5 3)))"),
     (stf.ST_MakeLine, ("line1", "line2"), "two_lines", "", "LINESTRING (0 0, 1 
1, 0 0, 3 2)"),
     (stf.ST_Polygon, ("geom", 4236), "closed_linestring_geom", "", "POLYGON 
((0 0, 1 0, 1 1, 0 0))"),
+    (stf.ST_Polygonize, ("geom",), "noded_linework", "ST_Normalize(geom)", 
"GEOMETRYCOLLECTION (POLYGON ((0 2, 1 3, 2 4, 2 3, 2 2, 1 2, 0 2)), POLYGON ((2 
2, 2 3, 2 4, 3 3, 4 2, 3 2, 2 2)))"),
     (stf.ST_MakePolygon, ("geom",), "closed_linestring_geom", "", "POLYGON ((0 
0, 1 0, 1 1, 0 0))"),
     (stf.ST_MinimumBoundingCircle, ("line", 8), "linestring_geom", 
"ST_ReducePrecision(geom, 2)", "POLYGON ((4.95 -0.49, 4.81 -0.96, 4.58 -1.39, 
4.27 -1.77, 3.89 -2.08, 3.46 -2.31, 2.99 -2.45, 2.5 -2.5, 2.01 -2.45, 1.54 
-2.31, 1.11 -2.08, 0.73 -1.77, 0.42 -1.39, 0.19 -0.96, 0.05 -0.49, 0 0, 0.05 
0.49, 0.19 0.96, 0.42 1.39, 0.73 1.77, 1.11 2.08, 1.54 2.31, 2.01 2.45, 2.5 
2.5, 2.99 2.45, 3.46 2.31, 3.89 2.08, 4.27 1.77, 4.58 1.39, 4.81 0.96, 4.95 
0.49, 5 0, 4.95 -0.49))"),
     (stf.ST_MinimumBoundingCircle, ("line", 2), "linestring_geom", 
"ST_ReducePrecision(geom, 2)", "POLYGON ((4.27 -1.77, 2.5 -2.5, 0.73 -1.77, 0 
0, 0.73 1.77, 2.5 2.5, 4.27 1.77, 5 0, 4.27 -1.77))"),
@@ -467,6 +468,8 @@ class TestDataFrameAPI(TestBase):
             return TestDataFrameAPI.spark.sql("SELECT 
array(ST_GeomFromWKT('POLYGON ((-3 -3, 3 -3, 3 3, -3 3, -3 -3))'), 
ST_GeomFromWKT('POLYGON ((-2 1, 2 1, 2 4, -2 4, -2 1))')) as polys")
         elif request.param == "poly_and_line":
             return TestDataFrameAPI.spark.sql("SELECT 
ST_GeomFromWKT('POLYGON((2.6 12.5, 2.6 20.0, 12.6 20.0, 12.6 12.5, 2.6 12.5 
))') as poly, ST_GeomFromWKT('LINESTRING (0.5 10.7, 5.4 8.4, 10.1 10.0)') as 
line")
+        elif request.param == "noded_linework":
+            return TestDataFrameAPI.spark.sql("SELECT 
ST_GeomFromWKT('GEOMETRYCOLLECTION (LINESTRING (2 0, 2 1, 2 2), LINESTRING (2 
2, 2 3, 2 4), LINESTRING (0 2, 1 2, 2 2), LINESTRING (2 2, 3 2, 4 2), 
LINESTRING (0 2, 1 3, 2 4), LINESTRING (2 4, 3 3, 4 2))') as geom")
         raise ValueError(f"Invalid base_df name passed: {request.param}")
 
     def _id_test_configuration(val):
diff --git a/python/tests/sql/test_function.py 
b/python/tests/sql/test_function.py
index 783ae1640..009459d5b 100644
--- a/python/tests/sql/test_function.py
+++ b/python/tests/sql/test_function.py
@@ -973,6 +973,30 @@ class TestPredicateJoin(TestBase):
         for actual, expected in result:
             assert actual == expected
 
+    def test_st_polygonize(self):
+        # Given
+        geometry_df = self.spark.createDataFrame(
+            [
+                # Adding only the input that will result in a non-null polygon
+                ["GEOMETRYCOLLECTION (LINESTRING (2 0, 2 1, 2 2), LINESTRING 
(2 2, 2 3, 2 4), LINESTRING (0 2, 1 2, 2 2), LINESTRING (2 2, 3 2, 4 2), 
LINESTRING (0 2, 1 3, 2 4), LINESTRING (2 4, 3 3, 4 2))", "GEOMETRYCOLLECTION 
(POLYGON ((0 2, 1 3, 2 4, 2 3, 2 2, 1 2, 0 2)), POLYGON ((2 2, 2 3, 2 4, 3 3, 4 
2, 3 2, 2 2)))"]
+            ]
+        ).selectExpr("ST_GeomFromText(_1) AS geom", "_2 AS expected")
+
+        # When calling st_polygonize
+        geom_poly = geometry_df.withColumn("actual", 
expr("st_normalize(st_polygonize(geom))"))
+
+        # Then only based on closed linestring geom is created
+        geom_poly.filter("actual IS NOT NULL").selectExpr("ST_AsText(actual)", 
"expected"). \
+            show()
+        result = geom_poly.filter("actual IS NOT 
NULL").selectExpr("ST_AsText(actual)", "expected"). \
+            collect()
+
+        assert result.__len__() == 1
+
+        for actual, expected in result:
+            assert actual == expected
+
+
     def test_st_make_polygon(self):
         # Given
         geometry_df = self.spark.createDataFrame(
diff --git 
a/snowflake-tester/src/test/java/org/apache/sedona/snowflake/snowsql/TestFunctions.java
 
b/snowflake-tester/src/test/java/org/apache/sedona/snowflake/snowsql/TestFunctions.java
index 0ba9b1b99..710baefd7 100644
--- 
a/snowflake-tester/src/test/java/org/apache/sedona/snowflake/snowsql/TestFunctions.java
+++ 
b/snowflake-tester/src/test/java/org/apache/sedona/snowflake/snowsql/TestFunctions.java
@@ -715,6 +715,16 @@ public class TestFunctions extends TestBase {
         );
     }
 
+    @Test
+    public void test_ST_Polygonize() {
+        registerUDF("ST_Polygonize", byte[].class);
+        registerUDF("ST_Area", byte[].class);
+        verifySqlSingleRes(
+                "select 
sedona.ST_Area(sedona.ST_Polygonize(sedona.ST_GeomFromText('GEOMETRYCOLLECTION 
(LINESTRING (2 0, 2 1, 2 2), LINESTRING (2 2, 2 3, 2 4), LINESTRING (0 2, 1 2, 
2 2), LINESTRING (2 2, 3 2, 4 2), LINESTRING (0 2, 1 3, 2 4), LINESTRING (2 4, 
3 3, 4 2))')))",
+                4.0
+        );
+    }
+
     @Test
     public void test_ST_PrecisionReduce() {
         registerUDF("ST_PrecisionReduce", byte[].class, int.class);
diff --git 
a/snowflake-tester/src/test/java/org/apache/sedona/snowflake/snowsql/TestFunctionsV2.java
 
b/snowflake-tester/src/test/java/org/apache/sedona/snowflake/snowsql/TestFunctionsV2.java
index b37c02cc1..3921b4f09 100644
--- 
a/snowflake-tester/src/test/java/org/apache/sedona/snowflake/snowsql/TestFunctionsV2.java
+++ 
b/snowflake-tester/src/test/java/org/apache/sedona/snowflake/snowsql/TestFunctionsV2.java
@@ -665,6 +665,16 @@ public class TestFunctionsV2
         );
     }
 
+    @Test
+    public void test_ST_Polygonize() {
+        registerUDFV2("ST_Polygonize", String.class);
+        registerUDFV2("ST_Area", String.class);
+        verifySqlSingleRes(
+                "select 
ST_Area(sedona.ST_Polygonize(ST_GeometryFromWKT('GEOMETRYCOLLECTION (LINESTRING 
(2 0, 2 1, 2 2), LINESTRING (2 2, 2 3, 2 4), LINESTRING (0 2, 1 2, 2 2), 
LINESTRING (2 2, 3 2, 4 2), LINESTRING (0 2, 1 3, 2 4), LINESTRING (2 4, 3 3, 4 
2))')))",
+                4.0
+        );
+    }
+
     @Test
     public void test_ST_PrecisionReduce() {
         registerUDFV2("ST_PrecisionReduce", String.class, int.class);
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 df9c4a109..bad3f0c89 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
@@ -947,6 +947,15 @@ public class UDFs {
         );
     }
 
+    @UDFAnnotations.ParamMeta(argNames = {"geometry"})
+    public static byte[] ST_Polygonize(byte[] geometry) {
+        return GeometrySerde.serialize(
+                Functions.polygonize(
+                        GeometrySerde.deserialize(geometry)
+                )
+        );
+    }
+
     @UDFAnnotations.ParamMeta(argNames = {"minX", "minY", "maxX", "maxY"})
     public static byte[] ST_PolygonFromEnvelope(double minX, double minY, 
double maxX, double maxY) {
         return GeometrySerde.serialize(
diff --git 
a/snowflake/src/main/java/org/apache/sedona/snowflake/snowsql/UDFsV2.java 
b/snowflake/src/main/java/org/apache/sedona/snowflake/snowsql/UDFsV2.java
index 177d7b00b..2c6ed7907 100644
--- a/snowflake/src/main/java/org/apache/sedona/snowflake/snowsql/UDFsV2.java
+++ b/snowflake/src/main/java/org/apache/sedona/snowflake/snowsql/UDFsV2.java
@@ -810,6 +810,15 @@ public class UDFsV2
         );
     }
 
+    @UDFAnnotations.ParamMeta(argNames = {"geometry"}, argTypes = 
{"Geometry"}, returnTypes = "Geometry")
+    public static String ST_Polygonize(String geometry) {
+        return GeometrySerde.serGeoJson(
+                Functions.polygonize(
+                        GeometrySerde.deserGeoJson(geometry)
+                )
+        );
+    }
+
     @UDFAnnotations.ParamMeta(argNames = {"geometry", "precisionScale"}, 
argTypes = {"Geometry", "int"}, returnTypes = "Geometry")
     public static String ST_PrecisionReduce(String geometry, int 
precisionScale) {
         return GeometrySerde.serGeoJson(
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 e727e0809..011e89b07 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
@@ -140,6 +140,7 @@ object Catalog {
     function[ST_SubDivide](),
     function[ST_MakeLine](),
     function[ST_Polygon](),
+    function[ST_Polygonize](),
     function[ST_MakePolygon](null),
     function[ST_GeoHash](),
     function[ST_GeomFromGeoHash](null),
diff --git 
a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/Functions.scala
 
b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/Functions.scala
index 1cf73cf6f..4b9c09c03 100644
--- 
a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/Functions.scala
+++ 
b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/Functions.scala
@@ -755,6 +755,14 @@ case class ST_Polygon(inputExpressions: Seq[Expression])
   }
 }
 
+case class ST_Polygonize(inputExpressions: Seq[Expression])
+  extends InferredExpression(Functions.polygonize _) {
+
+  protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]) = 
{
+    copy(inputExpressions = newChildren)
+  }
+}
+
 case class ST_MakePolygon(inputExpressions: Seq[Expression])
   extends 
InferredExpression(InferrableFunction.allowRightNull(Functions.makePolygon)) {
 
diff --git 
a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/st_functions.scala
 
b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/st_functions.scala
index cd637f3ec..b78917303 100644
--- 
a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/st_functions.scala
+++ 
b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/st_functions.scala
@@ -220,6 +220,9 @@ object st_functions extends DataFrameAPI {
   def ST_Polygon(lineString: Column, srid: Column): Column = 
wrapExpression[ST_Polygon](lineString, srid)
   def ST_Polygon(lineString: String, srid: Integer): Column = 
wrapExpression[ST_Polygon](lineString, srid)
 
+  def ST_Polygonize(geoms: Column): Column = 
wrapExpression[ST_Polygonize](geoms)
+  def ST_Polygonize(geoms: String): Column = 
wrapExpression[ST_Polygonize](geoms)
+
   def ST_MakePolygon(lineString: Column): Column = 
wrapExpression[ST_MakePolygon](lineString, null)
   def ST_MakePolygon(lineString: String): Column = 
wrapExpression[ST_MakePolygon](lineString, null)
   def ST_MakePolygon(lineString: Column, holes: Column): Column = 
wrapExpression[ST_MakePolygon](lineString, holes)
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 5ab1da700..1ff67eeeb 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
@@ -435,6 +435,15 @@ class dataFrameAPITestScala extends TestBaseScala {
       assert(actualResult.getSRID() == 4236)
     }
 
+    it("Passed ST_Polygonize") {
+      val invalidDf = sparkSession.sql("SELECT 
ST_GeomFromWKT('GEOMETRYCOLLECTION (LINESTRING (2 0, 2 1, 2 2), LINESTRING (2 
2, 2 3, 2 4), LINESTRING (0 2, 1 2, 2 2), LINESTRING (2 2, 3 2, 4 2), 
LINESTRING (0 2, 1 3, 2 4), LINESTRING (2 4, 3 3, 4 2))') AS geom")
+      val df = invalidDf.select(ST_Polygonize("geom"))
+      val actualResult = df.take(1)(0).get(0).asInstanceOf[Geometry]
+      actualResult.normalize()
+      val expectedResult = "GEOMETRYCOLLECTION (POLYGON ((0 2, 1 3, 2 4, 2 3, 
2 2, 1 2, 0 2)), POLYGON ((2 2, 2 3, 2 4, 3 3, 4 2, 3 2, 2 2)))"
+      assert(actualResult.toText()  == expectedResult)
+    }
+
     it("Passed `ST_MakePolygon`") {
       val invalidDf = sparkSession.sql("SELECT ST_GeomFromWKT('LINESTRING (0 
0, 1 0, 1 1, 0 0)') AS geom")
       val df = invalidDf.select(ST_MakePolygon("geom"))
diff --git 
a/spark/common/src/test/scala/org/apache/sedona/sql/functionTestScala.scala 
b/spark/common/src/test/scala/org/apache/sedona/sql/functionTestScala.scala
index c1cd9cfcf..3fee1e53c 100644
--- a/spark/common/src/test/scala/org/apache/sedona/sql/functionTestScala.scala
+++ b/spark/common/src/test/scala/org/apache/sedona/sql/functionTestScala.scala
@@ -399,6 +399,18 @@ class functionTestScala extends TestBaseScala with 
Matchers with GeometrySample
       assert(testtable.take(1)(0).get(0).asInstanceOf[Geometry].getSRID == 
4326)
     }
 
+    it("Passed ST_Polygonize") {
+
+      val testtable = sparkSession.sql(
+        "SELECT ST_Polygonize(ST_GeomFromText('GEOMETRYCOLLECTION (LINESTRING 
(180 40, 30 20, 20 90), LINESTRING (180 40, 160 160), LINESTRING (80 60, 120 
130, 150 80), LINESTRING (80 60, 150 80), LINESTRING (20 90, 70 70, 80 130), 
LINESTRING (80 130, 160 160), LINESTRING (20 90, 20 160, 70 190), LINESTRING 
(70 190, 80 130), LINESTRING (70 190, 160 160))'))"
+      )
+
+      val result = testtable.take(1)(0).get(0).asInstanceOf[Geometry]
+
+      result.normalize()
+      assert(result.toText()  == "GEOMETRYCOLLECTION (POLYGON ((20 90, 20 160, 
70 190, 80 130, 70 70, 20 90)), POLYGON ((20 90, 70 70, 80 130, 160 160, 180 
40, 30 20, 20 90), (80 60, 150 80, 120 130, 80 60)), POLYGON ((70 190, 160 160, 
80 130, 70 190)), POLYGON ((80 60, 120 130, 150 80, 80 60)))")
+    }
+
     it("Passed ST_MakeValid On Invalid Polygon") {
 
       val df = sparkSession.sql("SELECT ST_GeomFromWKT('POLYGON((1 5, 1 1, 3 
3, 5 3, 7 1, 7 5, 5 3, 3 3, 1 5))') AS polygon")

Reply via email to