From a77230ccc19383b7c39d1193bd59fca4e56bacdd Mon Sep 17 00:00:00 2001 From: radekaadek Date: Sat, 25 Oct 2025 14:49:45 +0200 Subject: [PATCH 1/5] Add Sedona Flink SQL module --- .../org/apache/sedona/flink/SedonaModule.java | 55 +++++++++++++++++++ .../sedona/flink/SedonaModuleFactory.java | 50 +++++++++++++++++ .../org.apache.flink.table.factories.Factory | 1 + 3 files changed, 106 insertions(+) create mode 100644 flink/src/main/java/org/apache/sedona/flink/SedonaModule.java create mode 100644 flink/src/main/java/org/apache/sedona/flink/SedonaModuleFactory.java create mode 100644 flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory diff --git a/flink/src/main/java/org/apache/sedona/flink/SedonaModule.java b/flink/src/main/java/org/apache/sedona/flink/SedonaModule.java new file mode 100644 index 0000000000..d57439deab --- /dev/null +++ b/flink/src/main/java/org/apache/sedona/flink/SedonaModule.java @@ -0,0 +1,55 @@ +/* + * 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. + */ +package org.apache.sedona.flink; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.module.Module; + +public class SedonaModule implements Module { + + private Map functions = new HashMap<>(); + + public SedonaModule() { + Stream funcs = Arrays.stream(Catalog.getFuncs()); + Stream predicates = Arrays.stream(Catalog.getPredicates()); + + this.functions = + Stream.concat(funcs, predicates) + .collect( + Collectors.toMap( + func -> func.getClass().getSimpleName().toLowerCase(), func -> func)); + } + + @Override + public Set listFunctions() { + return functions.keySet(); + } + + @Override + public Optional getFunctionDefinition(String name) { + return Optional.ofNullable(functions.get(name.toLowerCase())); + } +} diff --git a/flink/src/main/java/org/apache/sedona/flink/SedonaModuleFactory.java b/flink/src/main/java/org/apache/sedona/flink/SedonaModuleFactory.java new file mode 100644 index 0000000000..4d80285544 --- /dev/null +++ b/flink/src/main/java/org/apache/sedona/flink/SedonaModuleFactory.java @@ -0,0 +1,50 @@ +/* + * 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. + */ +package org.apache.sedona.flink; + +import java.util.Collections; +import java.util.Set; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.table.factories.ModuleFactory; +import org.apache.flink.table.module.Module; + +public class SedonaModuleFactory implements ModuleFactory { + + public static final String IDENTIFIER = "sedona"; + + @Override + public Module createModule(Context context) { + return new SedonaModule(); + } + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + return Collections.emptySet(); + } + + @Override + public Set> optionalOptions() { + return Collections.emptySet(); + } +} diff --git a/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 0000000000..ab4d717fb5 --- /dev/null +++ b/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -0,0 +1 @@ +org.apache.sedona.flink.SedonaModuleFactory From c798ce54f453288773d96ebd47cc3a13f72c301d Mon Sep 17 00:00:00 2001 From: radekaadek Date: Sat, 1 Nov 2025 14:20:59 +0100 Subject: [PATCH 2/5] Add Sedona Flink SQL module tests --- .../org/apache/sedona/flink/ModuleTest.java | 188 ++++++++++++++++++ 1 file changed, 188 insertions(+) create mode 100644 flink/src/test/java/org/apache/sedona/flink/ModuleTest.java diff --git a/flink/src/test/java/org/apache/sedona/flink/ModuleTest.java b/flink/src/test/java/org/apache/sedona/flink/ModuleTest.java new file mode 100644 index 0000000000..5f5716eaed --- /dev/null +++ b/flink/src/test/java/org/apache/sedona/flink/ModuleTest.java @@ -0,0 +1,188 @@ +/* + * 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. + */ +package org.apache.sedona.flink; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +public class ModuleTest extends TestBase { + + @BeforeClass + public static void onceExecutedBeforeAll() { + initialize(); + tableEnv.executeSql("LOAD MODULE sedona"); + } + + @AfterClass + public static void onceExecutedAfterAll() { + tableEnv.executeSql("UNLOAD MODULE sedona"); + } + + @Test + public void testSedonaModuleIsListed() throws Exception { + TableResult result = tableEnv.executeSql("SHOW MODULES"); + + List loadedModules = new ArrayList<>(); + try (CloseableIterator it = result.collect()) { + while (it.hasNext()) { + Row row = it.next(); + if (row.getField(0) != null) { + loadedModules.add(row.getField(0).toString().trim()); + } + } + } + + boolean isSedonaLoaded = + loadedModules.stream().anyMatch(moduleName -> moduleName.equalsIgnoreCase("sedona")); + + assertTrue("Module 'sedona' should be listed in SHOW MODULES", isSedonaLoaded); + + TableResult fullResult = tableEnv.executeSql("SHOW FULL MODULES"); + boolean foundAndUsed = false; + try (CloseableIterator it = fullResult.collect()) { + while (it.hasNext()) { + Row row = it.next(); + String moduleName = row.getField(0).toString().trim(); + boolean isUsed = (Boolean) row.getField(1); + if (moduleName.equalsIgnoreCase("sedona") && isUsed) { + foundAndUsed = true; + break; + } + } + } + + assertTrue("Module 'sedona' should be listed as 'used' in SHOW FULL MODULES", foundAndUsed); + } + + @Test + public void testRegularFunctionsAreLoaded() throws Exception { + Set loadedFunctions = getLoadedFunctions(); + + for (org.apache.flink.table.functions.UserDefinedFunction func : + org.apache.sedona.flink.Catalog.getFuncs()) { + String funcName = func.getClass().getSimpleName().toLowerCase(); + assertTrue("Function " + funcName + " should be loaded", loadedFunctions.contains(funcName)); + } + } + + @Test + public void testPredicateFunctionsAreLoaded() throws Exception { + Set loadedFunctions = getLoadedFunctions(); + + for (org.apache.flink.table.functions.UserDefinedFunction func : + org.apache.sedona.flink.Catalog.getPredicates()) { + String funcName = func.getClass().getSimpleName().toLowerCase(); + assertTrue("Predicate " + funcName + " should be loaded", loadedFunctions.contains(funcName)); + } + } + + @Test + public void testAggregateFunction() throws Exception { + Table pointTable = createPointTable(testDataSize); + tableEnv.createTemporaryView(pointTableName, pointTable); + + Table result = + tableEnv.sqlQuery( + "SELECT ST_AsText(ST_Envelope_Aggr(" + pointColNames[0] + ")) FROM " + pointTableName); + Row row = last(result); + String wkt = (String) row.getField(0); + + String expectedWkt = + String.format( + "POLYGON ((0 0, 0 %s, %s %s, %s 0, 0 0))", + testDataSize - 1, testDataSize - 1, testDataSize - 1, testDataSize - 1); + assertEquals(expectedWkt, wkt); + + tableEnv.dropTemporaryView(pointTableName); + } + + @Test + public void testConstructor() throws Exception { + Table table = tableEnv.sqlQuery("SELECT ST_AsText(ST_Point(1.0, 2.0))"); + Row row = first(table); + String wkt = (String) row.getField(0); + assertEquals("POINT (1 2)", wkt); + } + + @Test + public void testMeasurementFunction() throws Exception { + Table table = + tableEnv.sqlQuery("SELECT ST_Area(ST_GeomFromText('POLYGON((0 0, 2 0, 2 2, 0 2, 0 0))'))"); + Row row = first(table); + Double area = (Double) row.getField(0); + assertEquals(4.0, area, 0.0001); + } + + @Test + public void testPredicateFunction() throws Exception { + Table table = + tableEnv.sqlQuery( + "SELECT ST_Intersects(" + + "ST_GeomFromText('POINT(1 1)'), " + + "ST_GeomFromText('POLYGON((0 0, 2 0, 2 2, 0 2, 0 0))')" + + ")"); + Row row = first(table); + Boolean intersects = (Boolean) row.getField(0); + assertTrue("Point should intersect the polygon", intersects); + } + + private Set getLoadedFunctions() throws Exception { + TableResult result = tableEnv.executeSql("SHOW FUNCTIONS"); + Set loadedFunctions = new HashSet<>(); + + try (CloseableIterator it = result.collect()) { + while (it.hasNext()) { + Row row = it.next(); + if (row.getField(0) != null) { + loadedFunctions.add(row.getField(0).toString().trim().toLowerCase()); + } + } + } + return loadedFunctions; + } + + static void initialize() { + initialize(false); + } + + static void initialize(boolean enableWebUI) { + env = + enableWebUI + ? StreamExecutionEnvironment.createLocalEnvironmentWithWebUI(new Configuration()) + : StreamExecutionEnvironment.getExecutionEnvironment(); + EnvironmentSettings settings = EnvironmentSettings.newInstance().inStreamingMode().build(); + tableEnv = StreamTableEnvironment.create(env, settings); + } +} From 7b9bffaeb5cc909b1ef3d1415da0c6bc0ca3a0e5 Mon Sep 17 00:00:00 2001 From: radekaadek Date: Sat, 1 Nov 2025 14:24:25 +0100 Subject: [PATCH 3/5] Add GeometryTypeSerializer for Sedona Flink SQL module --- .../sedona/flink/GeometryTypeSerializer.java | 161 ++++++++++++++++++ 1 file changed, 161 insertions(+) create mode 100644 flink/src/main/java/org/apache/sedona/flink/GeometryTypeSerializer.java diff --git a/flink/src/main/java/org/apache/sedona/flink/GeometryTypeSerializer.java b/flink/src/main/java/org/apache/sedona/flink/GeometryTypeSerializer.java new file mode 100644 index 0000000000..99287f5f1a --- /dev/null +++ b/flink/src/main/java/org/apache/sedona/flink/GeometryTypeSerializer.java @@ -0,0 +1,161 @@ +/* + * 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. + */ +package org.apache.sedona.flink; + +import java.io.IOException; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.locationtech.jts.geom.Geometry; +import org.locationtech.jts.geom.GeometryFactory; + +public class GeometryTypeSerializer extends TypeSerializer { + + private static final long serialVersionUID = 1L; + + public static final GeometryTypeSerializer INSTANCE = new GeometryTypeSerializer(); + + private final GeometryFactory geometryFactory; + + public GeometryTypeSerializer() { + this.geometryFactory = new GeometryFactory(); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + return this; + } + + @Override + public Geometry createInstance() { + return geometryFactory.createPoint(); + } + + @Override + public Geometry copy(Geometry from) { + return (Geometry) from.copy(); + } + + @Override + public Geometry copy(Geometry from, Geometry reuse) { + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(Geometry record, DataOutputView target) throws IOException { + if (record == null) { + target.writeInt(-1); + } else { + byte[] data = org.apache.sedona.common.geometrySerde.GeometrySerializer.serialize(record); + target.writeInt(data.length); + target.write(data); + } + } + + @Override + public Geometry deserialize(DataInputView source) throws IOException { + int length = source.readInt(); + if (length == -1) { + return null; + } + byte[] data = new byte[length]; + source.readFully(data); + return org.apache.sedona.common.geometrySerde.GeometrySerializer.deserialize(data); + } + + @Override + public Geometry deserialize(Geometry reuse, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + int length = source.readInt(); + target.writeInt(length); + if (length > 0) { + target.write(source, length); + } + } + + @Override + public boolean equals(Object obj) { + return obj instanceof GeometryTypeSerializer; + } + + @Override + public int hashCode() { + return getClass().hashCode(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new GeometrySerializerSnapshot(); + } + + public static final class GeometrySerializerSnapshot implements TypeSerializerSnapshot { + private static final int CURRENT_VERSION = 1; + + @Override + public int getCurrentVersion() { + return CURRENT_VERSION; + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException {} + + @Override + public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) + throws IOException { + if (readVersion != CURRENT_VERSION) { + throw new IOException( + "Cannot read snapshot: Incompatible version " + + readVersion + + ". Expected version " + + CURRENT_VERSION); + } + } + + @Override + public TypeSerializer restoreSerializer() { + return GeometryTypeSerializer.INSTANCE; + } + + @Override + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializer newSerializer) { + if (newSerializer instanceof GeometryTypeSerializer) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } else { + return TypeSerializerSchemaCompatibility.incompatible(); + } + } + } +} From 50bff5f3b737606a2606937843e900682c42cbc6 Mon Sep 17 00:00:00 2001 From: radekaadek Date: Sun, 2 Nov 2025 16:52:20 +0100 Subject: [PATCH 4/5] Add GeometryArrayTypeSerializer for Sedona Flink SQL module --- .../flink/GeometryArrayTypeSerializer.java | 174 ++++++++++++++++++ 1 file changed, 174 insertions(+) create mode 100644 flink/src/main/java/org/apache/sedona/flink/GeometryArrayTypeSerializer.java diff --git a/flink/src/main/java/org/apache/sedona/flink/GeometryArrayTypeSerializer.java b/flink/src/main/java/org/apache/sedona/flink/GeometryArrayTypeSerializer.java new file mode 100644 index 0000000000..7e257bc6be --- /dev/null +++ b/flink/src/main/java/org/apache/sedona/flink/GeometryArrayTypeSerializer.java @@ -0,0 +1,174 @@ +/* + * 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. + */ +package org.apache.sedona.flink; + +import java.io.IOException; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.locationtech.jts.geom.Geometry; + +public class GeometryArrayTypeSerializer extends TypeSerializer { + private static final long serialVersionUID = 1L; + + public static final GeometryArrayTypeSerializer INSTANCE = new GeometryArrayTypeSerializer(); + + private GeometryArrayTypeSerializer() {} + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + return this; + } + + @Override + public Geometry[] createInstance() { + return new Geometry[0]; + } + + @Override + public Geometry[] copy(Geometry[] from) { + if (from == null) { + return null; + } + Geometry[] newArray = new Geometry[from.length]; + for (int i = 0; i < from.length; i++) { + if (from[i] != null) { + newArray[i] = GeometryTypeSerializer.INSTANCE.copy(from[i]); + } else { + newArray[i] = null; + } + } + return newArray; + } + + @Override + public Geometry[] copy(Geometry[] from, Geometry[] reuse) { + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(Geometry[] record, DataOutputView target) throws IOException { + if (record == null) { + target.writeInt(-1); + } else { + target.writeInt(record.length); + for (Geometry geom : record) { + GeometryTypeSerializer.INSTANCE.serialize(geom, target); + } + } + } + + @Override + public Geometry[] deserialize(DataInputView source) throws IOException { + int length = source.readInt(); + if (length == -1) { + return null; + } + + Geometry[] array = new Geometry[length]; + for (int i = 0; i < length; i++) { + array[i] = GeometryTypeSerializer.INSTANCE.deserialize(source); + } + return array; + } + + @Override + public Geometry[] deserialize(Geometry[] reuse, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + int length = source.readInt(); + target.writeInt(length); + + if (length > 0) { + for (int i = 0; i < length; i++) { + GeometryTypeSerializer.INSTANCE.copy(source, target); + } + } + } + + @Override + public boolean equals(Object obj) { + return obj instanceof GeometryArrayTypeSerializer; + } + + @Override + public int hashCode() { + return getClass().hashCode(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new GeometryArraySerializerSnapshot(); + } + + public static final class GeometryArraySerializerSnapshot + implements TypeSerializerSnapshot { + private static final int CURRENT_VERSION = 1; + + @Override + public int getCurrentVersion() { + return CURRENT_VERSION; + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException {} + + @Override + public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) + throws IOException { + if (readVersion != CURRENT_VERSION) { + throw new IOException( + "Cannot read snapshot: Incompatible version " + + readVersion + + ". Expected version " + + CURRENT_VERSION); + } + } + + @Override + public TypeSerializer restoreSerializer() { + return GeometryArrayTypeSerializer.INSTANCE; + } + + @Override + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializer newSerializer) { + if (newSerializer instanceof GeometryArrayTypeSerializer) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } else { + return TypeSerializerSchemaCompatibility.incompatible(); + } + } + } +} From 2ebe7934b6460864de73d11aab73a590076a6a87 Mon Sep 17 00:00:00 2001 From: radekaadek Date: Sat, 1 Nov 2025 14:24:51 +0100 Subject: [PATCH 5/5] Use GeometryTypeSerializers for Sedona Flink SQL --- .../flink/expressions/Accumulators.java | 3 +- .../sedona/flink/expressions/Aggregators.java | 25 +- .../flink/expressions/Constructors.java | 121 +-- .../sedona/flink/expressions/Functions.java | 918 +++++++----------- .../flink/expressions/FunctionsGeoTools.java | 13 +- .../sedona/flink/expressions/Predicates.java | 91 +- 6 files changed, 490 insertions(+), 681 deletions(-) diff --git a/flink/src/main/java/org/apache/sedona/flink/expressions/Accumulators.java b/flink/src/main/java/org/apache/sedona/flink/expressions/Accumulators.java index 4ed3627452..d4e1449e28 100644 --- a/flink/src/main/java/org/apache/sedona/flink/expressions/Accumulators.java +++ b/flink/src/main/java/org/apache/sedona/flink/expressions/Accumulators.java @@ -19,6 +19,7 @@ package org.apache.sedona.flink.expressions; import org.apache.flink.table.annotation.DataTypeHint; +import org.apache.sedona.flink.GeometryTypeSerializer; import org.locationtech.jts.geom.Geometry; /** Mutable accumulator of structured type for the aggregate function */ @@ -39,7 +40,7 @@ void reset() { } public static class AccGeometry { - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry geom; } diff --git a/flink/src/main/java/org/apache/sedona/flink/expressions/Aggregators.java b/flink/src/main/java/org/apache/sedona/flink/expressions/Aggregators.java index de3d2f525d..1ca183a9ca 100644 --- a/flink/src/main/java/org/apache/sedona/flink/expressions/Aggregators.java +++ b/flink/src/main/java/org/apache/sedona/flink/expressions/Aggregators.java @@ -20,6 +20,7 @@ import org.apache.flink.table.annotation.DataTypeHint; import org.apache.flink.table.functions.AggregateFunction; +import org.apache.sedona.flink.GeometryTypeSerializer; import org.locationtech.jts.geom.Coordinate; import org.locationtech.jts.geom.Envelope; import org.locationtech.jts.geom.Geometry; @@ -27,7 +28,7 @@ public class Aggregators { // Compute the rectangular boundary of a number of geometries - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public static class ST_Envelope_Aggr extends AggregateFunction { Geometry createPolygon(double minX, double minY, double maxX, double maxY) { @@ -47,14 +48,14 @@ public Accumulators.Envelope createAccumulator() { } @Override - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry getValue(Accumulators.Envelope acc) { return createPolygon(acc.minX, acc.minY, acc.maxX, acc.maxY); } public void accumulate( Accumulators.Envelope acc, - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Envelope envelope = ((Geometry) o).getEnvelopeInternal(); acc.minX = Math.min(acc.minX, envelope.getMinX()); acc.minY = Math.min(acc.minY, envelope.getMinY()); @@ -70,7 +71,7 @@ public void accumulate( */ public void retract( Accumulators.Envelope acc, - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geometry = (Geometry) o; assert (false); } @@ -91,7 +92,7 @@ public void resetAccumulator(Accumulators.Envelope acc) { // Compute the Union boundary of numbers of geometries // - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public static class ST_Intersection_Aggr extends AggregateFunction { @@ -101,14 +102,14 @@ public Accumulators.AccGeometry createAccumulator() { } @Override - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry getValue(Accumulators.AccGeometry acc) { return acc.geom; } public void accumulate( Accumulators.AccGeometry acc, - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { if (acc.geom == null) { acc.geom = (Geometry) o; } else { @@ -124,7 +125,7 @@ public void accumulate( */ public void retract( Accumulators.AccGeometry acc, - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geometry = (Geometry) o; assert (false); } @@ -147,7 +148,7 @@ public void resetAccumulator(Accumulators.AccGeometry acc) { // Compute the Union boundary of numbers of geometries // - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public static class ST_Union_Aggr extends AggregateFunction { @Override @@ -156,14 +157,14 @@ public Accumulators.AccGeometry createAccumulator() { } @Override - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry getValue(Accumulators.AccGeometry acc) { return acc.geom; } public void accumulate( Accumulators.AccGeometry acc, - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { if (acc.geom == null) { acc.geom = (Geometry) o; } else { @@ -179,7 +180,7 @@ public void accumulate( */ public void retract( Accumulators.AccGeometry acc, - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geometry = (Geometry) o; assert (false); } 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 2e43f4c76d..8df2afb9dc 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 @@ -25,6 +25,7 @@ import org.apache.sedona.common.enums.GeometryType; import org.apache.sedona.common.utils.FormatUtils; import org.apache.sedona.common.utils.GeoHashDecoder; +import org.apache.sedona.flink.GeometryTypeSerializer; import org.locationtech.jts.geom.*; import org.locationtech.jts.io.ParseException; import org.locationtech.jts.io.gml2.GMLReader; @@ -43,7 +44,7 @@ private static Geometry getGeometryByType( } public static class ST_Point extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("Double") Double x, @DataTypeHint("Double") Double y) throws ParseException { return org.apache.sedona.common.Constructors.point(x, y); @@ -51,7 +52,7 @@ public Geometry eval(@DataTypeHint("Double") Double x, @DataTypeHint("Double") D } public static class ST_PointZ extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( @DataTypeHint("Double") Double x, @DataTypeHint("Double") Double y, @@ -60,7 +61,7 @@ public Geometry eval( return eval(x, y, z, 0); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( @DataTypeHint("Double") Double x, @DataTypeHint("Double") Double y, @@ -72,7 +73,7 @@ public Geometry eval( } public static class ST_PointM extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( @DataTypeHint("Double") Double x, @DataTypeHint("Double") Double y, @@ -81,7 +82,7 @@ public Geometry eval( return eval(x, y, m, 0); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( @DataTypeHint("Double") Double x, @DataTypeHint("Double") Double y, @@ -93,7 +94,7 @@ public Geometry eval( } public static class ST_PointZM extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( @DataTypeHint("Double") Double x, @DataTypeHint("Double") Double y, @@ -103,7 +104,7 @@ public Geometry eval( return eval(x, y, z, m, 0); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( @DataTypeHint("Double") Double x, @DataTypeHint("Double") Double y, @@ -116,21 +117,21 @@ public Geometry eval( } public static class ST_PointFromText extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( @DataTypeHint("String") String s, @DataTypeHint("String") String inputDelimiter) throws ParseException { return getGeometryByType(s, inputDelimiter, GeometryType.POINT); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("String") String s) throws ParseException { return eval(s, null); } } public static class ST_MakePointM extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( @DataTypeHint("Double") Double x, @DataTypeHint("Double") Double y, @@ -141,13 +142,13 @@ public Geometry eval( } public static class ST_MakePoint extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("Double") Double x, @DataTypeHint("Double") Double y) throws ParseException { return org.apache.sedona.common.Constructors.makePoint(x, y, null, null); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( @DataTypeHint("Double") Double x, @DataTypeHint("Double") Double y, @@ -156,7 +157,7 @@ public Geometry eval( return org.apache.sedona.common.Constructors.makePoint(x, y, z, null); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( @DataTypeHint("Double") Double x, @DataTypeHint("Double") Double y, @@ -168,7 +169,7 @@ public Geometry eval( } public static class ST_LineFromText extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( @DataTypeHint("String") String lineString, @DataTypeHint("String") String inputDelimiter) throws ParseException { @@ -176,14 +177,14 @@ public Geometry eval( return getGeometryByType(lineString, inputDelimiter, GeometryType.LINESTRING); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("String") String lineString) throws ParseException { return eval(lineString, null); } } public static class ST_LineStringFromText extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( @DataTypeHint("String") String lineString, @DataTypeHint("String") String inputDelimiter) throws ParseException { @@ -191,14 +192,14 @@ public Geometry eval( return new ST_LineFromText().eval(lineString, inputDelimiter); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("String") String lineString) throws ParseException { return eval(lineString, null); } } public static class ST_PolygonFromText extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( @DataTypeHint("String") String polygonString, @DataTypeHint("String") String inputDelimiter) throws ParseException { @@ -206,14 +207,14 @@ public Geometry eval( return getGeometryByType(polygonString, inputDelimiter, GeometryType.POLYGON); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("String") String polygonString) throws ParseException { return eval(polygonString, null); } } public static class ST_MakeEnvelope extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( @DataTypeHint("Double") Double minX, @DataTypeHint("Double") Double minY, @@ -223,7 +224,7 @@ public Geometry eval( return org.apache.sedona.common.Constructors.makeEnvelope(minX, minY, maxX, maxY, srid); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( @DataTypeHint("Double") Double minX, @DataTypeHint("Double") Double minY, @@ -234,7 +235,7 @@ public Geometry eval( } public static class ST_PolygonFromEnvelope extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( @DataTypeHint("Double") Double minX, @DataTypeHint("Double") Double minY, @@ -258,12 +259,12 @@ private static Geometry getGeometryByFileData(String wktString, FileDataSplitter } public static class ST_GeomFromWKT extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("String") String wktString) throws ParseException { return org.apache.sedona.common.Constructors.geomFromWKT(wktString, 0); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( @DataTypeHint("String") String wktString, @DataTypeHint("Int") Integer srid) throws ParseException { @@ -272,19 +273,19 @@ public Geometry eval( } public static class ST_GeomFromEWKT extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("String") String wktString) throws ParseException { return org.apache.sedona.common.Constructors.geomFromEWKT(wktString); } } public static class ST_GeometryFromText extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("String") String wktString) throws ParseException { return org.apache.sedona.common.Constructors.geomFromWKT(wktString, 0); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( @DataTypeHint("String") String wktString, @DataTypeHint("Int") Integer srid) throws ParseException { @@ -293,12 +294,12 @@ public Geometry eval( } public static class ST_GeomFromText extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("String") String wktString) throws ParseException { return org.apache.sedona.common.Constructors.geomFromWKT(wktString, 0); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( @DataTypeHint("String") String wktString, @DataTypeHint("Int") Integer srid) throws ParseException { @@ -307,31 +308,31 @@ public Geometry eval( } public static class ST_GeomFromWKB extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.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) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("Bytes") byte[] wkb) throws ParseException { return org.apache.sedona.common.Constructors.geomFromWKB(wkb); } } public static class ST_GeomFromEWKB extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.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) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("Bytes") byte[] wkb) throws ParseException { return org.apache.sedona.common.Constructors.geomFromWKB(wkb); } } public static class ST_PointFromWKB extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("String") String wkbString) throws ParseException { Geometry geometry = getGeometryByFileData(wkbString, FileDataSplitter.WKB); if (geometry instanceof Point) { @@ -340,7 +341,7 @@ public Geometry eval(@DataTypeHint("String") String wkbString) throws ParseExcep return null; // Return null if geometry is not a Point } - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("String") String wkbString, int srid) throws ParseException { Geometry geometry = getGeometryByFileData(wkbString, FileDataSplitter.WKB); if (geometry instanceof Point) { @@ -350,19 +351,19 @@ public Geometry eval(@DataTypeHint("String") String wkbString, int srid) throws return null; // Return null if geometry is not a Point } - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("Bytes") byte[] wkb) throws ParseException { return org.apache.sedona.common.Constructors.pointFromWKB(wkb); } - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("Bytes") byte[] wkb, int srid) throws ParseException { return org.apache.sedona.common.Constructors.pointFromWKB(wkb, srid); } } public static class ST_LineFromWKB extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("String") String wkbString) throws ParseException { Geometry geometry = getGeometryByFileData(wkbString, FileDataSplitter.WKB); if (geometry instanceof LineString) { @@ -371,7 +372,7 @@ public Geometry eval(@DataTypeHint("String") String wkbString) throws ParseExcep return null; // Return null if geometry is not a Point } - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("String") String wkbString, int srid) throws ParseException { Geometry geometry = getGeometryByFileData(wkbString, FileDataSplitter.WKB); if (geometry instanceof LineString) { @@ -381,19 +382,19 @@ public Geometry eval(@DataTypeHint("String") String wkbString, int srid) throws return null; // Return null if geometry is not a Linestring } - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("Bytes") byte[] wkb) throws ParseException { return org.apache.sedona.common.Constructors.lineFromWKB(wkb); } - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("Bytes") byte[] wkb, int srid) throws ParseException { return org.apache.sedona.common.Constructors.lineFromWKB(wkb, srid); } } public static class ST_LinestringFromWKB extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("String") String wkbString) throws ParseException { Geometry geometry = getGeometryByFileData(wkbString, FileDataSplitter.WKB); if (geometry instanceof LineString) { @@ -402,7 +403,7 @@ public Geometry eval(@DataTypeHint("String") String wkbString) throws ParseExcep return null; // Return null if geometry is not a Linestring } - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("String") String wkbString, int srid) throws ParseException { Geometry geometry = getGeometryByFileData(wkbString, FileDataSplitter.WKB); if (geometry instanceof LineString) { @@ -412,26 +413,26 @@ public Geometry eval(@DataTypeHint("String") String wkbString, int srid) throws return null; // Return null if geometry is not a Linestring } - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("Bytes") byte[] wkb) throws ParseException { return org.apache.sedona.common.Constructors.lineFromWKB(wkb); } - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("Bytes") byte[] wkb, int srid) throws ParseException { return org.apache.sedona.common.Constructors.lineFromWKB(wkb, srid); } } public static class ST_GeomFromGeoJSON extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("String") String geoJson) throws ParseException { return getGeometryByFileData(geoJson, FileDataSplitter.GEOJSON); } } public static class ST_GeomFromGeoHash extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( @DataTypeHint("String") String value, @DataTypeHint("Int") Integer precision) throws ParseException, GeoHashDecoder.InvalidGeoHashException { @@ -439,7 +440,7 @@ public Geometry eval( return GeoHashDecoder.decode(value, precision); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("String") String value) throws ParseException, GeoHashDecoder.InvalidGeoHashException { return eval(value, null); @@ -447,21 +448,21 @@ public Geometry eval(@DataTypeHint("String") String value) } public static class ST_PointFromGeoHash extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( @DataTypeHint("String") String value, @DataTypeHint("Int") Integer precision) { // The default precision is the geohash length. Otherwise, use the precision given by the user return org.apache.sedona.common.Constructors.pointFromGeoHash(value, precision); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("String") String value) { return eval(value, null); } } public static class ST_GeomFromGML extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("String") String gml) throws ParseException { GMLReader reader = new GMLReader(); try { @@ -473,63 +474,63 @@ public Geometry eval(@DataTypeHint("String") String gml) throws ParseException { } public static class ST_GeomFromKML extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint("String") String kml) throws ParseException { return new KMLReader().read(kml); } } public static class ST_MPolyFromText extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( @DataTypeHint(value = "String") String wkt, @DataTypeHint("Int") Integer srid) throws ParseException { return org.apache.sedona.common.Constructors.mPolyFromText(wkt, srid); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint(value = "String") String wkt) throws ParseException { return org.apache.sedona.common.Constructors.mPolyFromText(wkt, 0); } } public static class ST_MLineFromText extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( @DataTypeHint(value = "String") String wkt, @DataTypeHint("Int") Integer srid) throws ParseException { return org.apache.sedona.common.Constructors.mLineFromText(wkt, srid); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint(value = "String") String wkt) throws ParseException { return org.apache.sedona.common.Constructors.mLineFromText(wkt, 0); } } public static class ST_MPointFromText extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( @DataTypeHint(value = "String") String wkt, @DataTypeHint("Int") Integer srid) throws ParseException { return org.apache.sedona.common.Constructors.mPointFromText(wkt, srid); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint(value = "String") String wkt) throws ParseException { return org.apache.sedona.common.Constructors.mPointFromText(wkt, 0); } } public static class ST_GeomCollFromText extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( @DataTypeHint(value = "String") String wkt, @DataTypeHint("Int") Integer srid) throws ParseException { return org.apache.sedona.common.Constructors.geomCollFromText(wkt, srid); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint(value = "String") String wkt) throws ParseException { return org.apache.sedona.common.Constructors.geomCollFromText(wkt, 0); } 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 211d7db796..12b3bf15da 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 @@ -24,6 +24,8 @@ import org.apache.flink.table.annotation.InputGroup; import org.apache.flink.table.functions.ScalarFunction; import org.apache.sedona.common.FunctionsGeoTools; +import org.apache.sedona.flink.GeometryArrayTypeSerializer; +import org.apache.sedona.flink.GeometryTypeSerializer; import org.geotools.api.referencing.FactoryException; import org.geotools.api.referencing.operation.TransformException; import org.locationtech.jts.geom.Geometry; @@ -33,33 +35,31 @@ public class Functions { public static class GeometryType extends ScalarFunction { @DataTypeHint("String") public String eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.geometryTypeWithMeasured(geom); } } public static class ST_LabelPoint extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.labelPoint(geom); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Integer") Integer gridResolution) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.labelPoint(geom, gridResolution); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Integer") Integer gridResolution, @DataTypeHint("Double") Double goodnessThreshold) { Geometry geom = (Geometry) o; @@ -70,8 +70,7 @@ public Geometry eval( public static class ST_Area extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.area(geom); } @@ -80,8 +79,7 @@ public Double eval( public static class ST_AreaSpheroid extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.sphere.Spheroid.area(geom); } @@ -90,10 +88,8 @@ public Double eval( public static class ST_Azimuth extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; return org.apache.sedona.common.Functions.azimuth(geom1, geom2); @@ -101,17 +97,16 @@ public Double eval( } public static class ST_ApproximateMedialAxis extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.approximateMedialAxis(geom); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Integer") Integer maxVertices) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.approximateMedialAxis(geom, maxVertices); @@ -119,28 +114,27 @@ public Geometry eval( } public static class ST_Boundary extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.boundary(geom); } } public static class ST_Buffer extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double radius) throws FactoryException, TransformException { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.buffer(geom, radius); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double radius, @DataTypeHint("Boolean") Boolean useSpheroid) throws FactoryException, TransformException { @@ -148,9 +142,9 @@ public Geometry eval( return org.apache.sedona.common.Functions.buffer(geom, radius, useSpheroid); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double radius, @DataTypeHint("Boolean") Boolean useSpheroid, @DataTypeHint("String") String params) @@ -162,27 +156,27 @@ public Geometry eval( public static class ST_BestSRID extends ScalarFunction { @DataTypeHint("Integer") - public int eval(@DataTypeHint(value = "RAW", bridgedTo = Geometry.class) Object o) { + public int eval( + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.bestSRID(geom); } } public static class ST_ShiftLongitude extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - public Geometry eval(@DataTypeHint(value = "RAW", bridgedTo = Geometry.class) Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) + public Geometry eval( + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.shiftLongitude(geom); } } public static class ST_ClosestPoint extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object g1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object g2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object g1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object g2) { Geometry geom1 = (Geometry) g1; Geometry geom2 = (Geometry) g2; return org.apache.sedona.common.Functions.closestPoint(geom1, geom2); @@ -190,29 +184,26 @@ public Geometry eval( } public static class ST_Centroid extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.getCentroid(geom); } } public static class ST_Collect extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; Geometry[] geoms = new Geometry[] {geom1, geom2}; return org.apache.sedona.common.Functions.createMultiGeometry(geoms); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint(inputGroup = InputGroup.ANY) Object o) { Geometry[] geoms = (Geometry[]) o; return org.apache.sedona.common.Functions.createMultiGeometry(geoms); @@ -220,17 +211,16 @@ public Geometry eval(@DataTypeHint(inputGroup = InputGroup.ANY) Object o) { } public static class ST_CollectionExtract extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.collectionExtract(geom); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Integer") Integer geoType) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.collectionExtract(geom, geoType); @@ -238,17 +228,17 @@ public Geometry eval( } public static class ST_ConcaveHull extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double pctConvex) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.concaveHull(geom, pctConvex, false); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double pctConvex, @DataTypeHint("Boolean") Boolean allowHoles) { Geometry geom = (Geometry) o; @@ -257,10 +247,9 @@ public Geometry eval( } public static class ST_ConvexHull extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.convexHull(geom); } @@ -272,44 +261,42 @@ public ST_CrossesDateLine() {} @DataTypeHint("Boolean") public Boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.crossesDateLine(geom); } } public static class ST_Envelope extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.envelope(geom); } } public static class ST_Expand extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint(value = "Double") Double uniformDelta) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.expand(geom, uniformDelta); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint(value = "Double") Double deltaX, @DataTypeHint(value = "Double") Double deltaY) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.expand(geom, deltaX, deltaY); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint(value = "Double") Double deltaX, @DataTypeHint(value = "Double") Double deltaY, @DataTypeHint(value = "Double") Double deltaZ) { @@ -321,20 +308,17 @@ public Geometry eval( public static class ST_Dimension extends ScalarFunction { @DataTypeHint("Integer") public Integer eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.dimension(geom); } } public static class ST_Difference extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; return org.apache.sedona.common.Functions.difference(geom1, geom2); @@ -344,10 +328,8 @@ public Geometry eval( public static class ST_Distance extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; return org.apache.sedona.common.Functions.distance(geom1, geom2); @@ -357,10 +339,8 @@ public Double eval( public static class ST_DistanceSphere extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; return org.apache.sedona.common.sphere.Haversine.distance(geom1, geom2); @@ -368,10 +348,8 @@ public Double eval( @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2, @DataTypeHint("Double") Double radius) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; @@ -382,10 +360,8 @@ public Double eval( public static class ST_DistanceSpheroid extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; return org.apache.sedona.common.sphere.Spheroid.distance(geom1, geom2); @@ -395,10 +371,8 @@ public Double eval( public static class ST_3DDistance extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; return org.apache.sedona.common.Functions.distance3d(geom1, geom2); @@ -406,30 +380,27 @@ public Double eval( } public static class ST_Dump extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry[].class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryArrayTypeSerializer.class) public Geometry[] eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom1 = (Geometry) o; return org.apache.sedona.common.Functions.dump(geom1); } } public static class ST_DumpPoints extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry[].class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryArrayTypeSerializer.class) public Geometry[] eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom1 = (Geometry) o; return org.apache.sedona.common.Functions.dumpPoints(geom1); } } public static class ST_EndPoint extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom1 = (Geometry) o; return org.apache.sedona.common.Functions.endPoint(geom1); } @@ -438,20 +409,17 @@ public Geometry eval( public static class ST_GeometryType extends ScalarFunction { @DataTypeHint("String") public String eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.geometryType(geom); } } public static class ST_Intersection extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object g1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object g2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object g1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object g2) { Geometry geom1 = (Geometry) g1; Geometry geom2 = (Geometry) g2; return org.apache.sedona.common.Functions.intersection(geom1, geom2); @@ -461,8 +429,7 @@ public Geometry eval( public static class ST_Length extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.length(geom); } @@ -471,8 +438,7 @@ public Double eval( public static class ST_Length2D extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.length(geom); } @@ -481,17 +447,16 @@ public Double eval( public static class ST_LengthSpheroid extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.sphere.Spheroid.length(geom); } } public static class ST_LineInterpolatePoint extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double fraction) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.lineInterpolatePoint(geom, fraction); @@ -501,9 +466,8 @@ public Geometry eval( public static class ST_LineLocatePoint extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object p) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object p) { Geometry geom = (Geometry) o; Geometry point = (Geometry) p; return org.apache.sedona.common.Functions.lineLocatePoint(geom, point); @@ -511,18 +475,18 @@ public Double eval( } public static class ST_LocateAlong extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint(value = "Double") Double measure, @DataTypeHint(value = "Double") Double offset) { Geometry linear = (Geometry) o; return org.apache.sedona.common.Functions.locateAlong(linear, measure, offset); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint(value = "Double") Double measure) { Geometry linear = (Geometry) o; return org.apache.sedona.common.Functions.locateAlong(linear, measure); @@ -530,12 +494,10 @@ public Geometry eval( } public static class ST_LongestLine extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object g1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object g2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object g1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object g2) { Geometry geom1 = (Geometry) g1; Geometry geom2 = (Geometry) g2; return org.apache.sedona.common.Functions.longestLine(geom1, geom2); @@ -545,8 +507,7 @@ public Geometry eval( public static class ST_YMin extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.yMin(geom); } @@ -555,8 +516,7 @@ public Double eval( public static class ST_YMax extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.yMax(geom); } @@ -565,8 +525,7 @@ public Double eval( public static class ST_ZMax extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.zMax(geom); } @@ -575,8 +534,7 @@ public Double eval( public static class ST_ZMin extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.zMin(geom); } @@ -585,18 +543,16 @@ public Double eval( public static class ST_NDims extends ScalarFunction { @DataTypeHint("Integer") public Integer eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.nDims(geom); } } public static class ST_FlipCoordinates extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.flipCoordinates(geom); } @@ -605,8 +561,7 @@ public Geometry eval( public static class ST_GeoHash extends ScalarFunction { @DataTypeHint("String") public String eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object geometry, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object geometry, Integer precision) { Geometry geom = (Geometry) geometry; return org.apache.sedona.common.Functions.geohash(geom, precision); @@ -616,15 +571,14 @@ public String eval( public static class ST_Perimeter extends ScalarFunction { @DataTypeHint(value = "Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.perimeter(geom); } @DataTypeHint(value = "Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, Boolean use_spheroid) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.perimeter(geom, use_spheroid); @@ -632,7 +586,7 @@ public Double eval( @DataTypeHint(value = "Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, Boolean use_spheroid, boolean lenient) { Geometry geom = (Geometry) o; @@ -643,15 +597,14 @@ public Double eval( public static class ST_Perimeter2D extends ScalarFunction { @DataTypeHint(value = "Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.perimeter(geom); } @DataTypeHint(value = "Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, Boolean use_spheroid) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.perimeter(geom, use_spheroid); @@ -659,7 +612,7 @@ public Double eval( @DataTypeHint(value = "Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, Boolean use_spheroid, boolean lenient) { Geometry geom = (Geometry) o; @@ -668,19 +621,18 @@ public Double eval( } public static class ST_PointOnSurface extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.pointOnSurface(geom); } } public static class ST_ReducePrecision extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Integer") Integer precisionScale) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.reducePrecision(geom, precisionScale); @@ -688,19 +640,18 @@ public Geometry eval( } public static class ST_Reverse extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.reverse(geom); } } public static class ST_GeometryN extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, int n) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.geometryN(geom, n); @@ -708,9 +659,9 @@ public Geometry eval( } public static class ST_InteriorRingN extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, int n) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.interiorRingN(geom, n); @@ -718,9 +669,9 @@ public Geometry eval( } public static class ST_PointN extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, int n) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.pointN(geom, n); @@ -730,8 +681,7 @@ public Geometry eval( public static class ST_NPoints extends ScalarFunction { @DataTypeHint("Integer") public Integer eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.nPoints(geom); } @@ -740,8 +690,7 @@ public Integer eval( public static class ST_NumGeometries extends ScalarFunction { @DataTypeHint("Integer") public Integer eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.numGeometries(geom); } @@ -750,8 +699,7 @@ public Integer eval( public static class ST_NumInteriorRings extends ScalarFunction { @DataTypeHint("Integer") public Integer eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.numInteriorRings(geom); } @@ -760,18 +708,16 @@ public Integer eval( public static class ST_NumInteriorRing extends ScalarFunction { @DataTypeHint("Integer") public Integer eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.numInteriorRings(geom); } } public static class ST_ExteriorRing extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.exteriorRing(geom); } @@ -780,8 +726,7 @@ public Geometry eval( public static class ST_AsEWKT extends ScalarFunction { @DataTypeHint("String") public String eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.asEWKT(geom); } @@ -790,8 +735,7 @@ public String eval( public static class ST_AsText extends ScalarFunction { @DataTypeHint("String") public String eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.asWKT(geom); } @@ -800,8 +744,7 @@ public String eval( public static class ST_AsEWKB extends ScalarFunction { @DataTypeHint("Bytes") public byte[] eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.asEWKB(geom); } @@ -810,7 +753,7 @@ public byte[] eval( public static class ST_AsHEXEWKB extends ScalarFunction { @DataTypeHint("String") public String eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("String") String endian) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.asHexEWKB(geom, endian); @@ -818,8 +761,7 @@ public String eval( @DataTypeHint("String") public String eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.asHexEWKB(geom); } @@ -828,8 +770,7 @@ public String eval( public static class ST_AsBinary extends ScalarFunction { @DataTypeHint("Bytes") public byte[] eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.asEWKB(geom); } @@ -838,15 +779,14 @@ public byte[] eval( public static class ST_AsGeoJSON extends ScalarFunction { @DataTypeHint("String") public String eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.asGeoJson(geom); } @DataTypeHint("String") public String eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, String type) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.asGeoJson(geom, type); @@ -856,8 +796,7 @@ public String eval( public static class ST_AsGML extends ScalarFunction { @DataTypeHint("String") public String eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.asGML(geom); } @@ -866,28 +805,25 @@ public String eval( public static class ST_AsKML extends ScalarFunction { @DataTypeHint("String") public String eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.asKML(geom); } } public static class ST_Force_2D extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.force2D(geom); } } public static class ST_Force2D extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.force2D(geom); } @@ -896,8 +832,7 @@ public Geometry eval( public static class ST_IsEmpty extends ScalarFunction { @DataTypeHint("Boolean") public boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.isEmpty(geom); } @@ -906,8 +841,7 @@ public boolean eval( public static class ST_X extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.x(geom); } @@ -916,8 +850,7 @@ public Double eval( public static class ST_Y extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.y(geom); } @@ -926,8 +859,7 @@ public Double eval( public static class ST_Z extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.z(geom); } @@ -936,8 +868,7 @@ public Double eval( public static class ST_Zmflag extends ScalarFunction { @DataTypeHint("Integer") public Integer eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.zmFlag(geom); } @@ -946,8 +877,7 @@ public Integer eval( public static class ST_XMax extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.xMax(geom); } @@ -956,27 +886,25 @@ public Double eval( public static class ST_XMin extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.xMin(geom); } } public static class ST_BuildArea extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.buildArea(geom); } } public static class ST_SetSRID extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, int srid) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.setSRID(geom, srid); @@ -986,8 +914,7 @@ public Geometry eval( public static class ST_SRID extends ScalarFunction { @DataTypeHint("Integer") public Integer eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.getSRID(geom); } @@ -996,8 +923,7 @@ public Integer eval( public static class ST_IsClosed extends ScalarFunction { @DataTypeHint("Boolean") public boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.isClosed(geom); } @@ -1006,8 +932,7 @@ public boolean eval( public static class ST_IsPolygonCW extends ScalarFunction { @DataTypeHint("Boolean") public boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.isPolygonCW(geom); } @@ -1016,8 +941,7 @@ public boolean eval( public static class ST_IsRing extends ScalarFunction { @DataTypeHint("Boolean") public boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.isRing(geom); } @@ -1026,8 +950,7 @@ public boolean eval( public static class ST_IsSimple extends ScalarFunction { @DataTypeHint("Boolean") public boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.isSimple(geom); } @@ -1036,15 +959,14 @@ public boolean eval( public static class ST_IsValid extends ScalarFunction { @DataTypeHint("Boolean") public boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.isValid(geom); } @DataTypeHint("Boolean") public Boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Integer") Integer flag) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.isValid(geom, flag); @@ -1052,20 +974,18 @@ public Boolean eval( } public static class ST_Normalize extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.normalize(geom); } } public static class ST_AddMeasure extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, @DataTypeHint(value = "Double") Double measureStart, @DataTypeHint(value = "Double") Double measureEnd) { Geometry geom = (Geometry) o1; @@ -1074,23 +994,19 @@ public Geometry eval( } public static class ST_AddPoint extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry linestring = (Geometry) o1; Geometry point = (Geometry) o2; return org.apache.sedona.common.Functions.addPoint(linestring, point); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2, int position) { Geometry linestring = (Geometry) o1; Geometry point = (Geometry) o2; @@ -1099,17 +1015,16 @@ public Geometry eval( } public static class ST_RemovePoint extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.removePoint(geom); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, int offset) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.removePoint(geom, offset); @@ -1117,17 +1032,16 @@ public Geometry eval( } public static class ST_RemoveRepeatedPoints extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.removeRepeatedPoints(geom); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, double tolerance) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.removeRepeatedPoints(geom, tolerance); @@ -1135,13 +1049,11 @@ public Geometry eval( } public static class ST_SetPoint extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, int position, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry linestring = (Geometry) o1; Geometry point = (Geometry) o2; return org.apache.sedona.common.Functions.setPoint(linestring, position, point); @@ -1149,27 +1061,25 @@ public Geometry eval( } public static class ST_LineFromMultiPoint extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.lineFromMultiPoint(geom); } } public static class ST_LineSegments extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry[].class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryArrayTypeSerializer.class) public Geometry[] eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.lineSegments(geometry); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry[].class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryArrayTypeSerializer.class) public Geometry[] eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint(value = "Boolean") Boolean lenient) { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.lineSegments(geometry, lenient); @@ -1177,19 +1087,18 @@ public Geometry[] eval( } public static class ST_LineMerge extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.lineMerge(geom); } } public static class ST_LineSubstring extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double startFraction, @DataTypeHint("Double") Double endFraction) { Geometry geom = (Geometry) o; @@ -1200,8 +1109,7 @@ public Geometry eval( public static class ST_HasM extends ScalarFunction { @DataTypeHint("Boolean") public Boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.hasM(geom); } @@ -1210,8 +1118,7 @@ public Boolean eval( public static class ST_HasZ extends ScalarFunction { @DataTypeHint("Boolean") public Boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.hasZ(geom); } @@ -1220,8 +1127,7 @@ public Boolean eval( public static class ST_M extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.m(geom); } @@ -1230,8 +1136,7 @@ public Double eval( public static class ST_MMin extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.mMin(geom); } @@ -1240,26 +1145,23 @@ public Double eval( public static class ST_MMax extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.mMax(geom); } } public static class ST_MakeLine extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; return org.apache.sedona.common.Functions.makeLine(geom1, geom2); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint(inputGroup = InputGroup.ANY) Object o) { Geometry[] geoms = (Geometry[]) o; return org.apache.sedona.common.Functions.makeLine(geoms); @@ -1267,40 +1169,36 @@ public Geometry eval(@DataTypeHint(inputGroup = InputGroup.ANY) Object o) { } public static class ST_MakePolygon extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, @DataTypeHint(inputGroup = InputGroup.ANY) Object o2) { Geometry outerLinestring = (Geometry) o1; Geometry[] interiorLinestrings = (Geometry[]) o2; return org.apache.sedona.common.Functions.makePolygon(outerLinestring, interiorLinestrings); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry linestring = (Geometry) o; return org.apache.sedona.common.Functions.makePolygon(linestring, null); } } public static class ST_Points extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1) { Geometry geom = (Geometry) o1; return org.apache.sedona.common.Functions.points(geom); } } public static class ST_Polygon extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, @DataTypeHint("Integer") Integer srid) { Geometry linestring = (Geometry) o1; return org.apache.sedona.common.Functions.makepolygonWithSRID(linestring, srid); @@ -1308,20 +1206,18 @@ public Geometry eval( } public static class ST_Polygonize extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1) { Geometry geom = (Geometry) o1; return org.apache.sedona.common.Functions.polygonize(geom); } } public static class ST_Project extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, @DataTypeHint(value = "Double") Double distance, @DataTypeHint(value = "Double") Double azimuth, @DataTypeHint("Boolean") Boolean lenient) { @@ -1329,10 +1225,9 @@ public Geometry eval( return org.apache.sedona.common.Functions.project(point, distance, azimuth, lenient); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, @DataTypeHint(value = "Double") Double distance, @DataTypeHint(value = "Double") Double azimuth) { Geometry point = (Geometry) o1; @@ -1341,18 +1236,17 @@ public Geometry eval( } public static class ST_MakeValid extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Boolean") Boolean keepCollapsed) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.makeValid(geom, keepCollapsed); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.makeValid(geom, false); } @@ -1361,10 +1255,8 @@ public Geometry eval( public static class ST_MaxDistance extends ScalarFunction { @DataTypeHint(value = "Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object g1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object g2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object g1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object g2) { Geometry geom1 = (Geometry) g1; Geometry geom2 = (Geometry) g2; return org.apache.sedona.common.Functions.maxDistance(geom1, geom2); @@ -1374,36 +1266,33 @@ public Double eval( public static class ST_MinimumClearance extends ScalarFunction { @DataTypeHint(value = "Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.minimumClearance(geometry); } } public static class ST_MinimumClearanceLine extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.minimumClearanceLine(geometry); } } public static class ST_MinimumBoundingCircle extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Integer") Integer quadrantSegments) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.minimumBoundingCircle(geom, quadrantSegments); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.minimumBoundingCircle( geom, BufferParameters.DEFAULT_QUADRANT_SEGMENTS * 6); @@ -1413,45 +1302,41 @@ public Geometry eval( public static class ST_MinimumBoundingRadius extends ScalarFunction { @DataTypeHint(value = "RAW") public Pair eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.minimumBoundingRadius(geom); } } public static class ST_Multi extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.createMultiGeometryFromOneElement(geom); } } public static class ST_StartPoint extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.startPoint(geom); } } public static class ST_StraightSkeleton extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.straightSkeleton(geom); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Integer") Integer maxVertices) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.straightSkeleton(geom, maxVertices); @@ -1459,12 +1344,10 @@ public Geometry eval( } public static class ST_Split extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry input = (Geometry) o1; Geometry blade = (Geometry) o2; return org.apache.sedona.common.Functions.split(input, blade); @@ -1472,12 +1355,10 @@ public Geometry eval( } public static class ST_Snap extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2, @DataTypeHint("Double") Double tolerance) { Geometry input = (Geometry) o1; Geometry reference = (Geometry) o2; @@ -1488,7 +1369,7 @@ public Geometry eval( public static class ST_S2CellIDs extends ScalarFunction { @DataTypeHint(value = "ARRAY") public Long[] eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("INT") Integer level) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.s2CellIDs(geom, level); @@ -1496,7 +1377,7 @@ public Long[] eval( } public static class ST_S2ToGeom extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry[].class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryArrayTypeSerializer.class) public Geometry[] eval(@DataTypeHint(value = "ARRAY") Long[] cellIds) { return org.apache.sedona.common.Functions.s2ToGeom( Arrays.stream(cellIds).mapToLong(Long::longValue).toArray()); @@ -1506,7 +1387,7 @@ public Geometry[] eval(@DataTypeHint(value = "ARRAY") Long[] cellIds) { public static class ST_H3CellIDs extends ScalarFunction { @DataTypeHint(value = "ARRAY") public Long[] eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("INT") Integer level, @DataTypeHint("Boolean") Boolean fullCover) { Geometry geom = (Geometry) o; @@ -1532,7 +1413,7 @@ public Long[] eval( } public static class ST_H3ToGeom extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry[].class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryArrayTypeSerializer.class) public Geometry[] eval(@DataTypeHint(value = "ARRAY") Long[] cells) { return org.apache.sedona.common.Functions.h3ToGeom( Arrays.stream(cells).mapToLong(Long::longValue).toArray()); @@ -1540,9 +1421,9 @@ public Geometry[] eval(@DataTypeHint(value = "ARRAY") Long[] cells) { } public static class ST_Simplify extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double distanceTolerance) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.simplify(geom, distanceTolerance); @@ -1550,9 +1431,9 @@ public Geometry eval( } public static class ST_SimplifyPreserveTopology extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double distanceTolerance) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.simplifyPreserveTopology(geom, distanceTolerance); @@ -1560,9 +1441,9 @@ public Geometry eval( } public static class ST_SimplifyVW extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double distanceTolerance) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.simplifyVW(geom, distanceTolerance); @@ -1570,18 +1451,18 @@ public Geometry eval( } public static class ST_SimplifyPolygonHull extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double vertexFactor, @DataTypeHint("Boolean") Boolean isOuter) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.simplifyPolygonHull(geom, vertexFactor, isOuter); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double vertexFactor) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.simplifyPolygonHull(geom, vertexFactor); @@ -1589,9 +1470,9 @@ public Geometry eval( } public static class ST_Subdivide extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry[].class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryArrayTypeSerializer.class) public Geometry[] eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("INT") Integer maxVertices) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.subDivide(geom, maxVertices); @@ -1599,9 +1480,9 @@ public Geometry[] eval( } public static class ST_Segmentize extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double maxSegmentLength) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.segmentize(geom, maxSegmentLength); @@ -1609,12 +1490,10 @@ public Geometry eval( } public static class ST_SymDifference extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; return org.apache.sedona.common.Functions.symDifference(geom1, geom2); @@ -1622,26 +1501,26 @@ public Geometry eval( } public static class ST_GeometricMedian extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) throws Exception { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.geometricMedian(geometry); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double tolerance) throws Exception { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.geometricMedian(geometry, tolerance); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double tolerance, int maxIter) throws Exception { @@ -1649,9 +1528,9 @@ public Geometry eval( return org.apache.sedona.common.Functions.geometricMedian(geometry, tolerance, maxIter); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double tolerance, int maxIter, @DataTypeHint("Boolean") Boolean failIfNotConverged) @@ -1665,10 +1544,8 @@ public Geometry eval( public static class ST_FrechetDistance extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object g1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object g2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object g1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object g2) { Geometry geom1 = (Geometry) g1; Geometry geom2 = (Geometry) g2; return org.apache.sedona.common.Functions.frechetDistance(geom1, geom2); @@ -1678,7 +1555,7 @@ public Double eval( public static class ST_NumPoints extends ScalarFunction { @DataTypeHint(value = "Integer") public int eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) throws Exception { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.numPoints(geometry); @@ -1687,18 +1564,17 @@ public int eval( public static class ST_Force3D extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double zValue) { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.force3D(geometry, zValue); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.force3D(geometry); } @@ -1706,18 +1582,17 @@ public Geometry eval( public static class ST_Force3DM extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double zValue) { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.force3DM(geometry, zValue); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.force3DM(geometry); } @@ -1725,18 +1600,17 @@ public Geometry eval( public static class ST_Force3DZ extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double zValue) { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.force3D(geometry, zValue); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.force3D(geometry); } @@ -1744,66 +1618,62 @@ public Geometry eval( public static class ST_Force4D extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double zValue, @DataTypeHint("Double") Double mValue) { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.force4D(geometry, zValue, mValue); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.force4D(geometry); } } public static class ST_ForceCollection extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.forceCollection(geometry); } } public static class ST_ForcePolygonCW extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.forcePolygonCW(geometry); } } public static class ST_ForceRHR extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.forcePolygonCW(geometry); } } public static class ST_GeneratePoints extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint(value = "Integer") Integer numPoints) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.generatePoints(geom, numPoints); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint(value = "Integer") Integer numPoints, @DataTypeHint(value = "BIGINT") Long seed) { Geometry geom = (Geometry) o; @@ -1814,7 +1684,7 @@ public Geometry eval( public static class ST_NRings extends ScalarFunction { @DataTypeHint(value = "Integer") public int eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) throws Exception { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.nRings(geom); @@ -1822,10 +1692,9 @@ public int eval( } public static class ST_ForcePolygonCCW extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.forcePolygonCCW(geometry); } @@ -1834,26 +1703,25 @@ public Geometry eval( public static class ST_IsPolygonCCW extends ScalarFunction { @DataTypeHint("Boolean") public boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.isPolygonCCW(geom); } } public static class ST_Translate extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double deltaX, @DataTypeHint("Double") Double deltaY) { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.translate(geometry, deltaX, deltaY); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double deltaX, @DataTypeHint("Double") Double deltaY, @DataTypeHint("Double") Double deltaZ) { @@ -1863,38 +1731,34 @@ public Geometry eval( } public static class ST_TriangulatePolygon extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.triangulatePolygon(geometry); } } public static class ST_UnaryUnion extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.unaryUnion(geometry); } } public static class ST_Union extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry a = (Geometry) o1; Geometry b = (Geometry) o2; return org.apache.sedona.common.Functions.union(a, b); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval(@DataTypeHint(inputGroup = InputGroup.ANY) Object o) { Geometry[] geoms = (Geometry[]) o; return org.apache.sedona.common.Functions.union(geoms); @@ -1902,38 +1766,36 @@ public Geometry eval(@DataTypeHint(inputGroup = InputGroup.ANY) Object o) { } public static class ST_VoronoiPolygons extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double tolerance, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object extend) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object extend) { Geometry geom = (Geometry) o; Geometry extendTo = (Geometry) extend; return FunctionsGeoTools.voronoiPolygons(geom, tolerance, extendTo); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double tolerance) { Geometry geom = (Geometry) o; return FunctionsGeoTools.voronoiPolygons(geom, tolerance, null); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return FunctionsGeoTools.voronoiPolygons(geom, 0, null); } } public static class ST_Affine extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double a, @DataTypeHint("Double") Double b, @DataTypeHint("Double") Double c, @@ -1951,9 +1813,9 @@ public Geometry eval( geometry, a, b, c, d, e, f, g, h, i, xOff, yOff, zOff); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Double") Double a, @DataTypeHint("Double") Double b, @DataTypeHint("Double") Double d, @@ -1966,10 +1828,9 @@ public Geometry eval( } public static class ST_BoundingDiagonal extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.boundingDiagonal(geometry); } @@ -1978,10 +1839,8 @@ public Geometry eval( public static class ST_HausdorffDistance extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object g1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object g2, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object g1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object g2, @DataTypeHint("Double") Double densityFrac) throws Exception { Geometry geom1 = (Geometry) g1; @@ -1991,10 +1850,8 @@ public Double eval( @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object g1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object g2) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object g1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object g2) throws Exception { Geometry geom1 = (Geometry) g1; Geometry geom2 = (Geometry) g2; @@ -2005,8 +1862,7 @@ public Double eval( public static class ST_CoordDim extends ScalarFunction { @DataTypeHint("Integer") public Integer eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.nDims(geom); } @@ -2015,8 +1871,7 @@ public Integer eval( public static class ST_IsCollection extends ScalarFunction { @DataTypeHint("Boolean") public boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.isCollection(geom); } @@ -2026,14 +1881,10 @@ public static class ST_Angle extends ScalarFunction { @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object p1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object p2, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object p3, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object p4) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object p1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object p2, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object p3, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object p4) { Geometry point1 = (Geometry) p1; Geometry point2 = (Geometry) p2; Geometry point3 = (Geometry) p3; @@ -2044,12 +1895,9 @@ public Double eval( @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object p1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object p2, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object p3) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object p1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object p2, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object p3) { Geometry point1 = (Geometry) p1; Geometry point2 = (Geometry) p2; Geometry point3 = (Geometry) p3; @@ -2059,10 +1907,8 @@ public Double eval( @DataTypeHint("Double") public Double eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object line1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object line2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object line1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object line2) { Geometry lineString1 = (Geometry) line1; Geometry lineString2 = (Geometry) line2; @@ -2078,27 +1924,26 @@ public Double eval(@DataTypeHint("Double") Double angleInRadian) { } public static class ST_DelaunayTriangles extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint(value = "Double") Double tolerance, @DataTypeHint(value = "Integer") Integer flag) { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.delaunayTriangle(geometry, tolerance, flag); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint(value = "Double") Double tolerance) { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.delaunayTriangle(geometry, tolerance); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.delaunayTriangle(geometry); } @@ -2107,8 +1952,7 @@ public Geometry eval( public static class ST_IsValidTrajectory extends ScalarFunction { @DataTypeHint("Boolean") public Boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geometry = (Geometry) o; return org.apache.sedona.common.Functions.isValidTrajectory(geometry); } @@ -2117,15 +1961,14 @@ public Boolean eval( public static class ST_IsValidReason extends ScalarFunction { @DataTypeHint("String") public String eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.isValidReason(geom); } @DataTypeHint("String") public String eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("Integer") Integer flag) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.isValidReason(geom, flag); @@ -2133,9 +1976,9 @@ public String eval( } public static class ST_Scale extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint(value = "Double") Double scaleX, @DataTypeHint(value = "Double") Double scaleY) { Geometry geometry = (Geometry) o; @@ -2144,26 +1987,21 @@ public Geometry eval( } public static class ST_ScaleGeom 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, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o3) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) + public Geometry eval( + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o3) { Geometry geometry = (Geometry) o1; Geometry factor = (Geometry) o2; Geometry origin = (Geometry) o3; return org.apache.sedona.common.Functions.scaleGeom(geometry, factor, origin); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry geometry = (Geometry) o1; Geometry factor = (Geometry) o2; return org.apache.sedona.common.Functions.scaleGeom(geometry, factor); @@ -2171,9 +2009,9 @@ public Geometry eval( } public static class ST_RotateX extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint(value = "Double") Double angle) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.rotateX(geom, angle); @@ -2181,9 +2019,9 @@ public Geometry eval( } public static class ST_RotateY extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint(value = "Double") Double angle) { Geometry geom = (Geometry) o; return org.apache.sedona.common.Functions.rotateY(geom, angle); @@ -2191,31 +2029,27 @@ public Geometry eval( } public static class ST_Rotate extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, @DataTypeHint(value = "Double") Double angle) { Geometry geom1 = (Geometry) o1; return org.apache.sedona.common.Functions.rotate(geom1, angle); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, @DataTypeHint(value = "Double") Double angle, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; return org.apache.sedona.common.Functions.rotate(geom1, angle, geom2); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, @DataTypeHint(value = "Double") Double angle, @DataTypeHint(value = "Double") Double originX, @DataTypeHint(value = "Double") Double originY) { @@ -2227,8 +2061,8 @@ public Geometry eval( public static class ST_InterpolatePoint extends ScalarFunction { @DataTypeHint("Double") public double eval( - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) Object o1, - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; return org.apache.sedona.common.Functions.interpolatePoint(geom1, geom2); diff --git a/flink/src/main/java/org/apache/sedona/flink/expressions/FunctionsGeoTools.java b/flink/src/main/java/org/apache/sedona/flink/expressions/FunctionsGeoTools.java index 1085bc6656..5703313f56 100644 --- a/flink/src/main/java/org/apache/sedona/flink/expressions/FunctionsGeoTools.java +++ b/flink/src/main/java/org/apache/sedona/flink/expressions/FunctionsGeoTools.java @@ -20,24 +20,25 @@ import org.apache.flink.table.annotation.DataTypeHint; import org.apache.flink.table.functions.ScalarFunction; +import org.apache.sedona.flink.GeometryTypeSerializer; import org.geotools.api.referencing.FactoryException; import org.geotools.api.referencing.operation.TransformException; import org.locationtech.jts.geom.Geometry; public class FunctionsGeoTools { public static class ST_Transform extends ScalarFunction { - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("String") String targetCRS) throws FactoryException, TransformException { Geometry geom = (Geometry) o; return org.apache.sedona.common.FunctionsGeoTools.transform(geom, targetCRS); } - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("String") String sourceCRS, @DataTypeHint("String") String targetCRS) throws FactoryException, TransformException { @@ -45,9 +46,9 @@ public Geometry eval( return org.apache.sedona.common.FunctionsGeoTools.transform(geom, sourceCRS, targetCRS); } - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) public Geometry eval( - @DataTypeHint(value = "RAW", bridgedTo = Geometry.class) Object o, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o, @DataTypeHint("String") String sourceCRS, @DataTypeHint("String") String targetCRS, @DataTypeHint("Boolean") Boolean lenient) diff --git a/flink/src/main/java/org/apache/sedona/flink/expressions/Predicates.java b/flink/src/main/java/org/apache/sedona/flink/expressions/Predicates.java index 069b49b79f..9489eea629 100644 --- a/flink/src/main/java/org/apache/sedona/flink/expressions/Predicates.java +++ b/flink/src/main/java/org/apache/sedona/flink/expressions/Predicates.java @@ -20,6 +20,7 @@ import org.apache.flink.table.annotation.DataTypeHint; import org.apache.flink.table.functions.ScalarFunction; +import org.apache.sedona.flink.GeometryTypeSerializer; import org.locationtech.jts.geom.Geometry; public class Predicates { @@ -30,10 +31,8 @@ public ST_Intersects() {} @DataTypeHint("Boolean") public Boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; return org.apache.sedona.common.Predicates.intersects(geom1, geom2); @@ -47,10 +46,8 @@ public ST_Contains() {} @DataTypeHint("Boolean") public Boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; return org.apache.sedona.common.Predicates.contains(geom1, geom2); @@ -63,10 +60,8 @@ public ST_Within() {} @DataTypeHint("Boolean") public Boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; return org.apache.sedona.common.Predicates.within(geom1, geom2); @@ -80,10 +75,8 @@ public ST_Covers() {} @DataTypeHint("Boolean") public Boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; return org.apache.sedona.common.Predicates.covers(geom1, geom2); @@ -97,10 +90,8 @@ public ST_CoveredBy() {} @DataTypeHint("Boolean") public Boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; return org.apache.sedona.common.Predicates.coveredBy(geom1, geom2); @@ -113,10 +104,8 @@ public ST_Crosses() {} @DataTypeHint("Boolean") public Boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; return org.apache.sedona.common.Predicates.crosses(geom1, geom2); @@ -130,10 +119,8 @@ public ST_Disjoint() {} @DataTypeHint("Boolean") public Boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; return org.apache.sedona.common.Predicates.disjoint(geom1, geom2); @@ -147,10 +134,8 @@ public ST_Equals() {} @DataTypeHint("Boolean") public Boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; return org.apache.sedona.common.Predicates.equals(geom1, geom2); @@ -164,10 +149,8 @@ public ST_OrderingEquals() {} @DataTypeHint("Boolean") public Boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; return org.apache.sedona.common.Predicates.orderingEquals(geom1, geom2); @@ -181,10 +164,8 @@ public ST_Overlaps() {} @DataTypeHint("Boolean") public Boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; return org.apache.sedona.common.Predicates.overlaps(geom1, geom2); @@ -197,10 +178,8 @@ public ST_Relate() {} @DataTypeHint("String") public String eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; return org.apache.sedona.common.Predicates.relate(geom1, geom2); @@ -208,10 +187,8 @@ public String eval( @DataTypeHint("Boolean") public Boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2, @DataTypeHint("String") String IM) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; @@ -237,10 +214,8 @@ public ST_Touches() {} @DataTypeHint("Boolean") public Boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2) { + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; return org.apache.sedona.common.Predicates.touches(geom1, geom2); @@ -253,10 +228,8 @@ public ST_DWithin() {} @DataTypeHint("Boolean") public Boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2, @DataTypeHint("Double") Double distance) { Geometry geom1 = (Geometry) o1; Geometry geom2 = (Geometry) o2; @@ -265,10 +238,8 @@ public Boolean eval( @DataTypeHint("Boolean") public Boolean eval( - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o1, - @DataTypeHint(value = "RAW", bridgedTo = org.locationtech.jts.geom.Geometry.class) - Object o2, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o1, + @DataTypeHint(value = "RAW", rawSerializer = GeometryTypeSerializer.class) Object o2, @DataTypeHint("Double") Double distance, @DataTypeHint("Boolean") Boolean useSphere) { Geometry geom1 = (Geometry) o1;