You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2020/07/02 21:20:56 UTC

[GitHub] [incubator-pinot] yupeng9 opened a new pull request #5654: add geo support

yupeng9 opened a new pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654


   ## Description
   First part of https://github.com/apache/incubator-pinot/issues/5280. [Design doc](https://docs.google.com/document/d/1Mkm5RHS_tof-vIUt5-UNeOgRYSBAN6M_pN-hedV6Q0g/edit#heading=h.i45os595j1sp)
   
   This PR added the following
   
   ### add geo-spatial data model
   The data model includes both geometry and geography, which is differentiated by a spatial reference identifier (SRID). Notably, uses SRID=4326 as the coordinate system of lat/lng per https://epsg.io/4326.
   
   ### add serde
   Added the serialization/deserialization from geo-spatial value to bytes with kryo library. Also added a benchmark for performance evaluation
    
   ### add geospatial functions
   geo constructors
   - ST_GEOG_FROM_TEXT
   - ST_GEOG_FROM_WKB
   - ST_GEOM_FROM_TEXT
   - ST_GEOM_FROM_WKB
   - ST_POINT
   - ST_POLYGON
   
   geo measurements
   - ST_AREA (the geography area implementation is similar to Presto's)
   - ST_DISTANCE (the great circle distance implementation is similar to Presto's)
   - ST_GEOMETRY_TYPE
   
   geo outputs
   - ST_AS_BINARY
   - ST_AS_TEXT
   
   geo relationship
    - ST_CONTAINS
    - ST_EQUALS
   
   ### Updates to MeetupRsvp quickstart example
   Added a new `location` field from the longitude and latitude of the event, using an inbuilt `stPoint` transform function
   
   ## Upgrade Notes
   Does this PR prevent a zero down-time upgrade? (Assume upgrade order: Controller, Broker, Server, Minion)
   * [ ] No 
   
   Does this PR fix a zero-downtime upgrade introduced earlier?
   * [ ] No 
   
   Does this PR otherwise need attention when creating release notes? Things to consider:
   - New configuration options
   - Deprecation of configurations
   - Signature changes to public methods/interfaces
   - New plugins added or old plugins removed
   * [ ] Yes (Please label this PR as **<code>release-notes</code>** and complete the section on Release Notes)
   ## Release Notes
   Yes, added a new experimental feature
   
   ## Documentation
   If you have introduced a new feature or configuration, please add it to the documentation as well.
   See https://docs.pinot.apache.org/developers/developers-and-contributors/update-document
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] yupeng9 commented on a change in pull request #5654: [Part 1] Add geo support

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654#discussion_r449837110



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StAsTextFunction.java
##########
@@ -0,0 +1,86 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;

Review comment:
       hmm, then I cannot define scalar function in this package for inbuilt transformation functions.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] yupeng9 commented on pull request #5654: [Part 1] Add geo support

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654#issuecomment-653845835


   @kishoreg @mayankshriv thanks for the review. Will address the comments.
   @siddharthteotia No hurry. Please take your time.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang merged pull request #5654: [Part 1] Add geo support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang merged pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #5654: [Part 1] Add geo support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654#discussion_r456811395



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/TransformFunctionType.java
##########
@@ -57,8 +57,27 @@
   MAPVALUE("mapValue"),
 
   // Special type for annotation based scalar functions
-  SCALAR("scalar");
+  SCALAR("scalar"),
+  // geo constructors
+  ST_GEOG_FROM_TEXT("ST_GeogFromText"),
+  ST_GEOM_FROM_TEXT("ST_GeomFromText"),
+  ST_GEOG_FROM_WKB("ST_GeogFromWKB"),
+  ST_GEOM_FROM_WKB("ST_GeomFromWKB"),
+  ST_POINT("ST_Point"),
+  ST_POLYGON("ST_Polygon"),
 
+  // geo measurements
+  ST_AREA("ST_Area"),
+  ST_DISTANCE("ST_Distance"),
+  ST_GEOMETRY_TYPE("ST_GEOMETRY_TYPE"),
+
+  // geo outputs
+  ST_AS_BINARY("ST_AsBinary"),
+  ST_AS_TEXT("ST_AsText"),
+
+  // geo relationship
+  ST_CONTAINS("ST_Contains"),
+  ST_EQUALS("ST_Equals");

Review comment:
       (nit) Empty line after

##########
File path: pinot-spi/src/main/java/org/apache/pinot/spi/data/Schema.java
##########
@@ -400,7 +400,7 @@ public String toSingleLineJsonString() {
    * Validates a pinot schema.
    * <p>The following validations are performed:
    * <ul>
-   *   <li>For dimension, time, date time fields, support {@link DataType}: INT, LONG, FLOAT, DOUBLE, STRING</li>
+   *   <li>For dimension, time, date time fields, support {@link DataType}: INT, LONG, FLOAT, DOUBLE, STRING, BYTES</li>
    *   <li>For non-derived metric fields, support {@link DataType}: INT, LONG, FLOAT, DOUBLE</li>

Review comment:
       Thanks for updating the javadoc. Add BYTES here as well

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/serde/GeometrySerializationType.java
##########
@@ -0,0 +1,85 @@
+/**
+ * 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.pinot.core.geospatial.serde;
+
+import org.apache.pinot.core.geospatial.GeometryType;
+
+
+/**
+ * The geometry type used in serialization
+ */
+public enum GeometrySerializationType {

Review comment:
       Can we use `GeometryType` instead of introducing this extra type? I don't think `ENVELOPE` is used

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/serde/GeometrySerde.java
##########
@@ -0,0 +1,439 @@
+/**
+ * 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.pinot.core.geospatial.serde;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.Envelope;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.LinearRing;
+import org.locationtech.jts.geom.MultiPoint;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+import org.locationtech.jts.geom.TopologyException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static com.google.common.base.Verify.verify;
+import static com.google.common.collect.Iterables.getOnlyElement;
+import static java.lang.Double.NaN;
+import static java.lang.Double.isNaN;
+import static java.util.Objects.requireNonNull;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_FACTORY;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_GET_MASK;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_SET_MASK;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_SRID;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOMETRY_FACTORY;
+
+
+/**
+ * Provides methods to efficiently serialize and deserialize geometry types.
+ */
+public class GeometrySerde extends Serializer {
+  private static final Logger LOGGER = LoggerFactory.getLogger(GeometrySerde.class);
+
+  @Override
+  public void write(Kryo kryo, Output output, Object object) {

Review comment:
       Checked the Kryo implementation, it is only providing a buffer to the stream (very similar to `BufferedInputStream` and `BufferedOutputStream`). So I'm pretty sure reading/writing with pre-sized `ByteBuffer` will be much faster and the garbage generated will be much less. Also, with `ByteBuffer` we can store values with `LITTLE_ENDIEN` (linux native) to further improve the performance.
   I'd recommend revisiting this part because once the data is persistent, it will be hard to change

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StAreaFunction.java
##########
@@ -0,0 +1,221 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+
+import java.util.List;
+import java.util.Map;
+
+import static com.google.common.base.Preconditions.checkState;
+import static java.lang.Math.PI;
+import static java.lang.Math.toRadians;
+
+
+/**
+ * Function that calculates the area of the given geo-spatial object. For geometry type, it returns the 2D Euclidean
+ * area of a geometry. For geography, returns the area of a polygon or multi-polygon in square meters using a spherical
+ * model for Earth.
+ */
+public class StAreaFunction extends BaseTransformFunction {
+  private TransformFunction _transformFunction;
+  public static final String FUNCTION_NAME = "ST_Area";
+  private double[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 1, "Exactly 1 argument is required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Argument must be single-valued for transform function: %s", getName());

Review comment:
       Also check data type, should be BYTES. Same for other classes

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/serde/GeometrySerializationType.java
##########
@@ -0,0 +1,85 @@
+/**
+ * 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.pinot.core.geospatial.serde;
+
+import org.apache.pinot.core.geospatial.GeometryType;
+
+
+/**
+ * The geometry type used in serialization
+ */
+public enum GeometrySerializationType {
+  POINT(0, GeometryType.POINT),
+  MULTI_POINT(1, GeometryType.MULTI_POINT),
+  LINE_STRING(2, GeometryType.LINE_STRING),
+  MULTI_LINE_STRING(3, GeometryType.MULTI_LINE_STRING),
+  POLYGON(4, GeometryType.POLYGON),
+  MULTI_POLYGON(5, GeometryType.MULTI_POLYGON),
+  GEOMETRY_COLLECTION(6, GeometryType.GEOMETRY_COLLECTION),
+  ENVELOPE(7, GeometryType.POLYGON);
+
+  private final int _id;
+  private final GeometryType _geometryType;
+
+  GeometrySerializationType(int id, GeometryType geometryType) {
+    _id = id;
+    _geometryType = geometryType;
+  }
+
+  /**
+   * @return the id of the serialization type
+   */
+  public int id() {
+    return _id;
+  }
+
+  /**
+   * @return the type in the geometry model
+   */
+  public GeometryType getGeometryType() {
+    return _geometryType;
+  }
+
+  /**
+   * Constructs the serialization type from the id
+   * @param id id of the serialization type
+   * @return the serialization type
+   */
+  public static GeometrySerializationType fromID(int id) {

Review comment:
       You can keep a static array with id as the index to prevent the branching of `switch`

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StDistanceFunction.java
##########
@@ -0,0 +1,173 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryType;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.Point;
+
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.atan2;
+import static java.lang.Math.cos;
+import static java.lang.Math.sin;
+import static java.lang.Math.sqrt;
+import static java.lang.Math.toRadians;
+
+
+/**
+ * Function that measures the distance between the two geo-spatial objects. For geometry type, returns the 2-dimensional
+ * cartesian minimum distance (based on spatial ref) between two geometries in projected units. For geography, returns
+ * the great-circle distance in meters between two SphericalGeography points. Note that g1, g2 shall have the same type.
+ */
+public class StDistanceFunction extends BaseTransformFunction {
+  private static final float MIN_LATITUDE = -90;
+  private static final float MAX_LATITUDE = 90;
+  private static final float MIN_LONGITUDE = -180;
+  private static final float MAX_LONGITUDE = 180;
+  public static final String FUNCTION_NAME = "ST_Distance";
+  private TransformFunction _firstArgument;
+  private TransformFunction _secondArgument;
+  private double[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 2, "2 arguments are required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "First argument must be single-valued for transform function: %s", getName());
+    _firstArgument = transformFunction;
+    transformFunction = arguments.get(1);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Second argument must be single-valued for transform function: %s", getName());
+    _secondArgument = transformFunction;
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return DOUBLE_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public double[] transformToDoubleValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new double[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+    byte[][] firstValues = _firstArgument.transformToBytesValuesSV(projectionBlock);
+    byte[][] secondValues = _secondArgument.transformToBytesValuesSV(projectionBlock);
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      Geometry firstGeometry = GeometrySerializer.deserialize(firstValues[i]);
+      Geometry secondGeometry = GeometrySerializer.deserialize(secondValues[i]);
+      if (GeometryUtils.isGeography(firstGeometry) != GeometryUtils.isGeography(secondGeometry)) {
+        throw new RuntimeException(
+            String.format("The first and second arguments shall either all be geometry" + " or all geography"));
+      }
+      if (GeometryUtils.isGeography(firstGeometry)) {
+        _results[i] = sphericalDistance(firstGeometry, secondGeometry);
+      } else {
+        _results[i] =
+            firstGeometry.isEmpty() || secondGeometry.isEmpty() ? null : firstGeometry.distance(secondGeometry);
+      }
+    }
+    return _results;
+  }
+
+  public static void checkLatitude(double latitude) {
+    if (Double.isNaN(latitude) || Double.isInfinite(latitude) || latitude < MIN_LATITUDE || latitude > MAX_LATITUDE) {
+      throw new RuntimeException("Latitude must be between -90 and 90");
+    }
+  }
+
+  public static void checkLongitude(double longitude) {
+    if (Double.isNaN(longitude) || Double.isInfinite(longitude) || longitude < MIN_LONGITUDE
+        || longitude > MAX_LONGITUDE) {
+      throw new RuntimeException("Longitude must be between -180 and 180");
+    }
+  }
+
+  public static Double sphericalDistance(Geometry leftGeometry, Geometry rightGeometry) {
+    if (leftGeometry.isEmpty() || rightGeometry.isEmpty()) {
+      return null;
+    }
+
+    validateGeographyType("ST_Distance", leftGeometry, EnumSet.of(GeometryType.POINT));
+    validateGeographyType("ST_Distance", rightGeometry, EnumSet.of(GeometryType.POINT));
+    Point leftPoint = (Point) leftGeometry;
+    Point rightPoint = (Point) rightGeometry;
+
+    // greatCircleDistance returns distance in KM.
+    return greatCircleDistance(leftPoint.getY(), leftPoint.getX(), rightPoint.getY(), rightPoint.getX()) * 1000;
+  }
+
+  /**
+   * Calculate the distance between two points on Earth.
+   * <p>
+   * This assumes a spherical Earth, and uses the Vincenty formula. (https://en.wikipedia
+   * .org/wiki/Great-circle_distance)
+   */
+  public static double greatCircleDistance(double latitude1, double longitude1, double latitude2, double longitude2) {

Review comment:
       Directly return value in meter instead of km?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StEqualsFunction.java
##########
@@ -0,0 +1,87 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * Function that returns true if the given geometries represent the same geometry.
+ */
+public class StEqualsFunction extends BaseTransformFunction {
+  public static final String FUNCTION_NAME = "ST_Equals";
+  private TransformFunction _firstArgument;
+  private TransformFunction _secondArgument;
+  private int[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 2, "2 arguments are required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "First argument must be single-valued for transform function: %s", getName());
+    _firstArgument = transformFunction;
+    transformFunction = arguments.get(1);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Second argument must be single-valued for transform function: %s", getName());
+    _secondArgument = transformFunction;
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return INT_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public int[] transformToIntValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new int[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+    byte[][] firstValues = _firstArgument.transformToBytesValuesSV(projectionBlock);
+    byte[][] secondValues = _secondArgument.transformToBytesValuesSV(projectionBlock);
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      Geometry firstGeometry = GeometrySerializer.deserialize(firstValues[i]);
+      Geometry secondGeometry = GeometrySerializer.deserialize(secondValues[i]);
+      if (GeometryUtils.isGeography(firstGeometry) || GeometryUtils.isGeography(secondGeometry)) {

Review comment:
       Equals should work on geography as well?

##########
File path: pinot-core/pom.xml
##########
@@ -227,5 +234,17 @@
       <artifactId>lucene-analyzers-common</artifactId>
       <version>${lucene.version}</version>
     </dependency>
+    <dependency>

Review comment:
       Don't include jmh in `pinot-core`, you can move the benchmark class to the `pinot-perf` package

##########
File path: pinot-perf/pom.xml
##########
@@ -98,7 +98,7 @@
     <dependency>
       <groupId>org.openjdk.jmh</groupId>
       <artifactId>jmh-core</artifactId>
-      <version>1.21</version>
+      <version>${jmh.version}</version>

Review comment:
       Move these to the root pom and specify the version there

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StAreaFunction.java
##########
@@ -0,0 +1,221 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+
+import java.util.List;
+import java.util.Map;
+
+import static com.google.common.base.Preconditions.checkState;
+import static java.lang.Math.PI;
+import static java.lang.Math.toRadians;
+
+
+/**
+ * Function that calculates the area of the given geo-spatial object. For geometry type, it returns the 2D Euclidean
+ * area of a geometry. For geography, returns the area of a polygon or multi-polygon in square meters using a spherical
+ * model for Earth.
+ */
+public class StAreaFunction extends BaseTransformFunction {
+  private TransformFunction _transformFunction;
+  public static final String FUNCTION_NAME = "ST_Area";
+  private double[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 1, "Exactly 1 argument is required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Argument must be single-valued for transform function: %s", getName());
+    _transformFunction = transformFunction;
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return DOUBLE_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public double[] transformToDoubleValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new double[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+
+    byte[][] values = _transformFunction.transformToBytesValuesSV(projectionBlock);
+    Geometry geometry;
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      geometry = GeometrySerializer.deserialize(values[i]);
+      _results[i] = GeometryUtils.isGeography(geometry) ? calculateGeographyArea(geometry) : geometry.getArea();
+    }
+    return _results;
+  }
+
+  private double calculateGeographyArea(Geometry geometry) {
+    Polygon polygon = (Polygon) geometry;
+
+    // See https://www.movable-type.co.uk/scripts/latlong.html
+    // and http://osgeo-org.1560.x6.nabble.com/Area-of-a-spherical-polygon-td3841625.html
+    // and https://www.element84.com/blog/determining-if-a-spherical-polygon-contains-a-pole
+    // for the underlying Maths
+
+    double sphericalExcess = Math.abs(computeSphericalExcess(polygon.getExteriorRing()));
+
+    for (int i = 0; i < polygon.getNumInteriorRing(); i++) {
+      sphericalExcess -= Math.abs(computeSphericalExcess(polygon.getInteriorRingN(i)));
+    }
+
+    // Math.abs is required here because for Polygons with a 2D area of 0
+    // isExteriorRing returns false for the exterior ring
+    return Math.abs(sphericalExcess * GeometryUtils.EARTH_RADIUS_M * GeometryUtils.EARTH_RADIUS_M);
+  }
+
+  private static double computeSphericalExcess(LineString lineString) {
+    if (lineString.getNumPoints() < 3) {
+      // A path with less than 3 distinct points is not valid for calculating an area
+      throw new RuntimeException("Polygon is not valid: a loop contains less then 3 vertices.");
+    }
+
+    // Initialize the calculator with the last point
+    SphericalExcessCalculator calculator = new SphericalExcessCalculator(lineString.getEndPoint());
+
+    // Our calculations rely on not processing the same point twice
+    int start = lineString.getStartPoint().equals(lineString.getEndPoint()) ? 1 : 0;
+
+    for (int i = start; i < lineString.getNumPoints(); i++) {
+      calculator.add(lineString.getPointN(i));
+    }
+    return calculator.computeSphericalExcess();
+  }
+
+  private static class SphericalExcessCalculator {
+    private static final double TWO_PI = 2 * Math.PI;
+    private static final double THREE_PI = 3 * Math.PI;
+
+    private double sphericalExcess;
+    private double courseDelta;
+
+    private boolean firstPoint;
+    private double firstInitialBearing;
+    private double previousFinalBearing;
+
+    private double previousPhi;
+    private double previousCos;
+    private double previousSin;
+    private double previousTan;
+    private double previousLongitude;
+
+    private boolean done;
+
+    public SphericalExcessCalculator(Point endPoint) {
+      previousPhi = toRadians(endPoint.getY());
+      previousSin = Math.sin(previousPhi);
+      previousCos = Math.cos(previousPhi);
+      previousTan = Math.tan(previousPhi / 2);
+      previousLongitude = toRadians(endPoint.getX());
+      firstPoint = true;
+    }
+
+    private void add(Point point)
+        throws IllegalStateException {
+      checkState(!done, "Computation of spherical excess is complete");
+
+      double phi = toRadians(point.getY());
+      double tan = Math.tan(phi / 2);
+      double longitude = toRadians(point.getX());
+
+      // We need to check for that specifically
+      // Otherwise calculating the bearing is not deterministic
+      if (longitude == previousLongitude && phi == previousPhi) {
+        throw new RuntimeException("Polygon is not valid: it has two identical consecutive vertices");
+      }
+
+      double deltaLongitude = longitude - previousLongitude;
+      sphericalExcess += 2 * Math.atan2(Math.tan(deltaLongitude / 2) * (previousTan + tan), 1 + previousTan * tan);
+
+      double cos = Math.cos(phi);
+      double sin = Math.sin(phi);
+      double sinOfDeltaLongitude = Math.sin(deltaLongitude);
+      double cosOfDeltaLongitude = Math.cos(deltaLongitude);
+
+      // Initial bearing from previous to current
+      double y = sinOfDeltaLongitude * cos;
+      double x = previousCos * sin - previousSin * cos * cosOfDeltaLongitude;
+      double initialBearing = (Math.atan2(y, x) + TWO_PI) % TWO_PI;
+
+      // Final bearing from previous to current = opposite of bearing from current to previous
+      double finalY = -sinOfDeltaLongitude * previousCos;
+      double finalX = previousSin * cos - previousCos * sin * cosOfDeltaLongitude;
+      double finalBearing = (Math.atan2(finalY, finalX) + PI) % TWO_PI;
+
+      // When processing our first point we don't yet have a previousFinalBearing
+      if (firstPoint) {
+        // So keep our initial bearing around, and we'll use it at the end
+        // with the last final bearing
+        firstInitialBearing = initialBearing;
+        firstPoint = false;
+      } else {
+        courseDelta += (initialBearing - previousFinalBearing + THREE_PI) % TWO_PI - PI;
+      }
+
+      courseDelta += (finalBearing - initialBearing + THREE_PI) % TWO_PI - PI;
+
+      previousFinalBearing = finalBearing;
+      previousCos = cos;
+      previousSin = sin;
+      previousPhi = phi;
+      previousTan = tan;
+      previousLongitude = longitude;
+    }
+
+    public double computeSphericalExcess() {
+      if (!done) {
+        // Now that we have our last final bearing, we can calculate the remaining course delta
+        courseDelta += (firstInitialBearing - previousFinalBearing + THREE_PI) % TWO_PI - PI;
+
+        // The courseDelta should be 2Pi or - 2Pi, unless a pole is enclosed (and then it should be ~ 0)

Review comment:
       Do you mean should be close to 2Pi or -2Pi?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/GeometryType.java
##########
@@ -0,0 +1,49 @@
+/**
+ * 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.pinot.core.geospatial;
+
+/**
+ * The geometry type.
+ */
+public enum GeometryType {
+
+  POINT(false, "ST_Point"),
+  MULTI_POINT(true, "ST_MultiPoint"),
+  LINE_STRING(false, "ST_LineString"),
+  MULTI_LINE_STRING(true, "ST_MultiLineString"),
+  POLYGON(false, "ST_Polygon"),
+  MULTI_POLYGON(true, "ST_MultiPolygon"),
+  GEOMETRY_COLLECTION(true, "ST_GeomCollection");
+
+  private final boolean _multitype;
+  private final String _name;
+
+  GeometryType(boolean multitype, String name) {
+    _multitype = multitype;
+    _name = name;
+  }
+
+  public boolean isMultitype() {

Review comment:
       Seems never used. How are you planning to use the multi-type info?

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/TransformFunctionType.java
##########
@@ -57,8 +57,27 @@
   MAPVALUE("mapValue"),
 
   // Special type for annotation based scalar functions
-  SCALAR("scalar");
+  SCALAR("scalar"),
+  // geo constructors

Review comment:
       (nit) Add an empty line in front, and capitalize the comment

##########
File path: pom.xml
##########
@@ -1186,6 +1189,22 @@
         <artifactId>grpc-stub</artifactId>
         <version>${grpc.version}</version>
       </dependency>
+      <dependency>
+        <groupId>org.objenesis</groupId>
+        <artifactId>objenesis</artifactId>

Review comment:
       Where is this used?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/serde/GeometrySerde.java
##########
@@ -0,0 +1,439 @@
+/**
+ * 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.pinot.core.geospatial.serde;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.Envelope;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.LinearRing;
+import org.locationtech.jts.geom.MultiPoint;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+import org.locationtech.jts.geom.TopologyException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static com.google.common.base.Verify.verify;

Review comment:
       Avoid static import.
   Please re-order the imports using PinotStyle within `config/codestyle-intellij.xml` or `config/codestyle-eclipse.xml`

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/GeometryType.java
##########
@@ -0,0 +1,49 @@
+/**
+ * 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.pinot.core.geospatial;
+
+/**
+ * The geometry type.
+ */
+public enum GeometryType {
+
+  POINT(false, "ST_Point"),
+  MULTI_POINT(true, "ST_MultiPoint"),
+  LINE_STRING(false, "ST_LineString"),
+  MULTI_LINE_STRING(true, "ST_MultiLineString"),
+  POLYGON(false, "ST_Polygon"),
+  MULTI_POLYGON(true, "ST_MultiPolygon"),
+  GEOMETRY_COLLECTION(true, "ST_GeomCollection");
+
+  private final boolean _multitype;

Review comment:
       ```suggestion
     private final boolean _multiType;
   ```

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/TransformFunctionType.java
##########
@@ -57,8 +57,27 @@
   MAPVALUE("mapValue"),
 
   // Special type for annotation based scalar functions
-  SCALAR("scalar");
+  SCALAR("scalar"),
+  // geo constructors
+  ST_GEOG_FROM_TEXT("ST_GeogFromText"),
+  ST_GEOM_FROM_TEXT("ST_GeomFromText"),
+  ST_GEOG_FROM_WKB("ST_GeogFromWKB"),
+  ST_GEOM_FROM_WKB("ST_GeomFromWKB"),
+  ST_POINT("ST_Point"),
+  ST_POLYGON("ST_Polygon"),
 
+  // geo measurements
+  ST_AREA("ST_Area"),
+  ST_DISTANCE("ST_Distance"),
+  ST_GEOMETRY_TYPE("ST_GEOMETRY_TYPE"),

Review comment:
       ```suggestion
     ST_GEOMETRY_TYPE("ST_GeometryType"),
   ```

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/ConstructFromWKBFunction.java
##########
@@ -0,0 +1,80 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.io.ParseException;
+import org.locationtech.jts.io.WKBReader;
+
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * An abstract class for implementing the geo constructor functions from well-known binary (WKB) format.
+ */
+abstract class ConstructFromWKBFunction extends BaseTransformFunction {
+  private TransformFunction _transformFunction;
+  private byte[][] _results;
+  private WKBReader _reader;
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 1, "Exactly 1 argument is required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "The argument must be single-valued for transform function: %s", getName());
+    _transformFunction = transformFunction;
+    _reader = new WKBReader(getGeometryFactory());
+  }
+
+  abstract protected GeometryFactory getGeometryFactory();
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return BYTES_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public byte[][] transformToBytesValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new byte[DocIdSetPlanNode.MAX_DOC_PER_CALL][];
+    }
+    byte[][] argumentValues = _transformFunction.transformToBytesValuesSV(projectionBlock);
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      try {
+        Geometry geometry = _reader.read(argumentValues[i]);
+        _results[i] = GeometrySerializer.serialize(geometry);
+      } catch (ParseException e) {
+        throw new RuntimeException(String.format("Failed to parse geometry from bytes %s", argumentValues[i]));

Review comment:
       ```suggestion
           throw new RuntimeException(String.format("Failed to parse geometry from bytes: %s", BytesUtils.toHexString(argumentValues[i])));
   ```

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/ConstructFromTextFunction.java
##########
@@ -0,0 +1,83 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.common.Utils;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.io.ParseException;
+import org.locationtech.jts.io.WKTReader;
+
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * An abstract class for implementing the geo constructor functions from text.
+ */
+abstract class ConstructFromTextFunction extends BaseTransformFunction {
+  private TransformFunction _transformFunction;
+  private byte[][] _results;
+  private WKTReader _reader;
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 1, "Exactly 1 argument is required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "The argument must be single-valued for transform function: %s", getName());
+    _transformFunction = transformFunction;
+    _reader = new WKTReader(getGeometryFactory());
+  }
+
+  abstract protected GeometryFactory getGeometryFactory();
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return BYTES_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public byte[][] transformToBytesValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new byte[DocIdSetPlanNode.MAX_DOC_PER_CALL][];
+    }
+    String[] argumentValues = _transformFunction.transformToStringValuesSV(projectionBlock);
+    int length = projectionBlock.getNumDocs();
+    for (int i = 0; i < length; i++) {
+      try {
+        Geometry geometry = _reader.read(argumentValues[i]);
+        _results[i] = GeometrySerializer.serialize(geometry);

Review comment:
       Move this line out of the try-catch

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/TransformFunctionType.java
##########
@@ -57,8 +57,27 @@
   MAPVALUE("mapValue"),
 
   // Special type for annotation based scalar functions
-  SCALAR("scalar");
+  SCALAR("scalar"),
+  // geo constructors
+  ST_GEOG_FROM_TEXT("ST_GeogFromText"),
+  ST_GEOM_FROM_TEXT("ST_GeomFromText"),
+  ST_GEOG_FROM_WKB("ST_GeogFromWKB"),
+  ST_GEOM_FROM_WKB("ST_GeomFromWKB"),
+  ST_POINT("ST_Point"),
+  ST_POLYGON("ST_Polygon"),
 
+  // geo measurements
+  ST_AREA("ST_Area"),
+  ST_DISTANCE("ST_Distance"),
+  ST_GEOMETRY_TYPE("ST_GEOMETRY_TYPE"),

Review comment:
       Is there a standard function to return the SRID of the geometry? (Identify whether it is geometry or geography)

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StAreaFunction.java
##########
@@ -0,0 +1,221 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+
+import java.util.List;
+import java.util.Map;
+
+import static com.google.common.base.Preconditions.checkState;
+import static java.lang.Math.PI;
+import static java.lang.Math.toRadians;
+
+
+/**
+ * Function that calculates the area of the given geo-spatial object. For geometry type, it returns the 2D Euclidean
+ * area of a geometry. For geography, returns the area of a polygon or multi-polygon in square meters using a spherical
+ * model for Earth.
+ */
+public class StAreaFunction extends BaseTransformFunction {
+  private TransformFunction _transformFunction;
+  public static final String FUNCTION_NAME = "ST_Area";
+  private double[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 1, "Exactly 1 argument is required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Argument must be single-valued for transform function: %s", getName());
+    _transformFunction = transformFunction;
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return DOUBLE_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public double[] transformToDoubleValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new double[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+
+    byte[][] values = _transformFunction.transformToBytesValuesSV(projectionBlock);
+    Geometry geometry;
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      geometry = GeometrySerializer.deserialize(values[i]);
+      _results[i] = GeometryUtils.isGeography(geometry) ? calculateGeographyArea(geometry) : geometry.getArea();
+    }
+    return _results;
+  }
+
+  private double calculateGeographyArea(Geometry geometry) {
+    Polygon polygon = (Polygon) geometry;
+
+    // See https://www.movable-type.co.uk/scripts/latlong.html
+    // and http://osgeo-org.1560.x6.nabble.com/Area-of-a-spherical-polygon-td3841625.html
+    // and https://www.element84.com/blog/determining-if-a-spherical-polygon-contains-a-pole
+    // for the underlying Maths
+
+    double sphericalExcess = Math.abs(computeSphericalExcess(polygon.getExteriorRing()));
+
+    for (int i = 0; i < polygon.getNumInteriorRing(); i++) {
+      sphericalExcess -= Math.abs(computeSphericalExcess(polygon.getInteriorRingN(i)));
+    }
+
+    // Math.abs is required here because for Polygons with a 2D area of 0
+    // isExteriorRing returns false for the exterior ring
+    return Math.abs(sphericalExcess * GeometryUtils.EARTH_RADIUS_M * GeometryUtils.EARTH_RADIUS_M);
+  }
+
+  private static double computeSphericalExcess(LineString lineString) {
+    if (lineString.getNumPoints() < 3) {
+      // A path with less than 3 distinct points is not valid for calculating an area
+      throw new RuntimeException("Polygon is not valid: a loop contains less then 3 vertices.");
+    }
+
+    // Initialize the calculator with the last point
+    SphericalExcessCalculator calculator = new SphericalExcessCalculator(lineString.getEndPoint());
+
+    // Our calculations rely on not processing the same point twice
+    int start = lineString.getStartPoint().equals(lineString.getEndPoint()) ? 1 : 0;
+
+    for (int i = start; i < lineString.getNumPoints(); i++) {
+      calculator.add(lineString.getPointN(i));
+    }
+    return calculator.computeSphericalExcess();
+  }
+
+  private static class SphericalExcessCalculator {
+    private static final double TWO_PI = 2 * Math.PI;
+    private static final double THREE_PI = 3 * Math.PI;
+
+    private double sphericalExcess;

Review comment:
       Prefix with `_` for member variables

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/ConstructFromWKBFunction.java
##########
@@ -0,0 +1,80 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.io.ParseException;
+import org.locationtech.jts.io.WKBReader;
+
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * An abstract class for implementing the geo constructor functions from well-known binary (WKB) format.
+ */
+abstract class ConstructFromWKBFunction extends BaseTransformFunction {
+  private TransformFunction _transformFunction;
+  private byte[][] _results;
+  private WKBReader _reader;
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 1, "Exactly 1 argument is required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "The argument must be single-valued for transform function: %s", getName());
+    _transformFunction = transformFunction;
+    _reader = new WKBReader(getGeometryFactory());
+  }
+
+  abstract protected GeometryFactory getGeometryFactory();
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return BYTES_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public byte[][] transformToBytesValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new byte[DocIdSetPlanNode.MAX_DOC_PER_CALL][];
+    }
+    byte[][] argumentValues = _transformFunction.transformToBytesValuesSV(projectionBlock);
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      try {
+        Geometry geometry = _reader.read(argumentValues[i]);
+        _results[i] = GeometrySerializer.serialize(geometry);

Review comment:
       Move this line out of the try-catch

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StDistanceFunction.java
##########
@@ -0,0 +1,173 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryType;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.Point;
+
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.atan2;
+import static java.lang.Math.cos;
+import static java.lang.Math.sin;
+import static java.lang.Math.sqrt;
+import static java.lang.Math.toRadians;
+
+
+/**
+ * Function that measures the distance between the two geo-spatial objects. For geometry type, returns the 2-dimensional
+ * cartesian minimum distance (based on spatial ref) between two geometries in projected units. For geography, returns
+ * the great-circle distance in meters between two SphericalGeography points. Note that g1, g2 shall have the same type.
+ */
+public class StDistanceFunction extends BaseTransformFunction {
+  private static final float MIN_LATITUDE = -90;
+  private static final float MAX_LATITUDE = 90;
+  private static final float MIN_LONGITUDE = -180;
+  private static final float MAX_LONGITUDE = 180;
+  public static final String FUNCTION_NAME = "ST_Distance";
+  private TransformFunction _firstArgument;
+  private TransformFunction _secondArgument;
+  private double[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 2, "2 arguments are required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "First argument must be single-valued for transform function: %s", getName());
+    _firstArgument = transformFunction;
+    transformFunction = arguments.get(1);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Second argument must be single-valued for transform function: %s", getName());
+    _secondArgument = transformFunction;
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return DOUBLE_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public double[] transformToDoubleValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new double[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+    byte[][] firstValues = _firstArgument.transformToBytesValuesSV(projectionBlock);
+    byte[][] secondValues = _secondArgument.transformToBytesValuesSV(projectionBlock);
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      Geometry firstGeometry = GeometrySerializer.deserialize(firstValues[i]);
+      Geometry secondGeometry = GeometrySerializer.deserialize(secondValues[i]);
+      if (GeometryUtils.isGeography(firstGeometry) != GeometryUtils.isGeography(secondGeometry)) {
+        throw new RuntimeException(
+            String.format("The first and second arguments shall either all be geometry" + " or all geography"));
+      }
+      if (GeometryUtils.isGeography(firstGeometry)) {
+        _results[i] = sphericalDistance(firstGeometry, secondGeometry);
+      } else {
+        _results[i] =
+            firstGeometry.isEmpty() || secondGeometry.isEmpty() ? null : firstGeometry.distance(secondGeometry);
+      }
+    }
+    return _results;
+  }
+
+  public static void checkLatitude(double latitude) {

Review comment:
       Make all helper methods private

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StAreaFunction.java
##########
@@ -0,0 +1,221 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+
+import java.util.List;
+import java.util.Map;
+
+import static com.google.common.base.Preconditions.checkState;
+import static java.lang.Math.PI;
+import static java.lang.Math.toRadians;
+
+
+/**
+ * Function that calculates the area of the given geo-spatial object. For geometry type, it returns the 2D Euclidean
+ * area of a geometry. For geography, returns the area of a polygon or multi-polygon in square meters using a spherical
+ * model for Earth.
+ */
+public class StAreaFunction extends BaseTransformFunction {
+  private TransformFunction _transformFunction;
+  public static final String FUNCTION_NAME = "ST_Area";
+  private double[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 1, "Exactly 1 argument is required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Argument must be single-valued for transform function: %s", getName());
+    _transformFunction = transformFunction;
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return DOUBLE_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public double[] transformToDoubleValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new double[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+
+    byte[][] values = _transformFunction.transformToBytesValuesSV(projectionBlock);
+    Geometry geometry;
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      geometry = GeometrySerializer.deserialize(values[i]);
+      _results[i] = GeometryUtils.isGeography(geometry) ? calculateGeographyArea(geometry) : geometry.getArea();
+    }
+    return _results;
+  }
+
+  private double calculateGeographyArea(Geometry geometry) {
+    Polygon polygon = (Polygon) geometry;
+
+    // See https://www.movable-type.co.uk/scripts/latlong.html
+    // and http://osgeo-org.1560.x6.nabble.com/Area-of-a-spherical-polygon-td3841625.html
+    // and https://www.element84.com/blog/determining-if-a-spherical-polygon-contains-a-pole
+    // for the underlying Maths
+
+    double sphericalExcess = Math.abs(computeSphericalExcess(polygon.getExteriorRing()));
+
+    for (int i = 0; i < polygon.getNumInteriorRing(); i++) {
+      sphericalExcess -= Math.abs(computeSphericalExcess(polygon.getInteriorRingN(i)));
+    }
+
+    // Math.abs is required here because for Polygons with a 2D area of 0
+    // isExteriorRing returns false for the exterior ring

Review comment:
       I don't quite follow the comment. Where is `isExteriorRing()` used?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/GeometryType.java
##########
@@ -0,0 +1,49 @@
+/**
+ * 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.pinot.core.geospatial;
+
+/**
+ * The geometry type.
+ */
+public enum GeometryType {
+
+  POINT(false, "ST_Point"),
+  MULTI_POINT(true, "ST_MultiPoint"),
+  LINE_STRING(false, "ST_LineString"),

Review comment:
       Based on https://locationtech.github.io/jts/javadoc/org/locationtech/jts/geom/Geometry.html, there is another type LINEAR_RING. Should we include that?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StAreaFunction.java
##########
@@ -0,0 +1,221 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+
+import java.util.List;
+import java.util.Map;
+
+import static com.google.common.base.Preconditions.checkState;
+import static java.lang.Math.PI;
+import static java.lang.Math.toRadians;
+
+
+/**
+ * Function that calculates the area of the given geo-spatial object. For geometry type, it returns the 2D Euclidean
+ * area of a geometry. For geography, returns the area of a polygon or multi-polygon in square meters using a spherical
+ * model for Earth.
+ */
+public class StAreaFunction extends BaseTransformFunction {
+  private TransformFunction _transformFunction;
+  public static final String FUNCTION_NAME = "ST_Area";
+  private double[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 1, "Exactly 1 argument is required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Argument must be single-valued for transform function: %s", getName());
+    _transformFunction = transformFunction;
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return DOUBLE_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public double[] transformToDoubleValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new double[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+
+    byte[][] values = _transformFunction.transformToBytesValuesSV(projectionBlock);
+    Geometry geometry;
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      geometry = GeometrySerializer.deserialize(values[i]);

Review comment:
       (nit) Cache `numDocs`, same for other classes
   ```suggestion
       int numDocs = projectionBlock.getNumDocs();
       for (int i = 0; i < numDocs; i++) {
         Geometry geometry = GeometrySerializer.deserialize(values[i]);
   ```

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/ConstructFromTextFunction.java
##########
@@ -0,0 +1,83 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.common.Utils;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.io.ParseException;
+import org.locationtech.jts.io.WKTReader;
+
+import java.util.List;

Review comment:
       (nit) re-order imports. Same for other classes

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StAsTextFunction.java
##########
@@ -0,0 +1,89 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.common.function.annotations.ScalarFunction;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.io.WKTWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * Returns the text representation of the geometry object.
+ */
+public class StAsTextFunction extends BaseTransformFunction {
+  private static final Logger LOGGER = LoggerFactory.getLogger(StAsTextFunction.class);
+  private TransformFunction _transformFunction;
+  private static WKTWriter _writer;
+  public static final String FUNCTION_NAME = "ST_AsText";
+  private String[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 1, "Exactly 1 argument is required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Argument must be single-valued for transform function: %s", getName());
+    _transformFunction = transformFunction;
+    _writer = new WKTWriter();
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return STRING_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public String[] transformToStringValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new String[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+    byte[][] values = _transformFunction.transformToBytesValuesSV(projectionBlock);
+    Geometry geometry;
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      geometry = GeometrySerializer.deserialize(values[i]);
+      _results[i] = _writer.write(geometry);
+    }
+    return _results;
+  }
+
+  @ScalarFunction
+  public static String stAsText(byte[] bytes) {

Review comment:
       Maybe better to use a separate class for all the scalar functions?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StDistanceFunction.java
##########
@@ -0,0 +1,173 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryType;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.Point;
+
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.atan2;
+import static java.lang.Math.cos;
+import static java.lang.Math.sin;
+import static java.lang.Math.sqrt;
+import static java.lang.Math.toRadians;
+
+
+/**
+ * Function that measures the distance between the two geo-spatial objects. For geometry type, returns the 2-dimensional
+ * cartesian minimum distance (based on spatial ref) between two geometries in projected units. For geography, returns
+ * the great-circle distance in meters between two SphericalGeography points. Note that g1, g2 shall have the same type.
+ */
+public class StDistanceFunction extends BaseTransformFunction {
+  private static final float MIN_LATITUDE = -90;
+  private static final float MAX_LATITUDE = 90;
+  private static final float MIN_LONGITUDE = -180;
+  private static final float MAX_LONGITUDE = 180;
+  public static final String FUNCTION_NAME = "ST_Distance";
+  private TransformFunction _firstArgument;
+  private TransformFunction _secondArgument;
+  private double[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 2, "2 arguments are required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "First argument must be single-valued for transform function: %s", getName());
+    _firstArgument = transformFunction;
+    transformFunction = arguments.get(1);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Second argument must be single-valued for transform function: %s", getName());
+    _secondArgument = transformFunction;
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return DOUBLE_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public double[] transformToDoubleValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new double[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+    byte[][] firstValues = _firstArgument.transformToBytesValuesSV(projectionBlock);
+    byte[][] secondValues = _secondArgument.transformToBytesValuesSV(projectionBlock);
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      Geometry firstGeometry = GeometrySerializer.deserialize(firstValues[i]);
+      Geometry secondGeometry = GeometrySerializer.deserialize(secondValues[i]);
+      if (GeometryUtils.isGeography(firstGeometry) != GeometryUtils.isGeography(secondGeometry)) {
+        throw new RuntimeException(
+            String.format("The first and second arguments shall either all be geometry" + " or all geography"));
+      }
+      if (GeometryUtils.isGeography(firstGeometry)) {
+        _results[i] = sphericalDistance(firstGeometry, secondGeometry);
+      } else {
+        _results[i] =
+            firstGeometry.isEmpty() || secondGeometry.isEmpty() ? null : firstGeometry.distance(secondGeometry);

Review comment:
       `null` cannot be added to `double[]`
   Check empty point before calculating the distance (for both geometry and geography)

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/ConstructFromTextFunction.java
##########
@@ -0,0 +1,83 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.common.Utils;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.io.ParseException;
+import org.locationtech.jts.io.WKTReader;
+
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * An abstract class for implementing the geo constructor functions from text.
+ */
+abstract class ConstructFromTextFunction extends BaseTransformFunction {
+  private TransformFunction _transformFunction;
+  private byte[][] _results;
+  private WKTReader _reader;
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 1, "Exactly 1 argument is required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "The argument must be single-valued for transform function: %s", getName());
+    _transformFunction = transformFunction;
+    _reader = new WKTReader(getGeometryFactory());
+  }
+
+  abstract protected GeometryFactory getGeometryFactory();
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return BYTES_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public byte[][] transformToBytesValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new byte[DocIdSetPlanNode.MAX_DOC_PER_CALL][];
+    }
+    String[] argumentValues = _transformFunction.transformToStringValuesSV(projectionBlock);
+    int length = projectionBlock.getNumDocs();
+    for (int i = 0; i < length; i++) {
+      try {
+        Geometry geometry = _reader.read(argumentValues[i]);
+        _results[i] = GeometrySerializer.serialize(geometry);
+      } catch (ParseException e) {
+        Utils.rethrowException(
+            new RuntimeException(String.format("Failed to parse geometry from String %s", argumentValues[i])));

Review comment:
       ```suggestion
           throw new RuntimeException(String.format("Failed to parse geometry from string: %s", argumentValues[i]));
   ```

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StDistanceFunction.java
##########
@@ -0,0 +1,173 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryType;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.Point;
+
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.atan2;
+import static java.lang.Math.cos;
+import static java.lang.Math.sin;
+import static java.lang.Math.sqrt;
+import static java.lang.Math.toRadians;
+
+
+/**
+ * Function that measures the distance between the two geo-spatial objects. For geometry type, returns the 2-dimensional
+ * cartesian minimum distance (based on spatial ref) between two geometries in projected units. For geography, returns
+ * the great-circle distance in meters between two SphericalGeography points. Note that g1, g2 shall have the same type.
+ */
+public class StDistanceFunction extends BaseTransformFunction {
+  private static final float MIN_LATITUDE = -90;
+  private static final float MAX_LATITUDE = 90;
+  private static final float MIN_LONGITUDE = -180;
+  private static final float MAX_LONGITUDE = 180;
+  public static final String FUNCTION_NAME = "ST_Distance";
+  private TransformFunction _firstArgument;
+  private TransformFunction _secondArgument;
+  private double[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 2, "2 arguments are required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "First argument must be single-valued for transform function: %s", getName());
+    _firstArgument = transformFunction;
+    transformFunction = arguments.get(1);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Second argument must be single-valued for transform function: %s", getName());
+    _secondArgument = transformFunction;
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return DOUBLE_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public double[] transformToDoubleValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new double[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+    byte[][] firstValues = _firstArgument.transformToBytesValuesSV(projectionBlock);
+    byte[][] secondValues = _secondArgument.transformToBytesValuesSV(projectionBlock);
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      Geometry firstGeometry = GeometrySerializer.deserialize(firstValues[i]);
+      Geometry secondGeometry = GeometrySerializer.deserialize(secondValues[i]);
+      if (GeometryUtils.isGeography(firstGeometry) != GeometryUtils.isGeography(secondGeometry)) {
+        throw new RuntimeException(
+            String.format("The first and second arguments shall either all be geometry" + " or all geography"));
+      }
+      if (GeometryUtils.isGeography(firstGeometry)) {
+        _results[i] = sphericalDistance(firstGeometry, secondGeometry);
+      } else {
+        _results[i] =
+            firstGeometry.isEmpty() || secondGeometry.isEmpty() ? null : firstGeometry.distance(secondGeometry);
+      }
+    }
+    return _results;
+  }
+
+  public static void checkLatitude(double latitude) {
+    if (Double.isNaN(latitude) || Double.isInfinite(latitude) || latitude < MIN_LATITUDE || latitude > MAX_LATITUDE) {
+      throw new RuntimeException("Latitude must be between -90 and 90");
+    }

Review comment:
       ```suggestion
       Preconditions.checkArgument(latitude >= MIN_LATITUDE && latitude <= MAX_LATITUDE, "Latitude must be between -90 and 90");
   ```

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StDistanceFunction.java
##########
@@ -0,0 +1,173 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryType;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.Point;
+
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.atan2;
+import static java.lang.Math.cos;
+import static java.lang.Math.sin;
+import static java.lang.Math.sqrt;
+import static java.lang.Math.toRadians;
+
+
+/**
+ * Function that measures the distance between the two geo-spatial objects. For geometry type, returns the 2-dimensional
+ * cartesian minimum distance (based on spatial ref) between two geometries in projected units. For geography, returns
+ * the great-circle distance in meters between two SphericalGeography points. Note that g1, g2 shall have the same type.
+ */
+public class StDistanceFunction extends BaseTransformFunction {
+  private static final float MIN_LATITUDE = -90;
+  private static final float MAX_LATITUDE = 90;
+  private static final float MIN_LONGITUDE = -180;
+  private static final float MAX_LONGITUDE = 180;
+  public static final String FUNCTION_NAME = "ST_Distance";
+  private TransformFunction _firstArgument;
+  private TransformFunction _secondArgument;
+  private double[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 2, "2 arguments are required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "First argument must be single-valued for transform function: %s", getName());
+    _firstArgument = transformFunction;
+    transformFunction = arguments.get(1);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Second argument must be single-valued for transform function: %s", getName());
+    _secondArgument = transformFunction;
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return DOUBLE_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public double[] transformToDoubleValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new double[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+    byte[][] firstValues = _firstArgument.transformToBytesValuesSV(projectionBlock);
+    byte[][] secondValues = _secondArgument.transformToBytesValuesSV(projectionBlock);
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      Geometry firstGeometry = GeometrySerializer.deserialize(firstValues[i]);
+      Geometry secondGeometry = GeometrySerializer.deserialize(secondValues[i]);
+      if (GeometryUtils.isGeography(firstGeometry) != GeometryUtils.isGeography(secondGeometry)) {
+        throw new RuntimeException(
+            String.format("The first and second arguments shall either all be geometry" + " or all geography"));

Review comment:
       ```suggestion
           throw new RuntimeException("The first and second arguments shall either all be geometry or all geography");
   ```

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StAreaFunction.java
##########
@@ -0,0 +1,221 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+
+import java.util.List;
+import java.util.Map;
+
+import static com.google.common.base.Preconditions.checkState;
+import static java.lang.Math.PI;
+import static java.lang.Math.toRadians;
+
+
+/**
+ * Function that calculates the area of the given geo-spatial object. For geometry type, it returns the 2D Euclidean
+ * area of a geometry. For geography, returns the area of a polygon or multi-polygon in square meters using a spherical
+ * model for Earth.
+ */
+public class StAreaFunction extends BaseTransformFunction {
+  private TransformFunction _transformFunction;
+  public static final String FUNCTION_NAME = "ST_Area";
+  private double[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 1, "Exactly 1 argument is required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Argument must be single-valued for transform function: %s", getName());
+    _transformFunction = transformFunction;
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return DOUBLE_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public double[] transformToDoubleValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new double[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+
+    byte[][] values = _transformFunction.transformToBytesValuesSV(projectionBlock);
+    Geometry geometry;
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      geometry = GeometrySerializer.deserialize(values[i]);
+      _results[i] = GeometryUtils.isGeography(geometry) ? calculateGeographyArea(geometry) : geometry.getArea();
+    }
+    return _results;
+  }
+
+  private double calculateGeographyArea(Geometry geometry) {
+    Polygon polygon = (Polygon) geometry;
+
+    // See https://www.movable-type.co.uk/scripts/latlong.html
+    // and http://osgeo-org.1560.x6.nabble.com/Area-of-a-spherical-polygon-td3841625.html
+    // and https://www.element84.com/blog/determining-if-a-spherical-polygon-contains-a-pole
+    // for the underlying Maths

Review comment:
       Cool, thanks for adding the reference of the algorithm.
   Didn't review the algorithm implementation carefully, please make sure it is covered by test

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StAreaFunction.java
##########
@@ -0,0 +1,221 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+
+import java.util.List;
+import java.util.Map;
+
+import static com.google.common.base.Preconditions.checkState;
+import static java.lang.Math.PI;
+import static java.lang.Math.toRadians;
+
+
+/**
+ * Function that calculates the area of the given geo-spatial object. For geometry type, it returns the 2D Euclidean
+ * area of a geometry. For geography, returns the area of a polygon or multi-polygon in square meters using a spherical
+ * model for Earth.
+ */
+public class StAreaFunction extends BaseTransformFunction {
+  private TransformFunction _transformFunction;
+  public static final String FUNCTION_NAME = "ST_Area";
+  private double[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 1, "Exactly 1 argument is required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Argument must be single-valued for transform function: %s", getName());
+    _transformFunction = transformFunction;
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return DOUBLE_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public double[] transformToDoubleValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new double[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+
+    byte[][] values = _transformFunction.transformToBytesValuesSV(projectionBlock);
+    Geometry geometry;
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      geometry = GeometrySerializer.deserialize(values[i]);
+      _results[i] = GeometryUtils.isGeography(geometry) ? calculateGeographyArea(geometry) : geometry.getArea();
+    }
+    return _results;
+  }
+
+  private double calculateGeographyArea(Geometry geometry) {
+    Polygon polygon = (Polygon) geometry;
+
+    // See https://www.movable-type.co.uk/scripts/latlong.html
+    // and http://osgeo-org.1560.x6.nabble.com/Area-of-a-spherical-polygon-td3841625.html
+    // and https://www.element84.com/blog/determining-if-a-spherical-polygon-contains-a-pole
+    // for the underlying Maths
+
+    double sphericalExcess = Math.abs(computeSphericalExcess(polygon.getExteriorRing()));
+
+    for (int i = 0; i < polygon.getNumInteriorRing(); i++) {
+      sphericalExcess -= Math.abs(computeSphericalExcess(polygon.getInteriorRingN(i)));
+    }
+
+    // Math.abs is required here because for Polygons with a 2D area of 0
+    // isExteriorRing returns false for the exterior ring
+    return Math.abs(sphericalExcess * GeometryUtils.EARTH_RADIUS_M * GeometryUtils.EARTH_RADIUS_M);
+  }
+
+  private static double computeSphericalExcess(LineString lineString) {
+    if (lineString.getNumPoints() < 3) {
+      // A path with less than 3 distinct points is not valid for calculating an area
+      throw new RuntimeException("Polygon is not valid: a loop contains less then 3 vertices.");
+    }
+
+    // Initialize the calculator with the last point
+    SphericalExcessCalculator calculator = new SphericalExcessCalculator(lineString.getEndPoint());
+
+    // Our calculations rely on not processing the same point twice
+    int start = lineString.getStartPoint().equals(lineString.getEndPoint()) ? 1 : 0;

Review comment:
       I believe start point is always the same as end point? Or the more than 3 points check does not stand?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/serde/GeometrySerde.java
##########
@@ -0,0 +1,439 @@
+/**
+ * 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.pinot.core.geospatial.serde;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.Envelope;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.LinearRing;
+import org.locationtech.jts.geom.MultiPoint;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+import org.locationtech.jts.geom.TopologyException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static com.google.common.base.Verify.verify;
+import static com.google.common.collect.Iterables.getOnlyElement;
+import static java.lang.Double.NaN;
+import static java.lang.Double.isNaN;
+import static java.util.Objects.requireNonNull;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_FACTORY;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_GET_MASK;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_SET_MASK;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_SRID;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOMETRY_FACTORY;
+
+
+/**
+ * Provides methods to efficiently serialize and deserialize geometry types.
+ */
+public class GeometrySerde extends Serializer {

Review comment:
       Seems the code is borrowed from Presto? (https://github.com/prestodb/presto/blob/master/presto-geospatial-toolkit/src/main/java/com/facebook/presto/geospatial/serde/JtsGeometrySerde.java)
   Can you please include that in the javadoc?
   Also, do we need to keep the same serialization format as presto? Using `LITTLE_ENDIEN` (linux native order) can improve the performance of ser/de

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/ConstructFromWKBFunction.java
##########
@@ -0,0 +1,80 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.io.ParseException;
+import org.locationtech.jts.io.WKBReader;
+
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * An abstract class for implementing the geo constructor functions from well-known binary (WKB) format.
+ */
+abstract class ConstructFromWKBFunction extends BaseTransformFunction {
+  private TransformFunction _transformFunction;
+  private byte[][] _results;
+  private WKBReader _reader;
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 1, "Exactly 1 argument is required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "The argument must be single-valued for transform function: %s", getName());

Review comment:
       Also check data type, should be BYTES

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StDistanceFunction.java
##########
@@ -0,0 +1,173 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryType;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.Point;
+
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.atan2;
+import static java.lang.Math.cos;
+import static java.lang.Math.sin;
+import static java.lang.Math.sqrt;
+import static java.lang.Math.toRadians;
+
+
+/**
+ * Function that measures the distance between the two geo-spatial objects. For geometry type, returns the 2-dimensional
+ * cartesian minimum distance (based on spatial ref) between two geometries in projected units. For geography, returns
+ * the great-circle distance in meters between two SphericalGeography points. Note that g1, g2 shall have the same type.
+ */
+public class StDistanceFunction extends BaseTransformFunction {
+  private static final float MIN_LATITUDE = -90;
+  private static final float MAX_LATITUDE = 90;
+  private static final float MIN_LONGITUDE = -180;
+  private static final float MAX_LONGITUDE = 180;
+  public static final String FUNCTION_NAME = "ST_Distance";
+  private TransformFunction _firstArgument;
+  private TransformFunction _secondArgument;
+  private double[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 2, "2 arguments are required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "First argument must be single-valued for transform function: %s", getName());
+    _firstArgument = transformFunction;
+    transformFunction = arguments.get(1);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Second argument must be single-valued for transform function: %s", getName());
+    _secondArgument = transformFunction;
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return DOUBLE_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public double[] transformToDoubleValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new double[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+    byte[][] firstValues = _firstArgument.transformToBytesValuesSV(projectionBlock);
+    byte[][] secondValues = _secondArgument.transformToBytesValuesSV(projectionBlock);
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      Geometry firstGeometry = GeometrySerializer.deserialize(firstValues[i]);
+      Geometry secondGeometry = GeometrySerializer.deserialize(secondValues[i]);
+      if (GeometryUtils.isGeography(firstGeometry) != GeometryUtils.isGeography(secondGeometry)) {
+        throw new RuntimeException(
+            String.format("The first and second arguments shall either all be geometry" + " or all geography"));
+      }
+      if (GeometryUtils.isGeography(firstGeometry)) {
+        _results[i] = sphericalDistance(firstGeometry, secondGeometry);
+      } else {
+        _results[i] =
+            firstGeometry.isEmpty() || secondGeometry.isEmpty() ? null : firstGeometry.distance(secondGeometry);
+      }
+    }
+    return _results;
+  }
+
+  public static void checkLatitude(double latitude) {
+    if (Double.isNaN(latitude) || Double.isInfinite(latitude) || latitude < MIN_LATITUDE || latitude > MAX_LATITUDE) {
+      throw new RuntimeException("Latitude must be between -90 and 90");
+    }
+  }
+
+  public static void checkLongitude(double longitude) {
+    if (Double.isNaN(longitude) || Double.isInfinite(longitude) || longitude < MIN_LONGITUDE
+        || longitude > MAX_LONGITUDE) {
+      throw new RuntimeException("Longitude must be between -180 and 180");
+    }
+  }
+
+  public static Double sphericalDistance(Geometry leftGeometry, Geometry rightGeometry) {

Review comment:
       Return primitive double, or it will throw NPE

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/ConstructFromTextFunction.java
##########
@@ -0,0 +1,83 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.common.Utils;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.io.ParseException;
+import org.locationtech.jts.io.WKTReader;
+
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * An abstract class for implementing the geo constructor functions from text.
+ */
+abstract class ConstructFromTextFunction extends BaseTransformFunction {
+  private TransformFunction _transformFunction;
+  private byte[][] _results;
+  private WKTReader _reader;
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 1, "Exactly 1 argument is required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "The argument must be single-valued for transform function: %s", getName());

Review comment:
       Also check the data type, should be STRING

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StDistanceFunction.java
##########
@@ -0,0 +1,173 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryType;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.Point;
+
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.atan2;
+import static java.lang.Math.cos;
+import static java.lang.Math.sin;
+import static java.lang.Math.sqrt;
+import static java.lang.Math.toRadians;
+
+
+/**
+ * Function that measures the distance between the two geo-spatial objects. For geometry type, returns the 2-dimensional
+ * cartesian minimum distance (based on spatial ref) between two geometries in projected units. For geography, returns
+ * the great-circle distance in meters between two SphericalGeography points. Note that g1, g2 shall have the same type.
+ */
+public class StDistanceFunction extends BaseTransformFunction {
+  private static final float MIN_LATITUDE = -90;
+  private static final float MAX_LATITUDE = 90;
+  private static final float MIN_LONGITUDE = -180;
+  private static final float MAX_LONGITUDE = 180;
+  public static final String FUNCTION_NAME = "ST_Distance";
+  private TransformFunction _firstArgument;
+  private TransformFunction _secondArgument;
+  private double[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 2, "2 arguments are required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "First argument must be single-valued for transform function: %s", getName());
+    _firstArgument = transformFunction;
+    transformFunction = arguments.get(1);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Second argument must be single-valued for transform function: %s", getName());
+    _secondArgument = transformFunction;
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return DOUBLE_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public double[] transformToDoubleValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new double[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+    byte[][] firstValues = _firstArgument.transformToBytesValuesSV(projectionBlock);
+    byte[][] secondValues = _secondArgument.transformToBytesValuesSV(projectionBlock);
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      Geometry firstGeometry = GeometrySerializer.deserialize(firstValues[i]);
+      Geometry secondGeometry = GeometrySerializer.deserialize(secondValues[i]);
+      if (GeometryUtils.isGeography(firstGeometry) != GeometryUtils.isGeography(secondGeometry)) {
+        throw new RuntimeException(
+            String.format("The first and second arguments shall either all be geometry" + " or all geography"));
+      }
+      if (GeometryUtils.isGeography(firstGeometry)) {
+        _results[i] = sphericalDistance(firstGeometry, secondGeometry);
+      } else {
+        _results[i] =
+            firstGeometry.isEmpty() || secondGeometry.isEmpty() ? null : firstGeometry.distance(secondGeometry);
+      }
+    }
+    return _results;
+  }
+
+  public static void checkLatitude(double latitude) {
+    if (Double.isNaN(latitude) || Double.isInfinite(latitude) || latitude < MIN_LATITUDE || latitude > MAX_LATITUDE) {
+      throw new RuntimeException("Latitude must be between -90 and 90");
+    }
+  }
+
+  public static void checkLongitude(double longitude) {
+    if (Double.isNaN(longitude) || Double.isInfinite(longitude) || longitude < MIN_LONGITUDE
+        || longitude > MAX_LONGITUDE) {
+      throw new RuntimeException("Longitude must be between -180 and 180");
+    }
+  }
+
+  public static Double sphericalDistance(Geometry leftGeometry, Geometry rightGeometry) {
+    if (leftGeometry.isEmpty() || rightGeometry.isEmpty()) {
+      return null;
+    }
+
+    validateGeographyType("ST_Distance", leftGeometry, EnumSet.of(GeometryType.POINT));

Review comment:
       Can be simplified to `geometry instanceof Point`

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StDistanceFunction.java
##########
@@ -0,0 +1,173 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryType;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.Point;
+
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.atan2;
+import static java.lang.Math.cos;
+import static java.lang.Math.sin;
+import static java.lang.Math.sqrt;
+import static java.lang.Math.toRadians;
+
+
+/**
+ * Function that measures the distance between the two geo-spatial objects. For geometry type, returns the 2-dimensional
+ * cartesian minimum distance (based on spatial ref) between two geometries in projected units. For geography, returns
+ * the great-circle distance in meters between two SphericalGeography points. Note that g1, g2 shall have the same type.
+ */
+public class StDistanceFunction extends BaseTransformFunction {
+  private static final float MIN_LATITUDE = -90;
+  private static final float MAX_LATITUDE = 90;
+  private static final float MIN_LONGITUDE = -180;
+  private static final float MAX_LONGITUDE = 180;
+  public static final String FUNCTION_NAME = "ST_Distance";
+  private TransformFunction _firstArgument;
+  private TransformFunction _secondArgument;
+  private double[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 2, "2 arguments are required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "First argument must be single-valued for transform function: %s", getName());
+    _firstArgument = transformFunction;
+    transformFunction = arguments.get(1);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Second argument must be single-valued for transform function: %s", getName());
+    _secondArgument = transformFunction;
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return DOUBLE_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public double[] transformToDoubleValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new double[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+    byte[][] firstValues = _firstArgument.transformToBytesValuesSV(projectionBlock);
+    byte[][] secondValues = _secondArgument.transformToBytesValuesSV(projectionBlock);
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      Geometry firstGeometry = GeometrySerializer.deserialize(firstValues[i]);
+      Geometry secondGeometry = GeometrySerializer.deserialize(secondValues[i]);
+      if (GeometryUtils.isGeography(firstGeometry) != GeometryUtils.isGeography(secondGeometry)) {
+        throw new RuntimeException(
+            String.format("The first and second arguments shall either all be geometry" + " or all geography"));
+      }
+      if (GeometryUtils.isGeography(firstGeometry)) {
+        _results[i] = sphericalDistance(firstGeometry, secondGeometry);
+      } else {
+        _results[i] =
+            firstGeometry.isEmpty() || secondGeometry.isEmpty() ? null : firstGeometry.distance(secondGeometry);
+      }
+    }
+    return _results;
+  }
+
+  public static void checkLatitude(double latitude) {
+    if (Double.isNaN(latitude) || Double.isInfinite(latitude) || latitude < MIN_LATITUDE || latitude > MAX_LATITUDE) {
+      throw new RuntimeException("Latitude must be between -90 and 90");
+    }
+  }
+
+  public static void checkLongitude(double longitude) {
+    if (Double.isNaN(longitude) || Double.isInfinite(longitude) || longitude < MIN_LONGITUDE
+        || longitude > MAX_LONGITUDE) {
+      throw new RuntimeException("Longitude must be between -180 and 180");
+    }
+  }
+
+  public static Double sphericalDistance(Geometry leftGeometry, Geometry rightGeometry) {
+    if (leftGeometry.isEmpty() || rightGeometry.isEmpty()) {
+      return null;
+    }
+
+    validateGeographyType("ST_Distance", leftGeometry, EnumSet.of(GeometryType.POINT));
+    validateGeographyType("ST_Distance", rightGeometry, EnumSet.of(GeometryType.POINT));
+    Point leftPoint = (Point) leftGeometry;
+    Point rightPoint = (Point) rightGeometry;
+
+    // greatCircleDistance returns distance in KM.
+    return greatCircleDistance(leftPoint.getY(), leftPoint.getX(), rightPoint.getY(), rightPoint.getX()) * 1000;
+  }
+
+  /**
+   * Calculate the distance between two points on Earth.
+   * <p>
+   * This assumes a spherical Earth, and uses the Vincenty formula. (https://en.wikipedia
+   * .org/wiki/Great-circle_distance)
+   */
+  public static double greatCircleDistance(double latitude1, double longitude1, double latitude2, double longitude2) {

Review comment:
       Does it work on line across dateline?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StPolygonFunction.java
##########
@@ -0,0 +1,40 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.locationtech.jts.geom.GeometryFactory;
+
+
+/**
+ * Constructor function for polygon object from text.
+ */
+public class StPolygonFunction extends ConstructFromTextFunction {

Review comment:
       This doesn't seem right that `St_Polygon` is the same as `ST_GeomFromText`

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/serde/GeometrySerde.java
##########
@@ -0,0 +1,439 @@
+/**
+ * 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.pinot.core.geospatial.serde;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.Envelope;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.LinearRing;
+import org.locationtech.jts.geom.MultiPoint;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+import org.locationtech.jts.geom.TopologyException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static com.google.common.base.Verify.verify;
+import static com.google.common.collect.Iterables.getOnlyElement;
+import static java.lang.Double.NaN;
+import static java.lang.Double.isNaN;
+import static java.util.Objects.requireNonNull;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_FACTORY;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_GET_MASK;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_SET_MASK;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_SRID;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOMETRY_FACTORY;
+
+
+/**
+ * Provides methods to efficiently serialize and deserialize geometry types.
+ */
+public class GeometrySerde extends Serializer {
+  private static final Logger LOGGER = LoggerFactory.getLogger(GeometrySerde.class);
+
+  @Override
+  public void write(Kryo kryo, Output output, Object object) {
+    if (!(object instanceof Geometry)) {
+      throw new UnsupportedOperationException("Cannot serialize object of type " + object.getClass().getName());
+    }
+    writeGeometry(output, (Geometry) object);
+  }
+
+  @Override
+  public Object read(Kryo kryo, Input input, Class aClass) {
+    byte typeByte = input.readByte();
+    GeometrySerializationType type = readGeometryType(typeByte);
+    GeometryFactory factory = getGeometryFactory(typeByte);
+
+    return readGeometry(input, type, factory);
+  }
+
+  private Geometry readGeometry(Input input, GeometrySerializationType type, GeometryFactory factory) {
+    switch (type) {
+      case POINT:
+        return readPoint(input, factory);
+      case MULTI_POINT:
+        return readMultiPoint(input, factory);
+      case LINE_STRING:
+        return readPolyline(input, false, factory);
+      case MULTI_LINE_STRING:
+        return readPolyline(input, true, factory);
+      case POLYGON:
+        return readPolygon(input, false, factory);
+      case MULTI_POLYGON:
+        return readPolygon(input, true, factory);
+      case GEOMETRY_COLLECTION:
+        return readGeometryCollection(input, factory);
+      default:
+        throw new UnsupportedOperationException("Unexpected type: " + type);
+    }
+  }
+
+  private Point readPoint(Input input, GeometryFactory factory) {

Review comment:
       Javadoc describing the format of the serialized bytes will be preferred

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/BaseTransformFunction.java
##########
@@ -41,7 +41,10 @@
       new TransformResultMetadata(DataType.STRING, true, false);
   protected static final TransformResultMetadata STRING_MV_NO_DICTIONARY_METADATA =
       new TransformResultMetadata(DataType.STRING, false, false);
+  protected static final TransformResultMetadata BYTES_SV_NO_DICTIONARY_METADATA =
+          new TransformResultMetadata(DataType.BYTES, true, false);
 
+  private boolean[] _booleanValuesSV;

Review comment:
       Remove?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] mayankshriv commented on a change in pull request #5654: [Part 1] Add geo support

Posted by GitBox <gi...@apache.org>.
mayankshriv commented on a change in pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654#discussion_r449826482



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/serde/GeometrySerde.java
##########
@@ -0,0 +1,438 @@
+/**
+ * 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.pinot.core.geospatial.serde;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.Envelope;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.LinearRing;
+import org.locationtech.jts.geom.MultiPoint;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+import org.locationtech.jts.geom.TopologyException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static com.google.common.base.Verify.verify;
+import static com.google.common.collect.Iterables.getOnlyElement;
+import static java.lang.Double.NaN;
+import static java.lang.Double.isNaN;
+import static java.util.Objects.requireNonNull;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_FACTORY;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_GET_MASK;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_SET_MASK;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_SRID;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOMETRY_FACTORY;
+
+/**
+ * Provides methods to efficiently serialize and deserialize geometry types.
+ */
+public class GeometrySerde extends Serializer {
+    private static final Logger LOGGER = LoggerFactory.getLogger(GeometrySerde.class);

Review comment:
       Tab space indicates not following Pinot code-styling.

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/serde/GeometrySerde.java
##########
@@ -0,0 +1,438 @@
+/**
+ * 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.pinot.core.geospatial.serde;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.Envelope;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.LinearRing;
+import org.locationtech.jts.geom.MultiPoint;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+import org.locationtech.jts.geom.TopologyException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static com.google.common.base.Verify.verify;
+import static com.google.common.collect.Iterables.getOnlyElement;
+import static java.lang.Double.NaN;
+import static java.lang.Double.isNaN;
+import static java.util.Objects.requireNonNull;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_FACTORY;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_GET_MASK;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_SET_MASK;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_SRID;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOMETRY_FACTORY;
+
+/**
+ * Provides methods to efficiently serialize and deserialize geometry types.
+ */
+public class GeometrySerde extends Serializer {
+    private static final Logger LOGGER = LoggerFactory.getLogger(GeometrySerde.class);
+
+    @Override
+    public void write(Kryo kryo, Output output, Object object) {
+        if (!(object instanceof Geometry)) {
+            throw new UnsupportedOperationException("Cannot serialize object of type " +
+                    object.getClass().getName());
+        }
+        writeGeometry(output, (Geometry) object);
+    }
+
+    @Override
+    public Object read(Kryo kryo, Input input, Class aClass) {
+        byte typeByte = input.readByte();
+        GeometrySerializationType type = readGeometryType(typeByte);
+        GeometryFactory factory = getGeometryFactory(typeByte);
+
+        return readGeometry(input, type, factory);
+    }
+
+    private Geometry readGeometry(Input input, GeometrySerializationType type, GeometryFactory factory) {
+        switch (type) {
+            case POINT:
+                return readPoint(input, factory);
+            case MULTI_POINT:
+                return readMultiPoint(input, factory);
+            case LINE_STRING:
+                return readPolyline(input, false, factory);
+            case MULTI_LINE_STRING:
+                return readPolyline(input, true, factory);
+            case POLYGON:
+                return readPolygon(input, false, factory);
+            case MULTI_POLYGON:
+                return readPolygon(input, true, factory);
+            case GEOMETRY_COLLECTION:
+                return readGeometryCollection(input, factory);
+            default:
+                throw new UnsupportedOperationException("Unexpected type: " + type);
+        }
+    }
+
+    private Point readPoint(Input input, GeometryFactory factory) {
+        Coordinate coordinates = readCoordinate(input);
+        if (isNaN(coordinates.x) || isNaN(coordinates.y)) {
+            return factory.createPoint();
+        }
+        return factory.createPoint(coordinates);
+    }
+
+    private Coordinate readCoordinate(Input input) {
+        return new Coordinate(input.readDouble(), input.readDouble());
+    }
+
+    private Coordinate[] readCoordinates(Input input, int count) {
+        requireNonNull(input, "input is null");
+        verify(count > 0);
+        Coordinate[] coordinates = new Coordinate[count];
+        for (int i = 0; i < count; i++) {
+            coordinates[i] = readCoordinate(input);
+        }
+        return coordinates;
+    }
+
+    private Geometry readMultiPoint(Input input, GeometryFactory factory) {
+        int pointCount = input.readInt();
+        Point[] points = new Point[pointCount];
+        for (int i = 0; i < pointCount; i++) {
+            points[i] = readPoint(input, factory);
+        }
+        return factory.createMultiPoint(points);
+    }
+
+    private GeometrySerializationType readGeometryType(byte typeByte) {
+        return GeometrySerializationType.fromID(typeByte & GEOGRAPHY_GET_MASK);
+    }
+
+    private Geometry readPolyline(Input input, boolean multitype, GeometryFactory factory) {
+        int partCount = input.readInt();
+        if (partCount == 0) {
+            if (multitype) {
+                return factory.createMultiLineString();
+            }
+            return factory.createLineString();
+        }
+
+        int pointCount = input.readInt();
+        int[] startIndexes = new int[partCount];
+        for (int i = 0; i < partCount; i++) {
+            startIndexes[i] = input.readInt();
+        }
+
+        int[] partLengths = new int[partCount];
+        if (partCount > 1) {
+            partLengths[0] = startIndexes[1];
+            for (int i = 1; i < partCount - 1; i++) {
+                partLengths[i] = startIndexes[i + 1] - startIndexes[i];
+            }
+        }
+        partLengths[partCount - 1] = pointCount - startIndexes[partCount - 1];
+
+        LineString[] lineStrings = new LineString[partCount];
+
+        for (int i = 0; i < partCount; i++) {
+            lineStrings[i] = factory.createLineString(readCoordinates(input, partLengths[i]));
+        }
+
+        if (multitype) {
+            return factory.createMultiLineString(lineStrings);
+        }
+        verify(lineStrings.length == 1);
+        return lineStrings[0];
+    }
+
+    private Geometry readPolygon(Input input, boolean multitype, GeometryFactory factory) {
+        int partCount = input.readInt();
+        if (partCount == 0) {
+            if (multitype) {
+                return factory.createMultiPolygon();
+            }
+            return factory.createPolygon();
+        }
+
+        int pointCount = input.readInt();
+        int[] startIndexes = new int[partCount];
+        for (int i = 0; i < partCount; i++) {
+            startIndexes[i] = input.readInt();
+        }
+
+        int[] partLengths = new int[partCount];
+        if (partCount > 1) {
+            partLengths[0] = startIndexes[1];
+            for (int i = 1; i < partCount - 1; i++) {
+                partLengths[i] = startIndexes[i + 1] - startIndexes[i];
+            }
+        }
+        partLengths[partCount - 1] = pointCount - startIndexes[partCount - 1];
+
+        LinearRing shell = null;
+        List<LinearRing> holes = new ArrayList<>();

Review comment:
       Initialize list with size if known.

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/ConstructFromTextFunction.java
##########
@@ -0,0 +1,78 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.io.ParseException;
+import org.locationtech.jts.io.WKTReader;
+
+import java.util.List;
+import java.util.Map;
+
+public abstract class ConstructFromTextFunction extends BaseTransformFunction {
+    private TransformFunction _transformFunction;
+    private byte[][] _results;
+    private WKTReader _reader;
+
+    @Override
+    public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+        Preconditions
+                .checkArgument(arguments.size() == 1, "Exactly 1 argument is required for transform function: %s",
+                        getName());
+        TransformFunction transformFunction = arguments.get(0);
+        Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+                "The argument must be single-valued for transform function: %s", getName());
+        _transformFunction = transformFunction;
+        _reader = new WKTReader(getGeometryFactory());
+    }
+
+    abstract protected GeometryFactory getGeometryFactory();
+
+    @Override
+    public TransformResultMetadata getResultMetadata() {
+        return BYTES_SV_NO_DICTIONARY_METADATA;
+    }
+
+    @Override
+    public byte[][] transformToBytesValuesSV(ProjectionBlock projectionBlock) {
+        if (_results == null) {
+            _results = new byte[DocIdSetPlanNode.MAX_DOC_PER_CALL][];
+        }
+        String[] argumentValues = _transformFunction.transformToStringValuesSV(projectionBlock);
+        int length = projectionBlock.getNumDocs();
+        for (int i = 0; i < length; i++) {
+            try {
+                Geometry geometry = _reader.read(argumentValues[i]);
+                _results[i] = GeometrySerializer.serialize(geometry);
+            } catch (ParseException e) {
+                throw new RuntimeException(String.format("Failed to parse geometry from String %s", argumentValues[i]));

Review comment:
       Utils.rethrow will preserve the original exception.

##########
File path: pinot-core/pom.xml
##########
@@ -159,7 +159,14 @@
       <groupId>com.jayway.jsonpath</groupId>
       <artifactId>json-path</artifactId>
     </dependency>
-
+    <dependency>
+      <groupId>org.locationtech.jts</groupId>

Review comment:
       Is Eclipse license ok to add? So far we have taken Apache/MIT/Gnu.

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/serde/GeometrySerializer.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.pinot.core.geospatial.serde;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.locationtech.jts.geom.Geometry;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+
+public class GeometrySerializer {

Review comment:
       Please add Java doc to all classes and their public methods.

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/serde/GeometrySerializationType.java
##########
@@ -0,0 +1,76 @@
+/**
+ * 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.pinot.core.geospatial.serde;
+
+import org.apache.pinot.core.geospatial.GeometryType;
+
+public enum GeometrySerializationType
+{
+    POINT(0, GeometryType.POINT),
+    MULTI_POINT(1, GeometryType.MULTI_POINT),
+    LINE_STRING(2, GeometryType.LINE_STRING),
+    MULTI_LINE_STRING(3, GeometryType.MULTI_LINE_STRING),
+    POLYGON(4, GeometryType.POLYGON),
+    MULTI_POLYGON(5, GeometryType.MULTI_POLYGON),
+    GEOMETRY_COLLECTION(6, GeometryType.GEOMETRY_COLLECTION),
+    ENVELOPE(7, GeometryType.POLYGON);
+
+    private final int id;
+    private final GeometryType geometryType;
+
+    GeometrySerializationType(int id, GeometryType geometryType)
+    {
+        this.id = id;
+        this.geometryType = geometryType;
+    }
+
+    public int id()
+    {
+        return id;
+    }
+
+    public GeometryType getGeometryType()
+    {
+        return geometryType;
+    }
+
+    public static GeometrySerializationType fromID(int id)
+    {
+        switch (id) {

Review comment:
       Consider using a Static Map, if this list has a chance to grow.

##########
File path: pinot-tools/src/main/resources/examples/stream/meetupRsvp/meetupRsvp_schema.json
##########
@@ -37,6 +37,19 @@
     {
       "dataType": "STRING",
       "name": "group_name"
+    },
+    {
+      "dataType": "DOUBLE",
+      "name": "group_lat"
+    },
+    {
+      "dataType": "DOUBLE",
+      "name": "group_lon"
+    },
+    {
+      "dataType": "BYTES",
+      "name": "location",
+      "transformFunction": "stPoint(group_lon, group_lat)"

Review comment:
       +1

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/serde/GeometrySerializationType.java
##########
@@ -0,0 +1,76 @@
+/**
+ * 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.pinot.core.geospatial.serde;
+
+import org.apache.pinot.core.geospatial.GeometryType;
+
+public enum GeometrySerializationType
+{
+    POINT(0, GeometryType.POINT),
+    MULTI_POINT(1, GeometryType.MULTI_POINT),
+    LINE_STRING(2, GeometryType.LINE_STRING),
+    MULTI_LINE_STRING(3, GeometryType.MULTI_LINE_STRING),
+    POLYGON(4, GeometryType.POLYGON),
+    MULTI_POLYGON(5, GeometryType.MULTI_POLYGON),
+    GEOMETRY_COLLECTION(6, GeometryType.GEOMETRY_COLLECTION),
+    ENVELOPE(7, GeometryType.POLYGON);
+
+    private final int id;
+    private final GeometryType geometryType;
+
+    GeometrySerializationType(int id, GeometryType geometryType)
+    {
+        this.id = id;

Review comment:
       Please use Pinot code style (name of member variables starts with `_` to avoid qualifying with `this`.

##########
File path: pinot-tools/src/main/java/org/apache/pinot/tools/streams/MeetupRsvpStream.java
##########
@@ -87,11 +90,14 @@ public void onMessage(String message) {
                   }
 
                   JsonNode group = messageJSON.get("group");
+                  System.out.println(String.format("reading group %s", group.get("group_id")));

Review comment:
       Unsure if LOGGER should be used here?

##########
File path: pinot-tools/src/main/resources/log4j2.xml
##########
@@ -44,7 +44,7 @@
 
   </Appenders>
   <Loggers>
-    <Root level="info" additivity="false">
+    <Root level="warn" additivity="false">

Review comment:
       This seems unrelated to this PR? Would be good to call it out in the description, along with the motivation for the change.

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/geospatial/serde/BenchmarkSerde.java
##########
@@ -0,0 +1,356 @@
+/**
+ * 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.pinot.core.geospatial.serde;
+
+import com.google.common.base.Joiner;
+import com.google.common.io.Resources;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.io.ParseException;
+import org.locationtech.jts.io.WKTReader;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Warmup;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.Options;
+import org.openjdk.jmh.runner.options.OptionsBuilder;
+import org.openjdk.jmh.runner.options.VerboseMode;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+
+import static com.google.common.io.Resources.getResource;
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.openjdk.jmh.annotations.Mode.Throughput;
+
+import static org.apache.pinot.core.geospatial.serde.GeometrySerializer.serialize;
+import static org.apache.pinot.core.geospatial.serde.GeometrySerializer.deserialize;
+
+@State(Scope.Thread)
+@Fork(2)
+@Warmup(iterations = 3, time = 3, timeUnit = SECONDS)
+@Measurement(iterations = 5, time = 4, timeUnit = SECONDS)
+@OutputTimeUnit(SECONDS)
+@BenchmarkMode(Throughput)
+public class BenchmarkSerde

Review comment:
       Would be good to add the benchmark results in the PR description.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] yupeng9 commented on a change in pull request #5654: [Part 1] Add geo support

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654#discussion_r459183733



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/ScalarFunctions.java
##########
@@ -0,0 +1,57 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import org.apache.pinot.common.function.annotations.ScalarFunction;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.io.WKTWriter;
+
+
+/**
+ * Geospatial scalar functions that can be used in transformation.
+ */
+public class ScalarFunctions {
+
+  /**
+   * Creates a point.
+   *
+   * @param longitude longitude
+   * @param latitude latitude
+   * @return the created point
+   */
+  @ScalarFunction
+  public static byte[] stPoint(double longitude, double latitude) {
+    return GeometrySerializer
+        .serialize(GeometryUtils.GEOMETRY_FACTORY.createPoint(new Coordinate(longitude, latitude)));

Review comment:
       good point. Changed the argument to `x,y`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] yupeng9 edited a comment on pull request #5654: [Part 1] Add geo support

Posted by GitBox <gi...@apache.org>.
yupeng9 edited a comment on pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654#issuecomment-656268431


   > High level question, why are we using JTS library to handle both geometry as well as geography? Shouldn't we use ESRI for geography?
   
   Good call.
   There are two reasons, as I found out during prototyping:
   - Performance. I found the serialization performance JTS is better than ESRI. This is also confirmed by Presto project, as they are migrating from ESRI to JTS: https://github.com/prestodb/presto/pull/13604
   - Simplicity on the dependency. With one less dependency, it's easier to manage the implementation. As in my current way, the geometry to geography conversion is just a simple change of coordinate system (SRID).
   
   The tradeoff of taking this approach is that JTS is a library for Euclidean planar linear geometry, so all the geography-related operations have to be implemented using JTS's primitives. That's why there is some lengthy logic on geography measurement functions. Those implementations are similar to what Presto is doing.
   
   Will update the design doc to reflect this change


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] kishoreg commented on a change in pull request #5654: [Part 1] Add geo support

Posted by GitBox <gi...@apache.org>.
kishoreg commented on a change in pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654#discussion_r449692895



##########
File path: pinot-core/pom.xml
##########
@@ -223,5 +230,17 @@
       <artifactId>lucene-analyzers-common</artifactId>
       <version>${lucene.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.openjdk.jmh</groupId>
+      <artifactId>jmh-core</artifactId>
+      <version>1.21</version>

Review comment:
       move version to pom.properties

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StAsTextFunction.java
##########
@@ -0,0 +1,86 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;

Review comment:
       function -> functions

##########
File path: pinot-tools/src/main/resources/examples/stream/meetupRsvp/meetupRsvp_schema.json
##########
@@ -37,6 +37,19 @@
     {
       "dataType": "STRING",
       "name": "group_name"
+    },
+    {
+      "dataType": "DOUBLE",
+      "name": "group_lat"
+    },
+    {
+      "dataType": "DOUBLE",
+      "name": "group_lon"
+    },
+    {
+      "dataType": "BYTES",
+      "name": "location",
+      "transformFunction": "stPoint(group_lon, group_lat)"

Review comment:
       Great to see this work

##########
File path: pinot-tools/src/main/java/org/apache/pinot/tools/streams/MeetupRsvpStream.java
##########
@@ -101,18 +107,18 @@ public void onMessage(String message) {
                     producer.produce("meetupRSVPEvents", extracted.toString().getBytes(StandardCharsets.UTF_8));
                   }
                 } catch (Exception e) {
-                  //LOGGER.error("error processing raw event ", e);
+                  LOGGER.error("error processing raw event ", e);
                 }
               }
             });
             session.getBasicRemote().sendText("");
           } catch (IOException e) {
-            //LOGGER.error("found an event where data did not have all the fields, don't care about for quickstart");
+            LOGGER.error("found an event where data did not have all the fields, don't care about for quickstart", e);
           }
         }
       }, cec, new URI("ws://stream.meetup.com/2/rsvps"));
     } catch (Exception e) {
-      //e.printStackTrace();
+      e.printStackTrace();

Review comment:
       logger?

##########
File path: pinot-tools/src/main/java/org/apache/pinot/tools/streams/MeetupRsvpStream.java
##########
@@ -87,11 +90,14 @@ public void onMessage(String message) {
                   }
 
                   JsonNode group = messageJSON.get("group");
+                  System.out.println(String.format("reading group %s", group.get("group_id")));
                   if (group != null) {
                     extracted.set("group_city", group.get("group_city"));
                     extracted.set("group_country", group.get("group_country"));
                     extracted.set("group_id", group.get("group_id"));
                     extracted.set("group_name", group.get("group_name"));
+                    extracted.set("group_lat", group.get("group_lat"));

Review comment:
       sweet

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/ConstructFromTextFunction.java
##########
@@ -0,0 +1,78 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.io.ParseException;
+import org.locationtech.jts.io.WKTReader;
+
+import java.util.List;
+import java.util.Map;
+
+public abstract class ConstructFromTextFunction extends BaseTransformFunction {

Review comment:
       what does this function do? please add java docs




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] yupeng9 commented on a change in pull request #5654: [Part 1] Add geo support

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654#discussion_r449272231



##########
File path: pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/BrokerUserDefinedMessageHandlerFactory.java
##########
@@ -96,7 +96,7 @@ public HelixTaskResult handleMessage() {
 
     @Override
     public void onError(Exception e, ErrorCode code, ErrorType type) {
-      LOGGER.error("Got error while refreshing segment: {} of table: {} (error code: {}, error type: {})", _segmentName,

Review comment:
       plz ignore this, seems some search/replace error




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on pull request #5654: [Part 1] Add geo support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654#issuecomment-656319848


   > > High level question, why are we using JTS library to handle both geometry as well as geography? Shouldn't we use ESRI for geography?
   > 
   > Good call.
   > There are two reasons, as I found out during prototyping:
   > 
   > * Performance. I found the serialization performance JTS is better than ESRI. This is also confirmed by Presto project, as they are migrating from ESRI to JTS: [prestodb/presto#13604](https://github.com/prestodb/presto/pull/13604)
   > * Simplicity on the dependency. With one less dependency, it's easier to manage the implementation. As in my current way, the geometry to geography conversion is just a simple change of coordinate system (SRID).
   > 
   > The tradeoff of taking this approach is that JTS is a library for Euclidean planar linear geometry, so all the geography-related operations have to be implemented using JTS's primitives. That's why there is some lengthy logic on geography measurement functions. Those implementations are similar to what Presto is doing.
   > 
   > Will update the design doc to reflect this change
   
   For ser-de, we are using the customized serializer, so I don't think there will be performance difference between these 2 libraries?
   For geography operations, we should leverage ESRI for the calculation (or are you planing to implement all geography operations yourself?).
   We need to support both geometry (SRID 0) and geography (SRID 4326) operations:
   - If ESRI supports both geometry and geography operations, we can use only ESRI to do the calculation
   - If not, we should use JTS for geometry and ESRI for geography calculation


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] yupeng9 commented on a change in pull request #5654: [Part 1] Add geo support

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654#discussion_r452419516



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/GeometryType.java
##########
@@ -0,0 +1,49 @@
+/**
+ * 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.pinot.core.geospatial;
+
+/**
+ * The geometry type.
+ */
+public enum GeometryType {
+
+  POINT(false, "ST_Point"),
+  MULTI_POINT(true, "ST_MultiPoint"),
+  LINE_STRING(false, "ST_LineString"),
+  MULTI_LINE_STRING(true, "ST_MultiLineString"),
+  POLYGON(false, "ST_Polygon"),
+  MULTI_POLYGON(true, "ST_MultiPolygon"),
+  GEOMETRY_COLLECTION(true, "ST_GeomCollection");
+
+  private final boolean multitype;
+  private final String name;
+
+  GeometryType(boolean multitype, String name) {
+    this.multitype = multitype;

Review comment:
       @Jackie-Jiang any reason we set checkstyle severity to `warning` but not `error`. I saw we have such a rule for member variable, but the maven checkstyle does not fail the build




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #5654: [Part 1] Add geo support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654#discussion_r458998237



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/GeometryType.java
##########
@@ -0,0 +1,84 @@
+/**
+ * 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.pinot.core.geospatial;
+
+/**
+ * The geometry type.
+ */
+public enum GeometryType {
+
+  POINT(false, 0,"ST_Point"),

Review comment:
       (nit) reformat

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/GeometryType.java
##########
@@ -0,0 +1,84 @@
+/**
+ * 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.pinot.core.geospatial;
+
+/**
+ * The geometry type.
+ */
+public enum GeometryType {
+
+  POINT(false, 0,"ST_Point"),
+  MULTI_POINT(true, 1,"ST_MultiPoint"),
+  LINE_STRING(false, 2,"ST_LineString"),
+  MULTI_LINE_STRING(true, 3,"ST_MultiLineString"),
+  POLYGON(false, 4,"ST_Polygon"),
+  MULTI_POLYGON(true, 5,"ST_MultiPolygon"),
+  GEOMETRY_COLLECTION(true, 6,"ST_GeomCollection");
+
+  private final boolean _multitype;

Review comment:
       (nit) `_multiType`? (IDE identify `multitype` as typo)

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/serde/GeometrySerializationType.java
##########
@@ -0,0 +1,82 @@
+/**
+ * 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.pinot.core.geospatial.serde;
+
+import org.apache.pinot.core.geospatial.GeometryType;
+
+
+/**
+ * The geometry type used in serialization
+ */
+public enum GeometrySerializationType {

Review comment:
       Remove this class

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/GeometryType.java
##########
@@ -0,0 +1,84 @@
+/**
+ * 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.pinot.core.geospatial;
+
+/**
+ * The geometry type.
+ */
+public enum GeometryType {
+
+  POINT(false, 0,"ST_Point"),
+  MULTI_POINT(true, 1,"ST_MultiPoint"),
+  LINE_STRING(false, 2,"ST_LineString"),
+  MULTI_LINE_STRING(true, 3,"ST_MultiLineString"),
+  POLYGON(false, 4,"ST_Polygon"),
+  MULTI_POLYGON(true, 5,"ST_MultiPolygon"),
+  GEOMETRY_COLLECTION(true, 6,"ST_GeomCollection");
+
+  private final boolean _multitype;
+  private final int _id;
+  private final String _name;
+
+  GeometryType(boolean multitype, int id, String name) {
+    _multitype = multitype;
+    _id = id;
+    _name = name;
+  }
+
+  public boolean isMultitype() {
+    return _multitype;
+  }
+
+  public String getName() {
+    return _name;
+  }
+
+  /**
+   * @return the id of the serialization type
+   */
+  public int id() {
+    return _id;
+  }
+
+  /**
+   * Constructs the serialization type from the id
+   * @param id id of the serialization type
+   * @return the serialization type
+   */
+  public static GeometryType fromID(int id) {
+    switch (id) {

Review comment:
       Keep an static GeometryType array
   `private static final GeometryType[] ID_TO_TYPE_MAP = new GeometryType[] {POINT, MULTI_POINT, ...};`
   Then you can avoid the switch branching for better performance
   `return ID_TO_TYPE_MAP[id];`

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/serde/GeometrySerializer.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.pinot.core.geospatial.serde;
+
+import org.locationtech.jts.geom.Geometry;
+
+
+/**
+ * A serializer that serializes a geometry object into bytes and vice versa.
+ */
+public class GeometrySerializer {

Review comment:
       Merge `GeometrySerde` into this class? I don't see the value of keeping them separate

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/serde/GeometrySerde.java
##########
@@ -0,0 +1,510 @@
+/**
+ * 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.pinot.core.geospatial.serde;
+
+import com.google.common.base.Verify;
+import com.google.common.collect.Iterables;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import org.apache.pinot.core.geospatial.GeometryType;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.LinearRing;
+import org.locationtech.jts.geom.MultiPoint;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+import org.locationtech.jts.geom.TopologyException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Provides methods to efficiently serialize and deserialize geometry types.
+ *
+ * This serialization is similar to Presto's https://github.com/prestodb/presto/blob/master/presto-geospatial-toolkit/src/main/java/com/facebook/presto/geospatial/serde/JtsGeometrySerde.java,
+ * with the following differences:
+ *  - The geometry vs geography info is encoded in the type byte.
+ *  - The envelope info is not serialized
+ */
+public class GeometrySerde {
+  private static final Logger LOGGER = LoggerFactory.getLogger(GeometrySerde.class);
+  private static final int TYPE_SIZE = Byte.BYTES;
+  private static final int COORDINATE_SIZE = Double.BYTES + Double.BYTES;
+
+  byte[] writeGeometry(Geometry geometry) {
+    byte[] bytes = new byte[getByteSize(geometry)];
+    ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
+    writeGeometryByteBuffer(byteBuffer, geometry);
+    return bytes;
+  }
+
+  public Geometry readGeometry(byte[] bytes) {
+    return readGeometry(ByteBuffer.wrap(bytes));
+  }
+
+  private Geometry readGeometry(ByteBuffer byteBuffer) {
+    byte typeByte = byteBuffer.get();
+    GeometryType type = readGeometryType(typeByte);
+    GeometryFactory factory = getGeometryFactory(typeByte);
+    Geometry geometry = readGeometry(byteBuffer, type, factory);
+    return geometry;
+  }
+
+  private Geometry readGeometry(ByteBuffer byteBuffer, GeometryType type, GeometryFactory factory) {
+    switch (type) {
+      case POINT:
+        return readPoint(byteBuffer, factory);
+      case MULTI_POINT:
+        return readMultiPoint(byteBuffer, factory);
+      case LINE_STRING:
+        return readPolyline(byteBuffer, false, factory);
+      case MULTI_LINE_STRING:
+        return readPolyline(byteBuffer, true, factory);
+      case POLYGON:
+        return readPolygon(byteBuffer, false, factory);
+      case MULTI_POLYGON:
+        return readPolygon(byteBuffer, true, factory);
+      case GEOMETRY_COLLECTION:
+        return readGeometryCollection(byteBuffer, factory);
+      default:
+        throw new UnsupportedOperationException("Unexpected type: " + type);
+    }
+  }
+
+  private Point readPoint(ByteBuffer byteBuffer, GeometryFactory factory) {
+    Coordinate coordinates = readCoordinate(byteBuffer);
+    if (Double.isNaN(coordinates.x) || Double.isNaN(coordinates.y)) {
+      return factory.createPoint();
+    }
+    return factory.createPoint(coordinates);
+  }
+
+  private Coordinate readCoordinate(ByteBuffer byteBuffer) {
+    return new Coordinate(byteBuffer.getDouble(), byteBuffer.getDouble());
+  }
+
+  private Coordinate[] readCoordinates(ByteBuffer byteBuffer, int count) {
+    Objects.requireNonNull(byteBuffer, "input is null");

Review comment:
       (nit) Redundant check

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/ScalarFunctions.java
##########
@@ -0,0 +1,57 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import org.apache.pinot.common.function.annotations.ScalarFunction;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.io.WKTWriter;
+
+
+/**
+ * Geospatial scalar functions that can be used in transformation.
+ */
+public class ScalarFunctions {
+
+  /**
+   * Creates a point.
+   *
+   * @param longitude longitude
+   * @param latitude latitude
+   * @return the created point
+   */
+  @ScalarFunction
+  public static byte[] stPoint(double longitude, double latitude) {
+    return GeometrySerializer
+        .serialize(GeometryUtils.GEOMETRY_FACTORY.createPoint(new Coordinate(longitude, latitude)));

Review comment:
       (Major) Should this be `GEOGRAPHY_FACTORY` for longitude and latitude?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StDistanceFunction.java
##########
@@ -0,0 +1,173 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryType;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.Point;
+
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.atan2;
+import static java.lang.Math.cos;
+import static java.lang.Math.sin;
+import static java.lang.Math.sqrt;
+import static java.lang.Math.toRadians;
+
+
+/**
+ * Function that measures the distance between the two geo-spatial objects. For geometry type, returns the 2-dimensional
+ * cartesian minimum distance (based on spatial ref) between two geometries in projected units. For geography, returns
+ * the great-circle distance in meters between two SphericalGeography points. Note that g1, g2 shall have the same type.
+ */
+public class StDistanceFunction extends BaseTransformFunction {
+  private static final float MIN_LATITUDE = -90;
+  private static final float MAX_LATITUDE = 90;
+  private static final float MIN_LONGITUDE = -180;
+  private static final float MAX_LONGITUDE = 180;
+  public static final String FUNCTION_NAME = "ST_Distance";
+  private TransformFunction _firstArgument;
+  private TransformFunction _secondArgument;
+  private double[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 2, "2 arguments are required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "First argument must be single-valued for transform function: %s", getName());
+    _firstArgument = transformFunction;
+    transformFunction = arguments.get(1);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Second argument must be single-valued for transform function: %s", getName());
+    _secondArgument = transformFunction;
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return DOUBLE_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public double[] transformToDoubleValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new double[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+    byte[][] firstValues = _firstArgument.transformToBytesValuesSV(projectionBlock);
+    byte[][] secondValues = _secondArgument.transformToBytesValuesSV(projectionBlock);
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      Geometry firstGeometry = GeometrySerializer.deserialize(firstValues[i]);
+      Geometry secondGeometry = GeometrySerializer.deserialize(secondValues[i]);
+      if (GeometryUtils.isGeography(firstGeometry) != GeometryUtils.isGeography(secondGeometry)) {
+        throw new RuntimeException(
+            String.format("The first and second arguments shall either all be geometry" + " or all geography"));
+      }
+      if (GeometryUtils.isGeography(firstGeometry)) {
+        _results[i] = sphericalDistance(firstGeometry, secondGeometry);
+      } else {
+        _results[i] =
+            firstGeometry.isEmpty() || secondGeometry.isEmpty() ? null : firstGeometry.distance(secondGeometry);
+      }
+    }
+    return _results;
+  }
+
+  public static void checkLatitude(double latitude) {
+    if (Double.isNaN(latitude) || Double.isInfinite(latitude) || latitude < MIN_LATITUDE || latitude > MAX_LATITUDE) {
+      throw new RuntimeException("Latitude must be between -90 and 90");
+    }
+  }
+
+  public static void checkLongitude(double longitude) {
+    if (Double.isNaN(longitude) || Double.isInfinite(longitude) || longitude < MIN_LONGITUDE
+        || longitude > MAX_LONGITUDE) {
+      throw new RuntimeException("Longitude must be between -180 and 180");
+    }
+  }
+
+  public static Double sphericalDistance(Geometry leftGeometry, Geometry rightGeometry) {
+    if (leftGeometry.isEmpty() || rightGeometry.isEmpty()) {
+      return null;
+    }
+
+    validateGeographyType("ST_Distance", leftGeometry, EnumSet.of(GeometryType.POINT));

Review comment:
       There will be quite big performance difference, especially for per-value check

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StAreaFunction.java
##########
@@ -0,0 +1,221 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+
+import java.util.List;
+import java.util.Map;
+
+import static com.google.common.base.Preconditions.checkState;
+import static java.lang.Math.PI;
+import static java.lang.Math.toRadians;
+
+
+/**
+ * Function that calculates the area of the given geo-spatial object. For geometry type, it returns the 2D Euclidean
+ * area of a geometry. For geography, returns the area of a polygon or multi-polygon in square meters using a spherical
+ * model for Earth.
+ */
+public class StAreaFunction extends BaseTransformFunction {
+  private TransformFunction _transformFunction;
+  public static final String FUNCTION_NAME = "ST_Area";
+  private double[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 1, "Exactly 1 argument is required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Argument must be single-valued for transform function: %s", getName());
+    _transformFunction = transformFunction;
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return DOUBLE_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public double[] transformToDoubleValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new double[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+
+    byte[][] values = _transformFunction.transformToBytesValuesSV(projectionBlock);
+    Geometry geometry;
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      geometry = GeometrySerializer.deserialize(values[i]);

Review comment:
       Not necessary. I did some benchmark on this and there is performance difference

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/serde/GeometrySerde.java
##########
@@ -0,0 +1,510 @@
+/**
+ * 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.pinot.core.geospatial.serde;
+
+import com.google.common.base.Verify;
+import com.google.common.collect.Iterables;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import org.apache.pinot.core.geospatial.GeometryType;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.LinearRing;
+import org.locationtech.jts.geom.MultiPoint;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+import org.locationtech.jts.geom.TopologyException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Provides methods to efficiently serialize and deserialize geometry types.
+ *
+ * This serialization is similar to Presto's https://github.com/prestodb/presto/blob/master/presto-geospatial-toolkit/src/main/java/com/facebook/presto/geospatial/serde/JtsGeometrySerde.java,
+ * with the following differences:
+ *  - The geometry vs geography info is encoded in the type byte.
+ *  - The envelope info is not serialized
+ */
+public class GeometrySerde {
+  private static final Logger LOGGER = LoggerFactory.getLogger(GeometrySerde.class);

Review comment:
       (nit) Remove the unused LOGGER (we don't want to log within serde as it is per-value based and can easily flood the log)

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/serde/GeometrySerde.java
##########
@@ -0,0 +1,510 @@
+/**
+ * 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.pinot.core.geospatial.serde;
+
+import com.google.common.base.Verify;
+import com.google.common.collect.Iterables;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import org.apache.pinot.core.geospatial.GeometryType;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.LinearRing;
+import org.locationtech.jts.geom.MultiPoint;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+import org.locationtech.jts.geom.TopologyException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Provides methods to efficiently serialize and deserialize geometry types.
+ *
+ * This serialization is similar to Presto's https://github.com/prestodb/presto/blob/master/presto-geospatial-toolkit/src/main/java/com/facebook/presto/geospatial/serde/JtsGeometrySerde.java,
+ * with the following differences:
+ *  - The geometry vs geography info is encoded in the type byte.
+ *  - The envelope info is not serialized
+ */
+public class GeometrySerde {
+  private static final Logger LOGGER = LoggerFactory.getLogger(GeometrySerde.class);
+  private static final int TYPE_SIZE = Byte.BYTES;
+  private static final int COORDINATE_SIZE = Double.BYTES + Double.BYTES;
+
+  byte[] writeGeometry(Geometry geometry) {
+    byte[] bytes = new byte[getByteSize(geometry)];
+    ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
+    writeGeometryByteBuffer(byteBuffer, geometry);
+    return bytes;
+  }
+
+  public Geometry readGeometry(byte[] bytes) {
+    return readGeometry(ByteBuffer.wrap(bytes));
+  }
+
+  private Geometry readGeometry(ByteBuffer byteBuffer) {
+    byte typeByte = byteBuffer.get();
+    GeometryType type = readGeometryType(typeByte);
+    GeometryFactory factory = getGeometryFactory(typeByte);
+    Geometry geometry = readGeometry(byteBuffer, type, factory);
+    return geometry;
+  }
+
+  private Geometry readGeometry(ByteBuffer byteBuffer, GeometryType type, GeometryFactory factory) {
+    switch (type) {
+      case POINT:
+        return readPoint(byteBuffer, factory);
+      case MULTI_POINT:
+        return readMultiPoint(byteBuffer, factory);
+      case LINE_STRING:
+        return readPolyline(byteBuffer, false, factory);
+      case MULTI_LINE_STRING:
+        return readPolyline(byteBuffer, true, factory);
+      case POLYGON:
+        return readPolygon(byteBuffer, false, factory);
+      case MULTI_POLYGON:
+        return readPolygon(byteBuffer, true, factory);
+      case GEOMETRY_COLLECTION:
+        return readGeometryCollection(byteBuffer, factory);
+      default:
+        throw new UnsupportedOperationException("Unexpected type: " + type);
+    }
+  }
+
+  private Point readPoint(ByteBuffer byteBuffer, GeometryFactory factory) {
+    Coordinate coordinates = readCoordinate(byteBuffer);
+    if (Double.isNaN(coordinates.x) || Double.isNaN(coordinates.y)) {
+      return factory.createPoint();
+    }
+    return factory.createPoint(coordinates);
+  }
+
+  private Coordinate readCoordinate(ByteBuffer byteBuffer) {
+    return new Coordinate(byteBuffer.getDouble(), byteBuffer.getDouble());
+  }
+
+  private Coordinate[] readCoordinates(ByteBuffer byteBuffer, int count) {
+    Objects.requireNonNull(byteBuffer, "input is null");
+    Verify.verify(count > 0);

Review comment:
       Let's use `Preconditions` for argument check. Same for other places

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/ConstructFromTextFunction.java
##########
@@ -0,0 +1,85 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import java.util.List;
+import java.util.Map;
+import org.apache.pinot.common.Utils;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.io.ParseException;
+import org.locationtech.jts.io.WKTReader;
+
+
+/**
+ * An abstract class for implementing the geo constructor functions from text.
+ */
+abstract class ConstructFromTextFunction extends BaseTransformFunction {
+  protected TransformFunction _transformFunction;
+  protected byte[][] _results;
+  protected WKTReader _reader;
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 1, "Exactly 1 argument is required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "The argument must be single-valued for transform function: %s", getName());
+    Preconditions.checkArgument(transformFunction.getResultMetadata().getDataType() == FieldSpec.DataType.STRING,
+        "The argument must be of string type");
+    _transformFunction = transformFunction;
+    _reader = new WKTReader(getGeometryFactory());
+  }
+
+  abstract protected GeometryFactory getGeometryFactory();
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return BYTES_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public byte[][] transformToBytesValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new byte[DocIdSetPlanNode.MAX_DOC_PER_CALL][];
+    }
+    String[] argumentValues = _transformFunction.transformToStringValuesSV(projectionBlock);
+    int length = projectionBlock.getNumDocs();
+    for (int i = 0; i < length; i++) {
+      try {
+        Geometry geometry = _reader.read(argumentValues[i]);
+        _results[i] = GeometrySerializer.serialize(geometry);
+      } catch (ParseException e) {
+        Utils.rethrowException(
+            new RuntimeException(String.format("Failed to parse geometry from string: %s", argumentValues[i])));

Review comment:
       ```suggestion
           throw new RuntimeException(String.format("Failed to parse geometry from string: %s", argumentValues[i]));
   ```

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StPolygonFunction.java
##########
@@ -0,0 +1,69 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import java.util.EnumSet;
+import org.apache.pinot.common.Utils;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.Polygon;
+import org.locationtech.jts.io.ParseException;
+
+
+/**
+ * Constructor function for polygon object from text.
+ */
+public class StPolygonFunction extends ConstructFromTextFunction {
+  public static final String FUNCTION_NAME = "ST_Polygon";
+
+  @Override
+  protected GeometryFactory getGeometryFactory() {
+    return GeometryUtils.GEOMETRY_FACTORY;
+  }
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public byte[][] transformToBytesValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new byte[DocIdSetPlanNode.MAX_DOC_PER_CALL][];
+    }
+    String[] argumentValues = _transformFunction.transformToStringValuesSV(projectionBlock);
+    int length = projectionBlock.getNumDocs();
+    for (int i = 0; i < length; i++) {
+      try {
+        Geometry geometry = _reader.read(argumentValues[i]);
+        Preconditions.checkArgument(geometry instanceof Polygon, "The geometry object must be polygon");
+        _results[i] = GeometrySerializer.serialize(geometry);
+      } catch (ParseException e) {
+        Utils.rethrowException(
+            new RuntimeException(String.format("Failed to parse geometry from string: %s", argumentValues[i])));

Review comment:
       ```suggestion
           throw new RuntimeException(String.format("Failed to parse geometry from string: %s", argumentValues[i]));
   ```

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StDistanceFunction.java
##########
@@ -0,0 +1,173 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryType;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.Point;
+
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.atan2;
+import static java.lang.Math.cos;
+import static java.lang.Math.sin;
+import static java.lang.Math.sqrt;
+import static java.lang.Math.toRadians;
+
+
+/**
+ * Function that measures the distance between the two geo-spatial objects. For geometry type, returns the 2-dimensional
+ * cartesian minimum distance (based on spatial ref) between two geometries in projected units. For geography, returns
+ * the great-circle distance in meters between two SphericalGeography points. Note that g1, g2 shall have the same type.
+ */
+public class StDistanceFunction extends BaseTransformFunction {
+  private static final float MIN_LATITUDE = -90;
+  private static final float MAX_LATITUDE = 90;
+  private static final float MIN_LONGITUDE = -180;
+  private static final float MAX_LONGITUDE = 180;
+  public static final String FUNCTION_NAME = "ST_Distance";
+  private TransformFunction _firstArgument;
+  private TransformFunction _secondArgument;
+  private double[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 2, "2 arguments are required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "First argument must be single-valued for transform function: %s", getName());
+    _firstArgument = transformFunction;
+    transformFunction = arguments.get(1);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Second argument must be single-valued for transform function: %s", getName());
+    _secondArgument = transformFunction;
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return DOUBLE_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public double[] transformToDoubleValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new double[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+    byte[][] firstValues = _firstArgument.transformToBytesValuesSV(projectionBlock);
+    byte[][] secondValues = _secondArgument.transformToBytesValuesSV(projectionBlock);
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      Geometry firstGeometry = GeometrySerializer.deserialize(firstValues[i]);
+      Geometry secondGeometry = GeometrySerializer.deserialize(secondValues[i]);
+      if (GeometryUtils.isGeography(firstGeometry) != GeometryUtils.isGeography(secondGeometry)) {
+        throw new RuntimeException(
+            String.format("The first and second arguments shall either all be geometry" + " or all geography"));
+      }
+      if (GeometryUtils.isGeography(firstGeometry)) {
+        _results[i] = sphericalDistance(firstGeometry, secondGeometry);
+      } else {
+        _results[i] =
+            firstGeometry.isEmpty() || secondGeometry.isEmpty() ? null : firstGeometry.distance(secondGeometry);

Review comment:
       I think you can return `Double.NaN` here to indicate empty geometry




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] yupeng9 commented on a change in pull request #5654: [Part 1] Add geo support

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654#discussion_r451068987



##########
File path: pinot-core/src/test/java/org/apache/pinot/core/geospatial/serde/BenchmarkSerde.java
##########
@@ -0,0 +1,356 @@
+/**
+ * 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.pinot.core.geospatial.serde;
+
+import com.google.common.base.Joiner;
+import com.google.common.io.Resources;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.io.ParseException;
+import org.locationtech.jts.io.WKTReader;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Warmup;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.Options;
+import org.openjdk.jmh.runner.options.OptionsBuilder;
+import org.openjdk.jmh.runner.options.VerboseMode;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+
+import static com.google.common.io.Resources.getResource;
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.openjdk.jmh.annotations.Mode.Throughput;
+
+import static org.apache.pinot.core.geospatial.serde.GeometrySerializer.serialize;
+import static org.apache.pinot.core.geospatial.serde.GeometrySerializer.deserialize;
+
+@State(Scope.Thread)
+@Fork(2)
+@Warmup(iterations = 3, time = 3, timeUnit = SECONDS)
+@Measurement(iterations = 5, time = 4, timeUnit = SECONDS)
+@OutputTimeUnit(SECONDS)
+@BenchmarkMode(Throughput)
+public class BenchmarkSerde

Review comment:
       Added https://gist.github.com/yupeng9/8e2b081ffb372593492ebb6a41da97fd to the description




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] yupeng9 commented on pull request #5654: [Part 1] Add geo support

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654#issuecomment-659799835


   @Jackie-Jiang I changed the relationship functions to work only for the Geometry objects, to align with Presto's behavior. PTAL


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] siddharthteotia commented on pull request #5654: [Part 1] Add geo support

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654#issuecomment-653821779


   @yupeng9 , I would like to review this PR. Please give a day or two to go over the code. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] yupeng9 commented on pull request #5654: [Part 1] Add geo support

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654#issuecomment-656481288


   > > > High level question, why are we using JTS library to handle both geometry as well as geography? Shouldn't we use ESRI for geography?
   > > 
   > > 
   > > Good call.
   > > There are two reasons, as I found out during prototyping:
   > > 
   > > * Performance. I found the serialization performance JTS is better than ESRI. This is also confirmed by Presto project, as they are migrating from ESRI to JTS: [prestodb/presto#13604](https://github.com/prestodb/presto/pull/13604)
   > > * Simplicity on the dependency. With one less dependency, it's easier to manage the implementation. As in my current way, the geometry to geography conversion is just a simple change of coordinate system (SRID).
   > > 
   > > The tradeoff of taking this approach is that JTS is a library for Euclidean planar linear geometry, so all the geography-related operations have to be implemented using JTS's primitives. That's why there is some lengthy logic on geography measurement functions. Those implementations are similar to what Presto is doing.
   > > Will update the design doc to reflect this change
   > 
   > For ser-de, we are using the customized serializer, so I don't think there will be performance difference between these 2 libraries?
   
   Though we use customized serializer, there could be some difference due to the internal representation of the fields, their accessor implementations. The PR I linked above shows about the 20% difference. 
   
   Another notable reason is that JTS conforms to the ISO standards better. I believe this is the primary reason that Presto community decided to move from ESRI to JTS. I suggest we take the lessons learned from them. 
   
   Lastly, many users query Pinot via the Presto connector, so it's also a desirable property that Pinot geo functions return same or similar results as Presto's for better unification. 
   
   > For geography operations, we should leverage ESRI for the calculation (or are you planing to implement all geography operations yourself?).
   
   There are not too many geographical functions, so I believe its implementation is still manageable.
   
   > We need to support both geometry (SRID 0) and geography (SRID 4326) operations:
   > 
   > * If ESRI supports both geometry and geography operations, we can use only ESRI to do the calculation
   > * If not, we should use JTS for geometry and ESRI for geography calculation
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] yupeng9 commented on a change in pull request #5654: [Part 1] Add geo support

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654#discussion_r452386815



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/GeometryType.java
##########
@@ -0,0 +1,49 @@
+/**
+ * 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.pinot.core.geospatial;
+
+/**
+ * The geometry type.
+ */
+public enum GeometryType {
+
+  POINT(false, "ST_Point"),
+  MULTI_POINT(true, "ST_MultiPoint"),
+  LINE_STRING(false, "ST_LineString"),
+  MULTI_LINE_STRING(true, "ST_MultiLineString"),
+  POLYGON(false, "ST_Polygon"),
+  MULTI_POLYGON(true, "ST_MultiPolygon"),
+  GEOMETRY_COLLECTION(true, "ST_GeomCollection");
+
+  private final boolean multitype;
+  private final String name;
+
+  GeometryType(boolean multitype, String name) {
+    this.multitype = multitype;

Review comment:
       Thanks for taking a pass, updated the style




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] kishoreg commented on pull request #5654: [Part 1] Add geo support

Posted by GitBox <gi...@apache.org>.
kishoreg commented on pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654#issuecomment-653804777


   Add sample queries to the description and also update the java docs.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] yupeng9 commented on a change in pull request #5654: [Part 1] Add geo support

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654#discussion_r450628803



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/serde/GeometrySerializationType.java
##########
@@ -0,0 +1,76 @@
+/**
+ * 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.pinot.core.geospatial.serde;
+
+import org.apache.pinot.core.geospatial.GeometryType;
+
+public enum GeometrySerializationType
+{
+    POINT(0, GeometryType.POINT),
+    MULTI_POINT(1, GeometryType.MULTI_POINT),
+    LINE_STRING(2, GeometryType.LINE_STRING),
+    MULTI_LINE_STRING(3, GeometryType.MULTI_LINE_STRING),
+    POLYGON(4, GeometryType.POLYGON),
+    MULTI_POLYGON(5, GeometryType.MULTI_POLYGON),
+    GEOMETRY_COLLECTION(6, GeometryType.GEOMETRY_COLLECTION),
+    ENVELOPE(7, GeometryType.POLYGON);
+
+    private final int id;
+    private final GeometryType geometryType;
+
+    GeometrySerializationType(int id, GeometryType geometryType)
+    {
+        this.id = id;

Review comment:
       Sure. thanks.

##########
File path: pinot-core/pom.xml
##########
@@ -159,7 +159,14 @@
       <groupId>com.jayway.jsonpath</groupId>
       <artifactId>json-path</artifactId>
     </dependency>
-
+    <dependency>
+      <groupId>org.locationtech.jts</groupId>

Review comment:
       My understanding is okay to include.
   Per https://www.apache.org/legal/resolved.html#category-a, `Eclipse Distribution License 1.0` can be included. And JTS is dual-licensed under Eclipse Public License 2.0 and Eclipse Distribution License 1.0 (https://github.com/locationtech/jts#license)

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/serde/GeometrySerializationType.java
##########
@@ -0,0 +1,76 @@
+/**
+ * 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.pinot.core.geospatial.serde;
+
+import org.apache.pinot.core.geospatial.GeometryType;
+
+public enum GeometrySerializationType
+{
+    POINT(0, GeometryType.POINT),
+    MULTI_POINT(1, GeometryType.MULTI_POINT),
+    LINE_STRING(2, GeometryType.LINE_STRING),
+    MULTI_LINE_STRING(3, GeometryType.MULTI_LINE_STRING),
+    POLYGON(4, GeometryType.POLYGON),
+    MULTI_POLYGON(5, GeometryType.MULTI_POLYGON),
+    GEOMETRY_COLLECTION(6, GeometryType.GEOMETRY_COLLECTION),
+    ENVELOPE(7, GeometryType.POLYGON);
+
+    private final int id;
+    private final GeometryType geometryType;
+
+    GeometrySerializationType(int id, GeometryType geometryType)
+    {
+        this.id = id;
+        this.geometryType = geometryType;
+    }
+
+    public int id()
+    {
+        return id;
+    }
+
+    public GeometryType getGeometryType()
+    {
+        return geometryType;
+    }
+
+    public static GeometrySerializationType fromID(int id)
+    {
+        switch (id) {

Review comment:
       This list is unlikely to grow, given the OGC geo is a well-defined standard per https://www.ogc.org/standards/sfa

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/ConstructFromTextFunction.java
##########
@@ -0,0 +1,78 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.io.ParseException;
+import org.locationtech.jts.io.WKTReader;
+
+import java.util.List;
+import java.util.Map;
+
+public abstract class ConstructFromTextFunction extends BaseTransformFunction {
+    private TransformFunction _transformFunction;
+    private byte[][] _results;
+    private WKTReader _reader;
+
+    @Override
+    public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+        Preconditions
+                .checkArgument(arguments.size() == 1, "Exactly 1 argument is required for transform function: %s",
+                        getName());
+        TransformFunction transformFunction = arguments.get(0);
+        Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+                "The argument must be single-valued for transform function: %s", getName());
+        _transformFunction = transformFunction;
+        _reader = new WKTReader(getGeometryFactory());
+    }
+
+    abstract protected GeometryFactory getGeometryFactory();
+
+    @Override
+    public TransformResultMetadata getResultMetadata() {
+        return BYTES_SV_NO_DICTIONARY_METADATA;
+    }
+
+    @Override
+    public byte[][] transformToBytesValuesSV(ProjectionBlock projectionBlock) {
+        if (_results == null) {
+            _results = new byte[DocIdSetPlanNode.MAX_DOC_PER_CALL][];
+        }
+        String[] argumentValues = _transformFunction.transformToStringValuesSV(projectionBlock);
+        int length = projectionBlock.getNumDocs();
+        for (int i = 0; i < length; i++) {
+            try {
+                Geometry geometry = _reader.read(argumentValues[i]);
+                _results[i] = GeometrySerializer.serialize(geometry);
+            } catch (ParseException e) {
+                throw new RuntimeException(String.format("Failed to parse geometry from String %s", argumentValues[i]));

Review comment:
       Good to see this util

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/ConstructFromTextFunction.java
##########
@@ -0,0 +1,78 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.io.ParseException;
+import org.locationtech.jts.io.WKTReader;
+
+import java.util.List;
+import java.util.Map;
+
+public abstract class ConstructFromTextFunction extends BaseTransformFunction {

Review comment:
       Added. This is an abstract class for implementing the geo constructor functions like `StGeomFromText` and `StGeogFromText`

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/geospatial/serde/BenchmarkSerde.java
##########
@@ -0,0 +1,356 @@
+/**
+ * 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.pinot.core.geospatial.serde;
+
+import com.google.common.base.Joiner;
+import com.google.common.io.Resources;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.io.ParseException;
+import org.locationtech.jts.io.WKTReader;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Warmup;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.Options;
+import org.openjdk.jmh.runner.options.OptionsBuilder;
+import org.openjdk.jmh.runner.options.VerboseMode;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+
+import static com.google.common.io.Resources.getResource;
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.openjdk.jmh.annotations.Mode.Throughput;
+
+import static org.apache.pinot.core.geospatial.serde.GeometrySerializer.serialize;
+import static org.apache.pinot.core.geospatial.serde.GeometrySerializer.deserialize;
+
+@State(Scope.Thread)
+@Fork(2)
+@Warmup(iterations = 3, time = 3, timeUnit = SECONDS)
+@Measurement(iterations = 5, time = 4, timeUnit = SECONDS)
+@OutputTimeUnit(SECONDS)
+@BenchmarkMode(Throughput)
+public class BenchmarkSerde

Review comment:
       Sounds good

##########
File path: pinot-tools/src/main/java/org/apache/pinot/tools/streams/MeetupRsvpStream.java
##########
@@ -87,11 +90,14 @@ public void onMessage(String message) {
                   }
 
                   JsonNode group = messageJSON.get("group");
+                  System.out.println(String.format("reading group %s", group.get("group_id")));

Review comment:
       removed. it was for debugging purpose.

##########
File path: pinot-tools/src/main/resources/log4j2.xml
##########
@@ -44,7 +44,7 @@
 
   </Appenders>
   <Loggers>
-    <Root level="info" additivity="false">
+    <Root level="warn" additivity="false">

Review comment:
       You are right. Reverted this since it was for local debugging purpose.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] yupeng9 commented on a change in pull request #5654: [Part 1] Add geo support

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654#discussion_r457859631



##########
File path: pinot-core/pom.xml
##########
@@ -227,5 +234,17 @@
       <artifactId>lucene-analyzers-common</artifactId>
       <version>${lucene.version}</version>
     </dependency>
+    <dependency>

Review comment:
       moved it to `pinot-perf` project.

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/TransformFunctionType.java
##########
@@ -57,8 +57,27 @@
   MAPVALUE("mapValue"),
 
   // Special type for annotation based scalar functions
-  SCALAR("scalar");
+  SCALAR("scalar"),
+  // geo constructors
+  ST_GEOG_FROM_TEXT("ST_GeogFromText"),
+  ST_GEOM_FROM_TEXT("ST_GeomFromText"),
+  ST_GEOG_FROM_WKB("ST_GeogFromWKB"),
+  ST_GEOM_FROM_WKB("ST_GeomFromWKB"),
+  ST_POINT("ST_Point"),
+  ST_POLYGON("ST_Polygon"),
 
+  // geo measurements
+  ST_AREA("ST_Area"),
+  ST_DISTANCE("ST_Distance"),
+  ST_GEOMETRY_TYPE("ST_GEOMETRY_TYPE"),

Review comment:
       Yes, it returns the type of the geometry as a string. EG: 'ST_Linestring', 'ST_Polygon','ST_MultiPolygon' etc

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/GeometryType.java
##########
@@ -0,0 +1,49 @@
+/**
+ * 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.pinot.core.geospatial;
+
+/**
+ * The geometry type.
+ */
+public enum GeometryType {
+
+  POINT(false, "ST_Point"),
+  MULTI_POINT(true, "ST_MultiPoint"),
+  LINE_STRING(false, "ST_LineString"),
+  MULTI_LINE_STRING(true, "ST_MultiLineString"),
+  POLYGON(false, "ST_Polygon"),
+  MULTI_POLYGON(true, "ST_MultiPolygon"),
+  GEOMETRY_COLLECTION(true, "ST_GeomCollection");
+
+  private final boolean _multitype;
+  private final String _name;
+
+  GeometryType(boolean multitype, String name) {
+    _multitype = multitype;
+    _name = name;
+  }
+
+  public boolean isMultitype() {

Review comment:
       not in this PR. It's useful in function like https://postgis.net/docs/ST_GeometryN.html

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/GeometryType.java
##########
@@ -0,0 +1,49 @@
+/**
+ * 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.pinot.core.geospatial;
+
+/**
+ * The geometry type.
+ */
+public enum GeometryType {
+
+  POINT(false, "ST_Point"),
+  MULTI_POINT(true, "ST_MultiPoint"),
+  LINE_STRING(false, "ST_LineString"),

Review comment:
       `LINEAR_RING` is a subtype of `LINEAR_STRING`

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/serde/GeometrySerializationType.java
##########
@@ -0,0 +1,85 @@
+/**
+ * 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.pinot.core.geospatial.serde;
+
+import org.apache.pinot.core.geospatial.GeometryType;
+
+
+/**
+ * The geometry type used in serialization
+ */
+public enum GeometrySerializationType {

Review comment:
       removed. 

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/serde/GeometrySerde.java
##########
@@ -0,0 +1,439 @@
+/**
+ * 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.pinot.core.geospatial.serde;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.Envelope;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.LinearRing;
+import org.locationtech.jts.geom.MultiPoint;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+import org.locationtech.jts.geom.TopologyException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static com.google.common.base.Verify.verify;
+import static com.google.common.collect.Iterables.getOnlyElement;
+import static java.lang.Double.NaN;
+import static java.lang.Double.isNaN;
+import static java.util.Objects.requireNonNull;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_FACTORY;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_GET_MASK;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_SET_MASK;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_SRID;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOMETRY_FACTORY;
+
+
+/**
+ * Provides methods to efficiently serialize and deserialize geometry types.
+ */
+public class GeometrySerde extends Serializer {
+  private static final Logger LOGGER = LoggerFactory.getLogger(GeometrySerde.class);
+
+  @Override
+  public void write(Kryo kryo, Output output, Object object) {

Review comment:
       Thanks. This is a very good suggestion. It's true that using pre-sized `ByteBuffer` would be more performant than the stream API. I run a benchmark on three implementations: stream, ByteBuffer with Big_Endian, ByteBuffer with Little_Endian: https://docs.google.com/spreadsheets/d/1ANkC7I4N-TxtzFI-fDbchJ9uGmyQtOG_ViYY0mSstNM/edit#gid=0. 
   
   The benchmark confirms the performance gain. Also, the result shows `BIG_ENDIAN` is more performant, as it's Java's default order. I will go with the default order.

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StAreaFunction.java
##########
@@ -0,0 +1,221 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+
+import java.util.List;
+import java.util.Map;
+
+import static com.google.common.base.Preconditions.checkState;
+import static java.lang.Math.PI;
+import static java.lang.Math.toRadians;
+
+
+/**
+ * Function that calculates the area of the given geo-spatial object. For geometry type, it returns the 2D Euclidean
+ * area of a geometry. For geography, returns the area of a polygon or multi-polygon in square meters using a spherical
+ * model for Earth.
+ */
+public class StAreaFunction extends BaseTransformFunction {
+  private TransformFunction _transformFunction;
+  public static final String FUNCTION_NAME = "ST_Area";
+  private double[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 1, "Exactly 1 argument is required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Argument must be single-valued for transform function: %s", getName());
+    _transformFunction = transformFunction;
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return DOUBLE_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public double[] transformToDoubleValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new double[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+
+    byte[][] values = _transformFunction.transformToBytesValuesSV(projectionBlock);
+    Geometry geometry;
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      geometry = GeometrySerializer.deserialize(values[i]);

Review comment:
       I don't think it's needed. JIT will take care of it

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/ConstructFromWKBFunction.java
##########
@@ -0,0 +1,80 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.io.ParseException;
+import org.locationtech.jts.io.WKBReader;
+
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * An abstract class for implementing the geo constructor functions from well-known binary (WKB) format.
+ */
+abstract class ConstructFromWKBFunction extends BaseTransformFunction {
+  private TransformFunction _transformFunction;
+  private byte[][] _results;
+  private WKBReader _reader;
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 1, "Exactly 1 argument is required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "The argument must be single-valued for transform function: %s", getName());
+    _transformFunction = transformFunction;
+    _reader = new WKBReader(getGeometryFactory());
+  }
+
+  abstract protected GeometryFactory getGeometryFactory();
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return BYTES_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public byte[][] transformToBytesValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new byte[DocIdSetPlanNode.MAX_DOC_PER_CALL][];
+    }
+    byte[][] argumentValues = _transformFunction.transformToBytesValuesSV(projectionBlock);
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      try {
+        Geometry geometry = _reader.read(argumentValues[i]);
+        _results[i] = GeometrySerializer.serialize(geometry);
+      } catch (ParseException e) {
+        throw new RuntimeException(String.format("Failed to parse geometry from bytes %s", argumentValues[i]));

Review comment:
       good catch

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/ConstructFromTextFunction.java
##########
@@ -0,0 +1,83 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.common.Utils;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.io.ParseException;
+import org.locationtech.jts.io.WKTReader;
+
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * An abstract class for implementing the geo constructor functions from text.
+ */
+abstract class ConstructFromTextFunction extends BaseTransformFunction {
+  private TransformFunction _transformFunction;
+  private byte[][] _results;
+  private WKTReader _reader;
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 1, "Exactly 1 argument is required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "The argument must be single-valued for transform function: %s", getName());

Review comment:
       added

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StAreaFunction.java
##########
@@ -0,0 +1,221 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+
+import java.util.List;
+import java.util.Map;
+
+import static com.google.common.base.Preconditions.checkState;
+import static java.lang.Math.PI;
+import static java.lang.Math.toRadians;
+
+
+/**
+ * Function that calculates the area of the given geo-spatial object. For geometry type, it returns the 2D Euclidean
+ * area of a geometry. For geography, returns the area of a polygon or multi-polygon in square meters using a spherical
+ * model for Earth.
+ */
+public class StAreaFunction extends BaseTransformFunction {
+  private TransformFunction _transformFunction;
+  public static final String FUNCTION_NAME = "ST_Area";
+  private double[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 1, "Exactly 1 argument is required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Argument must be single-valued for transform function: %s", getName());
+    _transformFunction = transformFunction;
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return DOUBLE_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public double[] transformToDoubleValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new double[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+
+    byte[][] values = _transformFunction.transformToBytesValuesSV(projectionBlock);
+    Geometry geometry;
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      geometry = GeometrySerializer.deserialize(values[i]);
+      _results[i] = GeometryUtils.isGeography(geometry) ? calculateGeographyArea(geometry) : geometry.getArea();
+    }
+    return _results;
+  }
+
+  private double calculateGeographyArea(Geometry geometry) {
+    Polygon polygon = (Polygon) geometry;
+
+    // See https://www.movable-type.co.uk/scripts/latlong.html
+    // and http://osgeo-org.1560.x6.nabble.com/Area-of-a-spherical-polygon-td3841625.html
+    // and https://www.element84.com/blog/determining-if-a-spherical-polygon-contains-a-pole
+    // for the underlying Maths
+
+    double sphericalExcess = Math.abs(computeSphericalExcess(polygon.getExteriorRing()));
+
+    for (int i = 0; i < polygon.getNumInteriorRing(); i++) {
+      sphericalExcess -= Math.abs(computeSphericalExcess(polygon.getInteriorRingN(i)));
+    }
+
+    // Math.abs is required here because for Polygons with a 2D area of 0
+    // isExteriorRing returns false for the exterior ring
+    return Math.abs(sphericalExcess * GeometryUtils.EARTH_RADIUS_M * GeometryUtils.EARTH_RADIUS_M);
+  }
+
+  private static double computeSphericalExcess(LineString lineString) {
+    if (lineString.getNumPoints() < 3) {
+      // A path with less than 3 distinct points is not valid for calculating an area
+      throw new RuntimeException("Polygon is not valid: a loop contains less then 3 vertices.");
+    }
+
+    // Initialize the calculator with the last point
+    SphericalExcessCalculator calculator = new SphericalExcessCalculator(lineString.getEndPoint());
+
+    // Our calculations rely on not processing the same point twice
+    int start = lineString.getStartPoint().equals(lineString.getEndPoint()) ? 1 : 0;

Review comment:
       true. can be optimized.

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StAreaFunction.java
##########
@@ -0,0 +1,221 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+
+import java.util.List;
+import java.util.Map;
+
+import static com.google.common.base.Preconditions.checkState;
+import static java.lang.Math.PI;
+import static java.lang.Math.toRadians;
+
+
+/**
+ * Function that calculates the area of the given geo-spatial object. For geometry type, it returns the 2D Euclidean
+ * area of a geometry. For geography, returns the area of a polygon or multi-polygon in square meters using a spherical
+ * model for Earth.
+ */
+public class StAreaFunction extends BaseTransformFunction {
+  private TransformFunction _transformFunction;
+  public static final String FUNCTION_NAME = "ST_Area";
+  private double[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 1, "Exactly 1 argument is required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Argument must be single-valued for transform function: %s", getName());
+    _transformFunction = transformFunction;
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return DOUBLE_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public double[] transformToDoubleValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new double[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+
+    byte[][] values = _transformFunction.transformToBytesValuesSV(projectionBlock);
+    Geometry geometry;
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      geometry = GeometrySerializer.deserialize(values[i]);
+      _results[i] = GeometryUtils.isGeography(geometry) ? calculateGeographyArea(geometry) : geometry.getArea();
+    }
+    return _results;
+  }
+
+  private double calculateGeographyArea(Geometry geometry) {
+    Polygon polygon = (Polygon) geometry;
+
+    // See https://www.movable-type.co.uk/scripts/latlong.html
+    // and http://osgeo-org.1560.x6.nabble.com/Area-of-a-spherical-polygon-td3841625.html
+    // and https://www.element84.com/blog/determining-if-a-spherical-polygon-contains-a-pole
+    // for the underlying Maths
+
+    double sphericalExcess = Math.abs(computeSphericalExcess(polygon.getExteriorRing()));
+
+    for (int i = 0; i < polygon.getNumInteriorRing(); i++) {
+      sphericalExcess -= Math.abs(computeSphericalExcess(polygon.getInteriorRingN(i)));
+    }
+
+    // Math.abs is required here because for Polygons with a 2D area of 0
+    // isExteriorRing returns false for the exterior ring
+    return Math.abs(sphericalExcess * GeometryUtils.EARTH_RADIUS_M * GeometryUtils.EARTH_RADIUS_M);
+  }
+
+  private static double computeSphericalExcess(LineString lineString) {
+    if (lineString.getNumPoints() < 3) {
+      // A path with less than 3 distinct points is not valid for calculating an area
+      throw new RuntimeException("Polygon is not valid: a loop contains less then 3 vertices.");
+    }
+
+    // Initialize the calculator with the last point
+    SphericalExcessCalculator calculator = new SphericalExcessCalculator(lineString.getEndPoint());
+
+    // Our calculations rely on not processing the same point twice
+    int start = lineString.getStartPoint().equals(lineString.getEndPoint()) ? 1 : 0;
+
+    for (int i = start; i < lineString.getNumPoints(); i++) {
+      calculator.add(lineString.getPointN(i));
+    }
+    return calculator.computeSphericalExcess();
+  }
+
+  private static class SphericalExcessCalculator {
+    private static final double TWO_PI = 2 * Math.PI;
+    private static final double THREE_PI = 3 * Math.PI;
+
+    private double sphericalExcess;
+    private double courseDelta;
+
+    private boolean firstPoint;
+    private double firstInitialBearing;
+    private double previousFinalBearing;
+
+    private double previousPhi;
+    private double previousCos;
+    private double previousSin;
+    private double previousTan;
+    private double previousLongitude;
+
+    private boolean done;
+
+    public SphericalExcessCalculator(Point endPoint) {
+      previousPhi = toRadians(endPoint.getY());
+      previousSin = Math.sin(previousPhi);
+      previousCos = Math.cos(previousPhi);
+      previousTan = Math.tan(previousPhi / 2);
+      previousLongitude = toRadians(endPoint.getX());
+      firstPoint = true;
+    }
+
+    private void add(Point point)
+        throws IllegalStateException {
+      checkState(!done, "Computation of spherical excess is complete");
+
+      double phi = toRadians(point.getY());
+      double tan = Math.tan(phi / 2);
+      double longitude = toRadians(point.getX());
+
+      // We need to check for that specifically
+      // Otherwise calculating the bearing is not deterministic
+      if (longitude == previousLongitude && phi == previousPhi) {
+        throw new RuntimeException("Polygon is not valid: it has two identical consecutive vertices");
+      }
+
+      double deltaLongitude = longitude - previousLongitude;
+      sphericalExcess += 2 * Math.atan2(Math.tan(deltaLongitude / 2) * (previousTan + tan), 1 + previousTan * tan);
+
+      double cos = Math.cos(phi);
+      double sin = Math.sin(phi);
+      double sinOfDeltaLongitude = Math.sin(deltaLongitude);
+      double cosOfDeltaLongitude = Math.cos(deltaLongitude);
+
+      // Initial bearing from previous to current
+      double y = sinOfDeltaLongitude * cos;
+      double x = previousCos * sin - previousSin * cos * cosOfDeltaLongitude;
+      double initialBearing = (Math.atan2(y, x) + TWO_PI) % TWO_PI;
+
+      // Final bearing from previous to current = opposite of bearing from current to previous
+      double finalY = -sinOfDeltaLongitude * previousCos;
+      double finalX = previousSin * cos - previousCos * sin * cosOfDeltaLongitude;
+      double finalBearing = (Math.atan2(finalY, finalX) + PI) % TWO_PI;
+
+      // When processing our first point we don't yet have a previousFinalBearing
+      if (firstPoint) {
+        // So keep our initial bearing around, and we'll use it at the end
+        // with the last final bearing
+        firstInitialBearing = initialBearing;
+        firstPoint = false;
+      } else {
+        courseDelta += (initialBearing - previousFinalBearing + THREE_PI) % TWO_PI - PI;
+      }
+
+      courseDelta += (finalBearing - initialBearing + THREE_PI) % TWO_PI - PI;
+
+      previousFinalBearing = finalBearing;
+      previousCos = cos;
+      previousSin = sin;
+      previousPhi = phi;
+      previousTan = tan;
+      previousLongitude = longitude;
+    }
+
+    public double computeSphericalExcess() {
+      if (!done) {
+        // Now that we have our last final bearing, we can calculate the remaining course delta
+        courseDelta += (firstInitialBearing - previousFinalBearing + THREE_PI) % TWO_PI - PI;
+
+        // The courseDelta should be 2Pi or - 2Pi, unless a pole is enclosed (and then it should be ~ 0)

Review comment:
       it shall be 2PI or 2PI per https://www.element84.com/blog/determining-if-a-spherical-polygon-contains-a-pole 
   Since it makes 360 degree complete turn

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StAsTextFunction.java
##########
@@ -0,0 +1,89 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.common.function.annotations.ScalarFunction;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.io.WKTWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * Returns the text representation of the geometry object.
+ */
+public class StAsTextFunction extends BaseTransformFunction {
+  private static final Logger LOGGER = LoggerFactory.getLogger(StAsTextFunction.class);
+  private TransformFunction _transformFunction;
+  private static WKTWriter _writer;
+  public static final String FUNCTION_NAME = "ST_AsText";
+  private String[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 1, "Exactly 1 argument is required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Argument must be single-valued for transform function: %s", getName());
+    _transformFunction = transformFunction;
+    _writer = new WKTWriter();
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return STRING_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public String[] transformToStringValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new String[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+    byte[][] values = _transformFunction.transformToBytesValuesSV(projectionBlock);
+    Geometry geometry;
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      geometry = GeometrySerializer.deserialize(values[i]);
+      _results[i] = _writer.write(geometry);
+    }
+    return _results;
+  }
+
+  @ScalarFunction
+  public static String stAsText(byte[] bytes) {

Review comment:
       sure. moved

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StDistanceFunction.java
##########
@@ -0,0 +1,173 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryType;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.Point;
+
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.atan2;
+import static java.lang.Math.cos;
+import static java.lang.Math.sin;
+import static java.lang.Math.sqrt;
+import static java.lang.Math.toRadians;
+
+
+/**
+ * Function that measures the distance between the two geo-spatial objects. For geometry type, returns the 2-dimensional
+ * cartesian minimum distance (based on spatial ref) between two geometries in projected units. For geography, returns
+ * the great-circle distance in meters between two SphericalGeography points. Note that g1, g2 shall have the same type.
+ */
+public class StDistanceFunction extends BaseTransformFunction {
+  private static final float MIN_LATITUDE = -90;
+  private static final float MAX_LATITUDE = 90;
+  private static final float MIN_LONGITUDE = -180;
+  private static final float MAX_LONGITUDE = 180;
+  public static final String FUNCTION_NAME = "ST_Distance";
+  private TransformFunction _firstArgument;
+  private TransformFunction _secondArgument;
+  private double[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 2, "2 arguments are required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "First argument must be single-valued for transform function: %s", getName());
+    _firstArgument = transformFunction;
+    transformFunction = arguments.get(1);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Second argument must be single-valued for transform function: %s", getName());
+    _secondArgument = transformFunction;
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return DOUBLE_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public double[] transformToDoubleValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new double[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+    byte[][] firstValues = _firstArgument.transformToBytesValuesSV(projectionBlock);
+    byte[][] secondValues = _secondArgument.transformToBytesValuesSV(projectionBlock);
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      Geometry firstGeometry = GeometrySerializer.deserialize(firstValues[i]);
+      Geometry secondGeometry = GeometrySerializer.deserialize(secondValues[i]);
+      if (GeometryUtils.isGeography(firstGeometry) != GeometryUtils.isGeography(secondGeometry)) {
+        throw new RuntimeException(
+            String.format("The first and second arguments shall either all be geometry" + " or all geography"));
+      }
+      if (GeometryUtils.isGeography(firstGeometry)) {
+        _results[i] = sphericalDistance(firstGeometry, secondGeometry);
+      } else {
+        _results[i] =
+            firstGeometry.isEmpty() || secondGeometry.isEmpty() ? null : firstGeometry.distance(secondGeometry);
+      }
+    }
+    return _results;
+  }
+
+  public static void checkLatitude(double latitude) {
+    if (Double.isNaN(latitude) || Double.isInfinite(latitude) || latitude < MIN_LATITUDE || latitude > MAX_LATITUDE) {
+      throw new RuntimeException("Latitude must be between -90 and 90");
+    }
+  }
+
+  public static void checkLongitude(double longitude) {
+    if (Double.isNaN(longitude) || Double.isInfinite(longitude) || longitude < MIN_LONGITUDE
+        || longitude > MAX_LONGITUDE) {
+      throw new RuntimeException("Longitude must be between -180 and 180");
+    }
+  }
+
+  public static Double sphericalDistance(Geometry leftGeometry, Geometry rightGeometry) {
+    if (leftGeometry.isEmpty() || rightGeometry.isEmpty()) {
+      return null;
+    }
+
+    validateGeographyType("ST_Distance", leftGeometry, EnumSet.of(GeometryType.POINT));

Review comment:
       try to reuse the same error msg template of several functions

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StDistanceFunction.java
##########
@@ -0,0 +1,173 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryType;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.Point;
+
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.atan2;
+import static java.lang.Math.cos;
+import static java.lang.Math.sin;
+import static java.lang.Math.sqrt;
+import static java.lang.Math.toRadians;
+
+
+/**
+ * Function that measures the distance between the two geo-spatial objects. For geometry type, returns the 2-dimensional
+ * cartesian minimum distance (based on spatial ref) between two geometries in projected units. For geography, returns
+ * the great-circle distance in meters between two SphericalGeography points. Note that g1, g2 shall have the same type.
+ */
+public class StDistanceFunction extends BaseTransformFunction {
+  private static final float MIN_LATITUDE = -90;
+  private static final float MAX_LATITUDE = 90;
+  private static final float MIN_LONGITUDE = -180;
+  private static final float MAX_LONGITUDE = 180;
+  public static final String FUNCTION_NAME = "ST_Distance";
+  private TransformFunction _firstArgument;
+  private TransformFunction _secondArgument;
+  private double[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 2, "2 arguments are required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "First argument must be single-valued for transform function: %s", getName());
+    _firstArgument = transformFunction;
+    transformFunction = arguments.get(1);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Second argument must be single-valued for transform function: %s", getName());
+    _secondArgument = transformFunction;
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return DOUBLE_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public double[] transformToDoubleValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new double[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+    byte[][] firstValues = _firstArgument.transformToBytesValuesSV(projectionBlock);
+    byte[][] secondValues = _secondArgument.transformToBytesValuesSV(projectionBlock);
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      Geometry firstGeometry = GeometrySerializer.deserialize(firstValues[i]);
+      Geometry secondGeometry = GeometrySerializer.deserialize(secondValues[i]);
+      if (GeometryUtils.isGeography(firstGeometry) != GeometryUtils.isGeography(secondGeometry)) {
+        throw new RuntimeException(
+            String.format("The first and second arguments shall either all be geometry" + " or all geography"));
+      }
+      if (GeometryUtils.isGeography(firstGeometry)) {
+        _results[i] = sphericalDistance(firstGeometry, secondGeometry);
+      } else {
+        _results[i] =
+            firstGeometry.isEmpty() || secondGeometry.isEmpty() ? null : firstGeometry.distance(secondGeometry);

Review comment:
       hmm, without null handling, what's the default value we return for bytes?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StDistanceFunction.java
##########
@@ -0,0 +1,173 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryType;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.Point;
+
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.atan2;
+import static java.lang.Math.cos;
+import static java.lang.Math.sin;
+import static java.lang.Math.sqrt;
+import static java.lang.Math.toRadians;
+
+
+/**
+ * Function that measures the distance between the two geo-spatial objects. For geometry type, returns the 2-dimensional
+ * cartesian minimum distance (based on spatial ref) between two geometries in projected units. For geography, returns
+ * the great-circle distance in meters between two SphericalGeography points. Note that g1, g2 shall have the same type.
+ */
+public class StDistanceFunction extends BaseTransformFunction {
+  private static final float MIN_LATITUDE = -90;
+  private static final float MAX_LATITUDE = 90;
+  private static final float MIN_LONGITUDE = -180;
+  private static final float MAX_LONGITUDE = 180;
+  public static final String FUNCTION_NAME = "ST_Distance";
+  private TransformFunction _firstArgument;
+  private TransformFunction _secondArgument;
+  private double[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 2, "2 arguments are required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "First argument must be single-valued for transform function: %s", getName());
+    _firstArgument = transformFunction;
+    transformFunction = arguments.get(1);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Second argument must be single-valued for transform function: %s", getName());
+    _secondArgument = transformFunction;
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return DOUBLE_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public double[] transformToDoubleValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new double[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+    byte[][] firstValues = _firstArgument.transformToBytesValuesSV(projectionBlock);
+    byte[][] secondValues = _secondArgument.transformToBytesValuesSV(projectionBlock);
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      Geometry firstGeometry = GeometrySerializer.deserialize(firstValues[i]);
+      Geometry secondGeometry = GeometrySerializer.deserialize(secondValues[i]);
+      if (GeometryUtils.isGeography(firstGeometry) != GeometryUtils.isGeography(secondGeometry)) {
+        throw new RuntimeException(
+            String.format("The first and second arguments shall either all be geometry" + " or all geography"));
+      }
+      if (GeometryUtils.isGeography(firstGeometry)) {
+        _results[i] = sphericalDistance(firstGeometry, secondGeometry);
+      } else {
+        _results[i] =
+            firstGeometry.isEmpty() || secondGeometry.isEmpty() ? null : firstGeometry.distance(secondGeometry);
+      }
+    }
+    return _results;
+  }
+
+  public static void checkLatitude(double latitude) {
+    if (Double.isNaN(latitude) || Double.isInfinite(latitude) || latitude < MIN_LATITUDE || latitude > MAX_LATITUDE) {
+      throw new RuntimeException("Latitude must be between -90 and 90");
+    }
+  }
+
+  public static void checkLongitude(double longitude) {
+    if (Double.isNaN(longitude) || Double.isInfinite(longitude) || longitude < MIN_LONGITUDE
+        || longitude > MAX_LONGITUDE) {
+      throw new RuntimeException("Longitude must be between -180 and 180");
+    }
+  }
+
+  public static Double sphericalDistance(Geometry leftGeometry, Geometry rightGeometry) {
+    if (leftGeometry.isEmpty() || rightGeometry.isEmpty()) {
+      return null;
+    }
+
+    validateGeographyType("ST_Distance", leftGeometry, EnumSet.of(GeometryType.POINT));
+    validateGeographyType("ST_Distance", rightGeometry, EnumSet.of(GeometryType.POINT));
+    Point leftPoint = (Point) leftGeometry;
+    Point rightPoint = (Point) rightGeometry;
+
+    // greatCircleDistance returns distance in KM.
+    return greatCircleDistance(leftPoint.getY(), leftPoint.getX(), rightPoint.getY(), rightPoint.getX()) * 1000;
+  }
+
+  /**
+   * Calculate the distance between two points on Earth.
+   * <p>
+   * This assumes a spherical Earth, and uses the Vincenty formula. (https://en.wikipedia
+   * .org/wiki/Great-circle_distance)
+   */
+  public static double greatCircleDistance(double latitude1, double longitude1, double latitude2, double longitude2) {

Review comment:
       yes, per https://en.wikipedia.org/wiki/Great-circle_distance

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/serde/GeometrySerde.java
##########
@@ -0,0 +1,439 @@
+/**
+ * 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.pinot.core.geospatial.serde;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.Envelope;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.LinearRing;
+import org.locationtech.jts.geom.MultiPoint;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+import org.locationtech.jts.geom.TopologyException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static com.google.common.base.Verify.verify;
+import static com.google.common.collect.Iterables.getOnlyElement;
+import static java.lang.Double.NaN;
+import static java.lang.Double.isNaN;
+import static java.util.Objects.requireNonNull;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_FACTORY;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_GET_MASK;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_SET_MASK;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOGRAPHY_SRID;
+import static org.apache.pinot.core.geospatial.GeometryUtils.GEOMETRY_FACTORY;
+
+
+/**
+ * Provides methods to efficiently serialize and deserialize geometry types.
+ */
+public class GeometrySerde extends Serializer {

Review comment:
       They are not exactly same, in particular, the differences are:
    - Presto uses schema to indicate geometry vs geography info, while we encode this in the type byte.
    - Presto serializes additional information such as envelope to be compatible with ESRI serialization, but the serde here does not, which is simpler and faster
   
   Added this to the comments

##########
File path: pinot-perf/pom.xml
##########
@@ -98,7 +98,7 @@
     <dependency>
       <groupId>org.openjdk.jmh</groupId>
       <artifactId>jmh-core</artifactId>
-      <version>1.21</version>
+      <version>${jmh.version}</version>

Review comment:
       it's already there

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StEqualsFunction.java
##########
@@ -0,0 +1,87 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * Function that returns true if the given geometries represent the same geometry.
+ */
+public class StEqualsFunction extends BaseTransformFunction {
+  public static final String FUNCTION_NAME = "ST_Equals";
+  private TransformFunction _firstArgument;
+  private TransformFunction _secondArgument;
+  private int[] _results;
+
+  @Override
+  public String getName() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 2, "2 arguments are required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "First argument must be single-valued for transform function: %s", getName());
+    _firstArgument = transformFunction;
+    transformFunction = arguments.get(1);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "Second argument must be single-valued for transform function: %s", getName());
+    _secondArgument = transformFunction;
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return INT_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public int[] transformToIntValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new int[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+    byte[][] firstValues = _firstArgument.transformToBytesValuesSV(projectionBlock);
+    byte[][] secondValues = _secondArgument.transformToBytesValuesSV(projectionBlock);
+    for (int i = 0; i < projectionBlock.getNumDocs(); i++) {
+      Geometry firstGeometry = GeometrySerializer.deserialize(firstValues[i]);
+      Geometry secondGeometry = GeometrySerializer.deserialize(secondValues[i]);
+      if (GeometryUtils.isGeography(firstGeometry) || GeometryUtils.isGeography(secondGeometry)) {

Review comment:
       yes, it should

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/ConstructFromTextFunction.java
##########
@@ -0,0 +1,83 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.common.Utils;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.geospatial.serde.GeometrySerializer;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.operator.transform.function.BaseTransformFunction;
+import org.apache.pinot.core.operator.transform.function.TransformFunction;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.io.ParseException;
+import org.locationtech.jts.io.WKTReader;
+
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * An abstract class for implementing the geo constructor functions from text.
+ */
+abstract class ConstructFromTextFunction extends BaseTransformFunction {
+  private TransformFunction _transformFunction;
+  private byte[][] _results;
+  private WKTReader _reader;
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    Preconditions
+        .checkArgument(arguments.size() == 1, "Exactly 1 argument is required for transform function: %s", getName());
+    TransformFunction transformFunction = arguments.get(0);
+    Preconditions.checkArgument(transformFunction.getResultMetadata().isSingleValue(),
+        "The argument must be single-valued for transform function: %s", getName());
+    _transformFunction = transformFunction;
+    _reader = new WKTReader(getGeometryFactory());
+  }
+
+  abstract protected GeometryFactory getGeometryFactory();
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return BYTES_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public byte[][] transformToBytesValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new byte[DocIdSetPlanNode.MAX_DOC_PER_CALL][];
+    }
+    String[] argumentValues = _transformFunction.transformToStringValuesSV(projectionBlock);
+    int length = projectionBlock.getNumDocs();
+    for (int i = 0; i < length; i++) {
+      try {
+        Geometry geometry = _reader.read(argumentValues[i]);
+        _results[i] = GeometrySerializer.serialize(geometry);

Review comment:
       then i have to have another line to initialize `geometry`

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/transform/function/StPolygonFunction.java
##########
@@ -0,0 +1,40 @@
+/**
+ * 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.pinot.core.geospatial.transform.function;
+
+import org.apache.pinot.core.geospatial.GeometryUtils;
+import org.locationtech.jts.geom.GeometryFactory;
+
+
+/**
+ * Constructor function for polygon object from text.
+ */
+public class StPolygonFunction extends ConstructFromTextFunction {

Review comment:
       yup, added the constraint of checking polygon type




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #5654: [Part 1] Add geo support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654#discussion_r451243023



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/GeometryType.java
##########
@@ -0,0 +1,49 @@
+/**
+ * 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.pinot.core.geospatial;
+
+/**
+ * The geometry type.
+ */
+public enum GeometryType {
+
+  POINT(false, "ST_Point"),
+  MULTI_POINT(true, "ST_MultiPoint"),
+  LINE_STRING(false, "ST_LineString"),
+  MULTI_LINE_STRING(true, "ST_MultiLineString"),
+  POLYGON(false, "ST_Polygon"),
+  MULTI_POLYGON(true, "ST_MultiPolygon"),
+  GEOMETRY_COLLECTION(true, "ST_GeomCollection");
+
+  private final boolean multitype;
+  private final String name;
+
+  GeometryType(boolean multitype, String name) {
+    this.multitype = multitype;

Review comment:
       As an example of Pinot coding convention
   ```suggestion
       _multitype = multitype;
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #5654: [Part 1] Add geo support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654#discussion_r451242850



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/GeometryType.java
##########
@@ -0,0 +1,49 @@
+/**
+ * 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.pinot.core.geospatial;
+
+/**
+ * The geometry type.
+ */
+public enum GeometryType {
+
+  POINT(false, "ST_Point"),
+  MULTI_POINT(true, "ST_MultiPoint"),
+  LINE_STRING(false, "ST_LineString"),
+  MULTI_LINE_STRING(true, "ST_MultiLineString"),
+  POLYGON(false, "ST_Polygon"),
+  MULTI_POLYGON(true, "ST_MultiPolygon"),
+  GEOMETRY_COLLECTION(true, "ST_GeomCollection");
+
+  private final boolean multitype;

Review comment:
       Please follow the Pinot coding convention of using underscore as the prefix for member variables. Same for other classes




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] yupeng9 commented on pull request #5654: [Part 1] Add geo support

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654#issuecomment-662793026


   @Jackie-Jiang thanks for taking another pass. Comments addressed, and feel free to merge


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] yupeng9 commented on pull request #5654: [Part 1] Add geo support

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654#issuecomment-656268431


   > High level question, why are we using JTS library to handle both geometry as well as geography? Shouldn't we use ESRI for geography?
   
   Good call.
   There are two reasons, as I found out during prototyping:
   - Performance. I found the serialization performance JTS is better than ESRI. This is also confirmed by Presto project, as they are migrating from ESRI to JTS: https://github.com/prestodb/presto/pull/13604
   - Simplicity on the dependency. With one less dependency, it's easier to manage the implementation. As in my current way, the geometry to geography conversion is just a simple change of coordinate system (SRID).
   
   The tradeoff of taking this approach is that JTS is a library for Euclidean planar linear geometry, so all the geography-related operations have to be implemented using JTS's primitives. That's why there is some lengthy logic on geography measurement functions. Those implementations are similar to what Presto is doing.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #5654: [Part 1] Add geo support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #5654:
URL: https://github.com/apache/incubator-pinot/pull/5654#discussion_r452448439



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/geospatial/GeometryType.java
##########
@@ -0,0 +1,49 @@
+/**
+ * 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.pinot.core.geospatial;
+
+/**
+ * The geometry type.
+ */
+public enum GeometryType {
+
+  POINT(false, "ST_Point"),
+  MULTI_POINT(true, "ST_MultiPoint"),
+  LINE_STRING(false, "ST_LineString"),
+  MULTI_LINE_STRING(true, "ST_MultiLineString"),
+  POLYGON(false, "ST_Polygon"),
+  MULTI_POLYGON(true, "ST_MultiPolygon"),
+  GEOMETRY_COLLECTION(true, "ST_GeomCollection");
+
+  private final boolean multitype;
+  private final String name;
+
+  GeometryType(boolean multitype, String name) {
+    this.multitype = multitype;

Review comment:
       Good suggestion. We can switch it to `error` once we fixed all the existing code with wrong code styles. Opened issue #5675 to track this




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org