diff --git a/Cargo.lock b/Cargo.lock index c0af964a..20e02b3e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5108,6 +5108,7 @@ dependencies = [ "geos", "rstest", "sedona", + "sedona-common", "sedona-expr", "sedona-functions", "sedona-geometry", diff --git a/c/sedona-geos/Cargo.toml b/c/sedona-geos/Cargo.toml index d92f076f..549a76bd 100644 --- a/c/sedona-geos/Cargo.toml +++ b/c/sedona-geos/Cargo.toml @@ -39,6 +39,7 @@ arrow-array = { workspace = true } datafusion-common = { workspace = true } datafusion-expr = { workspace = true } geos = { workspace = true } +sedona-common = { path = "../../rust/sedona-common" } sedona-expr = { path = "../../rust/sedona-expr" } sedona-functions = { path = "../../rust/sedona-functions" } sedona-geometry = { path = "../../rust/sedona-geometry" } diff --git a/c/sedona-geos/src/lib.rs b/c/sedona-geos/src/lib.rs index fc589a1e..282c7687 100644 --- a/c/sedona-geos/src/lib.rs +++ b/c/sedona-geos/src/lib.rs @@ -31,6 +31,7 @@ mod st_isvalid; mod st_isvalidreason; mod st_length; mod st_perimeter; +mod st_polygonize_agg; mod st_reverse; mod st_simplifypreservetopology; mod st_unaryunion; diff --git a/c/sedona-geos/src/register.rs b/c/sedona-geos/src/register.rs index f24404ef..705f234b 100644 --- a/c/sedona-geos/src/register.rs +++ b/c/sedona-geos/src/register.rs @@ -14,6 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. +use sedona_expr::aggregate_udf::SedonaAccumulatorRef; use sedona_expr::scalar_udf::ScalarKernelRef; use crate::{ @@ -29,6 +30,7 @@ use crate::{ st_isvalidreason::st_is_valid_reason_impl, st_length::st_length_impl, st_perimeter::st_perimeter_impl, + st_polygonize_agg::st_polygonize_agg_impl, st_reverse::st_reverse_impl, st_simplifypreservetopology::st_simplify_preserve_topology_impl, st_unaryunion::st_unary_union_impl, @@ -80,3 +82,7 @@ pub fn scalar_kernels() -> Vec<(&'static str, ScalarKernelRef)> { ("st_within", st_within_impl()), ] } + +pub fn aggregate_kernels() -> Vec<(&'static str, SedonaAccumulatorRef)> { + vec![("st_polygonize_agg", st_polygonize_agg_impl())] +} diff --git a/c/sedona-geos/src/st_polygonize_agg.rs b/c/sedona-geos/src/st_polygonize_agg.rs new file mode 100644 index 00000000..13e35d02 --- /dev/null +++ b/c/sedona-geos/src/st_polygonize_agg.rs @@ -0,0 +1,454 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use crate::wkb_to_geos::GEOSWkbFactory; +use arrow_array::{cast::AsArray, types::UInt64Type, Array, ArrayRef}; +use arrow_schema::{DataType, Field, FieldRef}; +use datafusion_common::{cast::as_binary_array, error::Result, DataFusionError, ScalarValue}; +use datafusion_expr::{Accumulator, ColumnarValue}; +use geo_traits::Dimensions; +use geos::Geom; +use sedona_common::sedona_internal_err; +use sedona_expr::aggregate_udf::{SedonaAccumulator, SedonaAccumulatorRef}; +use sedona_geometry::wkb_factory::write_wkb_geometrycollection_header; +use sedona_schema::{ + datatypes::{SedonaType, WKB_GEOMETRY}, + matchers::ArgMatcher, +}; +use wkb::reader::read_wkb; + +/// ST_Polygonize_Agg() aggregate implementation using GEOS +pub fn st_polygonize_agg_impl() -> SedonaAccumulatorRef { + Arc::new(STPolygonizeAgg {}) +} + +#[derive(Debug)] +struct STPolygonizeAgg {} + +impl SedonaAccumulator for STPolygonizeAgg { + fn return_type(&self, args: &[SedonaType]) -> Result> { + let matcher = ArgMatcher::new(vec![ArgMatcher::is_geometry()], WKB_GEOMETRY); + matcher.match_args(args) + } + + fn accumulator( + &self, + args: &[SedonaType], + _output_type: &SedonaType, + ) -> Result> { + Ok(Box::new(PolygonizeAccumulator::new(args[0].clone()))) + } + + fn state_fields(&self, _args: &[SedonaType]) -> Result> { + Ok(vec![ + Arc::new(Field::new("count", DataType::UInt64, false)), + Arc::new(Field::new("item", DataType::Binary, true)), + ]) + } +} + +#[derive(Debug)] +struct PolygonizeAccumulator { + input_type: SedonaType, + item: Option>, + count: usize, +} + +const WKB_HEADER_SIZE: usize = 1 + 4 + 4; + +impl PolygonizeAccumulator { + pub fn new(input_type: SedonaType) -> Self { + let mut item = Vec::new(); + write_wkb_geometrycollection_header(&mut item, Dimensions::Xy, 0) + .expect("Failed to write initial GeometryCollection header"); + + Self { + input_type, + item: Some(item), + count: 0, + } + } + + fn make_wkb_result(&mut self) -> Result>> { + if self.count == 0 { + return Ok(None); + } + + let collection_wkb = self.item.as_mut().unwrap(); + let mut header = Vec::new(); + write_wkb_geometrycollection_header(&mut header, Dimensions::Xy, self.count) + .map_err(|e| DataFusionError::Execution(format!("Failed to write header: {e}")))?; + collection_wkb[0..WKB_HEADER_SIZE].copy_from_slice(&header); + + let wkb = read_wkb(collection_wkb) + .map_err(|e| DataFusionError::Execution(format!("Failed to read WKB: {e}")))?; + + let factory = GEOSWkbFactory::new(); + let collection = factory.create(&wkb).map_err(|e| { + DataFusionError::Execution(format!("Failed to create geometry from WKB: {e}")) + })?; + + let num_geoms = collection.get_num_geometries().map_err(|e| { + DataFusionError::Execution(format!("Failed to get number of geometries: {e}")) + })?; + + let mut geos_geoms = Vec::with_capacity(num_geoms); + for i in 0..num_geoms { + let geom = collection.get_geometry_n(i).map_err(|e| { + DataFusionError::Execution(format!("Failed to get geometry {}: {e}", i)) + })?; + geos_geoms.push(geom.clone()); + } + + let result = geos::Geometry::polygonize(&geos_geoms) + .map_err(|e| DataFusionError::Execution(format!("Failed to polygonize: {e}")))?; + + let wkb = result.to_wkb().map_err(|e| { + DataFusionError::Execution(format!("Failed to convert result to WKB: {e}")) + })?; + + Ok(Some(wkb.into())) + } +} + +impl Accumulator for PolygonizeAccumulator { + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + if values.is_empty() { + return sedona_internal_err!("No input arrays provided to accumulator in update_batch"); + } + + let arg_types = std::slice::from_ref(&self.input_type); + let args = [ColumnarValue::Array(values[0].clone())]; + let executor = sedona_functions::executor::WkbExecutor::new(arg_types, &args); + + let item_ref = self.item.as_mut().unwrap(); + executor.execute_wkb_void(|maybe_item| { + if let Some(item) = maybe_item { + item_ref.extend_from_slice(item.buf()); + self.count += 1; + } + Ok(()) + })?; + + Ok(()) + } + + fn evaluate(&mut self) -> Result { + let wkb = self.make_wkb_result()?; + Ok(ScalarValue::Binary(wkb)) + } + + fn size(&self) -> usize { + let item_capacity = self.item.as_ref().map(|e| e.capacity()).unwrap_or(0); + size_of::() + item_capacity + } + + fn state(&mut self) -> Result> { + let serialized_count = ScalarValue::UInt64(Some(self.count as u64)); + let serialized_item = ScalarValue::Binary(self.item.take()); + + let mut item = Vec::new(); + write_wkb_geometrycollection_header(&mut item, Dimensions::Xy, 0) + .expect("Failed to write initial GeometryCollection header"); + self.item = Some(item); + self.count = 0; + + Ok(vec![serialized_count, serialized_item]) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + if states.len() != 2 { + return sedona_internal_err!( + "Unexpected number of state fields for st_polygonize() (expected 2, got {})", + states.len() + ); + } + + let item_ref = match self.item.as_mut() { + Some(item) => item, + None => return sedona_internal_err!("Unexpected internal state in ST_Polygonize()"), + }; + + let count_array = states[0].as_primitive::(); + let item_array = as_binary_array(&states[1])?; + + for i in 0..count_array.len() { + let count = count_array.value(i) as usize; + if count > 0 && !item_array.is_null(i) { + let item = item_array.value(i); + // Skip the header and append the geometry data + item_ref.extend_from_slice(&item[WKB_HEADER_SIZE..]); + self.count += count; + } + } + + Ok(()) + } +} + +#[cfg(test)] +mod tests { + use datafusion_expr::AggregateUDF; + use rstest::rstest; + use sedona_expr::aggregate_udf::SedonaAggregateUDF; + use sedona_schema::datatypes::{WKB_GEOMETRY, WKB_VIEW_GEOMETRY}; + use sedona_testing::{compare::assert_scalar_equal_wkb_geometry, testers::AggregateUdfTester}; + + use super::*; + + fn create_udf() -> SedonaAggregateUDF { + SedonaAggregateUDF::new( + "st_polygonize_agg", + vec![st_polygonize_agg_impl()], + datafusion_expr::Volatility::Immutable, + None, + ) + } + + #[test] + fn udf_metadata() { + let udf = create_udf(); + let aggregate_udf: AggregateUDF = udf.into(); + assert_eq!(aggregate_udf.name(), "st_polygonize_agg"); + } + + #[rstest] + fn basic_triangle(#[values(WKB_GEOMETRY, WKB_VIEW_GEOMETRY)] sedona_type: SedonaType) { + let udf = create_udf(); + let tester = AggregateUdfTester::new(udf.into(), vec![sedona_type.clone()]); + assert_eq!(tester.return_type().unwrap(), WKB_GEOMETRY); + + let batches = vec![vec![ + Some("LINESTRING (0 0, 10 0)"), + Some("LINESTRING (10 0, 10 10)"), + Some("LINESTRING (10 10, 0 0)"), + ]]; + + let result = tester.aggregate_wkt(batches).unwrap(); + assert_scalar_equal_wkb_geometry( + &result, + Some("GEOMETRYCOLLECTION (POLYGON ((10 0, 0 0, 10 10, 10 0)))"), + ); + } + + #[rstest] + fn polygonize_with_nulls(#[values(WKB_GEOMETRY, WKB_VIEW_GEOMETRY)] sedona_type: SedonaType) { + let udf = create_udf(); + let tester = AggregateUdfTester::new(udf.into(), vec![sedona_type.clone()]); + + let batches = vec![vec![ + Some("LINESTRING (0 0, 10 0)"), + None, + Some("LINESTRING (10 0, 10 10)"), + None, + Some("LINESTRING (10 10, 0 0)"), + ]]; + + let result = tester.aggregate_wkt(batches).unwrap(); + assert_scalar_equal_wkb_geometry( + &result, + Some("GEOMETRYCOLLECTION (POLYGON ((10 0, 0 0, 10 10, 10 0)))"), + ); + } + + #[rstest] + fn polygonize_empty_input(#[values(WKB_GEOMETRY, WKB_VIEW_GEOMETRY)] sedona_type: SedonaType) { + let udf = create_udf(); + let tester = AggregateUdfTester::new(udf.into(), vec![sedona_type.clone()]); + + let batches: Vec>> = vec![]; + assert_scalar_equal_wkb_geometry(&tester.aggregate_wkt(batches).unwrap(), None); + } + + #[rstest] + fn polygonize_no_polygons_formed( + #[values(WKB_GEOMETRY, WKB_VIEW_GEOMETRY)] sedona_type: SedonaType, + ) { + let udf = create_udf(); + let tester = AggregateUdfTester::new(udf.into(), vec![sedona_type.clone()]); + + let batches = vec![vec![ + Some("LINESTRING (0 0, 10 0)"), + Some("LINESTRING (20 0, 30 0)"), + ]]; + assert_scalar_equal_wkb_geometry( + &tester.aggregate_wkt(batches).unwrap(), + Some("GEOMETRYCOLLECTION EMPTY"), + ); + } + + #[rstest] + fn polygonize_multiple_polygons( + #[values(WKB_GEOMETRY, WKB_VIEW_GEOMETRY)] sedona_type: SedonaType, + ) { + let udf = create_udf(); + let tester = AggregateUdfTester::new(udf.into(), vec![sedona_type.clone()]); + + let batches = vec![vec![ + Some("LINESTRING (0 0, 10 0)"), + Some("LINESTRING (10 0, 5 10)"), + Some("LINESTRING (5 10, 0 0)"), + Some("LINESTRING (20 0, 30 0)"), + Some("LINESTRING (30 0, 25 10)"), + Some("LINESTRING (25 10, 20 0)"), + ]]; + + let result = tester.aggregate_wkt(batches).unwrap(); + assert_scalar_equal_wkb_geometry( + &result, + Some("GEOMETRYCOLLECTION (POLYGON ((10 0, 0 0, 5 10, 10 0)), POLYGON ((30 0, 20 0, 25 10, 30 0)))"), + ); + } + + #[rstest] + fn polygonize_multiple_batches( + #[values(WKB_GEOMETRY, WKB_VIEW_GEOMETRY)] sedona_type: SedonaType, + ) { + // Testing merge_batch + let udf = create_udf(); + let tester = AggregateUdfTester::new(udf.into(), vec![sedona_type.clone()]); + + let batches = vec![ + vec![Some("LINESTRING (0 0, 10 0)")], + vec![Some("LINESTRING (10 0, 10 10)")], + vec![Some("LINESTRING (10 10, 0 0)")], + ]; + + let result = tester.aggregate_wkt(batches).unwrap(); + assert_scalar_equal_wkb_geometry( + &result, + Some("GEOMETRYCOLLECTION (POLYGON ((10 0, 0 0, 10 10, 10 0)))"), + ); + } + + #[rstest] + fn polygonize_single_polygon( + #[values(WKB_GEOMETRY, WKB_VIEW_GEOMETRY)] sedona_type: SedonaType, + ) { + let udf = create_udf(); + let tester = AggregateUdfTester::new(udf.into(), vec![sedona_type.clone()]); + + let batches = vec![vec![Some("POLYGON ((10 0, 0 0, 10 10, 10 0))")]]; + + let result = tester.aggregate_wkt(batches).unwrap(); + assert_scalar_equal_wkb_geometry( + &result, + Some("GEOMETRYCOLLECTION (POLYGON ((10 0, 0 0, 10 10, 10 0)))"), + ); + } + + #[rstest] + fn polygonize_multipolygon(#[values(WKB_GEOMETRY, WKB_VIEW_GEOMETRY)] sedona_type: SedonaType) { + let udf = create_udf(); + let tester = AggregateUdfTester::new(udf.into(), vec![sedona_type.clone()]); + + let batches = vec![vec![Some( + "MULTIPOLYGON (((0 0, 1 0, 0 1, 0 0)), ((10 10, 11 10, 10 11, 10 10)))", + )]]; + + let result = tester.aggregate_wkt(batches).unwrap(); + assert_scalar_equal_wkb_geometry( + &result, + Some("GEOMETRYCOLLECTION (POLYGON ((0 0, 0 1, 1 0, 0 0)), POLYGON ((10 10, 10 11, 11 10, 10 10)))"), + ); + } + + #[rstest] + fn polygonize_closed_ring_linestring( + #[values(WKB_GEOMETRY, WKB_VIEW_GEOMETRY)] sedona_type: SedonaType, + ) { + let udf = create_udf(); + let tester = AggregateUdfTester::new(udf.into(), vec![sedona_type.clone()]); + + let batches = vec![vec![Some("LINESTRING (0 0, 0 1, 1 1, 1 0, 0 0)")]]; + + let result = tester.aggregate_wkt(batches).unwrap(); + assert_scalar_equal_wkb_geometry( + &result, + Some("GEOMETRYCOLLECTION (POLYGON ((0 0, 0 1, 1 1, 1 0, 0 0)))"), + ); + } + + #[rstest] + fn polygonize_point_returns_empty( + #[values(WKB_GEOMETRY, WKB_VIEW_GEOMETRY)] sedona_type: SedonaType, + ) { + let udf = create_udf(); + let tester = AggregateUdfTester::new(udf.into(), vec![sedona_type.clone()]); + + let batches = vec![vec![Some("POINT (0 0)")]]; + + let result = tester.aggregate_wkt(batches).unwrap(); + assert_scalar_equal_wkb_geometry(&result, Some("GEOMETRYCOLLECTION EMPTY")); + } + + #[rstest] + fn polygonize_multipoint_returns_empty( + #[values(WKB_GEOMETRY, WKB_VIEW_GEOMETRY)] sedona_type: SedonaType, + ) { + let udf = create_udf(); + let tester = AggregateUdfTester::new(udf.into(), vec![sedona_type.clone()]); + + let batches = vec![vec![Some("MULTIPOINT ((0 0), (1 1))")]]; + + let result = tester.aggregate_wkt(batches).unwrap(); + assert_scalar_equal_wkb_geometry(&result, Some("GEOMETRYCOLLECTION EMPTY")); + } + + #[rstest] + fn polygonize_multilinestring_returns_empty( + #[values(WKB_GEOMETRY, WKB_VIEW_GEOMETRY)] sedona_type: SedonaType, + ) { + let udf = create_udf(); + let tester = AggregateUdfTester::new(udf.into(), vec![sedona_type.clone()]); + + let batches = vec![vec![Some("MULTILINESTRING ((0 0, 1 1), (2 2, 3 3))")]]; + + let result = tester.aggregate_wkt(batches).unwrap(); + assert_scalar_equal_wkb_geometry(&result, Some("GEOMETRYCOLLECTION EMPTY")); + } + + #[rstest] + fn polygonize_geometrycollection_returns_empty( + #[values(WKB_GEOMETRY, WKB_VIEW_GEOMETRY)] sedona_type: SedonaType, + ) { + let udf = create_udf(); + let tester = AggregateUdfTester::new(udf.into(), vec![sedona_type.clone()]); + + let batches = vec![vec![Some( + "GEOMETRYCOLLECTION (POINT (0 0), LINESTRING (0 0, 1 1))", + )]]; + + let result = tester.aggregate_wkt(batches).unwrap(); + assert_scalar_equal_wkb_geometry(&result, Some("GEOMETRYCOLLECTION EMPTY")); + } + + #[rstest] + fn polygonize_empty_linestring_returns_empty( + #[values(WKB_GEOMETRY, WKB_VIEW_GEOMETRY)] sedona_type: SedonaType, + ) { + let udf = create_udf(); + let tester = AggregateUdfTester::new(udf.into(), vec![sedona_type.clone()]); + + let batches = vec![vec![Some("LINESTRING EMPTY")]]; + + let result = tester.aggregate_wkt(batches).unwrap(); + assert_scalar_equal_wkb_geometry(&result, Some("GEOMETRYCOLLECTION EMPTY")); + } +} diff --git a/python/sedonadb/tests/functions/test_aggregate.py b/python/sedonadb/tests/functions/test_aggregate.py index 15defddf..5fddd9ad 100644 --- a/python/sedonadb/tests/functions/test_aggregate.py +++ b/python/sedonadb/tests/functions/test_aggregate.py @@ -19,6 +19,11 @@ from sedonadb.testing import PostGIS, SedonaDB +def polygonize_fn_suffix(eng): + """Return the appropriate suffix for the polygonize function for the given engine.""" + return "" if isinstance(eng, PostGIS) else "_Agg" + + @pytest.mark.parametrize("eng", [SedonaDB, PostGIS]) def test_st_collect_points(eng): eng = eng.create_or_skip() @@ -115,3 +120,104 @@ def test_st_collect_zero_input(eng): ) AS t(geom) WHERE false""", None, ) + + +@pytest.mark.parametrize("eng", [SedonaDB, PostGIS]) +def test_st_polygonize_basic_triangle(eng): + eng = eng.create_or_skip() + suffix = polygonize_fn_suffix(eng) + eng.assert_query_result( + f"""SELECT ST_Polygonize{suffix}(ST_GeomFromText(geom)) FROM ( + VALUES + ('LINESTRING (0 0, 10 0)'), + ('LINESTRING (10 0, 10 10)'), + ('LINESTRING (10 10, 0 0)') + ) AS t(geom)""", + "GEOMETRYCOLLECTION (POLYGON ((10 0, 0 0, 10 10, 10 0)))", + ) + + +@pytest.mark.parametrize("eng", [SedonaDB, PostGIS]) +def test_st_polygonize_with_nulls(eng): + eng = eng.create_or_skip() + suffix = polygonize_fn_suffix(eng) + eng.assert_query_result( + f"""SELECT ST_Polygonize{suffix}(ST_GeomFromText(geom)) FROM ( + VALUES + ('LINESTRING (0 0, 10 0)'), + (NULL), + ('LINESTRING (10 0, 10 10)'), + (NULL), + ('LINESTRING (10 10, 0 0)') + ) AS t(geom)""", + "GEOMETRYCOLLECTION (POLYGON ((10 0, 0 0, 10 10, 10 0)))", + ) + + +@pytest.mark.parametrize("eng", [SedonaDB, PostGIS]) +def test_st_polygonize_no_polygons_formed(eng): + eng = eng.create_or_skip() + suffix = polygonize_fn_suffix(eng) + eng.assert_query_result( + f"""SELECT ST_Polygonize{suffix}(ST_GeomFromText(geom)) FROM ( + VALUES + ('LINESTRING (0 0, 10 0)'), + ('LINESTRING (20 0, 30 0)') + ) AS t(geom)""", + "GEOMETRYCOLLECTION EMPTY", + ) + + +@pytest.mark.parametrize("eng", [SedonaDB, PostGIS]) +def test_st_polygonize_multiple_polygons(eng): + eng = eng.create_or_skip() + suffix = polygonize_fn_suffix(eng) + eng.assert_query_result( + f"""SELECT ST_Polygonize{suffix}(ST_GeomFromText(geom)) FROM ( + VALUES + ('LINESTRING (0 0, 10 0)'), + ('LINESTRING (10 0, 5 10)'), + ('LINESTRING (5 10, 0 0)'), + ('LINESTRING (20 0, 30 0)'), + ('LINESTRING (30 0, 25 10)'), + ('LINESTRING (25 10, 20 0)') + ) AS t(geom)""", + "GEOMETRYCOLLECTION (POLYGON ((10 0, 0 0, 5 10, 10 0)), POLYGON ((30 0, 20 0, 25 10, 30 0)))", + ) + + +@pytest.mark.parametrize("eng", [SedonaDB, PostGIS]) +@pytest.mark.parametrize( + ("geom", "expected"), + [ + ( + "POLYGON ((10 0, 0 0, 10 10, 10 0))", + "GEOMETRYCOLLECTION (POLYGON ((10 0, 0 0, 10 10, 10 0)))", + ), + ( + "LINESTRING (0 0, 0 1, 1 1, 1 0, 0 0)", + "GEOMETRYCOLLECTION (POLYGON ((0 0, 0 1, 1 1, 1 0, 0 0)))", + ), + ("POINT (0 0)", "GEOMETRYCOLLECTION EMPTY"), + ("MULTIPOINT ((0 0), (1 1))", "GEOMETRYCOLLECTION EMPTY"), + ("MULTILINESTRING ((0 0, 1 1), (2 2, 3 3))", "GEOMETRYCOLLECTION EMPTY"), + ( + "MULTIPOLYGON (((0 0, 1 0, 0 1, 0 0)), ((10 10, 11 10, 10 11, 10 10)))", + "GEOMETRYCOLLECTION (POLYGON ((0 0, 0 1, 1 0, 0 0)), POLYGON ((10 10, 10 11, 11 10, 10 10)))", + ), + ( + "GEOMETRYCOLLECTION (POINT (0 0), LINESTRING (0 0, 1 1))", + "GEOMETRYCOLLECTION EMPTY", + ), + ("LINESTRING EMPTY", "GEOMETRYCOLLECTION EMPTY"), + ], +) +def test_st_polygonize_single_geom(eng, geom, expected): + eng = eng.create_or_skip() + suffix = polygonize_fn_suffix(eng) + eng.assert_query_result( + f"""SELECT ST_Polygonize{suffix}(ST_GeomFromText(geom)) FROM ( + VALUES ('{geom}') + ) AS t(geom)""", + expected, + ) diff --git a/rust/sedona-functions/src/lib.rs b/rust/sedona-functions/src/lib.rs index 0014629e..25417e06 100644 --- a/rust/sedona-functions/src/lib.rs +++ b/rust/sedona-functions/src/lib.rs @@ -52,6 +52,7 @@ mod st_point; mod st_pointn; mod st_points; mod st_pointzm; +mod st_polygonize_agg; mod st_setsrid; mod st_srid; mod st_start_point; diff --git a/rust/sedona-functions/src/register.rs b/rust/sedona-functions/src/register.rs index ba9f82d0..9a06f258 100644 --- a/rust/sedona-functions/src/register.rs +++ b/rust/sedona-functions/src/register.rs @@ -123,6 +123,7 @@ pub fn default_function_set() -> FunctionSet { crate::st_collect::st_collect_udf, crate::st_envelope_aggr::st_envelope_aggr_udf, crate::st_intersection_aggr::st_intersection_aggr_udf, + crate::st_polygonize_agg::st_polygonize_agg_udf, crate::st_union_aggr::st_union_aggr_udf, ); diff --git a/rust/sedona-functions/src/st_polygonize_agg.rs b/rust/sedona-functions/src/st_polygonize_agg.rs new file mode 100644 index 00000000..477a6523 --- /dev/null +++ b/rust/sedona-functions/src/st_polygonize_agg.rs @@ -0,0 +1,65 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +use std::vec; + +use datafusion_expr::{scalar_doc_sections::DOC_SECTION_OTHER, Documentation, Volatility}; +use sedona_expr::aggregate_udf::SedonaAggregateUDF; +use sedona_schema::{datatypes::WKB_GEOMETRY, matchers::ArgMatcher}; + +/// ST_Polygonize_Agg() aggregate UDF implementation +/// +/// Creates polygons from a set of linework that forms closed rings. +pub fn st_polygonize_agg_udf() -> SedonaAggregateUDF { + SedonaAggregateUDF::new_stub( + "st_polygonize_agg", + ArgMatcher::new(vec![ArgMatcher::is_geometry()], WKB_GEOMETRY), + Volatility::Immutable, + Some(st_polygonize_agg_doc()), + ) +} + +fn st_polygonize_agg_doc() -> Documentation { + Documentation::builder( + DOC_SECTION_OTHER, + "Creates a GeometryCollection containing polygons formed from the linework of a set of geometries. \ + Returns an empty GeometryCollection if no polygons can be formed.", + "ST_Polygonize_Agg (geom: Geometry)", + ) + .with_argument("geom", "geometry: Input geometry (typically linestrings that form closed rings)") + .with_sql_example( + "SELECT ST_AsText(ST_Polygonize_Agg(geom)) FROM (VALUES \ + (ST_GeomFromText('LINESTRING (0 0, 10 0)')), \ + (ST_GeomFromText('LINESTRING (10 0, 10 10)')), \ + (ST_GeomFromText('LINESTRING (10 10, 0 0)')) \ + ) AS t(geom)" + ) + .build() +} + +#[cfg(test)] +mod test { + use datafusion_expr::AggregateUDF; + + use super::*; + + #[test] + fn udf_metadata() { + let udf: AggregateUDF = st_polygonize_agg_udf().into(); + assert_eq!(udf.name(), "st_polygonize_agg"); + assert!(udf.documentation().is_some()); + } +} diff --git a/rust/sedona/src/context.rs b/rust/sedona/src/context.rs index 744dca22..bd503e5d 100644 --- a/rust/sedona/src/context.rs +++ b/rust/sedona/src/context.rs @@ -129,6 +129,10 @@ impl SedonaContext { #[cfg(feature = "geos")] out.register_scalar_kernels(sedona_geos::register::scalar_kernels().into_iter())?; + // Register geos aggregate kernels if built with geos support + #[cfg(feature = "geos")] + out.register_aggregate_kernels(sedona_geos::register::aggregate_kernels().into_iter())?; + // Register geo kernels if built with geo support #[cfg(feature = "geo")] out.register_scalar_kernels(sedona_geo::register::scalar_kernels().into_iter())?;